|
@@ -19,25 +19,19 @@ package org.apache.hadoop.ozone.protocolPB;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
|
|
|
-import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
|
|
|
-import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
|
|
+import org.apache.hadoop.ozone.om.OzoneManager;
|
|
|
import org.apache.hadoop.ozone.om.helpers.OmDeleteVolumeResponse;
|
|
|
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
|
|
|
import org.apache.hadoop.ozone.om.helpers.OmVolumeOwnerChangeResponse;
|
|
|
-import org.apache.hadoop.ozone.om.protocol.OzoneManagerServerProtocol;
|
|
|
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer;
|
|
|
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
|
|
|
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
|
|
|
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
|
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
|
|
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
|
- .CreateBucketRequest;
|
|
|
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
|
- .CreateBucketResponse;
|
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
|
.CreateVolumeRequest;
|
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
|
.CreateVolumeResponse;
|
|
|
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
|
- .DeleteBucketRequest;
|
|
|
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
|
- .DeleteBucketResponse;
|
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
|
.DeleteVolumeRequest;
|
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
@@ -46,10 +40,6 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
|
.OMRequest;
|
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
|
.OMResponse;
|
|
|
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
|
- .SetBucketPropertyRequest;
|
|
|
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
|
- .SetBucketPropertyResponse;
|
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
|
.SetVolumePropertyRequest;
|
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
@@ -70,8 +60,12 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
|
public class OzoneManagerHARequestHandlerImpl
|
|
|
extends OzoneManagerRequestHandler implements OzoneManagerHARequestHandler {
|
|
|
|
|
|
- public OzoneManagerHARequestHandlerImpl(OzoneManagerServerProtocol om) {
|
|
|
+ private OzoneManagerDoubleBuffer ozoneManagerDoubleBuffer;
|
|
|
+
|
|
|
+ public OzoneManagerHARequestHandlerImpl(OzoneManager om,
|
|
|
+ OzoneManagerDoubleBuffer ozoneManagerDoubleBuffer) {
|
|
|
super(om);
|
|
|
+ this.ozoneManagerDoubleBuffer = ozoneManagerDoubleBuffer;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -90,15 +84,6 @@ public class OzoneManagerHARequestHandlerImpl
|
|
|
case DeleteVolume:
|
|
|
newOmRequest = handleDeleteVolumeStart(omRequest);
|
|
|
break;
|
|
|
- case CreateBucket:
|
|
|
- newOmRequest = handleCreateBucketStart(omRequest);
|
|
|
- break;
|
|
|
- case SetBucketProperty:
|
|
|
- newOmRequest = handleSetBucketPropertyStart(omRequest);
|
|
|
- break;
|
|
|
- case DeleteBucket:
|
|
|
- newOmRequest = handleDeleteBucketRequestStart(omRequest);
|
|
|
- break;
|
|
|
default:
|
|
|
throw new IOException("Unrecognized Command Type:" + cmdType);
|
|
|
}
|
|
@@ -107,7 +92,8 @@ public class OzoneManagerHARequestHandlerImpl
|
|
|
|
|
|
|
|
|
@Override
|
|
|
- public OMResponse handleApplyTransaction(OMRequest omRequest) {
|
|
|
+ public OMResponse handleApplyTransaction(OMRequest omRequest,
|
|
|
+ long transactionLogIndex) {
|
|
|
LOG.debug("Received OMRequest: {}, ", omRequest);
|
|
|
Type cmdType = omRequest.getCmdType();
|
|
|
OMResponse.Builder responseBuilder =
|
|
@@ -128,17 +114,26 @@ public class OzoneManagerHARequestHandlerImpl
|
|
|
handleDeleteVolumeApply(omRequest));
|
|
|
break;
|
|
|
case CreateBucket:
|
|
|
- responseBuilder.setCreateBucketResponse(
|
|
|
- handleCreateBucketApply(omRequest));
|
|
|
- break;
|
|
|
- case SetBucketProperty:
|
|
|
- responseBuilder.setSetBucketPropertyResponse(
|
|
|
- handleSetBucketPropertyApply(omRequest));
|
|
|
- break;
|
|
|
case DeleteBucket:
|
|
|
- responseBuilder.setDeleteBucketResponse(
|
|
|
- handleDeleteBucketApply(omRequest));
|
|
|
- break;
|
|
|
+ case SetBucketProperty:
|
|
|
+ //TODO: We don't need to pass transactionID, this will be removed when
|
|
|
+ // complete write requests is changed to new model. And also we can
|
|
|
+ // return OMClientResponse, then adding to doubleBuffer can be taken
|
|
|
+ // care by stateMachine. And also integrate both HA and NON HA code
|
|
|
+ // paths.
|
|
|
+ OMClientRequest omClientRequest =
|
|
|
+ OzoneManagerRatisUtils.createClientRequest(omRequest);
|
|
|
+ OMClientResponse omClientResponse =
|
|
|
+ omClientRequest.validateAndUpdateCache(getOzoneManager(),
|
|
|
+ transactionLogIndex);
|
|
|
+
|
|
|
+ // If any error we have got when validateAndUpdateCache, OMResponse
|
|
|
+ // Status is set with Error Code other than OK, in that case don't
|
|
|
+ // add this to double buffer.
|
|
|
+ if (omClientResponse.getOMResponse().getStatus() == Status.OK) {
|
|
|
+ ozoneManagerDoubleBuffer.add(omClientResponse, transactionLogIndex);
|
|
|
+ }
|
|
|
+ return omClientResponse.getOMResponse();
|
|
|
default:
|
|
|
// As all request types are not changed so we need to call handle
|
|
|
// here.
|
|
@@ -160,7 +155,7 @@ public class OzoneManagerHARequestHandlerImpl
|
|
|
throws IOException {
|
|
|
VolumeInfo volumeInfo = omRequest.getCreateVolumeRequest().getVolumeInfo();
|
|
|
OzoneManagerProtocolProtos.VolumeList volumeList =
|
|
|
- getOzoneManagerServerProtocol().startCreateVolume(
|
|
|
+ getOzoneManager().startCreateVolume(
|
|
|
OmVolumeArgs.getFromProtobuf(volumeInfo));
|
|
|
|
|
|
CreateVolumeRequest createVolumeRequest =
|
|
@@ -176,7 +171,7 @@ public class OzoneManagerHARequestHandlerImpl
|
|
|
omRequest.getCreateVolumeRequest().getVolumeInfo();
|
|
|
VolumeList volumeList =
|
|
|
omRequest.getCreateVolumeRequest().getVolumeList();
|
|
|
- getOzoneManagerServerProtocol().applyCreateVolume(
|
|
|
+ getOzoneManager().applyCreateVolume(
|
|
|
OmVolumeArgs.getFromProtobuf(volumeInfo),
|
|
|
volumeList);
|
|
|
return CreateVolumeResponse.newBuilder().build();
|
|
@@ -191,7 +186,7 @@ public class OzoneManagerHARequestHandlerImpl
|
|
|
if (setVolumePropertyRequest.hasQuotaInBytes()) {
|
|
|
long quota = setVolumePropertyRequest.getQuotaInBytes();
|
|
|
OmVolumeArgs omVolumeArgs =
|
|
|
- getOzoneManagerServerProtocol().startSetQuota(volume, quota);
|
|
|
+ getOzoneManager().startSetQuota(volume, quota);
|
|
|
SetVolumePropertyRequest newSetVolumePropertyRequest =
|
|
|
SetVolumePropertyRequest.newBuilder().setVolumeName(volume)
|
|
|
.setVolumeInfo(omVolumeArgs.getProtobuf()).build();
|
|
@@ -201,7 +196,7 @@ public class OzoneManagerHARequestHandlerImpl
|
|
|
} else {
|
|
|
String owner = setVolumePropertyRequest.getOwnerName();
|
|
|
OmVolumeOwnerChangeResponse omVolumeOwnerChangeResponse =
|
|
|
- getOzoneManagerServerProtocol().startSetOwner(volume, owner);
|
|
|
+ getOzoneManager().startSetOwner(volume, owner);
|
|
|
// If volumeLists become large and as ratis writes the request to disk we
|
|
|
// might take more space if the lists become very big in size. We might
|
|
|
// need to revisit this if it becomes problem
|
|
@@ -230,11 +225,11 @@ public class OzoneManagerHARequestHandlerImpl
|
|
|
omRequest.getSetVolumePropertyRequest();
|
|
|
|
|
|
if (setVolumePropertyRequest.hasQuotaInBytes()) {
|
|
|
- getOzoneManagerServerProtocol().applySetQuota(
|
|
|
+ getOzoneManager().applySetQuota(
|
|
|
OmVolumeArgs.getFromProtobuf(
|
|
|
setVolumePropertyRequest.getVolumeInfo()));
|
|
|
} else {
|
|
|
- getOzoneManagerServerProtocol().applySetOwner(
|
|
|
+ getOzoneManager().applySetOwner(
|
|
|
setVolumePropertyRequest.getOriginalOwner(),
|
|
|
setVolumePropertyRequest.getOldOwnerVolumeList(),
|
|
|
setVolumePropertyRequest.getNewOwnerVolumeList(),
|
|
@@ -252,7 +247,7 @@ public class OzoneManagerHARequestHandlerImpl
|
|
|
String volume = deleteVolumeRequest.getVolumeName();
|
|
|
|
|
|
OmDeleteVolumeResponse omDeleteVolumeResponse =
|
|
|
- getOzoneManagerServerProtocol().startDeleteVolume(volume);
|
|
|
+ getOzoneManager().startDeleteVolume(volume);
|
|
|
|
|
|
DeleteVolumeRequest newDeleteVolumeRequest =
|
|
|
DeleteVolumeRequest.newBuilder().setVolumeList(
|
|
@@ -272,97 +267,10 @@ public class OzoneManagerHARequestHandlerImpl
|
|
|
DeleteVolumeRequest deleteVolumeRequest =
|
|
|
omRequest.getDeleteVolumeRequest();
|
|
|
|
|
|
- getOzoneManagerServerProtocol().applyDeleteVolume(
|
|
|
+ getOzoneManager().applyDeleteVolume(
|
|
|
deleteVolumeRequest.getVolumeName(), deleteVolumeRequest.getOwner(),
|
|
|
deleteVolumeRequest.getVolumeList());
|
|
|
|
|
|
return DeleteVolumeResponse.newBuilder().build();
|
|
|
}
|
|
|
-
|
|
|
- private OMRequest handleCreateBucketStart(OMRequest omRequest)
|
|
|
- throws IOException {
|
|
|
-
|
|
|
- CreateBucketRequest createBucketRequest =
|
|
|
- omRequest.getCreateBucketRequest();
|
|
|
-
|
|
|
- OmBucketInfo omBucketInfo =
|
|
|
- getOzoneManagerServerProtocol().startCreateBucket(
|
|
|
- OmBucketInfo.getFromProtobuf(createBucketRequest.getBucketInfo()));
|
|
|
-
|
|
|
- CreateBucketRequest newCreateBucketRequest =
|
|
|
- CreateBucketRequest.newBuilder().setBucketInfo(
|
|
|
- omBucketInfo.getProtobuf()).build();
|
|
|
- return omRequest.toBuilder().setCreateBucketRequest(newCreateBucketRequest)
|
|
|
- .build();
|
|
|
-
|
|
|
- }
|
|
|
-
|
|
|
-
|
|
|
- private CreateBucketResponse handleCreateBucketApply(OMRequest omRequest)
|
|
|
- throws IOException {
|
|
|
- CreateBucketRequest createBucketRequest =
|
|
|
- omRequest.getCreateBucketRequest();
|
|
|
-
|
|
|
- getOzoneManagerServerProtocol().applyCreateBucket(
|
|
|
- OmBucketInfo.getFromProtobuf(createBucketRequest.getBucketInfo()));
|
|
|
-
|
|
|
- return CreateBucketResponse.newBuilder().build();
|
|
|
- }
|
|
|
-
|
|
|
-
|
|
|
- private OMRequest handleDeleteBucketRequestStart(OMRequest omRequest)
|
|
|
- throws IOException {
|
|
|
-
|
|
|
- DeleteBucketRequest deleteBucketRequest =
|
|
|
- omRequest.getDeleteBucketRequest();
|
|
|
- getOzoneManagerServerProtocol().startDeleteBucket(
|
|
|
- deleteBucketRequest.getVolumeName(),
|
|
|
- deleteBucketRequest.getBucketName());
|
|
|
-
|
|
|
- return omRequest;
|
|
|
- }
|
|
|
-
|
|
|
- private DeleteBucketResponse handleDeleteBucketApply(OMRequest omRequest)
|
|
|
- throws IOException {
|
|
|
-
|
|
|
- DeleteBucketRequest deleteBucketRequest =
|
|
|
- omRequest.getDeleteBucketRequest();
|
|
|
-
|
|
|
- getOzoneManagerServerProtocol().applyDeleteBucket(
|
|
|
- deleteBucketRequest.getVolumeName(),
|
|
|
- deleteBucketRequest.getBucketName());
|
|
|
-
|
|
|
- return DeleteBucketResponse.newBuilder().build();
|
|
|
- }
|
|
|
-
|
|
|
- private OMRequest handleSetBucketPropertyStart(
|
|
|
- OMRequest omRequest) throws IOException {
|
|
|
- SetBucketPropertyRequest setBucketPropertyRequest =
|
|
|
- omRequest.getSetBucketPropertyRequest();
|
|
|
-
|
|
|
- OmBucketInfo omBucketInfo =
|
|
|
- getOzoneManagerServerProtocol().startSetBucketProperty(
|
|
|
- OmBucketArgs.getFromProtobuf(setBucketPropertyRequest.getBucketArgs()));
|
|
|
-
|
|
|
- SetBucketPropertyRequest newSetBucketPropertyRequest =
|
|
|
- SetBucketPropertyRequest.newBuilder()
|
|
|
- .setBucketInfo(omBucketInfo.getProtobuf()).build();
|
|
|
-
|
|
|
- return omRequest.toBuilder().setSetBucketPropertyRequest(
|
|
|
- newSetBucketPropertyRequest).build();
|
|
|
- }
|
|
|
-
|
|
|
- private SetBucketPropertyResponse handleSetBucketPropertyApply(
|
|
|
- OMRequest omRequest) throws IOException {
|
|
|
- SetBucketPropertyRequest setBucketPropertyRequest =
|
|
|
- omRequest.getSetBucketPropertyRequest();
|
|
|
-
|
|
|
- getOzoneManagerServerProtocol().applySetBucketProperty(
|
|
|
- OmBucketInfo.getFromProtobuf(setBucketPropertyRequest.getBucketInfo()));
|
|
|
-
|
|
|
- return SetBucketPropertyResponse.newBuilder().build();
|
|
|
- }
|
|
|
-
|
|
|
-
|
|
|
-
|
|
|
}
|