|
@@ -18,14 +18,21 @@ package org.apache.hadoop.ozone.om;
|
|
|
|
|
|
|
|
|
import org.apache.commons.lang3.RandomStringUtils;
|
|
|
+import org.apache.hadoop.hdds.client.ReplicationFactor;
|
|
|
+import org.apache.hadoop.hdds.client.ReplicationType;
|
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
|
import org.apache.hadoop.hdfs.LogVerificationAppender;
|
|
|
import org.apache.hadoop.ipc.RemoteException;
|
|
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
|
|
import org.apache.hadoop.ozone.MiniOzoneHAClusterImpl;
|
|
|
import org.apache.hadoop.ozone.client.ObjectStore;
|
|
|
+import org.apache.hadoop.ozone.client.OzoneBucket;
|
|
|
import org.apache.hadoop.ozone.client.OzoneClient;
|
|
|
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
|
|
|
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
|
|
import org.apache.hadoop.ozone.om.ha.OMFailoverProxyProvider;
|
|
|
+import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
|
|
|
+import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.ozone.client.OzoneClientFactory;
|
|
|
import org.apache.hadoop.ozone.client.OzoneVolume;
|
|
@@ -42,7 +49,9 @@ import org.junit.rules.Timeout;
|
|
|
import java.io.IOException;
|
|
|
import java.net.ConnectException;
|
|
|
import java.net.InetSocketAddress;
|
|
|
+import java.util.HashMap;
|
|
|
import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
import java.util.UUID;
|
|
|
|
|
|
import static org.apache.hadoop.ozone.MiniOzoneHAClusterImpl
|
|
@@ -120,6 +129,7 @@ public class TestOzoneManagerHA {
|
|
|
@Test
|
|
|
public void testAllOMNodesRunning() throws Exception {
|
|
|
createVolumeTest(true);
|
|
|
+ createKeyTest(true);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -131,6 +141,8 @@ public class TestOzoneManagerHA {
|
|
|
Thread.sleep(NODE_FAILURE_TIMEOUT * 2);
|
|
|
|
|
|
createVolumeTest(true);
|
|
|
+
|
|
|
+ createKeyTest(true);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -143,8 +155,181 @@ public class TestOzoneManagerHA {
|
|
|
Thread.sleep(NODE_FAILURE_TIMEOUT * 2);
|
|
|
|
|
|
createVolumeTest(false);
|
|
|
+
|
|
|
+ createKeyTest(false);
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
+ private OzoneBucket setupBucket() throws Exception {
|
|
|
+ String userName = "user" + RandomStringUtils.randomNumeric(5);
|
|
|
+ String adminName = "admin" + RandomStringUtils.randomNumeric(5);
|
|
|
+ String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
|
|
+
|
|
|
+ VolumeArgs createVolumeArgs = VolumeArgs.newBuilder()
|
|
|
+ .setOwner(userName)
|
|
|
+ .setAdmin(adminName)
|
|
|
+ .build();
|
|
|
+
|
|
|
+ objectStore.createVolume(volumeName, createVolumeArgs);
|
|
|
+ OzoneVolume retVolumeinfo = objectStore.getVolume(volumeName);
|
|
|
+
|
|
|
+ Assert.assertTrue(retVolumeinfo.getName().equals(volumeName));
|
|
|
+ Assert.assertTrue(retVolumeinfo.getOwner().equals(userName));
|
|
|
+ Assert.assertTrue(retVolumeinfo.getAdmin().equals(adminName));
|
|
|
+
|
|
|
+ String bucketName = UUID.randomUUID().toString();
|
|
|
+ retVolumeinfo.createBucket(bucketName);
|
|
|
+
|
|
|
+ OzoneBucket ozoneBucket = retVolumeinfo.getBucket(bucketName);
|
|
|
+
|
|
|
+ Assert.assertTrue(ozoneBucket.getName().equals(bucketName));
|
|
|
+ Assert.assertTrue(ozoneBucket.getVolumeName().equals(volumeName));
|
|
|
+
|
|
|
+ return ozoneBucket;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testMultipartUpload() throws Exception {
|
|
|
+
|
|
|
+ // Happy scenario when all OM's are up.
|
|
|
+ OzoneBucket ozoneBucket = setupBucket();
|
|
|
+
|
|
|
+ String keyName = UUID.randomUUID().toString();
|
|
|
+ String uploadID = initiateMultipartUpload(ozoneBucket, keyName);
|
|
|
+
|
|
|
+ createMultipartKeyAndReadKey(ozoneBucket, keyName, uploadID);
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testMultipartUploadWithOneOmNodeDown() throws Exception {
|
|
|
+
|
|
|
+ OzoneBucket ozoneBucket = setupBucket();
|
|
|
+
|
|
|
+ String keyName = UUID.randomUUID().toString();
|
|
|
+ String uploadID = initiateMultipartUpload(ozoneBucket, keyName);
|
|
|
+
|
|
|
+ // After initiate multipartupload, shutdown leader OM.
|
|
|
+ // Stop leader OM, to see when the OM leader changes
|
|
|
+ // multipart upload is happening successfully or not.
|
|
|
+
|
|
|
+ OMFailoverProxyProvider omFailoverProxyProvider =
|
|
|
+ objectStore.getClientProxy().getOMProxyProvider();
|
|
|
+
|
|
|
+ // The OMFailoverProxyProvider will point to the current leader OM node.
|
|
|
+ String leaderOMNodeId = omFailoverProxyProvider.getCurrentProxyOMNodeId();
|
|
|
+
|
|
|
+ // Stop one of the ozone manager, to see when the OM leader changes
|
|
|
+ // multipart upload is happening successfully or not.
|
|
|
+ cluster.stopOzoneManager(leaderOMNodeId);
|
|
|
+
|
|
|
+
|
|
|
+ createMultipartKeyAndReadKey(ozoneBucket, keyName, uploadID);
|
|
|
+
|
|
|
+ String newLeaderOMNodeId =
|
|
|
+ omFailoverProxyProvider.getCurrentProxyOMNodeId();
|
|
|
+
|
|
|
+ Assert.assertTrue(leaderOMNodeId != newLeaderOMNodeId);
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+ private String initiateMultipartUpload(OzoneBucket ozoneBucket,
|
|
|
+ String keyName) throws Exception {
|
|
|
+
|
|
|
+ OmMultipartInfo omMultipartInfo =
|
|
|
+ ozoneBucket.initiateMultipartUpload(keyName,
|
|
|
+ ReplicationType.RATIS,
|
|
|
+ ReplicationFactor.ONE);
|
|
|
+
|
|
|
+ String uploadID = omMultipartInfo.getUploadID();
|
|
|
+ Assert.assertTrue(uploadID != null);
|
|
|
+ return uploadID;
|
|
|
}
|
|
|
|
|
|
+ private void createMultipartKeyAndReadKey(OzoneBucket ozoneBucket,
|
|
|
+ String keyName, String uploadID) throws Exception {
|
|
|
+
|
|
|
+ String value = "random data";
|
|
|
+ OzoneOutputStream ozoneOutputStream = ozoneBucket.createMultipartKey(
|
|
|
+ keyName, value.length(), 1, uploadID);
|
|
|
+ ozoneOutputStream.write(value.getBytes(), 0, value.length());
|
|
|
+ ozoneOutputStream.close();
|
|
|
+
|
|
|
+
|
|
|
+ Map<Integer, String> partsMap = new HashMap<>();
|
|
|
+ partsMap.put(1, ozoneOutputStream.getCommitUploadPartInfo().getPartName());
|
|
|
+ OmMultipartUploadCompleteInfo omMultipartUploadCompleteInfo =
|
|
|
+ ozoneBucket.completeMultipartUpload(keyName, uploadID, partsMap);
|
|
|
+
|
|
|
+ Assert.assertTrue(omMultipartUploadCompleteInfo != null);
|
|
|
+ Assert.assertTrue(omMultipartUploadCompleteInfo.getHash() != null);
|
|
|
+
|
|
|
+
|
|
|
+ OzoneInputStream ozoneInputStream = ozoneBucket.readKey(keyName);
|
|
|
+
|
|
|
+ byte[] fileContent = new byte[value.getBytes().length];
|
|
|
+ ozoneInputStream.read(fileContent);
|
|
|
+ Assert.assertEquals(value, new String(fileContent));
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+ private void createKeyTest(boolean checkSuccess) throws Exception {
|
|
|
+ String userName = "user" + RandomStringUtils.randomNumeric(5);
|
|
|
+ String adminName = "admin" + RandomStringUtils.randomNumeric(5);
|
|
|
+ String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
|
|
+
|
|
|
+ VolumeArgs createVolumeArgs = VolumeArgs.newBuilder()
|
|
|
+ .setOwner(userName)
|
|
|
+ .setAdmin(adminName)
|
|
|
+ .build();
|
|
|
+
|
|
|
+ try {
|
|
|
+ objectStore.createVolume(volumeName, createVolumeArgs);
|
|
|
+
|
|
|
+ OzoneVolume retVolumeinfo = objectStore.getVolume(volumeName);
|
|
|
+
|
|
|
+ Assert.assertTrue(retVolumeinfo.getName().equals(volumeName));
|
|
|
+ Assert.assertTrue(retVolumeinfo.getOwner().equals(userName));
|
|
|
+ Assert.assertTrue(retVolumeinfo.getAdmin().equals(adminName));
|
|
|
+
|
|
|
+ String bucketName = UUID.randomUUID().toString();
|
|
|
+ String keyName = UUID.randomUUID().toString();
|
|
|
+ retVolumeinfo.createBucket(bucketName);
|
|
|
+
|
|
|
+ OzoneBucket ozoneBucket = retVolumeinfo.getBucket(bucketName);
|
|
|
+
|
|
|
+ Assert.assertTrue(ozoneBucket.getName().equals(bucketName));
|
|
|
+ Assert.assertTrue(ozoneBucket.getVolumeName().equals(volumeName));
|
|
|
+
|
|
|
+ String value = "random data";
|
|
|
+ OzoneOutputStream ozoneOutputStream = ozoneBucket.createKey(keyName,
|
|
|
+ value.length(), ReplicationType.STAND_ALONE,
|
|
|
+ ReplicationFactor.ONE, new HashMap<>());
|
|
|
+ ozoneOutputStream.write(value.getBytes(), 0, value.length());
|
|
|
+ ozoneOutputStream.close();
|
|
|
+
|
|
|
+ OzoneInputStream ozoneInputStream = ozoneBucket.readKey(keyName);
|
|
|
+
|
|
|
+ byte[] fileContent = new byte[value.getBytes().length];
|
|
|
+ ozoneInputStream.read(fileContent);
|
|
|
+ Assert.assertEquals(value, new String(fileContent));
|
|
|
+
|
|
|
+ } catch (ConnectException | RemoteException e) {
|
|
|
+ if (!checkSuccess) {
|
|
|
+ // If the last OM to be tried by the RetryProxy is down, we would get
|
|
|
+ // ConnectException. Otherwise, we would get a RemoteException from the
|
|
|
+ // last running OM as it would fail to get a quorum.
|
|
|
+ if (e instanceof RemoteException) {
|
|
|
+ GenericTestUtils.assertExceptionContains(
|
|
|
+ "RaftRetryFailureException", e);
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+ }
|
|
|
/**
|
|
|
* Create a volume and test its attribute.
|
|
|
*/
|
|
@@ -186,6 +371,8 @@ public class TestOzoneManagerHA {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+
|
|
|
+
|
|
|
/**
|
|
|
* Test that OMFailoverProxyProvider creates an OM proxy for each OM in the
|
|
|
* cluster.
|