[jira] [Assigned] (HADOOP-18716) [JDK-17] Failed unit tests , with Java 17 runtime and compiled Java 8
[ https://issues.apache.org/jira/browse/HADOOP-18716?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bilwa S T reassigned HADOOP-18716: -- Assignee: Bilwa S T > [JDK-17] Failed unit tests , with Java 17 runtime and compiled Java 8 > - > > Key: HADOOP-18716 > URL: https://issues.apache.org/jira/browse/HADOOP-18716 > Project: Hadoop Common > Issue Type: Bug >Reporter: Vinay Devadiga >Assignee: Bilwa S T >Priority: Critical > > Compiled Hadoop - Hadoop branch 3.3.3 > mvn clean install - DskipTests=True > Java_Home -> points to Java-8 > maven version - 3.8.8 (Quite latest) > > Ran various whole test suit on my private cloud environment - > Changed Java_Home to Java-17 > > mvn surefire:test > > Out of 22k tests - 2.5 k tests failed . -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Updated] (HADOOP-19052) Hadoop use Shell command to get the count of the hard link which takes a lot of time
[ https://issues.apache.org/jira/browse/HADOOP-19052?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] ASF GitHub Bot updated HADOOP-19052: Labels: pull-request-available (was: ) > Hadoop use Shell command to get the count of the hard link which takes a lot > of time > > > Key: HADOOP-19052 > URL: https://issues.apache.org/jira/browse/HADOOP-19052 > Project: Hadoop Common > Issue Type: Improvement > Environment: Hadopp 3.3.4 >Reporter: liang yu >Priority: Major > Labels: pull-request-available > > Using Hadoop 3.3.4 > > When the QPS of `append` executions is very high, at a rate of above 1/s. > > We found that the write speed in hadoop is very slow. We traced some > datanodes' log and find that there is a warning : > {code:java} > 2024-01-26 11:09:44,292 WARN impl.FsDatasetImpl > (InstrumentedLock.java:logwaitWarning(165)) Waited above threshold(300 ms) to > acquire lock: lock identifier: FsDatasetRwlock waitTimeMs=336 ms.Suppressed 0 > lock wait warnings.Longest supressed waitTimeMs=0.The stack trace is > java.lang.Thread,getStackTrace(Thread.java:1559) > org.apache.hadoop.util.StringUtils.getStackTrace(StringUtils.java:1060) > org.apache.hadoop.util.Instrumentedlock.logWaitWarning(InstrumentedLock.java:171) > org.apache.hadoop.util.InstrumentedLock.check(InstrumentedLock.java:222) > org.apache.hadoop.util.InstrumentedLock.lock(InstrumentedLock, iaya:105) > org.apache.hadoop.util.AutocloseableLock.acquire(AutocloseableLock.java:67) > org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl.append(FsDatasetImpl.java:1239) > org.apache.hadoop.hdfs.server.datanode.BlockReceiver.(BlockReceiver.java:230) > org.apache.hadoop.hdfs.server.datanode.DataXceiver.getBlockReceiver > (DataXceiver.java:1313) > org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock > (DataXceiver.java:764) > org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.opWriteBlock(Receiver.java:176) > org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.processOp(Receiver.java:110) > org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:293) > java.lang.Thread.run(Thread.java:748) > {code} > > Then we traced the method > _org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl.append(FsDatasetImpl. > java:1239),_ and print how long each command take to finish the execution, > and find that it takes us 700ms to get the linkCount of the file which is > really slow. > > We traced the code and find that java1.8 use a Shell Command to get the > linkCount, in which execution it will start a new Process and wait for the > Process to fork, when the QPS is very high, it will sometimes take a long > time to fork the process. > Here is the shell command. > {code:java} > stat -c%h /path/to/file > {code} > > Solution: > For the FileStore that supports the file attributes "unix", we can use the > method _Files.getAttribute(f.toPath(), "unix:nlink")_ to get the linkCount, > this method doesn't need to start a new process, and will return the result > in a very short time. > > When we use this method to get the file linkCount, we rarely get the WARN log > above when the QPS of append execution is high. > . > -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-19052) Hadoop use Shell command to get the count of the hard link which takes a lot of time
[ https://issues.apache.org/jira/browse/HADOOP-19052?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17814191#comment-17814191 ] ASF GitHub Bot commented on HADOOP-19052: - liangyu-1 opened a new pull request, #6527: URL: https://github.com/apache/hadoop/pull/6527 …nk which takes a lot of time ### Description of PR As described in [HADOOP_19052](https://issues.apache.org/jira/browse/HADOOP-19052). When we try to append a file, we will execute method `getHardLinkCount` twice, inside method `getHardLinkCount`, java start a new process to execute a shell command and wait for it to fork. When the QPS of `append` execution is very high, method `getHardLinkCount` will take a long time to finish which will cause a long-time wait to acquire lock. I used another method to get the linkCount of a file whose file store supports the file attributes identified by the given file attribute view. This method does not start a new process and will finish in very short time even if the QPS of `append` execution is high. ### How was this patch tested? I add a new UT testGetLinkCountFromFileAttribute and a public method supportsHardLink to get whether or not this file store supports the file attributes identified by the given file attribute view. ### For code changes: - [ ] Does the title or this PR starts with the corresponding JIRA issue id (e.g. 'HADOOP-17799. Your PR title ...')? - [ ] Object storage: have the integration tests been executed and the endpoint declared according to the connector-specific documentation? - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? - [ ] If applicable, have you updated the `LICENSE`, `LICENSE-binary`, `NOTICE-binary` files? > Hadoop use Shell command to get the count of the hard link which takes a lot > of time > > > Key: HADOOP-19052 > URL: https://issues.apache.org/jira/browse/HADOOP-19052 > Project: Hadoop Common > Issue Type: Improvement > Environment: Hadopp 3.3.4 >Reporter: liang yu >Priority: Major > > Using Hadoop 3.3.4 > > When the QPS of `append` executions is very high, at a rate of above 1/s. > > We found that the write speed in hadoop is very slow. We traced some > datanodes' log and find that there is a warning : > {code:java} > 2024-01-26 11:09:44,292 WARN impl.FsDatasetImpl > (InstrumentedLock.java:logwaitWarning(165)) Waited above threshold(300 ms) to > acquire lock: lock identifier: FsDatasetRwlock waitTimeMs=336 ms.Suppressed 0 > lock wait warnings.Longest supressed waitTimeMs=0.The stack trace is > java.lang.Thread,getStackTrace(Thread.java:1559) > org.apache.hadoop.util.StringUtils.getStackTrace(StringUtils.java:1060) > org.apache.hadoop.util.Instrumentedlock.logWaitWarning(InstrumentedLock.java:171) > org.apache.hadoop.util.InstrumentedLock.check(InstrumentedLock.java:222) > org.apache.hadoop.util.InstrumentedLock.lock(InstrumentedLock, iaya:105) > org.apache.hadoop.util.AutocloseableLock.acquire(AutocloseableLock.java:67) > org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl.append(FsDatasetImpl.java:1239) > org.apache.hadoop.hdfs.server.datanode.BlockReceiver.(BlockReceiver.java:230) > org.apache.hadoop.hdfs.server.datanode.DataXceiver.getBlockReceiver > (DataXceiver.java:1313) > org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock > (DataXceiver.java:764) > org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.opWriteBlock(Receiver.java:176) > org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.processOp(Receiver.java:110) > org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:293) > java.lang.Thread.run(Thread.java:748) > {code} > > Then we traced the method > _org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl.append(FsDatasetImpl. > java:1239),_ and print how long each command take to finish the execution, > and find that it takes us 700ms to get the linkCount of the file which is > really slow. > > We traced the code and find that java1.8 use a Shell Command to get the > linkCount, in which execution it will start a new Process and wait for the > Process to fork, when the QPS is very high, it will sometimes take a long > time to fork the process. > Here is the shell command. > {code:java} > stat -c%h /path/to/file > {code} > > Solution: > For the FileStore that supports the file attributes "unix", we can use the > method _Files.getAttribute(f.toPath(), "unix:nlink")_ to get the linkCount, > this method doesn't need to start a new process, and will return the result > in a very short time. > > When
[jira] [Commented] (HADOOP-19047) Support InMemory Tracking Of S3A Magic Commits
[ https://issues.apache.org/jira/browse/HADOOP-19047?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17814189#comment-17814189 ] ASF GitHub Bot commented on HADOOP-19047: - shameersss1 commented on code in PR #6468: URL: https://github.com/apache/hadoop/pull/6468#discussion_r1477749474 ## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java: ## @@ -264,9 +326,14 @@ public void abortTask(TaskAttemptContext context) throws IOException { try (DurationInfo d = new DurationInfo(LOG, "Abort task %s", context.getTaskAttemptID()); CommitContext commitContext = initiateTaskOperation(context)) { - getCommitOperations().abortAllSinglePendingCommits(attemptPath, - commitContext, - true); + if (isTrackMagicCommitsInMemoryEnabled(context.getConfiguration())) { +List pendingCommits = loadPendingCommitsFromMemory(context); +for (SinglePendingCommit singleCommit : pendingCommits) { + commitContext.abortSingleCommit(singleCommit); +} + } else { +getCommitOperations().abortAllSinglePendingCommits(attemptPath, commitContext, true); Review Comment: AFIK, Spark calls abortTask from the same process (executor), When the job fails, The abortJob operation is called which basically lists all the pending uploads and aborts it as mentioned in the comment [here](https://github.com/apache/hadoop/pull/6468#issuecomment-1926348440) I am not sure why would a different process call abortTask, The driver process should ideally call abortJob if a job fails. > Support InMemory Tracking Of S3A Magic Commits > -- > > Key: HADOOP-19047 > URL: https://issues.apache.org/jira/browse/HADOOP-19047 > Project: Hadoop Common > Issue Type: Improvement > Components: fs/s3 >Reporter: Syed Shameerur Rahman >Assignee: Syed Shameerur Rahman >Priority: Major > Labels: pull-request-available > > The following are the operations which happens within a Task when it uses S3A > Magic Committer. > *During closing of stream* > 1. A 0-byte file with a same name of the original file is uploaded to S3 > using PUT operation. Refer > [here|https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java#L152] > for more information. This is done so that the downstream application like > Spark could get the size of the file which is being written. > 2. MultiPartUpload(MPU) metadata is uploaded to S3. Refer > [here|https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java#L176] > for more information. > *During TaskCommit* > 1. All the MPU metadata which the task wrote to S3 (There will be 'x' number > of metadata file in S3 if a single task writes to 'x' files) are read and > rewritten to S3 as a single metadata file. Refer > [here|https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java#L201] > for more information > Since these operations happens with the Task JVM, We could optimize as well > as save cost by storing these information in memory when Task memory usage is > not a constraint. Hence the proposal here is to introduce a new MagicCommit > Tracker called "InMemoryMagicCommitTracker" which will store the > 1. Metadata of MPU in memory till the Task is committed > 2. Store the size of the file which can be used by the downstream application > to get the file size before it is committed/visible to the output path. > This optimization will save 2 PUT S3 calls, 1 LIST S3 call, and 1 GET S3 call > given a Task writes only 1 file. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
Re: [PR] HDFS-17121. BPServiceActor to provide new thread to handle FBR [hadoop]
ZanderXu commented on PR #5888: URL: https://github.com/apache/hadoop/pull/5888#issuecomment-1926356252 @LiuGuH are you still working on this PR? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
Re: [PR] HADOOP-19047: Support InMemory Tracking Of S3A Magic Commits [hadoop]
shameersss1 commented on code in PR #6468: URL: https://github.com/apache/hadoop/pull/6468#discussion_r1477749474 ## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java: ## @@ -264,9 +326,14 @@ public void abortTask(TaskAttemptContext context) throws IOException { try (DurationInfo d = new DurationInfo(LOG, "Abort task %s", context.getTaskAttemptID()); CommitContext commitContext = initiateTaskOperation(context)) { - getCommitOperations().abortAllSinglePendingCommits(attemptPath, - commitContext, - true); + if (isTrackMagicCommitsInMemoryEnabled(context.getConfiguration())) { +List pendingCommits = loadPendingCommitsFromMemory(context); +for (SinglePendingCommit singleCommit : pendingCommits) { + commitContext.abortSingleCommit(singleCommit); +} + } else { +getCommitOperations().abortAllSinglePendingCommits(attemptPath, commitContext, true); Review Comment: AFIK, Spark calls abortTask from the same process (executor), When the job fails, The abortJob operation is called which basically lists all the pending uploads and aborts it as mentioned in the comment [here](https://github.com/apache/hadoop/pull/6468#issuecomment-1926348440) I am not sure why would a different process call abortTask, The driver process should ideally call abortJob if a job fails. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-19047) Support InMemory Tracking Of S3A Magic Commits
[ https://issues.apache.org/jira/browse/HADOOP-19047?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17814186#comment-17814186 ] ASF GitHub Bot commented on HADOOP-19047: - shameersss1 commented on code in PR #6468: URL: https://github.com/apache/hadoop/pull/6468#discussion_r1477747332 ## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java: ## @@ -3906,6 +3908,21 @@ public void access(final Path f, final FsAction mode) @Retries.RetryTranslated public FileStatus getFileStatus(final Path f) throws IOException { Path path = qualify(f); +if (isTrackMagicCommitsInMemoryEnabled(getConf()) && isMagicCommitPath(path)) { Review Comment: I agree this an ugly hack. I couldn't find any better alternative. This will be used by downstream application like Spark which wants to get the file size of the file written by the task. This is supposed to be used in the same process which writes the file/initiated and upload the MPU. > Support InMemory Tracking Of S3A Magic Commits > -- > > Key: HADOOP-19047 > URL: https://issues.apache.org/jira/browse/HADOOP-19047 > Project: Hadoop Common > Issue Type: Improvement > Components: fs/s3 >Reporter: Syed Shameerur Rahman >Assignee: Syed Shameerur Rahman >Priority: Major > Labels: pull-request-available > > The following are the operations which happens within a Task when it uses S3A > Magic Committer. > *During closing of stream* > 1. A 0-byte file with a same name of the original file is uploaded to S3 > using PUT operation. Refer > [here|https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java#L152] > for more information. This is done so that the downstream application like > Spark could get the size of the file which is being written. > 2. MultiPartUpload(MPU) metadata is uploaded to S3. Refer > [here|https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java#L176] > for more information. > *During TaskCommit* > 1. All the MPU metadata which the task wrote to S3 (There will be 'x' number > of metadata file in S3 if a single task writes to 'x' files) are read and > rewritten to S3 as a single metadata file. Refer > [here|https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java#L201] > for more information > Since these operations happens with the Task JVM, We could optimize as well > as save cost by storing these information in memory when Task memory usage is > not a constraint. Hence the proposal here is to introduce a new MagicCommit > Tracker called "InMemoryMagicCommitTracker" which will store the > 1. Metadata of MPU in memory till the Task is committed > 2. Store the size of the file which can be used by the downstream application > to get the file size before it is committed/visible to the output path. > This optimization will save 2 PUT S3 calls, 1 LIST S3 call, and 1 GET S3 call > given a Task writes only 1 file. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
Re: [PR] HADOOP-19047: Support InMemory Tracking Of S3A Magic Commits [hadoop]
shameersss1 commented on code in PR #6468: URL: https://github.com/apache/hadoop/pull/6468#discussion_r1477747332 ## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java: ## @@ -3906,6 +3908,21 @@ public void access(final Path f, final FsAction mode) @Retries.RetryTranslated public FileStatus getFileStatus(final Path f) throws IOException { Path path = qualify(f); +if (isTrackMagicCommitsInMemoryEnabled(getConf()) && isMagicCommitPath(path)) { Review Comment: I agree this an ugly hack. I couldn't find any better alternative. This will be used by downstream application like Spark which wants to get the file size of the file written by the task. This is supposed to be used in the same process which writes the file/initiated and upload the MPU. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-19047) Support InMemory Tracking Of S3A Magic Commits
[ https://issues.apache.org/jira/browse/HADOOP-19047?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17814185#comment-17814185 ] ASF GitHub Bot commented on HADOOP-19047: - shameersss1 commented on PR #6468: URL: https://github.com/apache/hadoop/pull/6468#issuecomment-1926348440 >1. static map of path to metadata. This will grow without constraint on a long live process. The entries to the Map are removed during commitTask or abortTask operation to keep memory under control. --- > 2. Two jobs writing to same path will it corrupt the Map ? No, The path (complete) is guaranteed to be unique The paths stored here as part of `private static Map> taskAttemptIdToPath = new ConcurrentHashMap<>();` is the magic path, Eventhough the file name might be same, The magic path for two different jobs will be different since the jobId is included in the path. - >3. the static map would be a weak ref to something held strongly by the actual committer (see WeakReferenceMap). Once the actual task attempt is gc'd, Since the entries from the HashMap are removed during commitTask or abortTask operation is WeakHashMap still required? >4. static structures should be per fs instances, so when an fs is cleaned up I am not sure why it should be scoped under fs object. For a simiar behaviour with storing in s3, Shouldn't the static structure be available to the whole JVM ? I mean shouldn't we able to access static structure irrespective of the fs object. >5. 'm also worried about how a job could abort a task attempt on a different process which has failed. Before worrying about that too much, why don't you look in spark to see how it calls abort. I'm not worried about MapReduce except for testing -so how do itself calls the committee isn't so important. For example: we don't care about recovery from a failed attempt as spark itself cannot do this. I have covered this as part of the comment [here](https://github.com/apache/hadoop/pull/6468#issuecomment-1926304528). --- > Support InMemory Tracking Of S3A Magic Commits > -- > > Key: HADOOP-19047 > URL: https://issues.apache.org/jira/browse/HADOOP-19047 > Project: Hadoop Common > Issue Type: Improvement > Components: fs/s3 >Reporter: Syed Shameerur Rahman >Assignee: Syed Shameerur Rahman >Priority: Major > Labels: pull-request-available > > The following are the operations which happens within a Task when it uses S3A > Magic Committer. > *During closing of stream* > 1. A 0-byte file with a same name of the original file is uploaded to S3 > using PUT operation. Refer > [here|https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java#L152] > for more information. This is done so that the downstream application like > Spark could get the size of the file which is being written. > 2. MultiPartUpload(MPU) metadata is uploaded to S3. Refer > [here|https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java#L176] > for more information. > *During TaskCommit* > 1. All the MPU metadata which the task wrote to S3 (There will be 'x' number > of metadata file in S3 if a single task writes to 'x' files) are read and > rewritten to S3 as a single metadata file. Refer > [here|https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java#L201] > for more information > Since these operations happens with the Task JVM, We could optimize as well > as save cost by storing these information in memory when Task memory usage is > not a constraint. Hence the proposal here is to introduce a new MagicCommit > Tracker called "InMemoryMagicCommitTracker" which will store the > 1. Metadata of MPU in memory till the Task is committed > 2. Store the size of the file which can be used by the downstream application > to get the file size before it is committed/visible to the output path. > This optimization will save 2 PUT S3 calls, 1 LIST S3 call, and 1 GET S3 call > given a Task writes only 1 file. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
Re: [PR] HADOOP-19047: Support InMemory Tracking Of S3A Magic Commits [hadoop]
shameersss1 commented on PR #6468: URL: https://github.com/apache/hadoop/pull/6468#issuecomment-1926348440 >1. static map of path to metadata. This will grow without constraint on a long live process. The entries to the Map are removed during commitTask or abortTask operation to keep memory under control. --- > 2. Two jobs writing to same path will it corrupt the Map ? No, The path (complete) is guaranteed to be unique The paths stored here as part of `private static Map> taskAttemptIdToPath = new ConcurrentHashMap<>();` is the magic path, Eventhough the file name might be same, The magic path for two different jobs will be different since the jobId is included in the path. - >3. the static map would be a weak ref to something held strongly by the actual committer (see WeakReferenceMap). Once the actual task attempt is gc'd, Since the entries from the HashMap are removed during commitTask or abortTask operation is WeakHashMap still required? >4. static structures should be per fs instances, so when an fs is cleaned up I am not sure why it should be scoped under fs object. For a simiar behaviour with storing in s3, Shouldn't the static structure be available to the whole JVM ? I mean shouldn't we able to access static structure irrespective of the fs object. >5. 'm also worried about how a job could abort a task attempt on a different process which has failed. Before worrying about that too much, why don't you look in spark to see how it calls abort. I'm not worried about MapReduce except for testing -so how do itself calls the committee isn't so important. For example: we don't care about recovery from a failed attempt as spark itself cannot do this. I have covered this as part of the comment [here](https://github.com/apache/hadoop/pull/6468#issuecomment-1926304528). --- -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-19047) Support InMemory Tracking Of S3A Magic Commits
[ https://issues.apache.org/jira/browse/HADOOP-19047?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17814175#comment-17814175 ] ASF GitHub Bot commented on HADOOP-19047: - shameersss1 commented on PR #6468: URL: https://github.com/apache/hadoop/pull/6468#issuecomment-1926304528 @steveloughran - Thanks a lot a for a detailed review and some amazing question, The following are my thoughts on the different asks. > 1. Marker files at the end of each path so that spark status reporting on different processes can get an update on an active job. As far i know (Please correct me if i am wrong) 1. 0-size marker files was specially added for Spark's use case. 2. After writing files, Spark tries to get the size of the files written for the statistic purpose (like showing the output bytes written) in the Spark History server UI. 3. This operation is being done as part of the [BasicWriteStatsTracker](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala#L86) class in Spark. 4. As i could see in my experiment, BasicWriteStatsTracker#getFileSize is called in the executor process itself. That being said, Since the same process is calling BasicWriteStatsTracker#getFileSize is it still required to have 0 marker file? I have solved this by adding a check in FileStatus method by returing the file size corresponding to the magic path/file. -- > 2. A way to abort all uploads of a failed task attempt -even from a different process. Probably also a way to abort the entire job. Thinking from Spark's perspective, 1. When a taskAttempt fails (gracefully), [abortTask](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala#L176) operation is called. This is operation is called within the same process and hence we can fetch the MPU metadata from the memory itself. 2. If a taskAttempt fails (ungracefully and all retries) are exhausted, When [abortJob](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala#L69) operation is called which will internally invoke [cleanup](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java#L965) which lists all the pending multi part upload and aborts them. That being said, I am not sure if there is any such use case of abortingTask from another process. In such cases, The abortJob will handle it i guess. >3. Confidence that the inner memory store of pending uploads Will not grow it definitely. 1. The static map entry is removed is during taskCommit or abortTask operations and hence it guaranteed that there is no memory leak (unless there is some unexplored corner case). 2. The only case when it grows large, is when there are large number of concurrent jobs reusing the same executor JVM, Since we don't enable the "inmemory" by default we should be good. That being said, maybe we should call this out in the documentation. > 4. Two jobs writing to same path will it corrupt the Map ? The paths stored here as part of `private static Map> taskAttemptIdToPath = new ConcurrentHashMap<>();` is the magic path, Eventhough the file name might be same, The magic path for two different jobs will be different since the jobId is included in the path. Doe it make sense? Or am i missing anything? > Support InMemory Tracking Of S3A Magic Commits > -- > > Key: HADOOP-19047 > URL: https://issues.apache.org/jira/browse/HADOOP-19047 > Project: Hadoop Common > Issue Type: Improvement > Components: fs/s3 >Reporter: Syed Shameerur Rahman >Assignee: Syed Shameerur Rahman >Priority: Major > Labels: pull-request-available > > The following are the operations which happens within a Task when it uses S3A > Magic Committer. > *During closing of stream* > 1. A 0-byte file with a same name of the original file is uploaded to S3 > using PUT operation. Refer > [here|https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java#L152] > for more information. This is done so that the downstream application like > Spark could get the size of the file which is being written. > 2. MultiPartUpload(MPU) metadata is uploaded to S3. Refer > [here|https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java#L176] > for more information. > *During TaskCommit* > 1. All the
Re: [PR] HADOOP-19047: Support InMemory Tracking Of S3A Magic Commits [hadoop]
shameersss1 commented on PR #6468: URL: https://github.com/apache/hadoop/pull/6468#issuecomment-1926304528 @steveloughran - Thanks a lot a for a detailed review and some amazing question, The following are my thoughts on the different asks. > 1. Marker files at the end of each path so that spark status reporting on different processes can get an update on an active job. As far i know (Please correct me if i am wrong) 1. 0-size marker files was specially added for Spark's use case. 2. After writing files, Spark tries to get the size of the files written for the statistic purpose (like showing the output bytes written) in the Spark History server UI. 3. This operation is being done as part of the [BasicWriteStatsTracker](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala#L86) class in Spark. 4. As i could see in my experiment, BasicWriteStatsTracker#getFileSize is called in the executor process itself. That being said, Since the same process is calling BasicWriteStatsTracker#getFileSize is it still required to have 0 marker file? I have solved this by adding a check in FileStatus method by returing the file size corresponding to the magic path/file. -- > 2. A way to abort all uploads of a failed task attempt -even from a different process. Probably also a way to abort the entire job. Thinking from Spark's perspective, 1. When a taskAttempt fails (gracefully), [abortTask](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala#L176) operation is called. This is operation is called within the same process and hence we can fetch the MPU metadata from the memory itself. 2. If a taskAttempt fails (ungracefully and all retries) are exhausted, When [abortJob](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala#L69) operation is called which will internally invoke [cleanup](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java#L965) which lists all the pending multi part upload and aborts them. That being said, I am not sure if there is any such use case of abortingTask from another process. In such cases, The abortJob will handle it i guess. >3. Confidence that the inner memory store of pending uploads Will not grow it definitely. 1. The static map entry is removed is during taskCommit or abortTask operations and hence it guaranteed that there is no memory leak (unless there is some unexplored corner case). 2. The only case when it grows large, is when there are large number of concurrent jobs reusing the same executor JVM, Since we don't enable the "inmemory" by default we should be good. That being said, maybe we should call this out in the documentation. > 4. Two jobs writing to same path will it corrupt the Map ? The paths stored here as part of `private static Map> taskAttemptIdToPath = new ConcurrentHashMap<>();` is the magic path, Eventhough the file name might be same, The magic path for two different jobs will be different since the jobId is included in the path. Doe it make sense? Or am i missing anything? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-19066) AWS SDK V2 - Enabling FIPS should be allowed with central endpoint
[ https://issues.apache.org/jira/browse/HADOOP-19066?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17814171#comment-17814171 ] Viraj Jasani commented on HADOOP-19066: --- Will run the whole suite with FIPS support + central endpoint. > AWS SDK V2 - Enabling FIPS should be allowed with central endpoint > -- > > Key: HADOOP-19066 > URL: https://issues.apache.org/jira/browse/HADOOP-19066 > Project: Hadoop Common > Issue Type: Sub-task > Components: fs/s3 >Affects Versions: 3.5.0, 3.4.1 >Reporter: Viraj Jasani >Assignee: Viraj Jasani >Priority: Major > > FIPS support can be enabled by setting "fs.s3a.endpoint.fips". Since the SDK > considers overriding endpoint and enabling fips as mutually exclusive, we > fail fast if fs.s3a.endpoint is set with fips support (details on > HADOOP-18975). > Now, we no longer override SDK endpoint for central endpoint since we enable > cross region access (details on HADOOP-19044) but we would still fail fast if > endpoint is central and fips is enabled. > Changes proposed: > * S3A to fail fast only if FIPS is enabled and non-central endpoint is > configured. > * Tests to ensure S3 bucket is accessible with default region us-east-2 with > cross region access (expected with central endpoint). > * Document FIPS support with central endpoint on connecting.html. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Assigned] (HADOOP-19066) AWS SDK V2 - Enabling FIPS should be allowed with central endpoint
[ https://issues.apache.org/jira/browse/HADOOP-19066?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Viraj Jasani reassigned HADOOP-19066: - Assignee: Viraj Jasani > AWS SDK V2 - Enabling FIPS should be allowed with central endpoint > -- > > Key: HADOOP-19066 > URL: https://issues.apache.org/jira/browse/HADOOP-19066 > Project: Hadoop Common > Issue Type: Sub-task > Components: fs/s3 >Affects Versions: 3.5.0, 3.4.1 >Reporter: Viraj Jasani >Assignee: Viraj Jasani >Priority: Major > > FIPS support can be enabled by setting "fs.s3a.endpoint.fips". Since the SDK > considers overriding endpoint and enabling fips as mutually exclusive, we > fail fast if fs.s3a.endpoint is set with fips support (details on > HADOOP-18975). > Now, we no longer override SDK endpoint for central endpoint since we enable > cross region access (details on HADOOP-19044) but we would still fail fast if > endpoint is central and fips is enabled. > Changes proposed: > * S3A to fail fast only if FIPS is enabled and non-central endpoint is > configured. > * Tests to ensure S3 bucket is accessible with default region us-east-2 with > cross region access (expected with central endpoint). > * Document FIPS support with central endpoint on connecting.html. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Created] (HADOOP-19066) AWS SDK V2 - Enabling FIPS should be allowed with central endpoint
Viraj Jasani created HADOOP-19066: - Summary: AWS SDK V2 - Enabling FIPS should be allowed with central endpoint Key: HADOOP-19066 URL: https://issues.apache.org/jira/browse/HADOOP-19066 Project: Hadoop Common Issue Type: Sub-task Components: fs/s3 Affects Versions: 3.5.0, 3.4.1 Reporter: Viraj Jasani FIPS support can be enabled by setting "fs.s3a.endpoint.fips". Since the SDK considers overriding endpoint and enabling fips as mutually exclusive, we fail fast if fs.s3a.endpoint is set with fips support (details on HADOOP-18975). Now, we no longer override SDK endpoint for central endpoint since we enable cross region access (details on HADOOP-19044) but we would still fail fast if endpoint is central and fips is enabled. Changes proposed: * S3A to fail fast only if FIPS is enabled and non-central endpoint is configured. * Tests to ensure S3 bucket is accessible with default region us-east-2 with cross region access (expected with central endpoint). * Document FIPS support with central endpoint on connecting.html. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-19050) Add S3 Access Grants Support in S3A
[ https://issues.apache.org/jira/browse/HADOOP-19050?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17814162#comment-17814162 ] ASF GitHub Bot commented on HADOOP-19050: - adnanhemani commented on PR #6507: URL: https://github.com/apache/hadoop/pull/6507#issuecomment-1926257843 Hi @steveloughran and @ahmarsuhail - I think this code is in a much more ready state than before and we've attempted to answer the questions you had earlier. Please let us know what other thoughts you have on this. (Also, I think the Unit Tests are a bit flaky here - not sure what to do about those) > Add S3 Access Grants Support in S3A > --- > > Key: HADOOP-19050 > URL: https://issues.apache.org/jira/browse/HADOOP-19050 > Project: Hadoop Common > Issue Type: New Feature > Components: fs/s3 >Affects Versions: 3.4.0 >Reporter: Jason Han >Assignee: Jason Han >Priority: Minor > Labels: pull-request-available > > Add support for S3 Access Grants > (https://aws.amazon.com/s3/features/access-grants/) in S3A. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
Re: [PR] HADOOP-19050, Add Support for AWS S3 Access Grants [hadoop]
adnanhemani commented on PR #6507: URL: https://github.com/apache/hadoop/pull/6507#issuecomment-1926257843 Hi @steveloughran and @ahmarsuhail - I think this code is in a much more ready state than before and we've attempted to answer the questions you had earlier. Please let us know what other thoughts you have on this. (Also, I think the Unit Tests are a bit flaky here - not sure what to do about those) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-19050) Add S3 Access Grants Support in S3A
[ https://issues.apache.org/jira/browse/HADOOP-19050?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17814156#comment-17814156 ] ASF GitHub Bot commented on HADOOP-19050: - adnanhemani commented on code in PR #6507: URL: https://github.com/apache/hadoop/pull/6507#discussion_r1477676885 ## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java: ## @@ -401,4 +409,32 @@ private static Region getS3RegionFromEndpoint(final String endpoint, return Region.of(AWS_S3_DEFAULT_REGION); } + public static , ClientT> void + applyS3AccessGrantsConfigurations(BuilderT builder, Configuration conf) { +boolean s3agEnabled = conf.getBoolean(AWS_S3_ACCESS_GRANTS_ENABLED, false); +if (!s3agEnabled){ + LOG_EXACTLY_ONCE.debug("s3ag plugin is not enabled."); Review Comment: On all logs, can we use the full name: "S3 Access Grants..." to make it clear for users looking through the logs? > Add S3 Access Grants Support in S3A > --- > > Key: HADOOP-19050 > URL: https://issues.apache.org/jira/browse/HADOOP-19050 > Project: Hadoop Common > Issue Type: New Feature > Components: fs/s3 >Affects Versions: 3.4.0 >Reporter: Jason Han >Assignee: Jason Han >Priority: Minor > Labels: pull-request-available > > Add support for S3 Access Grants > (https://aws.amazon.com/s3/features/access-grants/) in S3A. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
Re: [PR] HADOOP-19050, Add Support for AWS S3 Access Grants [hadoop]
adnanhemani commented on code in PR #6507: URL: https://github.com/apache/hadoop/pull/6507#discussion_r1477676885 ## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java: ## @@ -401,4 +409,32 @@ private static Region getS3RegionFromEndpoint(final String endpoint, return Region.of(AWS_S3_DEFAULT_REGION); } + public static , ClientT> void + applyS3AccessGrantsConfigurations(BuilderT builder, Configuration conf) { +boolean s3agEnabled = conf.getBoolean(AWS_S3_ACCESS_GRANTS_ENABLED, false); +if (!s3agEnabled){ + LOG_EXACTLY_ONCE.debug("s3ag plugin is not enabled."); Review Comment: On all logs, can we use the full name: "S3 Access Grants..." to make it clear for users looking through the logs? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
Re: [PR] HDFS-17369. Add uuid into datanode info for NameNodeMXBean [hadoop]
haiyang1987 commented on PR #6521: URL: https://github.com/apache/hadoop/pull/6521#issuecomment-1926162162 Thanks @slfan1989 @ZanderXu for your review and merge it. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-18342) Upgrade to Avro 1.11.1
[ https://issues.apache.org/jira/browse/HADOOP-18342?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17814079#comment-17814079 ] ASF GitHub Bot commented on HADOOP-18342: - hadoop-yetus commented on PR #4854: URL: https://github.com/apache/hadoop/pull/4854#issuecomment-1925734173 :broken_heart: **-1 overall** | Vote | Subsystem | Runtime | Logfile | Comment | |::|--:|:|::|:---:| | +0 :ok: | reexec | 17m 21s | | Docker mode activated. | _ Prechecks _ | | +1 :green_heart: | dupname | 0m 1s | | No case conflicting files found. | | +0 :ok: | codespell | 0m 0s | | codespell was not available. | | +0 :ok: | detsecrets | 0m 0s | | detect-secrets was not available. | | +0 :ok: | xmllint | 0m 0s | | xmllint was not available. | | +0 :ok: | shelldocs | 0m 0s | | Shelldocs was not available. | | +1 :green_heart: | @author | 0m 0s | | The patch does not contain any @author tags. | | +1 :green_heart: | test4tests | 0m 0s | | The patch appears to include 3 new or modified test files. | _ trunk Compile Tests _ | | +0 :ok: | mvndep | 14m 29s | | Maven dependency ordering for branch | | +1 :green_heart: | mvninstall | 30m 47s | | trunk passed | | +1 :green_heart: | compile | 16m 29s | | trunk passed with JDK Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 | | +1 :green_heart: | compile | 14m 53s | | trunk passed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08 | | +1 :green_heart: | checkstyle | 4m 15s | | trunk passed | | +1 :green_heart: | mvnsite | 18m 22s | | trunk passed | | +1 :green_heart: | javadoc | 8m 30s | | trunk passed with JDK Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 | | +1 :green_heart: | javadoc | 7m 37s | | trunk passed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08 | | +0 :ok: | spotbugs | 0m 19s | | branch/hadoop-project no spotbugs output file (spotbugsXml.xml) | | +0 :ok: | spotbugs | 0m 20s | | branch/hadoop-client-modules/hadoop-client no spotbugs output file (spotbugsXml.xml) | | +0 :ok: | spotbugs | 0m 20s | | branch/hadoop-client-modules/hadoop-client-minicluster no spotbugs output file (spotbugsXml.xml) | | +1 :green_heart: | shadedclient | 60m 58s | | branch has no errors when building and testing our client artifacts. | _ Patch Compile Tests _ | | +0 :ok: | mvndep | 0m 55s | | Maven dependency ordering for patch | | -1 :x: | mvninstall | 28m 54s | [/patch-mvninstall-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4854/2/artifact/out/patch-mvninstall-root.txt) | root in the patch failed. | | -1 :x: | mvninstall | 1m 6s | [/patch-mvninstall-hadoop-mapreduce-project.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4854/2/artifact/out/patch-mvninstall-hadoop-mapreduce-project.txt) | hadoop-mapreduce-project in the patch failed. | | -1 :x: | compile | 14m 38s | [/patch-compile-root-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4854/2/artifact/out/patch-compile-root-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt) | root in the patch failed with JDK Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04. | | -1 :x: | javac | 14m 38s | [/patch-compile-root-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4854/2/artifact/out/patch-compile-root-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt) | root in the patch failed with JDK Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04. | | -1 :x: | compile | 13m 49s | [/patch-compile-root-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4854/2/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt) | root in the patch failed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08. | | -1 :x: | javac | 13m 49s | [/patch-compile-root-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4854/2/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt) | root in the patch failed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08. | | +1 :green_heart: | blanks | 0m 0s | | The patch has no blanks issues. | | -0 :warning: | checkstyle | 3m 58s | [/buildtool-patch-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4854/2/artifact/out/buildtool-patch-checkstyle-root.txt) | The patch fails to run checkstyle in root | | -1 :x: | mvnsite | 4m 17s |
Re: [PR] HADOOP-18342: shaded avro jar [hadoop]
hadoop-yetus commented on PR #4854: URL: https://github.com/apache/hadoop/pull/4854#issuecomment-1925734173 :broken_heart: **-1 overall** | Vote | Subsystem | Runtime | Logfile | Comment | |::|--:|:|::|:---:| | +0 :ok: | reexec | 17m 21s | | Docker mode activated. | _ Prechecks _ | | +1 :green_heart: | dupname | 0m 1s | | No case conflicting files found. | | +0 :ok: | codespell | 0m 0s | | codespell was not available. | | +0 :ok: | detsecrets | 0m 0s | | detect-secrets was not available. | | +0 :ok: | xmllint | 0m 0s | | xmllint was not available. | | +0 :ok: | shelldocs | 0m 0s | | Shelldocs was not available. | | +1 :green_heart: | @author | 0m 0s | | The patch does not contain any @author tags. | | +1 :green_heart: | test4tests | 0m 0s | | The patch appears to include 3 new or modified test files. | _ trunk Compile Tests _ | | +0 :ok: | mvndep | 14m 29s | | Maven dependency ordering for branch | | +1 :green_heart: | mvninstall | 30m 47s | | trunk passed | | +1 :green_heart: | compile | 16m 29s | | trunk passed with JDK Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 | | +1 :green_heart: | compile | 14m 53s | | trunk passed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08 | | +1 :green_heart: | checkstyle | 4m 15s | | trunk passed | | +1 :green_heart: | mvnsite | 18m 22s | | trunk passed | | +1 :green_heart: | javadoc | 8m 30s | | trunk passed with JDK Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 | | +1 :green_heart: | javadoc | 7m 37s | | trunk passed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08 | | +0 :ok: | spotbugs | 0m 19s | | branch/hadoop-project no spotbugs output file (spotbugsXml.xml) | | +0 :ok: | spotbugs | 0m 20s | | branch/hadoop-client-modules/hadoop-client no spotbugs output file (spotbugsXml.xml) | | +0 :ok: | spotbugs | 0m 20s | | branch/hadoop-client-modules/hadoop-client-minicluster no spotbugs output file (spotbugsXml.xml) | | +1 :green_heart: | shadedclient | 60m 58s | | branch has no errors when building and testing our client artifacts. | _ Patch Compile Tests _ | | +0 :ok: | mvndep | 0m 55s | | Maven dependency ordering for patch | | -1 :x: | mvninstall | 28m 54s | [/patch-mvninstall-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4854/2/artifact/out/patch-mvninstall-root.txt) | root in the patch failed. | | -1 :x: | mvninstall | 1m 6s | [/patch-mvninstall-hadoop-mapreduce-project.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4854/2/artifact/out/patch-mvninstall-hadoop-mapreduce-project.txt) | hadoop-mapreduce-project in the patch failed. | | -1 :x: | compile | 14m 38s | [/patch-compile-root-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4854/2/artifact/out/patch-compile-root-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt) | root in the patch failed with JDK Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04. | | -1 :x: | javac | 14m 38s | [/patch-compile-root-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4854/2/artifact/out/patch-compile-root-jdkUbuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04.txt) | root in the patch failed with JDK Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04. | | -1 :x: | compile | 13m 49s | [/patch-compile-root-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4854/2/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt) | root in the patch failed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08. | | -1 :x: | javac | 13m 49s | [/patch-compile-root-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4854/2/artifact/out/patch-compile-root-jdkPrivateBuild-1.8.0_392-8u392-ga-1~20.04-b08.txt) | root in the patch failed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08. | | +1 :green_heart: | blanks | 0m 0s | | The patch has no blanks issues. | | -0 :warning: | checkstyle | 3m 58s | [/buildtool-patch-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4854/2/artifact/out/buildtool-patch-checkstyle-root.txt) | The patch fails to run checkstyle in root | | -1 :x: | mvnsite | 4m 17s | [/patch-mvnsite-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4854/2/artifact/out/patch-mvnsite-root.txt) | root in the patch failed. | | +1 :green_heart: | shellcheck | 0m 0s | | No new issues. | | -1 :x: | javadoc | 7m 2s |