|
@@ -50,17 +50,22 @@ import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
|
|
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
|
|
import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
|
|
import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType;
|
|
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
|
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
|
|
|
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsDriverException;
|
|
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
|
|
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
|
|
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConcurrentWriteOperationDetectedException;
|
|
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConcurrentWriteOperationDetectedException;
|
|
|
|
+import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
|
|
import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter;
|
|
import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter;
|
|
import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient;
|
|
import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient;
|
|
import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
|
|
import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
|
|
import org.apache.hadoop.fs.azurebfs.services.AbfsClientHandler;
|
|
import org.apache.hadoop.fs.azurebfs.services.AbfsClientHandler;
|
|
import org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil;
|
|
import org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil;
|
|
|
|
+import org.apache.hadoop.fs.azurebfs.services.AbfsDfsClient;
|
|
|
|
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
|
|
import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
|
|
import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
|
|
import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
|
|
import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
|
|
import org.apache.hadoop.fs.azurebfs.services.ITestAbfsClient;
|
|
import org.apache.hadoop.fs.azurebfs.services.ITestAbfsClient;
|
|
import org.apache.hadoop.fs.azurebfs.services.RenameAtomicity;
|
|
import org.apache.hadoop.fs.azurebfs.services.RenameAtomicity;
|
|
|
|
+import org.apache.hadoop.fs.azurebfs.services.TestAbfsClient;
|
|
import org.apache.hadoop.fs.azurebfs.utils.DirectoryStateHelper;
|
|
import org.apache.hadoop.fs.azurebfs.utils.DirectoryStateHelper;
|
|
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
|
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
|
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
|
|
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
|
|
@@ -70,16 +75,23 @@ import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.apache.hadoop.test.LambdaTestUtils;
|
|
import org.apache.hadoop.test.LambdaTestUtils;
|
|
import org.apache.hadoop.test.ReflectionUtils;
|
|
import org.apache.hadoop.test.ReflectionUtils;
|
|
|
|
|
|
|
|
+import static java.net.HttpURLConnection.HTTP_CLIENT_TIMEOUT;
|
|
import static java.net.HttpURLConnection.HTTP_CONFLICT;
|
|
import static java.net.HttpURLConnection.HTTP_CONFLICT;
|
|
import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR;
|
|
import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR;
|
|
import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
|
|
import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
|
|
import static java.net.HttpURLConnection.HTTP_OK;
|
|
import static java.net.HttpURLConnection.HTTP_OK;
|
|
import static java.net.HttpURLConnection.HTTP_PRECON_FAILED;
|
|
import static java.net.HttpURLConnection.HTTP_PRECON_FAILED;
|
|
import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CONNECTIONS_MADE;
|
|
import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CONNECTIONS_MADE;
|
|
|
|
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
|
|
|
|
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ROOT_PATH;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ROOT_PATH;
|
|
|
|
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_CLIENT_TRANSACTION_ID;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_CONDITIONAL_CREATE_OVERWRITE;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_CONDITIONAL_CREATE_OVERWRITE;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_MKDIR_OVERWRITE;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_MKDIR_OVERWRITE;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
|
|
|
|
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_CLIENT_TRANSACTION_ID;
|
|
|
|
+import static org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil.mockAddClientTransactionIdToHeader;
|
|
|
|
+import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_CREATE_RECOVERY;
|
|
import static org.apache.hadoop.fs.azurebfs.services.RenameAtomicity.SUFFIX;
|
|
import static org.apache.hadoop.fs.azurebfs.services.RenameAtomicity.SUFFIX;
|
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile;
|
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile;
|
|
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
|
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
|
@@ -2062,4 +2074,220 @@ public class ITestAzureBlobFileSystemCreate extends
|
|
op = client.getPathStatus(fileName, true, testTracingContext, null);
|
|
op = client.getPathStatus(fileName, true, testTracingContext, null);
|
|
return AzureBlobFileSystemStore.extractEtagHeader(op.getResult());
|
|
return AzureBlobFileSystemStore.extractEtagHeader(op.getResult());
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Tests the idempotency of creating a path with retries by simulating
|
|
|
|
+ * a conflict response (HTTP 409) from the Azure Blob File System client.
|
|
|
|
+ * The method ensures that the path creation operation retries correctly
|
|
|
|
+ * with the proper transaction ID headers, verifying idempotency during
|
|
|
|
+ * failure recovery.
|
|
|
|
+ *
|
|
|
|
+ * @throws Exception if any error occurs during the operation.
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testCreatePathRetryIdempotency() throws Exception {
|
|
|
|
+ Configuration configuration = new Configuration(getRawConfiguration());
|
|
|
|
+ configuration.set(FS_AZURE_ENABLE_CLIENT_TRANSACTION_ID, "true");
|
|
|
|
+ try (AzureBlobFileSystem fs = getFileSystem(configuration)) {
|
|
|
|
+ assumeRecoveryThroughClientTransactionID(true);
|
|
|
|
+ AbfsDfsClient abfsClient = mockIngressClientHandler(fs);
|
|
|
|
+ final Path nonOverwriteFile = new Path(
|
|
|
|
+ "/NonOverwriteTest_FileName_" + UUID.randomUUID());
|
|
|
|
+ final List<AbfsHttpHeader> headers = new ArrayList<>();
|
|
|
|
+ mockRetriedRequest(abfsClient, headers);
|
|
|
|
+ AbfsRestOperation getPathRestOp = Mockito.mock(AbfsRestOperation.class);
|
|
|
|
+ AbfsHttpOperation op = Mockito.mock(AbfsHttpOperation.class);
|
|
|
|
+ Mockito.doAnswer(answer -> {
|
|
|
|
+ String requiredHeader = null;
|
|
|
|
+ for (AbfsHttpHeader httpHeader : headers) {
|
|
|
|
+ if (X_MS_CLIENT_TRANSACTION_ID.equalsIgnoreCase(
|
|
|
|
+ httpHeader.getName())) {
|
|
|
|
+ requiredHeader = httpHeader.getValue();
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ return requiredHeader;
|
|
|
|
+ }).when(op).getResponseHeader(X_MS_CLIENT_TRANSACTION_ID);
|
|
|
|
+ Mockito.doReturn(true).when(getPathRestOp).hasResult();
|
|
|
|
+ Mockito.doReturn(op).when(getPathRestOp).getResult();
|
|
|
|
+ Mockito.doReturn(getPathRestOp).when(abfsClient).getPathStatus(
|
|
|
|
+ Mockito.nullable(String.class), Mockito.nullable(Boolean.class),
|
|
|
|
+ Mockito.nullable(TracingContext.class),
|
|
|
|
+ Mockito.nullable(ContextEncryptionAdapter.class));
|
|
|
|
+ fs.create(nonOverwriteFile, false);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Test to verify that the client transaction ID is included in the response header
|
|
|
|
+ * during the creation of a new file in Azure Blob Storage.
|
|
|
|
+ *
|
|
|
|
+ * This test ensures that when a new file is created, the Azure Blob FileSystem client
|
|
|
|
+ * correctly includes the client transaction ID in the response header for the created file.
|
|
|
|
+ * The test uses a configuration where client transaction ID is enabled and verifies
|
|
|
|
+ * its presence after the file creation operation.
|
|
|
|
+ *
|
|
|
|
+ * @throws Exception if any error occurs during test execution
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testGetClientTransactionIdAfterCreate() throws Exception {
|
|
|
|
+ try (AzureBlobFileSystem fs = getFileSystem()) {
|
|
|
|
+ assumeRecoveryThroughClientTransactionID(true);
|
|
|
|
+ final String[] clientTransactionId = new String[1];
|
|
|
|
+ AbfsDfsClient abfsDfsClient = mockIngressClientHandler(fs);
|
|
|
|
+ mockAddClientTransactionIdToHeader(abfsDfsClient, clientTransactionId);
|
|
|
|
+ final Path nonOverwriteFile = new Path(
|
|
|
|
+ "/NonOverwriteTest_FileName_" + UUID.randomUUID());
|
|
|
|
+ fs.create(nonOverwriteFile, false);
|
|
|
|
+
|
|
|
|
+ final AbfsHttpOperation getPathStatusOp =
|
|
|
|
+ abfsDfsClient.getPathStatus(nonOverwriteFile.toUri().getPath(), false,
|
|
|
|
+ getTestTracingContext(fs, true), null).getResult();
|
|
|
|
+ Assertions.assertThat(
|
|
|
|
+ getPathStatusOp.getResponseHeader(X_MS_CLIENT_TRANSACTION_ID))
|
|
|
|
+ .describedAs("Client transaction ID should be set during create")
|
|
|
|
+ .isNotNull();
|
|
|
|
+ Assertions.assertThat(
|
|
|
|
+ getPathStatusOp.getResponseHeader(X_MS_CLIENT_TRANSACTION_ID))
|
|
|
|
+ .describedAs("Client transaction ID should be equal to the one set in the header")
|
|
|
|
+ .isEqualTo(clientTransactionId[0]);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Test to verify that the client transaction ID is included in the response header
|
|
|
|
+ * after two consecutive create operations on the same file in Azure Blob Storage.
|
|
|
|
+ *
|
|
|
|
+ * This test ensures that even after performing two create operations (with overwrite)
|
|
|
|
+ * on the same file, the Azure Blob FileSystem client includes the client transaction ID
|
|
|
|
+ * in the response header for the created file. The test checks for the presence of
|
|
|
|
+ * the client transaction ID in the response after the second create call.
|
|
|
|
+ *
|
|
|
|
+ * @throws Exception if any error occurs during test execution
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testClientTransactionIdAfterTwoCreateCalls() throws Exception {
|
|
|
|
+ try (AzureBlobFileSystem fs = getFileSystem()) {
|
|
|
|
+ assumeRecoveryThroughClientTransactionID(true);
|
|
|
|
+ final String[] clientTransactionId = new String[1];
|
|
|
|
+ AbfsDfsClient abfsDfsClient = mockIngressClientHandler(fs);
|
|
|
|
+ mockAddClientTransactionIdToHeader(abfsDfsClient, clientTransactionId);
|
|
|
|
+ Path testPath = path("testfile");
|
|
|
|
+ AzureBlobFileSystemStore.Permissions permissions
|
|
|
|
+ = new AzureBlobFileSystemStore.Permissions(false,
|
|
|
|
+ FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
|
|
|
|
+ fs.create(testPath, false);
|
|
|
|
+ fs.create(testPath, true);
|
|
|
|
+ final AbfsHttpOperation getPathStatusOp =
|
|
|
|
+ abfsDfsClient.getPathStatus(testPath.toUri().getPath(), false,
|
|
|
|
+ getTestTracingContext(fs, true), null).getResult();
|
|
|
|
+ Assertions.assertThat(
|
|
|
|
+ getPathStatusOp.getResponseHeader(X_MS_CLIENT_TRANSACTION_ID))
|
|
|
|
+ .describedAs("Client transaction ID should be set during create")
|
|
|
|
+ .isNotNull();
|
|
|
|
+ Assertions.assertThat(
|
|
|
|
+ getPathStatusOp.getResponseHeader(X_MS_CLIENT_TRANSACTION_ID))
|
|
|
|
+ .describedAs("Client transaction ID should be equal to the one set in the header")
|
|
|
|
+ .isEqualTo(clientTransactionId[0]);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Test case to simulate a failure scenario during the recovery process while
|
|
|
|
+ * creating a file in Azure Blob File System. This test verifies that when the
|
|
|
|
+ * `getPathStatus` method encounters a timeout exception during recovery, it
|
|
|
|
+ * triggers an appropriate failure response.
|
|
|
|
+ *
|
|
|
|
+ * The test mocks the `AbfsDfsClient` to simulate the failure behavior, including
|
|
|
|
+ * a retry logic. It also verifies that an exception is correctly thrown and the
|
|
|
|
+ * error message contains the expected details for recovery failure.
|
|
|
|
+ *
|
|
|
|
+ * @throws Exception If an error occurs during the test setup or execution.
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testFailureInGetPathStatusDuringCreateRecovery() throws Exception {
|
|
|
|
+ try (AzureBlobFileSystem fs = getFileSystem()) {
|
|
|
|
+ assumeRecoveryThroughClientTransactionID(true);
|
|
|
|
+ final String[] clientTransactionId = new String[1];
|
|
|
|
+ AbfsDfsClient abfsDfsClient = mockIngressClientHandler(fs);
|
|
|
|
+ mockAddClientTransactionIdToHeader(abfsDfsClient, clientTransactionId);
|
|
|
|
+ mockRetriedRequest(abfsDfsClient, new ArrayList<>());
|
|
|
|
+ boolean[] flag = new boolean[1];
|
|
|
|
+ Mockito.doAnswer(getPathStatus -> {
|
|
|
|
+ if (!flag[0]) {
|
|
|
|
+ flag[0] = true;
|
|
|
|
+ throw new AbfsRestOperationException(HTTP_CLIENT_TIMEOUT, "", "", new Exception());
|
|
|
|
+ }
|
|
|
|
+ return getPathStatus.callRealMethod();
|
|
|
|
+ }).when(abfsDfsClient).getPathStatus(
|
|
|
|
+ Mockito.nullable(String.class), Mockito.nullable(Boolean.class),
|
|
|
|
+ Mockito.nullable(TracingContext.class),
|
|
|
|
+ Mockito.nullable(ContextEncryptionAdapter.class));
|
|
|
|
+
|
|
|
|
+ final Path nonOverwriteFile = new Path(
|
|
|
|
+ "/NonOverwriteTest_FileName_" + UUID.randomUUID());
|
|
|
|
+ String errorMessage = intercept(AbfsDriverException.class,
|
|
|
|
+ () -> fs.create(nonOverwriteFile, false)).getErrorMessage();
|
|
|
|
+
|
|
|
|
+ Assertions.assertThat(errorMessage)
|
|
|
|
+ .describedAs("getPathStatus should fail while recovering")
|
|
|
|
+ .contains(ERR_CREATE_RECOVERY);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Mocks and returns an instance of {@link AbfsDfsClient} for the given AzureBlobFileSystem.
|
|
|
|
+ * This method sets up the necessary mock behavior for the client handler and ingress client.
|
|
|
|
+ *
|
|
|
|
+ * @param fs The {@link AzureBlobFileSystem} instance for which the client handler will be mocked.
|
|
|
|
+ * @return A mocked {@link AbfsDfsClient} instance associated with the provided file system.
|
|
|
|
+ */
|
|
|
|
+ private AbfsDfsClient mockIngressClientHandler(AzureBlobFileSystem fs) {
|
|
|
|
+ AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
|
|
|
|
+ AbfsClientHandler clientHandler = Mockito.spy(store.getClientHandler());
|
|
|
|
+ AbfsDfsClient abfsDfsClient = (AbfsDfsClient) Mockito.spy(
|
|
|
|
+ clientHandler.getClient());
|
|
|
|
+ fs.getAbfsStore().setClient(abfsDfsClient);
|
|
|
|
+ fs.getAbfsStore().setClientHandler(clientHandler);
|
|
|
|
+ Mockito.doReturn(abfsDfsClient).when(clientHandler).getIngressClient();
|
|
|
|
+ return abfsDfsClient;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Mocks the retry behavior for an AbfsDfsClient request. The method intercepts
|
|
|
|
+ * the Abfs operation and simulates an HTTP conflict (HTTP 409) error on the
|
|
|
|
+ * first invocation. It creates a mock HTTP operation with a PUT method and
|
|
|
|
+ * specific status codes and error messages.
|
|
|
|
+ *
|
|
|
|
+ * @param abfsDfsClient The AbfsDfsClient to mock operations for.
|
|
|
|
+ * @param headers The list of HTTP headers to which request headers will be added.
|
|
|
|
+ *
|
|
|
|
+ * @throws Exception If an error occurs during mock creation or operation execution.
|
|
|
|
+ */
|
|
|
|
+ private void mockRetriedRequest(AbfsDfsClient abfsDfsClient,
|
|
|
|
+ final List<AbfsHttpHeader> headers) throws Exception {
|
|
|
|
+ TestAbfsClient.mockAbfsOperationCreation(abfsDfsClient,
|
|
|
|
+ new MockIntercept<AbfsRestOperation>() {
|
|
|
|
+ private int count = 0;
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public void answer(final AbfsRestOperation mockedObj,
|
|
|
|
+ final InvocationOnMock answer)
|
|
|
|
+ throws AbfsRestOperationException {
|
|
|
|
+ if (count == 0) {
|
|
|
|
+ count = 1;
|
|
|
|
+ AbfsHttpOperation op = Mockito.mock(AbfsHttpOperation.class);
|
|
|
|
+ Mockito.doReturn(HTTP_METHOD_PUT).when(op).getMethod();
|
|
|
|
+ Mockito.doReturn(EMPTY_STRING).when(op).getStorageErrorMessage();
|
|
|
|
+ Mockito.doReturn(true).when(mockedObj).hasResult();
|
|
|
|
+ Mockito.doReturn(op).when(mockedObj).getResult();
|
|
|
|
+ Mockito.doReturn(HTTP_CONFLICT).when(op).getStatusCode();
|
|
|
|
+ headers.addAll(mockedObj.getRequestHeaders());
|
|
|
|
+ throw new AbfsRestOperationException(HTTP_CONFLICT,
|
|
|
|
+ AzureServiceErrorCode.PATH_CONFLICT.getErrorCode(), EMPTY_STRING,
|
|
|
|
+ null, op);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ });
|
|
|
|
+ }
|
|
}
|
|
}
|