Skip to content

Conversation

@anujmodi2021
Copy link
Contributor

Description of PR

Since the onset of ABFS Driver, there has been a single implementation of AbfsInputStream. Different kinds of workloads require different heuristics to give the best performance for that type of workload. For example: 

Sequential Read Workloads like DFSIO and DistCP gain performance improvement from prefetched 
Random Read Workloads on other hand do not need Prefetches and enabling prefetches for them is an overhead and TPS heavy 
Query Workloads involving Parquet/ORC files benefit from improvements like Footer Read and Small Files Reads

To accomodate this we need to determine the pattern and accordingly create Input Streams implemented for that particular pattern.

image

Moving ahead more relevant policies and specialized implementation of AbfsInputStream can be added.

This PR only refactors the way we create input streams. No logical change introduced. As today by default we will continue to use AbfsAdaptiveInputStream which can cater to all kind of workloads.

How was this patch tested?

New tests were added.

@hadoop-yetus

This comment was marked as outdated.

@ahmarsuhail
Copy link
Contributor

@anujmodi2021 I am trying to propose a single optimised implementation of an input stream across cloud implementations, as I think we all need this kind of logic. Ideally I want to get to a place where 80% of the logic is shared in a common layer, and then we only implement cloud specific clients to actually make the requests separately.

There is some consensus to move the shared logic into the parquet-java repo: https://lists.apache.org/thread/nbksq32cs8h1ldj8762y6wh9zzp8gqx6 , and some buy-in from the team at google. I'll be following up on this in the new year.

Would be great to get your thoughts and if your team would also like to collaborate on this.

@anujmodi2021 anujmodi2021 marked this pull request as ready for review December 31, 2025 10:58
@hadoop-yetus

This comment was marked as outdated.

