|
@@ -20,15 +20,31 @@ package org.apache.hadoop.fs.azurebfs;
|
|
|
|
|
|
import java.io.FileNotFoundException;
|
|
|
import java.io.IOException;
|
|
|
+import java.io.OutputStream;
|
|
|
+import java.util.HashSet;
|
|
|
import java.util.Random;
|
|
|
+import java.util.Set;
|
|
|
|
|
|
+import org.assertj.core.api.Assertions;
|
|
|
import org.junit.Test;
|
|
|
+import org.mockito.Mockito;
|
|
|
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
+import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
|
|
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
|
|
|
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
|
|
+import org.apache.hadoop.fs.store.BlockUploadStatistics;
|
|
|
+import org.apache.hadoop.fs.store.DataBlocks;
|
|
|
+
|
|
|
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.DATA_BLOCKS_BUFFER;
|
|
|
+import static org.apache.hadoop.fs.store.DataBlocks.DATA_BLOCKS_BUFFER_ARRAY;
|
|
|
+import static org.apache.hadoop.fs.store.DataBlocks.DATA_BLOCKS_BUFFER_DISK;
|
|
|
+import static org.apache.hadoop.fs.store.DataBlocks.DATA_BLOCKS_BYTEBUFFER;
|
|
|
+import static org.apache.hadoop.fs.store.DataBlocks.DataBlock.DestState.Closed;
|
|
|
+import static org.apache.hadoop.fs.store.DataBlocks.DataBlock.DestState.Writing;
|
|
|
|
|
|
/**
|
|
|
* Test append operations.
|
|
@@ -90,4 +106,47 @@ public class ITestAzureBlobFileSystemAppend extends
|
|
|
fs.getFileSystemId(), FSOperationType.APPEND, false, 0));
|
|
|
fs.append(testPath, 10);
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testCloseOfDataBlockOnAppendComplete() throws Exception {
|
|
|
+ Set<String> blockBufferTypes = new HashSet<>();
|
|
|
+ blockBufferTypes.add(DATA_BLOCKS_BUFFER_DISK);
|
|
|
+ blockBufferTypes.add(DATA_BLOCKS_BYTEBUFFER);
|
|
|
+ blockBufferTypes.add(DATA_BLOCKS_BUFFER_ARRAY);
|
|
|
+ for (String blockBufferType : blockBufferTypes) {
|
|
|
+ Configuration configuration = new Configuration(getRawConfiguration());
|
|
|
+ configuration.set(DATA_BLOCKS_BUFFER, blockBufferType);
|
|
|
+ AzureBlobFileSystem fs = Mockito.spy(
|
|
|
+ (AzureBlobFileSystem) FileSystem.newInstance(configuration));
|
|
|
+ AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
|
|
|
+ Mockito.doReturn(store).when(fs).getAbfsStore();
|
|
|
+ DataBlocks.DataBlock[] dataBlock = new DataBlocks.DataBlock[1];
|
|
|
+ Mockito.doAnswer(getBlobFactoryInvocation -> {
|
|
|
+ DataBlocks.BlockFactory factory = Mockito.spy(
|
|
|
+ (DataBlocks.BlockFactory) getBlobFactoryInvocation.callRealMethod());
|
|
|
+ Mockito.doAnswer(factoryCreateInvocation -> {
|
|
|
+ dataBlock[0] = Mockito.spy(
|
|
|
+ (DataBlocks.DataBlock) factoryCreateInvocation.callRealMethod());
|
|
|
+ return dataBlock[0];
|
|
|
+ })
|
|
|
+ .when(factory)
|
|
|
+ .create(Mockito.anyLong(), Mockito.anyInt(), Mockito.any(
|
|
|
+ BlockUploadStatistics.class));
|
|
|
+ return factory;
|
|
|
+ }).when(store).getBlockFactory();
|
|
|
+ try (OutputStream os = fs.create(
|
|
|
+ new Path(getMethodName() + "_" + blockBufferType))) {
|
|
|
+ os.write(new byte[1]);
|
|
|
+ Assertions.assertThat(dataBlock[0].getState())
|
|
|
+ .describedAs(
|
|
|
+ "On write of data in outputStream, state should become Writing")
|
|
|
+ .isEqualTo(Writing);
|
|
|
+ os.close();
|
|
|
+ Mockito.verify(dataBlock[0], Mockito.times(1)).close();
|
|
|
+ Assertions.assertThat(dataBlock[0].getState())
|
|
|
+ .describedAs("On close of outputStream, state should become Closed")
|
|
|
+ .isEqualTo(Closed);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|