Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HADOOP-17296. ABFS: Force reads to be always of buffer size. #2368

Merged
merged 21 commits into from Nov 27, 2020

Conversation

snvijaya
Copy link
Contributor

@snvijaya snvijaya commented Oct 7, 2020

Customers migrating from Gen1 to Gen2 often are observing different read patterns for the same workload. The optimization in Gen2 which reads only requested data size once detected as random read pattern is usually the cause of difference.

In this PR, config option to force Gen2 driver to read always in buffer size even for random is being introduced. With this enabled the read pattern for the job will be similar to Gen1 and be full buffer sizes to backend.

Have also accommodated the request to config control the readahead size to help cases such as small row groups in parquet files, where more data can be captured.

These configs are not determined to be performant on the official parquet recommended row group sizes of 512-1024 MB and hence will not be enabled by default.

Tests are added to verify various combinations of config values. Also modified tests in file ITestAzureBlobFileSystemRandomRead which were using same file and hence test debugging was getting harder.

@snvijaya
Copy link
Contributor Author

snvijaya commented Oct 7, 2020

Test results from accounts in East US 2 regions:

NON-HNS:

SharedKey:
	[INFO] Tests run: 88, Failures: 0, Errors: 0, Skipped: 0
	[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 245
	[ERROR] Errors: 
	[ERROR]   ITestAbfsFileSystemContractGetFileStatus>AbstractContractGetFileStatusTest.testComplexDirActions:153->AbstractContractGetFileStatusTest.checkListStatusIteratorComplexDir:191 » IllegalState
	[ERROR]   ITestAbfsFileSystemContractGetFileStatus>AbstractContractGetFileStatusTest.testListStatusIteratorFile:366 » IllegalState
	[ERROR] Tests run: 208, Failures: 0, Errors: 2, Skipped: 24
		ERROR] testComplexDirActions(org.apache.hadoop.fs.azurebfs.contract.ITestAbfsFileSystemContractGetFileStatus)  Time elapsed: 31.122 s  <<< ERROR!
		java.lang.IllegalStateException: No more items in iterator
			at com.google.common.base.Preconditions.checkState(Preconditions.java:507)
			at org.apache.hadoop.fs.FileSystem$DirListingIterator.next(FileSystem.java:2232)
			at org.apache.hadoop.fs.FileSystem$DirListingIterator.next(FileSystem.java:2205)
			at org.apache.hadoop.fs.contract.ContractTestUtils.iteratorToListThroughNextCallsAlone(ContractTestUtils.java:1494)
			at org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest.checkListStatusIteratorComplexDir(AbstractContractGetFileStatusTest.java:191)
			at org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest.testComplexDirActions(AbstractContractGetFileStatusTest.java:153)
			at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
			at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
			at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
			at java.lang.reflect.Method.invoke(Method.java:498)
			at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
			at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
			at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
			at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
			at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
			at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
			at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
			at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
			at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
			at java.util.concurrent.FutureTask.run(FutureTask.java:266)
			at java.lang.Thread.run(Thread.java:748)

		[ERROR] testListStatusIteratorFile(org.apache.hadoop.fs.azurebfs.contract.ITestAbfsFileSystemContractGetFileStatus)  Time elapsed: 3.038 s  <<< ERROR!
		java.lang.IllegalStateException: No more items in iterator
			at com.google.common.base.Preconditions.checkState(Preconditions.java:507)
			at org.apache.hadoop.fs.FileSystem$DirListingIterator.next(FileSystem.java:2232)
			at org.apache.hadoop.fs.FileSystem$DirListingIterator.next(FileSystem.java:2205)
			at org.apache.hadoop.fs.contract.ContractTestUtils.iteratorToListThroughNextCallsAlone(ContractTestUtils.java:1494)
			at org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest.testListStatusIteratorFile(AbstractContractGetFileStatusTest.java:366)
			at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
			at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
			at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
			at java.lang.reflect.Method.invoke(Method.java:498)
			at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
			at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
			at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
			at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
			at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
			at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
			at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
			at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
			at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
			at java.util.concurrent.FutureTask.run(FutureTask.java:266)
			at java.lang.Thread.run(Thread.java:748)