*/
public class AbfsAdaptiveInputStream extends AbfsInputStream {

public AbfsAdaptiveInputStream(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: needs javadoc comment

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added

* Enum for ABFS Input Policies.
* Each policy maps to a particular implementation of {@link AbfsInputStream}
*/
public enum AbfsInputPolicy {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should be ReadPolicy or input stream policy instead of input policy?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Changed to ReadPolicy.
Anything is fine IMO

RANDOM(FS_OPTION_OPENFILE_READ_POLICY_RANDOM),
ADAPTIVE(FS_OPTION_OPENFILE_READ_POLICY_ADAPTIVE);

private final String policy;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

same as above

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Taken

abfsInputStreamContext, eTag, tracingContext);
}

@Override
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

inherit doc

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Taken

import static org.apache.hadoop.fs.azurebfs.constants.ReadType.NORMAL_READ;
import static org.apache.hadoop.fs.azurebfs.constants.ReadType.PREFETCH_READ;
import static org.apache.hadoop.fs.azurebfs.constants.ReadType.SMALLFILE_READ;
import static org.apache.hadoop.fs.azurebfs.constants.ReadType.*;
Copy link
Contributor

@anmolanmol1234 anmolanmol1234 Jan 6, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

same as above

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Takne

import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderVersion;
import org.apache.hadoop.fs.impl.OpenFileParameters;

import static org.apache.hadoop.fs.Options.OpenFileOptions.*;
Copy link
Contributor

@anmolanmol1234 anmolanmol1234 Jan 6, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

"*" import should be reverted

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Taken


/*
* Test to verify Random Read Type.
* Settin Read Policy to Parquet ensures Random Read Type.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: setting

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Taken

ArgumentCaptor<TracingContext> captor9 = ArgumentCaptor.forClass(TracingContext.class);

List<String> paths = captor1.getAllValues();
System.out.println(paths);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove the sysouts

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for catching. Taken

@hadoop-yetus

This comment was marked as outdated.

@anujmodi2021
Copy link
Contributor Author


:::: AGGREGATED TEST RESULT ::::

============================================================
HNS-OAuth-DFS

[ERROR] org.apache.hadoop.fs.azurebfs.ITestAzureBlobFileSystemListStatus.testListPath -- Time elapsed: 126.0 s <<< FAILURE!

[WARNING] Tests run: 225, Failures: 0, Errors: 0, Skipped: 3
[WARNING] Tests run: 892, Failures: 0, Errors: 0, Skipped: 220
[ERROR] Tests run: 158, Failures: 1, Errors: 0, Skipped: 8
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 23

============================================================
HNS-SharedKey-DFS

[ERROR] org.apache.hadoop.fs.azurebfs.ITestAzureBlobFileSystemListStatus.testListPath -- Time elapsed: 122.7 s <<< FAILURE!

[WARNING] Tests run: 225, Failures: 0, Errors: 0, Skipped: 4
[WARNING] Tests run: 895, Failures: 0, Errors: 0, Skipped: 166
[ERROR] Tests run: 158, Failures: 1, Errors: 0, Skipped: 8
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 10

============================================================
NonHNS-SharedKey-DFS

[ERROR] org.apache.hadoop.fs.azurebfs.ITestAzureBlobFileSystemListStatus.testListPath -- Time elapsed: 104.4 s <<< FAILURE!

[WARNING] Tests run: 225, Failures: 0, Errors: 0, Skipped: 10
[WARNING] Tests run: 732, Failures: 0, Errors: 0, Skipped: 287
[ERROR] Tests run: 158, Failures: 1, Errors: 0, Skipped: 9
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 11

============================================================
AppendBlob-HNS-OAuth-DFS

[ERROR] org.apache.hadoop.fs.azurebfs.ITestAzureBlobFileSystemListStatus.testListPath -- Time elapsed: 114.1 s <<< FAILURE!

[WARNING] Tests run: 225, Failures: 0, Errors: 0, Skipped: 3
[WARNING] Tests run: 892, Failures: 0, Errors: 0, Skipped: 231
[ERROR] Tests run: 135, Failures: 1, Errors: 0, Skipped: 9
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 23

============================================================
NonHNS-SharedKey-Blob

[ERROR] org.apache.hadoop.fs.azurebfs.ITestAzureBlobFileSystemListStatus.testListPath -- Time elapsed: 204.2 s <<< FAILURE!

[WARNING] Tests run: 225, Failures: 0, Errors: 0, Skipped: 10
[WARNING] Tests run: 740, Failures: 0, Errors: 0, Skipped: 144
[ERROR] Tests run: 158, Failures: 1, Errors: 0, Skipped: 3
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 11

============================================================
NonHNS-OAuth-DFS

[ERROR] org.apache.hadoop.fs.azurebfs.ITestAzureBlobFileSystemListStatus.testListPath -- Time elapsed: 108.8 s <<< FAILURE!

[WARNING] Tests run: 225, Failures: 0, Errors: 0, Skipped: 10
[WARNING] Tests run: 729, Failures: 0, Errors: 0, Skipped: 289
[ERROR] Tests run: 158, Failures: 1, Errors: 0, Skipped: 9
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 24

============================================================
NonHNS-OAuth-Blob

[ERROR] org.apache.hadoop.fs.azurebfs.ITestAzureBlobFileSystemListStatus.testListPath -- Time elapsed: 206.9 s <<< FAILURE!

[WARNING] Tests run: 225, Failures: 0, Errors: 0, Skipped: 10
[WARNING] Tests run: 737, Failures: 0, Errors: 0, Skipped: 156
[ERROR] Tests run: 158, Failures: 1, Errors: 0, Skipped: 3
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 24

============================================================
AppendBlob-NonHNS-OAuth-Blob

[ERROR] org.apache.hadoop.fs.azurebfs.ITestAzureBlobFileSystemListStatus.testListPath -- Time elapsed: 206.8 s <<< FAILURE!

[WARNING] Tests run: 225, Failures: 0, Errors: 0, Skipped: 10
[WARNING] Tests run: 732, Failures: 0, Errors: 0, Skipped: 202
[ERROR] Tests run: 135, Failures: 1, Errors: 0, Skipped: 4
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 24

============================================================
HNS-Oauth-DFS-IngressBlob

[ERROR] org.apache.hadoop.fs.azurebfs.ITestAzureBlobFileSystemListStatus.testListPath -- Time elapsed: 118.3 s <<< FAILURE!

[WARNING] Tests run: 225, Failures: 0, Errors: 0, Skipped: 3
[WARNING] Tests run: 766, Failures: 0, Errors: 0, Skipped: 229
[ERROR] Tests run: 158, Failures: 1, Errors: 0, Skipped: 8
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 23

============================================================
NonHNS-OAuth-DFS-IngressBlob

[ERROR] org.apache.hadoop.fs.azurebfs.ITestAzureBlobFileSystemListStatus.testListPath -- Time elapsed: 206.0 s <<< FAILURE!

[WARNING] Tests run: 225, Failures: 0, Errors: 0, Skipped: 10
[WARNING] Tests run: 729, Failures: 0, Errors: 0, Skipped: 286
[ERROR] Tests run: 158, Failures: 1, Errors: 0, Skipped: 9
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 24

Time taken: 306 mins 9 secs.

@hadoop-yetus

This comment was marked as outdated.

@hadoop-yetus

This comment was marked as outdated.

/**
* Constructs AbfsAdaptiveInputStream
* @param client AbfsClient to be used for read operations
* @param statistics to recordinput stream statistics
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: space

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Taken

}

/**
* inheritDoc
Copy link
Contributor

@manika137 manika137 Jan 8, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit {@ inheritDoc}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Taken

}

@Test
public void testRandomInputStreamDoesNotQueuePrefetches() throws Exception {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: javadoc

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Taken

protected final int readAheadRange;
private final int readAheadRange;

private boolean firstRead = true;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

comment as other params can be added

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added

public static final boolean DEFAULT_FS_AZURE_ENABLE_CREATE_BLOB_IDEMPOTENCY = true;

public static final boolean DEFAULT_FS_AZURE_ENABLE_PREFETCH_REQUEST_PRIORITY = true;
public static final boolean DEFAULT_FS_AZURE_ENABLE_PREFETCH_REQUEST_PRIORITY = false;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we dont require to disable it I think. We only add the request header if the read type is set to prefetch

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for catching. Will revert

*/
@VisibleForTesting
long getLimit() {
int getLimit() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why changed from long to int ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

limit is already int. The caller would have needed unncessary handling.

* @throws IOException if there is an error
*/
protected abstract int readOneBlock(final byte[] b, final int off, final int len) throws IOException;
protected abstract int readOneBlock(byte[] b, int off, int len) throws IOException;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

shouldn't these parameters be still kept final ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is abstract method's definition. Real implementatio still has final.
Checkstyle reported final as redundant modifier here.


private final Stack<Integer> removedBufferList = new Stack<>();
private final ConcurrentSkipListSet<Integer> removedBufferList = new ConcurrentSkipListSet<>();
private ConcurrentSkipListSet<Integer> freeList = new ConcurrentSkipListSet<>();
Copy link
Contributor

@anmolanmol1234 anmolanmol1234 Jan 8, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add comments for this change

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added

getInProgressList().clear();
getCompletedReadList().clear();
getFreeList().clear();
this.freeList.clear();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

clearFreeList() method can be called here

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Taken

if (!validate(b, off, len)) {
return -1;
}
//If buffer is empty, then fill the buffer.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Space missing between // and If.
Same thing at multiple other places as well, please correct it at all other places as well.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same thing is present in AbfsRandomInputStream and AbfsPrefetchInputStream as well.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Taken

import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Options;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

instead of importing entire Options class, we can just import OpenFileOptions class and directly mention OpenFileOptions class below in comments.
import org.apache.hadoop.fs.Options.OpenFileOptions;

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Taken

public static final String FS_AZURE_READ_AHEAD_BLOCK_SIZE = "fs.azure.read.readahead.blocksize";
/**
* Provides hint for the read workload pattern.
* Possible Values Exposed in {@link Options.OpenFileOptions}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should it be OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICIES?
OpenFileOptions has other values as well.
If so, we can do
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICIES;
and mention FS_OPTION_OPENFILE_READ_POLICIES only

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Tried but that variable is not resolvable here

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You can do something like this:
import org.apache.hadoop.fs.Options.OpenFileOptions;

  • Possible Values Exposed in {@link OpenFileOptions#FS_OPTION_OPENFILE_READ_POLICIES}

}
//If buffer is empty, then fill the buffer.
if (getBCursor() == getLimit()) {
//If EOF, then return -1
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same as above

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Taken

}

long bytesRead = 0;
//reset buffer to initial state - i.e., throw away existing data
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same as above

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Taken

return buffer;
}

protected void setBuffer(byte[] buffer) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Java doc missing

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Taken

* @return the corresponding AbsInputPolicy to be used
*/
public static AbfsReadPolicy getAbfsReadPolicy(String name) {
String trimmed = name.trim().toLowerCase(Locale.ENGLISH);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This variable can be renamed to something else.
Like readPolicyStr

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Taken

parameters.map(OpenFileParameters::getOptions),
contextEncryptionAdapter),
eTag, tracingContext);
AbfsReadPolicy inputPolicy = AbfsReadPolicy.getAbfsReadPolicy(getAbfsConfiguration().getAbfsReadPolicy());
Copy link
Contributor

@bhattmanish98 bhattmanish98 Jan 8, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This method contains lot of lines already. Instead of defining this switch case here, it would be better to define a new method so that tomorrow if new read pattern is introduced, we can just update that method.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Taken

@anujmodi2021
Copy link
Contributor Author

@anujmodi2021 I am trying to propose a single optimised implementation of an input stream across cloud implementations, as I think we all need this kind of logic. Ideally I want to get to a place where 80% of the logic is shared in a common layer, and then we only implement cloud specific clients to actually make the requests separately.

There is some consensus to move the shared logic into the parquet-java repo: https://lists.apache.org/thread/nbksq32cs8h1ldj8762y6wh9zzp8gqx6 , and some buy-in from the team at google. I'll be following up on this in the new year.

Would be great to get your thoughts and if your team would also like to collaborate on this.

Thanks for heads up @ahmarsuhail
This sounds like a good plan to me as well. We will surely keep a close eye on the updates on this thread and try to contribute to make things better in best way possible.

With this change we are not chaning how ABFS handles parquet file though. This just improves the infra and add capability for future improvements to be plugged in seemlessly. We will surely help address any gaps in ABFS to make things better for the common ground you are gearing up to improve.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 9m 6s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 1s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+0 🆗 detsecrets 0m 0s detect-secrets was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 4 new or modified test files.
_ trunk Compile Tests _
+1 💚 mvninstall 21m 50s trunk passed
+1 💚 compile 0m 25s trunk passed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04
+1 💚 compile 0m 23s trunk passed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
+1 💚 checkstyle 0m 18s trunk passed
+1 💚 mvnsite 0m 29s trunk passed
+1 💚 javadoc 0m 26s trunk passed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04
+1 💚 javadoc 0m 24s trunk passed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
-1 ❌ spotbugs 0m 48s /branch-spotbugs-hadoop-tools_hadoop-azure-warnings.html hadoop-tools/hadoop-azure in trunk has 1 extant spotbugs warnings.
+1 💚 shadedclient 14m 19s branch has no errors when building and testing our client artifacts.
-0 ⚠️ patch 14m 31s Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.
_ Patch Compile Tests _
+1 💚 mvninstall 0m 21s the patch passed
+1 💚 compile 0m 21s the patch passed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04
+1 💚 javac 0m 21s the patch passed
+1 💚 compile 0m 20s the patch passed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
+1 💚 javac 0m 20s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
+1 💚 checkstyle 0m 11s hadoop-tools/hadoop-azure: The patch generated 0 new + 3 unchanged - 1 fixed = 3 total (was 4)
+1 💚 mvnsite 0m 21s the patch passed
-1 ❌ javadoc 0m 18s /results-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-21.0.7+6-Ubuntu-0ubuntu120.04.txt hadoop-tools_hadoop-azure-jdkUbuntu-21.0.7+6-Ubuntu-0ubuntu120.04 with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04 generated 2 new + 1571 unchanged - 14 fixed = 1573 total (was 1585)
+1 💚 javadoc 0m 18s hadoop-tools_hadoop-azure-jdkUbuntu-17.0.15+6-Ubuntu-0ubuntu120.04 with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04 generated 0 new + 1452 unchanged - 12 fixed = 1452 total (was 1464)
-1 ❌ spotbugs 0m 44s /new-spotbugs-hadoop-tools_hadoop-azure.html hadoop-tools/hadoop-azure generated 5 new + 0 unchanged - 1 fixed = 5 total (was 1)
+1 💚 shadedclient 14m 15s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 💚 unit 2m 16s hadoop-azure in the patch passed.
+1 💚 asflicense 0m 21s The patch does not generate ASF License warnings.
69m 43s
Reason Tests
SpotBugs module:hadoop-tools/hadoop-azure
Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsInputStream.bCursor; locked 87% of time Unsynchronized access at AbfsInputStream.java:87% of time Unsynchronized access at AbfsInputStream.java:[line 970]
Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsInputStream.buffer; locked 66% of time Unsynchronized access at AbfsInputStream.java:66% of time Unsynchronized access at AbfsInputStream.java:[line 814]
Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsInputStream.fCursor; locked 90% of time Unsynchronized access at AbfsInputStream.java:90% of time Unsynchronized access at AbfsInputStream.java:[line 995]
Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsInputStream.fCursorAfterLastRead; locked 66% of time Unsynchronized access at AbfsInputStream.java:66% of time Unsynchronized access at AbfsInputStream.java:[line 1004]
Inconsistent synchronization of org.apache.hadoop.fs.azurebfs.services.AbfsInputStream.limit; locked 86% of time Unsynchronized access at AbfsInputStream.java:86% of time Unsynchronized access at AbfsInputStream.java:[line 1021]
Subsystem Report/Notes
Docker ClientAPI=1.52 ServerAPI=1.52 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-8153/6/artifact/out/Dockerfile
GITHUB PR #8153
JIRA Issue HADOOP-19767
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets
uname Linux e75afb8907fa 5.15.0-164-generic #174-Ubuntu SMP Fri Nov 14 20:25:16 UTC 2025 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / c53a82f
Default Java Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
Multi-JDK versions /usr/lib/jvm/java-21-openjdk-amd64:Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-17-openjdk-amd64:Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-8153/6/testReport/
Max. process+thread count 604 (vs. ulimit of 5500)
modules C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-8153/6/console
versions git=2.25.1 maven=3.9.11 spotbugs=4.9.7
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants