|
@@ -124,7 +124,7 @@ public class TestOzoneContainer {
|
|
pipeline, conf);
|
|
pipeline, conf);
|
|
|
|
|
|
try {
|
|
try {
|
|
- runTestOzoneContainerViaDataNode(containerName, pipeline, client);
|
|
|
|
|
|
+ runTestOzoneContainerViaDataNode(containerName, client);
|
|
} finally {
|
|
} finally {
|
|
cluster.shutdown();
|
|
cluster.shutdown();
|
|
}
|
|
}
|
|
@@ -170,7 +170,7 @@ public class TestOzoneContainer {
|
|
// This client talks to ozone container via datanode.
|
|
// This client talks to ozone container via datanode.
|
|
XceiverClient client = new XceiverClient(pipeline, conf);
|
|
XceiverClient client = new XceiverClient(pipeline, conf);
|
|
|
|
|
|
- runTestOzoneContainerViaDataNode(containerName, pipeline, client);
|
|
|
|
|
|
+ runTestOzoneContainerViaDataNode(containerName, client);
|
|
} finally {
|
|
} finally {
|
|
if (cluster != null) {
|
|
if (cluster != null) {
|
|
cluster.shutdown();
|
|
cluster.shutdown();
|
|
@@ -178,30 +178,19 @@ public class TestOzoneContainer {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- static void runTestOzoneContainerViaDataNode(
|
|
|
|
- String containerName, Pipeline pipeline, XceiverClientSpi client)
|
|
|
|
- throws Exception {
|
|
|
|
|
|
+ static void runTestOzoneContainerViaDataNode(String containerName,
|
|
|
|
+ XceiverClientSpi client) throws Exception {
|
|
|
|
+ ContainerProtos.ContainerCommandRequestProto
|
|
|
|
+ request, writeChunkRequest, putKeyRequest,
|
|
|
|
+ updateRequest1, updateRequest2;
|
|
|
|
+ ContainerProtos.ContainerCommandResponseProto response,
|
|
|
|
+ updateResponse1, updateResponse2;
|
|
try {
|
|
try {
|
|
client.connect();
|
|
client.connect();
|
|
|
|
|
|
// Create container
|
|
// Create container
|
|
- ContainerProtos.ContainerCommandRequestProto request =
|
|
|
|
- ContainerTestHelper.getCreateContainerRequest(containerName);
|
|
|
|
- ContainerProtos.ContainerCommandResponseProto response =
|
|
|
|
- client.sendCommand(request);
|
|
|
|
- Assert.assertNotNull(response);
|
|
|
|
- Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
|
|
|
|
-
|
|
|
|
- // Write Chunk
|
|
|
|
- final String keyName = OzoneUtils.getRequestID();
|
|
|
|
- ContainerProtos.ContainerCommandRequestProto writeChunkRequest =
|
|
|
|
- ContainerTestHelper.getWriteChunkRequest(pipeline, containerName,
|
|
|
|
- keyName, 1024);
|
|
|
|
-
|
|
|
|
- response = client.sendCommand(writeChunkRequest);
|
|
|
|
- Assert.assertNotNull(response);
|
|
|
|
- Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult());
|
|
|
|
- Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
|
|
|
|
|
|
+ createContainerForTesting(client, containerName);
|
|
|
|
+ writeChunkRequest = writeChunkForContainer(client, containerName, 1024);
|
|
|
|
|
|
// Read Chunk
|
|
// Read Chunk
|
|
request = ContainerTestHelper.getReadChunkRequest(writeChunkRequest
|
|
request = ContainerTestHelper.getReadChunkRequest(writeChunkRequest
|
|
@@ -213,8 +202,7 @@ public class TestOzoneContainer {
|
|
Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
|
|
Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
|
|
|
|
|
|
// Put Key
|
|
// Put Key
|
|
- ContainerProtos.ContainerCommandRequestProto putKeyRequest =
|
|
|
|
- ContainerTestHelper.getPutKeyRequest(writeChunkRequest
|
|
|
|
|
|
+ putKeyRequest = ContainerTestHelper.getPutKeyRequest(writeChunkRequest
|
|
.getWriteChunk());
|
|
.getWriteChunk());
|
|
|
|
|
|
|
|
|
|
@@ -249,21 +237,17 @@ public class TestOzoneContainer {
|
|
//Update an existing container
|
|
//Update an existing container
|
|
Map<String, String> containerUpdate = new HashMap<String, String>();
|
|
Map<String, String> containerUpdate = new HashMap<String, String>();
|
|
containerUpdate.put("container_updated_key", "container_updated_value");
|
|
containerUpdate.put("container_updated_key", "container_updated_value");
|
|
- ContainerProtos.ContainerCommandRequestProto updateRequest1 =
|
|
|
|
- ContainerTestHelper.getUpdateContainerRequest(
|
|
|
|
|
|
+ updateRequest1 = ContainerTestHelper.getUpdateContainerRequest(
|
|
containerName, containerUpdate);
|
|
containerName, containerUpdate);
|
|
- ContainerProtos.ContainerCommandResponseProto updateResponse1 =
|
|
|
|
- client.sendCommand(updateRequest1);
|
|
|
|
|
|
+ updateResponse1 = client.sendCommand(updateRequest1);
|
|
Assert.assertNotNull(updateResponse1);
|
|
Assert.assertNotNull(updateResponse1);
|
|
Assert.assertEquals(ContainerProtos.Result.SUCCESS,
|
|
Assert.assertEquals(ContainerProtos.Result.SUCCESS,
|
|
response.getResult());
|
|
response.getResult());
|
|
|
|
|
|
//Update an non-existing container
|
|
//Update an non-existing container
|
|
- ContainerProtos.ContainerCommandRequestProto updateRequest2 =
|
|
|
|
- ContainerTestHelper.getUpdateContainerRequest(
|
|
|
|
|
|
+ updateRequest2 = ContainerTestHelper.getUpdateContainerRequest(
|
|
"non_exist_container", containerUpdate);
|
|
"non_exist_container", containerUpdate);
|
|
- ContainerProtos.ContainerCommandResponseProto updateResponse2 =
|
|
|
|
- client.sendCommand(updateRequest2);
|
|
|
|
|
|
+ updateResponse2 = client.sendCommand(updateRequest2);
|
|
Assert.assertEquals(ContainerProtos.Result.CONTAINER_NOT_FOUND,
|
|
Assert.assertEquals(ContainerProtos.Result.CONTAINER_NOT_FOUND,
|
|
updateResponse2.getResult());
|
|
updateResponse2.getResult());
|
|
} finally {
|
|
} finally {
|
|
@@ -277,52 +261,31 @@ public class TestOzoneContainer {
|
|
public void testBothGetandPutSmallFile() throws Exception {
|
|
public void testBothGetandPutSmallFile() throws Exception {
|
|
MiniOzoneCluster cluster = null;
|
|
MiniOzoneCluster cluster = null;
|
|
XceiverClient client = null;
|
|
XceiverClient client = null;
|
|
|
|
+ ContainerProtos.ContainerCommandResponseProto response;
|
|
|
|
+ ContainerProtos.ContainerCommandRequestProto
|
|
|
|
+ smallFileRequest, getSmallFileRequest;
|
|
try {
|
|
try {
|
|
String keyName = OzoneUtils.getRequestID();
|
|
String keyName = OzoneUtils.getRequestID();
|
|
- String containerName = OzoneUtils.getRequestID();
|
|
|
|
OzoneConfiguration conf = new OzoneConfiguration();
|
|
OzoneConfiguration conf = new OzoneConfiguration();
|
|
- URL p = conf.getClass().getResource("");
|
|
|
|
- String path = p.getPath().concat(
|
|
|
|
- TestOzoneContainer.class.getSimpleName());
|
|
|
|
- path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
|
|
|
|
- OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
|
|
|
|
- conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
|
|
|
|
-
|
|
|
|
- // Start ozone container Via Datanode create.
|
|
|
|
-
|
|
|
|
- Pipeline pipeline =
|
|
|
|
- ContainerTestHelper.createSingleNodePipeline(containerName);
|
|
|
|
- conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
|
|
|
|
- pipeline.getLeader().getContainerPort());
|
|
|
|
|
|
|
|
|
|
+ client = createClientForTesting(conf);
|
|
cluster = new MiniOzoneCluster.Builder(conf)
|
|
cluster = new MiniOzoneCluster.Builder(conf)
|
|
.setRandomContainerPort(false)
|
|
.setRandomContainerPort(false)
|
|
.setHandlerType("distributed").build();
|
|
.setHandlerType("distributed").build();
|
|
-
|
|
|
|
- // This client talks to ozone container via datanode.
|
|
|
|
- client = new XceiverClient(pipeline, conf);
|
|
|
|
client.connect();
|
|
client.connect();
|
|
|
|
|
|
- // Create container
|
|
|
|
- ContainerProtos.ContainerCommandRequestProto request =
|
|
|
|
- ContainerTestHelper.getCreateContainerRequest(containerName);
|
|
|
|
- ContainerProtos.ContainerCommandResponseProto response =
|
|
|
|
- client.sendCommand(request);
|
|
|
|
- Assert.assertNotNull(response);
|
|
|
|
- Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
|
|
|
|
-
|
|
|
|
-
|
|
|
|
- ContainerProtos.ContainerCommandRequestProto smallFileRequest =
|
|
|
|
- ContainerTestHelper.getWriteSmallFileRequest(pipeline, containerName,
|
|
|
|
- keyName, 1024);
|
|
|
|
|
|
+ String containerName = client.getPipeline().getContainerName();
|
|
|
|
+ createContainerForTesting(client, containerName);
|
|
|
|
|
|
|
|
+ smallFileRequest = ContainerTestHelper.getWriteSmallFileRequest(
|
|
|
|
+ client.getPipeline(), containerName, keyName, 1024);
|
|
|
|
|
|
response = client.sendCommand(smallFileRequest);
|
|
response = client.sendCommand(smallFileRequest);
|
|
Assert.assertNotNull(response);
|
|
Assert.assertNotNull(response);
|
|
Assert.assertTrue(smallFileRequest.getTraceID()
|
|
Assert.assertTrue(smallFileRequest.getTraceID()
|
|
.equals(response.getTraceID()));
|
|
.equals(response.getTraceID()));
|
|
|
|
|
|
- ContainerProtos.ContainerCommandRequestProto getSmallFileRequest =
|
|
|
|
|
|
+ getSmallFileRequest =
|
|
ContainerTestHelper.getReadSmallFileRequest(smallFileRequest
|
|
ContainerTestHelper.getReadSmallFileRequest(smallFileRequest
|
|
.getPutSmallFile().getKey());
|
|
.getPutSmallFile().getKey());
|
|
response = client.sendCommand(getSmallFileRequest);
|
|
response = client.sendCommand(getSmallFileRequest);
|
|
@@ -343,58 +306,25 @@ public class TestOzoneContainer {
|
|
public void testCloseContainer() throws Exception {
|
|
public void testCloseContainer() throws Exception {
|
|
MiniOzoneCluster cluster = null;
|
|
MiniOzoneCluster cluster = null;
|
|
XceiverClient client = null;
|
|
XceiverClient client = null;
|
|
|
|
+ ContainerProtos.ContainerCommandResponseProto response;
|
|
|
|
+ ContainerProtos.ContainerCommandRequestProto
|
|
|
|
+ writeChunkRequest, putKeyRequest, request;
|
|
try {
|
|
try {
|
|
|
|
|
|
- String keyName = OzoneUtils.getRequestID();
|
|
|
|
- String containerName = OzoneUtils.getRequestID();
|
|
|
|
OzoneConfiguration conf = new OzoneConfiguration();
|
|
OzoneConfiguration conf = new OzoneConfiguration();
|
|
- URL p = conf.getClass().getResource("");
|
|
|
|
- String path = p.getPath().concat(
|
|
|
|
- TestOzoneContainer.class.getSimpleName());
|
|
|
|
- path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
|
|
|
|
- OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
|
|
|
|
- conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
|
|
|
|
-
|
|
|
|
- // Start ozone container Via Datanode create.
|
|
|
|
-
|
|
|
|
- Pipeline pipeline =
|
|
|
|
- ContainerTestHelper.createSingleNodePipeline(containerName);
|
|
|
|
- conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
|
|
|
|
- pipeline.getLeader().getContainerPort());
|
|
|
|
|
|
|
|
|
|
+ client = createClientForTesting(conf);
|
|
cluster = new MiniOzoneCluster.Builder(conf)
|
|
cluster = new MiniOzoneCluster.Builder(conf)
|
|
.setRandomContainerPort(false)
|
|
.setRandomContainerPort(false)
|
|
.setHandlerType("distributed").build();
|
|
.setHandlerType("distributed").build();
|
|
-
|
|
|
|
- // This client talks to ozone container via datanode.
|
|
|
|
- client = new XceiverClient(pipeline, conf);
|
|
|
|
client.connect();
|
|
client.connect();
|
|
|
|
|
|
|
|
+ String containerName = client.getPipeline().getContainerName();
|
|
|
|
+ createContainerForTesting(client, containerName);
|
|
|
|
+ writeChunkRequest = writeChunkForContainer(client, containerName, 1024);
|
|
|
|
|
|
|
|
|
|
- // Create container
|
|
|
|
- ContainerProtos.ContainerCommandRequestProto request =
|
|
|
|
- ContainerTestHelper.getCreateContainerRequest(containerName);
|
|
|
|
- ContainerProtos.ContainerCommandResponseProto response =
|
|
|
|
- client.sendCommand(request);
|
|
|
|
- Assert.assertNotNull(response);
|
|
|
|
- Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
|
|
|
|
-
|
|
|
|
- ContainerProtos.ContainerCommandRequestProto writeChunkRequest =
|
|
|
|
- ContainerTestHelper.getWriteChunkRequest(pipeline, containerName,
|
|
|
|
- keyName, 1024);
|
|
|
|
-
|
|
|
|
- // Write Chunk before closing
|
|
|
|
- response = client.sendCommand(writeChunkRequest);
|
|
|
|
- Assert.assertNotNull(response);
|
|
|
|
- Assert.assertEquals(ContainerProtos.Result.SUCCESS,
|
|
|
|
- response.getResult());
|
|
|
|
- Assert.assertTrue(writeChunkRequest.getTraceID().equals(response
|
|
|
|
- .getTraceID()));
|
|
|
|
-
|
|
|
|
-
|
|
|
|
- ContainerProtos.ContainerCommandRequestProto putKeyRequest =
|
|
|
|
- ContainerTestHelper.getPutKeyRequest(writeChunkRequest
|
|
|
|
|
|
+ putKeyRequest = ContainerTestHelper.getPutKeyRequest(writeChunkRequest
|
|
.getWriteChunk());
|
|
.getWriteChunk());
|
|
// Put key before closing.
|
|
// Put key before closing.
|
|
response = client.sendCommand(putKeyRequest);
|
|
response = client.sendCommand(putKeyRequest);
|
|
@@ -405,7 +335,7 @@ public class TestOzoneContainer {
|
|
putKeyRequest.getTraceID().equals(response.getTraceID()));
|
|
putKeyRequest.getTraceID().equals(response.getTraceID()));
|
|
|
|
|
|
// Close the contianer.
|
|
// Close the contianer.
|
|
- request = ContainerTestHelper.getCloseContainer(pipeline);
|
|
|
|
|
|
+ request = ContainerTestHelper.getCloseContainer(client.getPipeline());
|
|
response = client.sendCommand(request);
|
|
response = client.sendCommand(request);
|
|
Assert.assertNotNull(response);
|
|
Assert.assertNotNull(response);
|
|
Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult());
|
|
Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult());
|
|
@@ -460,4 +390,133 @@ public class TestOzoneContainer {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testDeleteContainer() throws Exception {
|
|
|
|
+ MiniOzoneCluster cluster = null;
|
|
|
|
+ XceiverClient client = null;
|
|
|
|
+ ContainerProtos.ContainerCommandResponseProto response;
|
|
|
|
+ ContainerProtos.ContainerCommandRequestProto request,
|
|
|
|
+ writeChunkRequest, putKeyRequest;
|
|
|
|
+ try {
|
|
|
|
+ OzoneConfiguration conf = new OzoneConfiguration();
|
|
|
|
+
|
|
|
|
+ client = createClientForTesting(conf);
|
|
|
|
+ cluster = new MiniOzoneCluster.Builder(conf)
|
|
|
|
+ .setRandomContainerPort(false)
|
|
|
|
+ .setHandlerType("distributed").build();
|
|
|
|
+ client.connect();
|
|
|
|
+
|
|
|
|
+ String containerName = client.getPipeline().getContainerName();
|
|
|
|
+ createContainerForTesting(client, containerName);
|
|
|
|
+ writeChunkRequest = writeChunkForContainer(client, containerName, 1024);
|
|
|
|
+
|
|
|
|
+ putKeyRequest = ContainerTestHelper.getPutKeyRequest(writeChunkRequest
|
|
|
|
+ .getWriteChunk());
|
|
|
|
+ // Put key before deleting.
|
|
|
|
+ response = client.sendCommand(putKeyRequest);
|
|
|
|
+ Assert.assertNotNull(response);
|
|
|
|
+ Assert.assertEquals(ContainerProtos.Result.SUCCESS,
|
|
|
|
+ response.getResult());
|
|
|
|
+ Assert.assertTrue(
|
|
|
|
+ putKeyRequest.getTraceID().equals(response.getTraceID()));
|
|
|
|
+
|
|
|
|
+ // Container cannot be deleted because the container is not empty.
|
|
|
|
+ request = ContainerTestHelper.getDeleteContainer(
|
|
|
|
+ client.getPipeline(), false);
|
|
|
|
+ response = client.sendCommand(request);
|
|
|
|
+
|
|
|
|
+ Assert.assertNotNull(response);
|
|
|
|
+ Assert.assertEquals(ContainerProtos.Result.ERROR_CONTAINER_NOT_EMPTY,
|
|
|
|
+ response.getResult());
|
|
|
|
+ Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
|
|
|
|
+
|
|
|
|
+ // Container cannot be deleted forcibly because
|
|
|
|
+ // the container is not closed.
|
|
|
|
+ request = ContainerTestHelper.getDeleteContainer(
|
|
|
|
+ client.getPipeline(), true);
|
|
|
|
+ response = client.sendCommand(request);
|
|
|
|
+
|
|
|
|
+ Assert.assertNotNull(response);
|
|
|
|
+ Assert.assertEquals(ContainerProtos.Result.UNCLOSED_CONTAINER_IO,
|
|
|
|
+ response.getResult());
|
|
|
|
+ Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
|
|
|
|
+
|
|
|
|
+ // Close the container.
|
|
|
|
+ request = ContainerTestHelper.getCloseContainer(client.getPipeline());
|
|
|
|
+ response = client.sendCommand(request);
|
|
|
|
+ Assert.assertNotNull(response);
|
|
|
|
+ Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult());
|
|
|
|
+ Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
|
|
|
|
+
|
|
|
|
+ // Container can be deleted forcibly because
|
|
|
|
+ // it has been closed.
|
|
|
|
+ request = ContainerTestHelper.getDeleteContainer(
|
|
|
|
+ client.getPipeline(), true);
|
|
|
|
+ response = client.sendCommand(request);
|
|
|
|
+
|
|
|
|
+ Assert.assertNotNull(response);
|
|
|
|
+ Assert.assertEquals(ContainerProtos.Result.SUCCESS,
|
|
|
|
+ response.getResult());
|
|
|
|
+ Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
|
|
|
|
+
|
|
|
|
+ } finally {
|
|
|
|
+ if (client != null) {
|
|
|
|
+ client.close();
|
|
|
|
+ }
|
|
|
|
+ if (cluster != null) {
|
|
|
|
+ cluster.shutdown();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private XceiverClient createClientForTesting(OzoneConfiguration conf)
|
|
|
|
+ throws Exception {
|
|
|
|
+
|
|
|
|
+ String containerName = OzoneUtils.getRequestID();
|
|
|
|
+ URL p = conf.getClass().getResource("");
|
|
|
|
+ String path = p.getPath().concat(
|
|
|
|
+ TestOzoneContainer.class.getSimpleName());
|
|
|
|
+ path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
|
|
|
|
+ OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
|
|
|
|
+ conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
|
|
|
|
+
|
|
|
|
+ // Start ozone container Via Datanode create.
|
|
|
|
+
|
|
|
|
+ Pipeline pipeline =
|
|
|
|
+ ContainerTestHelper.createSingleNodePipeline(containerName);
|
|
|
|
+ conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
|
|
|
|
+ pipeline.getLeader().getContainerPort());
|
|
|
|
+
|
|
|
|
+ // This client talks to ozone container via datanode.
|
|
|
|
+ return new XceiverClient(pipeline, conf);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private static void createContainerForTesting(XceiverClientSpi client,
|
|
|
|
+ String containerName) throws Exception {
|
|
|
|
+ // Create container
|
|
|
|
+ ContainerProtos.ContainerCommandRequestProto request =
|
|
|
|
+ ContainerTestHelper.getCreateContainerRequest(containerName);
|
|
|
|
+ ContainerProtos.ContainerCommandResponseProto response =
|
|
|
|
+ client.sendCommand(request);
|
|
|
|
+ Assert.assertNotNull(response);
|
|
|
|
+ Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private static ContainerProtos.ContainerCommandRequestProto
|
|
|
|
+ writeChunkForContainer(XceiverClientSpi client,
|
|
|
|
+ String containerName, int dataLen) throws Exception {
|
|
|
|
+ // Write Chunk
|
|
|
|
+ final String keyName = OzoneUtils.getRequestID();
|
|
|
|
+ ContainerProtos.ContainerCommandRequestProto writeChunkRequest =
|
|
|
|
+ ContainerTestHelper.getWriteChunkRequest(client.getPipeline(),
|
|
|
|
+ containerName, keyName, dataLen);
|
|
|
|
+
|
|
|
|
+ ContainerProtos.ContainerCommandResponseProto response =
|
|
|
|
+ client.sendCommand(writeChunkRequest);
|
|
|
|
+ Assert.assertNotNull(response);
|
|
|
|
+ Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult());
|
|
|
|
+ Assert.assertTrue(response.getTraceID().equals(response.getTraceID()));
|
|
|
|
+ return writeChunkRequest;
|
|
|
|
+ }
|
|
}
|
|
}
|