OAuth:
	INFO] Tests run: 88, Failures: 0, Errors: 0, Skipped: 0
	[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 24
	[ERROR] Errors: 
	[ERROR]   ITestAbfsFileSystemContractGetFileStatus>AbstractContractGetFileStatusTest.testComplexDirActions:153->AbstractContractGetFileStatusTest.checkListStatusIteratorComplexDir:191 » IllegalState
	[ERROR]   ITestAbfsFileSystemContractGetFileStatus>AbstractContractGetFileStatusTest.testListStatusIteratorFile:366 » IllegalState
	[ERROR] Tests run: 208, Failures: 0, Errors: 2, Skipped: 24

HNS:

SharedKey:
	[INFO] Tests run: 88, Failures: 0, Errors: 0, Skipped: 0
	[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 24
	ERROR] Errors:
	[ERROR]   ITestAbfsFileSystemContractGetFileStatus>AbstractContractGetFileStatusTest.testComplexDirActions:153->AbstractContractGetFileStatusTest.checkListStatusIteratorComplexDir:191 » IllegalState
	[ERROR]   ITestAbfsFileSystemContractGetFileStatus>AbstractContractGetFileStatusTest.testListStatusIteratorFile:366 » IllegalState
	[INFO]
	[ERROR] Tests run: 208, Failures: 0, Errors: 2, Skipped: 24


OAuth: * In this run the ListIterator test got skipped due to isSecure config being off
	[INFO] Tests run: 88, Failures: 0, Errors: 0, Skipped: 0
	[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 66
	[WARNING] Tests run: 208, Failures: 0, Errors: 0, Skipped: 141

2 failures seen in most combination call fs.listStatusIterator() APIs. These APIs are not supported by ABFS FS currently. Need to check if a recent change to test introduced this call.

@@ -178,11 +195,15 @@ private int readOneBlock(final byte[] b, final int off, final int len) throws IO
buffer = new byte[bufferSize];
}

// Enable readAhead when reading sequentially
if (-1 == fCursorAfterLastRead || fCursorAfterLastRead == fCursor || b.length >= bufferSize) {
if (alwaysReadBufferSize) {
bytesRead = readInternal(fCursor, buffer, 0, bufferSize, false);
Copy link
Contributor

Choose a reason for hiding this comment

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

JIRA and PR description says we are trying to read till bufferSize always rather than just the requested length but as per this line we are enabling the buffer manager readahead as well which is bypassed in random read in gen2 as per line 205 below. PS: I have never seen gen1 code though.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

AlwaysReadBufferSize helped the IO pattern to match the Gen1 run. But to be performant readAhead had to be enabled. For the customer scenario explained in the JIRA , for the small row groups for an overall small parquet file size, reading whole buffer size along with readAhead bought good performance.

long nextOffset = position;
// First read to queue needs to be of readBufferSize and later
Copy link
Contributor

Choose a reason for hiding this comment

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

Would like to understand the reasoning behind this. Thanks.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Couple of things here:

  1. The earlier code allowed bufferSize to be configurable whereas ReadAhead buffer size was fixed. And each time loop is done, read issued was always for bufferSize which can lead to gaps/holes in the readAhead range done.
  2. There is no validation for 4MB as a fixed size for readAhead is optimal for all sequential reads. Having a higher readAhead range for apps like DFSIO which are guaranteed sequential and doing higher readAhead ranges in background can be performant.
    In this PR, the bug in point 1 is fixed and also a provision to configure readAhead buffer size is provided.

Copy link
Contributor

Choose a reason for hiding this comment

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

What do you mean by gaps/holes in the readAhead range done here?
Have you done any experiments on this readAheadBlockSize config? If so, please share.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

If the buffer gets overwritten by config to 16MB, the readAhead buffer size will still remain to be 4MB as it was a code static. The loop done will start issuing readAheads in 16 MB buffer sizes, the request to readAhead will be:
offset=0, Length=16MB
offset=16MB, Length=32MB

But the readAhead buffer size is stuck at 4 MB. so it will read only:
offset=0 Length=4MB
offset=16MB Length=4MB

Gap being at 4MB to 16MB here.

This bug is getting fixed. Tests for all possible combinations here has been added to the tests of this PR.

Copy link
Contributor

Choose a reason for hiding this comment

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

I don't think there is any bug in the current production code as such. As far as I understand the code the change is introduced becuase new config is introduced.
Now my question is why not use readAheadBlockSize for the first call as well? The calls would be like
offset=0 Length=4MB
offset=4MB Length=4MB

Sorry to say this but honestly speaking, introducing so many configs is making the code complex and confusing.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Read buffer size config being available to be modified, fixed read ahead buffer size and issuing read aheads by buffer size is the current prod behaviour and will function as the picture attached. This will need fixing.
And as for deprecating read buffer size config and only use the new read ahead buffer size config. The config has been available since GA, and hence deprecating it would not be feasible. (Also for clients who are disabling readAheads to use readahead buffer size for reads might be confusing too).
As for the number of different configs present for read, 1 and 2 configs already were present while this PR is introducing 3 and 4. So total of 4 configs.

  1. fs.azure.read.request.size
  2. fs.azure.readaheadqueue.depth
  3. fs.azure.read.alwaysReadBufferSize => For Gen1 migrating customers
  4. fs.azure.read.readahead.blocksize => Was one that needed fixing long back as there is no validation on 4 MB being the right size for all workloads. Just the way read buffer size can be modified.

All these changes are being added based on various customer issues and experiences that we are dealing with. Instead of spending our time in providing patches that can enable them to test various combinations, having these options over a config for their testing saves our dev time to improve the service. As you can see in the PR, the defaults introduced by these configs will retain the current prod behavior.
ReadAheadBufferConfigExplanation

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Have synced with @mukund-thakur over mail thread further to clear the understanding.

static {
BUFFER_MANAGER = new ReadBufferManager();
BUFFER_MANAGER.init();
static ReadBufferManager getBufferManager() {
Copy link
Contributor

Choose a reason for hiding this comment

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

Why all these changes ? Why not just initilize the blockSize in init() ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

For singleton classes its a common practice to lock around the new instance creation within the getInstance() method. Also, didnt want to make any changes to init 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.

Retaining the change as current change has no functional issues.

Copy link
Contributor

Choose a reason for hiding this comment

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

FWIW, synchronize on the Class object is a standard pattern here, but the lock is fine, just a bit of extra coding

@mukund-thakur
Copy link
Contributor

Reviewed the production code. Will review the test code as well once I get some time.

@snvijaya
Copy link
Contributor Author

Test results from accounts on East US 2 region:

NON-HNS:

SharedKey:
	[INFO] Results:
	[INFO] 
	[INFO] Tests run: 88, Failures: 0, Errors: 0, Skipped: 0
	[INFO] Results:
	[INFO] 
	[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 245
	[INFO] Results:
	[INFO] 
	[WARNING] Tests run: 207, Failures: 0, Errors: 0, Skipped: 24

HNS:

SharedKey:
	[INFO] Results:
	[INFO] 
	[INFO] Tests run: 88, Failures: 0, Errors: 0, Skipped: 0
	[INFO] Results:
	[INFO] 
	[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 24
	[INFO] Results:
	[INFO] 
	[WARNING] Tests run: 207, Failures: 0, Errors: 0, Skipped: 24
OAuth:
	[INFO] Results:
	[INFO] 
	[INFO] Tests run: 88, Failures: 0, Errors: 0, Skipped: 0
	[INFO] Results:
	[INFO] 
	[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 66
	[INFO] Results:
	[INFO] 
	[WARNING] Tests run: 207, Failures: 0, Errors: 0, Skipped: 140

@snvijaya
Copy link
Contributor Author

@mukund-thakur Thanks for your review. I have updated the PR with suggestions. Kindly request you to review.

Copy link
Contributor

@mukund-thakur mukund-thakur left a comment

Choose a reason for hiding this comment

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

Production code almost ready apart from one query here #2368 (comment)

Added some minor comments on test code mostly around asserstions. Try if we can use assertJ asserstions which provide rich capabilities and good error messaging.
For example : https://github.com/apache/hadoop/pull/2307/files#diff-58b8ac52c9e03341fd23fdf90d318e6972570d010f6a12fe4a0ebaf72a59251dR249

assertTrue("Read should be of exact requested size",
inputStream.read(firstReadBuffer, 0, readRequestSize) == readRequestSize);
assertTrue("Data mismatch found in RAH1",
Arrays.equals(firstReadBuffer,
Copy link
Contributor

Choose a reason for hiding this comment

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

AssertJ has rich api's to tackle these kind of assertions. Try that
example : Assertions.assertThat(list)
.hasSameElementsAs(list2)

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. Have updated except for the content check assert.


assertTrue("Read should be of exact requested size",
inputStream.read(secondReadBuffer, 0, readAheadRequestSize) == readAheadRequestSize);
assertTrue("Data mismatch found in RAH2",
Copy link
Contributor

Choose a reason for hiding this comment

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

Better to use assert equals here inspite of assertTrue no?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

:) Just a force of habit. Will try to use the assertions.
Have modified in the other test code areas of this PR as well.

}
}

private AzureBlobFileSystem createTestFile(Path testFilePath, long testFileSize,
Copy link
Contributor

Choose a reason for hiding this comment

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

See if you can reuse the data generation and new file creation code from ContractTestUtils.dataset() and ContractTestUtils.createFile.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I need the file created to also have a specific pattern of data ingested and have another method which needs to return the expected data content from offset to range. This is being used to cross check with readahead buffer data. Will retain for these tests. But thanks, hadnt observed the other APIs, will default to them where there isnt a specific needs on the file content.

testAlwaysReadBufferSizeConfig(true);
}

private void assertStatistics(AzureBlobFileSystem fs,
Copy link
Contributor

Choose a reason for hiding this comment

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

Why creating a new method here if we are just doing a passthrough?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Redundant method removed.

@steveloughran
Copy link
Contributor

this will need to be rebased now that #2369

@snvijaya
Copy link
Contributor Author

@mukund-thakur - Have addressed the review comments. Please have a look.

@snvijaya
Copy link
Contributor Author

Re-ran the oAuth tests with the latest changes:
### OAuth:
[INFO] Results:
[INFO]
[INFO] Tests run: 89, Failures: 0, Errors: 0, Skipped: 0
[INFO] Results:
[INFO]
[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 66
[INFO] Results:
[INFO]
[WARNING] Tests run: 207, Failures: 0, Errors: 0, Skipped: 140

@snvijaya
Copy link
Contributor Author

Test failures reported in Yetus are not related to ABFS driver. Have created JIRA for tracking the failures.
https://issues.apache.org/jira/browse/HADOOP-17325

@snvijaya
Copy link
Contributor Author

Hi @mukund-thakur, Kindly request your review, so that I can freeze on comments from your side.

@snvijaya
Copy link
Contributor Author

@mukund-thakur - Will appreciate if you can confirm your review status on this PR. Have provided updates for your comments.

@snvijaya
Copy link
Contributor Author

HNS-OAuth

[INFO] Results:
[INFO]
[INFO] Tests run: 89, Failures: 0, Errors: 0, Skipped: 0
[INFO] Results:
[INFO]
[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 66
[INFO] Results:
[INFO]
[WARNING] Tests run: 208, Failures: 0, Errors: 0, Skipped: 141

HNS-SharedKey

[INFO] Results:
[INFO]
[INFO] Tests run: 89, Failures: 0, Errors: 0, Skipped: 0
[INFO] Results:
[INFO]
[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 24
[INFO] Results:
[INFO]
[WARNING] Tests run: 208, Failures: 0, Errors: 0, Skipped: 24

NonHNS-SharedKey

[INFO] Results:
[INFO]
[ERROR] Failures:
[ERROR] TestAbfsClientThrottlingAnalyzer.testManySuccessAndErrorsAndWaiting:171->fuzzyValidate:49 The actual value 20 is not within the expected range: [5.60, 8.40].
[INFO]
[ERROR] Tests run: 89, Failures: 1, Errors: 0, Skipped: 0
[INFO] Results:
[INFO]
[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 245
[INFO] Results:
[INFO]
[WARNING] Tests run: 208, Failures: 0, Errors: 0, Skipped: 24

NonHNS-OAuth

[INFO] Results:
[INFO]
[INFO] Tests run: 89, Failures: 0, Errors: 0, Skipped: 0
[INFO] Results:
[INFO]
[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 249
[INFO] Results:
[INFO]
[WARNING] Tests run: 208, Failures: 0, Errors: 0, Skipped: 141

Test failures reported in Yetus are not related to ABFS driver. Have created JIRA for tracking the failures.
https://issues.apache.org/jira/browse/HADOOP-17325

@apache apache deleted a comment from hadoop-yetus Oct 30, 2020
@mukund-thakur
Copy link
Contributor

@mukund-thakur - Will appreciate if you can confirm your review status on this PR. Have provided updates for your comments.

I was on a vacation. Will check once I get some time.

@apache apache deleted a comment from hadoop-yetus Nov 17, 2020
@apache apache deleted a comment from hadoop-yetus Nov 17, 2020
@apache apache deleted a comment from hadoop-yetus Nov 17, 2020
@apache apache deleted a comment from hadoop-yetus Nov 17, 2020
@apache apache deleted a comment from hadoop-yetus Nov 17, 2020
@snvijaya
Copy link
Contributor Author

HNS-OAuth

[INFO] Results:
[INFO]
[INFO] Tests run: 89, Failures: 0, Errors: 0, Skipped: 0
[INFO] Results:
[INFO]
[WARNING] Tests run: 446, Failures: 0, Errors: 0, Skipped: 66
[INFO] Results:
[INFO]
[WARNING] Tests run: 208, Failures: 0, Errors: 0, Skipped: 141

NonHNS-OAuth

[INFO] Results:
[INFO]
[INFO] Tests run: 89, Failures: 0, Errors: 0, Skipped: 0
[INFO] Results:
[INFO]
[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 249
[INFO] Results:
[INFO]
[WARNING] Tests run: 206, Failures: 0, Errors: 0, Skipped: 141

HNS-SharedKey

[INFO] Results:
[INFO]
[INFO] Tests run: 89, Failures: 0, Errors: 0, Skipped: 0
[INFO] Results:
[INFO]
[ERROR] Errors:
[ERROR] ITestAzureBlobFileSystemDelegationSAS.testCheckAccess:94 » AbfsRestOperation O...
[ERROR] ITestAzureBlobFileSystemDelegationSAS.testRootPath:343 » AbfsRestOperation Ope...
[INFO]
[ERROR] Tests run: 446, Failures: 0, Errors: 2, Skipped: 24
[INFO] Results:
[INFO]
[ERROR] Failures:
[ERROR] ITestAbfsFileSystemContractRename>AbstractContractRenameTest.testRenameFileOverExistingFile:131->Assert.fail:88 expected rename(abfs://gen2contracttest@snvijayacontracttest.dfs.core.windows.net/fork-6/test/source-256.txt, abfs://gen2contracttest@snvijayacontracttest.dfs.core.windows.net/fork-6/test/dest-512.txt) to be rejected with exception, but got false
[INFO]
[ERROR] Tests run: 196, Failures: 1, Errors: 0, Skipped: 24

NonHNS-SharedKey

[INFO] Results:
[INFO]
[INFO] Tests run: 89, Failures: 0, Errors: 0, Skipped: 0
[INFO] Results:
[INFO]
[WARNING] Tests run: 424, Failures: 0, Errors: 0, Skipped: 242
[INFO] Results:
[INFO]
[ERROR] Failures:
[ERROR] ITestAbfsFileSystemContractRename>AbstractContractRenameTest.testRenameFileOverExistingFile:131->Assert.fail:88 expected rename(abfs://gen2contracttest@snvijayanonhnstest.dfs.core.windows.net/fork-7/test/source-256.txt, abfs://gen2contracttest@snvijayanonhnstest.dfs.core.windows.net/fork-7/test/dest-512.txt) to be rejected with exception, but got false
[INFO]
[ERROR] Tests run: 208, Failures: 1, Errors: 0, Skipped: 24

Rename Test failure is tracked in JIRA: https://issues.apache.org/jira/browse/HADOOP-17396
SAS Test failure is tracked in JIRA: https://issues.apache.org/jira/browse/HADOOP-17397

@hadoop-yetus
Copy link

🎊 +1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 31m 5s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 1s No case conflicting files found.
+0 🆗 markdownlint 0m 0s markdownlint was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 0m 0s test4tests The patch appears to include 3 new or modified test files.
_ trunk Compile Tests _
+1 💚 mvninstall 35m 20s trunk passed
+1 💚 compile 0m 35s trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04
+1 💚 compile 0m 31s trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01
+1 💚 checkstyle 0m 23s trunk passed
+1 💚 mvnsite 0m 36s trunk passed
+1 💚 shadedclient 18m 14s branch has no errors when building and testing our client artifacts.
+1 💚 javadoc 0m 27s trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04
+1 💚 javadoc 0m 25s trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01
+0 🆗 spotbugs 0m 57s Used deprecated FindBugs config; considering switching to SpotBugs.
+1 💚 findbugs 0m 55s trunk passed
-0 ⚠️ patch 1m 12s 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 28s the patch passed
+1 💚 compile 0m 30s the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04
+1 💚 javac 0m 30s the patch passed
+1 💚 compile 0m 24s the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01
+1 💚 javac 0m 24s the patch passed
+1 💚 checkstyle 0m 15s the patch passed
+1 💚 mvnsite 0m 27s the patch passed
+1 💚 whitespace 0m 0s The patch has no whitespace issues.
+1 💚 shadedclient 16m 34s patch has no errors when building and testing our client artifacts.
+1 💚 javadoc 0m 25s the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04
+1 💚 javadoc 0m 21s the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01
+1 💚 findbugs 0m 59s the patch passed
_ Other Tests _
+1 💚 unit 1m 25s hadoop-azure in the patch passed.
+1 💚 asflicense 0m 29s The patch does not generate ASF License warnings.
112m 53s
Subsystem Report/Notes
Docker ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/7/artifact/out/Dockerfile
GITHUB PR #2368
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle markdownlint
uname Linux fd8c2773496c 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / 235947e
Default Java Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/7/testReport/
Max. process+thread count 622 (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-2368/7/console
versions git=2.17.1 maven=3.6.0 findbugs=4.0.6
Powered by Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org

This message was automatically generated.

@snvijaya
Copy link
Contributor Author

@mukund-thakur Have synced with latest trunk. Please have a look.

Copy link
Contributor

@mukund-thakur mukund-thakur left a comment

Choose a reason for hiding this comment

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

LGTM +1

@steveloughran steveloughran added bug enhancement fs/azure changes related to azure; submitter must declare test endpoint labels Nov 27, 2020
@@ -99,12 +115,14 @@ public void testBasicRead() throws Exception {
public void testRandomRead() throws Exception {
Assume.assumeFalse("This test does not support namespace enabled account",
this.getFileSystem().getIsNamespaceEnabled());
assumeHugeFileExists();
Path testPath = new Path(TEST_FILE_PREFIX + "_testRandomRead");
Copy link
Contributor

Choose a reason for hiding this comment

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

FWIW I use a Junit rule to get the method name, then you can hava a path() method which dynamically creates the unique path, including when you use parameterized tests.

Copy link
Contributor

@steveloughran steveloughran left a comment

Choose a reason for hiding this comment

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

+1

@steveloughran steveloughran changed the title Hadoop-17296. ABFS: Force reads to be always of buffer size HADOOP -17296. ABFS: Force reads to be always of buffer size. Nov 27, 2020
@steveloughran steveloughran merged commit 142941b into apache:trunk Nov 27, 2020
@steveloughran steveloughran changed the title HADOOP -17296. ABFS: Force reads to be always of buffer size. HADOOP-17296. ABFS: Force reads to be always of buffer size. Nov 27, 2020
@steveloughran
Copy link
Contributor

ok, merged into trunk. For 3.3.x can you do the backport, retest and put up a new PR for me to merge in there? That branch doesn't have guava under the thirdparty packaging, so a safe cherrypick isn't enough to prove it will compile/run. Don't worry about reordering the import references, just edit out the o.a.h.thirdparty prefixes and leave as is.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug enhancement fs/azure changes related to azure; submitter must declare test endpoint
Projects
None yet
6 participants