diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java index cd7c6ff61864..3f6d7a82320e 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java @@ -100,7 +100,6 @@ public final class OmUtils { public static final long EPOCH_ID_SHIFT = 62; // 64 - 2 public static final long REVERSE_EPOCH_ID_SHIFT = 2; // 64 - EPOCH_ID_SHIFT public static final long MAX_TRXN_ID = (1L << 54) - 2; - public static final int EPOCH_WHEN_RATIS_NOT_ENABLED = 1; public static final int EPOCH_WHEN_RATIS_ENABLED = 2; private OmUtils() { @@ -601,9 +600,8 @@ public static long getOMClientRpcTimeOut(ConfigurationSource configuration) { return configuration.getObject(OMClientConfig.class).getRpcTimeOut(); } - public static int getOMEpoch(boolean isRatisEnabled) { - return isRatisEnabled ? EPOCH_WHEN_RATIS_ENABLED : - EPOCH_WHEN_RATIS_NOT_ENABLED; + public static int getOMEpoch() { + return EPOCH_WHEN_RATIS_ENABLED; } /** diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisRequest.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisRequest.java index 3fc991ff5808..0b517539b321 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisRequest.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisRequest.java @@ -127,8 +127,7 @@ public void testUnknownRequestHandling() OzoneManagerProtocolServerSideTranslatorPB serverSideTranslatorPB = new OzoneManagerProtocolServerSideTranslatorPB(ozoneManager, - ratisServer, protocolMessageMetrics, true, - 100L); + ratisServer, protocolMessageMetrics); OzoneManagerProtocolProtos.OMResponse actualResponse = serverSideTranslatorPB.processRequest(omRequest); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java index 8f4c070b76c6..4e8fa8daa056 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java @@ -310,7 +310,6 @@ public class OmMetadataManagerImpl implements OMMetadataManager, private Table snapshotRenamedTable; private Table compactionLogTable; - private boolean isRatisEnabled; private boolean ignorePipelineinKey; private Table deletedDirTable; @@ -351,8 +350,7 @@ public OmMetadataManagerImpl(OzoneConfiguration conf, this.ozoneManager = ozoneManager; this.perfMetrics = perfMetrics; this.lock = new OzoneManagerLock(conf); - isRatisEnabled = true; - this.omEpoch = OmUtils.getOMEpoch(isRatisEnabled); + this.omEpoch = OmUtils.getOMEpoch(); // For test purpose only ignorePipelineinKey = conf.getBoolean( "ozone.om.ignore.pipeline", Boolean.TRUE); @@ -567,11 +565,6 @@ public void start(OzoneConfiguration configuration) throws IOException { // enabled, ratis log provides us this guaranty. This check is needed // until HA code path becomes default in OM. - // When ratis is not enabled override and set the sync. - if (!isRatisEnabled) { - rocksDBConfiguration.setSyncOption(true); - } - int maxOpenFiles = configuration.getInt(OZONE_OM_DB_MAX_OPEN_FILES, OZONE_OM_DB_MAX_OPEN_FILES_DEFAULT); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java index f1d31d130cf1..4a54f2f0f252 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java @@ -249,7 +249,6 @@ import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_BLOCK_TOKEN_ENABLED_DEFAULT; import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients; import static org.apache.hadoop.hdds.HddsUtils.preserveThreadName; -import static org.apache.hadoop.hdds.ratis.RatisHelper.newJvmPauseMonitor; import static org.apache.hadoop.hdds.server.ServerUtils.updateRPCListenAddress; import static org.apache.hadoop.hdds.utils.HAUtils.getScmInfo; import static org.apache.hadoop.hdds.utils.HddsServerUtil.getRemoteUser; @@ -338,7 +337,6 @@ import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.util.ExitUtils; import org.apache.ratis.util.FileUtils; -import org.apache.ratis.util.JvmPauseMonitor; import org.apache.ratis.util.LifeCycle; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -416,7 +414,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl private final File omMetaDir; private boolean isAclEnabled; private final boolean isSpnegoEnabled; - private final JvmPauseMonitor jvmPauseMonitor; private final SecurityConfig secConfig; private S3SecretManager s3SecretManager; private final boolean isOmGrpcServerEnabled; @@ -425,7 +422,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl private String omComponent; private OzoneManagerProtocolServerSideTranslatorPB omServerProtocol; - private final boolean isRatisEnabled; private OzoneManagerRatisServer omRatisServer; private OmRatisSnapshotProvider omRatisSnapshotProvider; private OMNodeDetails omNodeDetails; @@ -583,11 +579,6 @@ private OzoneManager(OzoneConfiguration conf, StartupOption startupOption) OZONE_OM_NAMESPACE_STRICT_S3, OZONE_OM_NAMESPACE_STRICT_S3_DEFAULT); - isRatisEnabled = true; - - // Ratis server comes with JvmPauseMonitor, no need to start another - jvmPauseMonitor = !isRatisEnabled ? newJvmPauseMonitor(omId) : null; - String defaultBucketLayoutString = configuration.getTrimmed(OZONE_DEFAULT_BUCKET_LAYOUT, OZONE_DEFAULT_BUCKET_LAYOUT_DEFAULT); @@ -859,7 +850,7 @@ private void instantiateServices(boolean withNewSnapshot) throws IOException { delegationTokenMgr = createDelegationTokenSecretManager(configuration); } - prefixManager = new PrefixManagerImpl(this, metadataManager, isRatisEnabled); + prefixManager = new PrefixManagerImpl(this, metadataManager, true); keyManager = new KeyManagerImpl(this, scmClient, configuration, perfMetrics); // If authorizer is not initialized or the authorizer is Native @@ -1262,14 +1253,12 @@ private RPC.Server getRpcServer(OzoneConfiguration conf) throws IOException { ProtobufRpcEngine.class); this.omServerProtocol = new OzoneManagerProtocolServerSideTranslatorPB( - this, omRatisServer, omClientProtocolMetrics, isRatisEnabled, - getLastTrxnIndexForNonRatis()); + this, omRatisServer, omClientProtocolMetrics); BlockingService omService = OzoneManagerService.newReflectiveBlockingService(omServerProtocol); OMInterServiceProtocolServerSideImpl omInterServerProtocol = - new OMInterServiceProtocolServerSideImpl(this, omRatisServer, - isRatisEnabled); + new OMInterServiceProtocolServerSideImpl(this, omRatisServer); BlockingService omInterService = OzoneManagerInterService.newReflectiveBlockingService( omInterServerProtocol); @@ -1467,75 +1456,73 @@ public static void initializeSecurity(OzoneConfiguration conf, } private void initializeRatisDirs(OzoneConfiguration conf) throws IOException { - if (isRatisEnabled) { - // Create Ratis storage dir - String omRatisDirectory = - OzoneManagerRatisUtils.getOMRatisDirectory(conf); - if (omRatisDirectory == null || omRatisDirectory.isEmpty()) { - throw new IllegalArgumentException(HddsConfigKeys.OZONE_METADATA_DIRS + - " must be defined."); - } - OmUtils.createOMDir(omRatisDirectory); - - String scmStorageDir = SCMHAUtils.getRatisStorageDir(conf); - if (!Strings.isNullOrEmpty(omRatisDirectory) && !Strings - .isNullOrEmpty(scmStorageDir) && omRatisDirectory - .equals(scmStorageDir)) { - throw new IOException( - "Path of " + OMConfigKeys.OZONE_OM_RATIS_STORAGE_DIR + " and " - + ScmConfigKeys.OZONE_SCM_HA_RATIS_STORAGE_DIR - + " should not be co located. Please change atleast one path."); - } - - // Create Ratis snapshot dir - omRatisSnapshotDir = OmUtils.createOMDir( - OzoneManagerRatisUtils.getOMRatisSnapshotDirectory(conf)); - - // Before starting ratis server, check if previous installation has - // snapshot directory in Ratis storage directory. if yes, move it to - // new snapshot directory. - - File snapshotDir = new File(omRatisDirectory, OZONE_RATIS_SNAPSHOT_DIR); - - if (snapshotDir.isDirectory()) { - FileUtils.moveDirectory(snapshotDir.toPath(), - omRatisSnapshotDir.toPath()); - } - - File omRatisDir = new File(omRatisDirectory); - String groupIDfromServiceID = RaftGroupId.valueOf( - getRaftGroupIdFromOmServiceId(getOMServiceId())).getUuid().toString(); - - // If a directory exists in ratis storage dir - // Check the Ratis group Dir is same as the one generated from - // om service id. - - // This will help to catch if some one has changed service id later on. - File[] ratisDirFiles = omRatisDir.listFiles(); - if (ratisDirFiles != null) { - for (File ratisGroupDir : ratisDirFiles) { - if (ratisGroupDir.isDirectory()) { - if (!ratisGroupDir.getName().equals(groupIDfromServiceID)) { - throw new IOException("Ratis group Dir on disk " - + ratisGroupDir.getName() + " does not match with RaftGroupID" - + groupIDfromServiceID + " generated from service id " - + getOMServiceId() + ". Looks like there is a change to " + - OMConfigKeys.OZONE_OM_SERVICE_IDS_KEY + " value after the " + - "cluster is setup. Currently change to this value is not " + - "supported."); - } - } else { - LOG.warn("Unknown file {} exists in ratis storage dir {}." - + " It is recommended not to share the ratis storage dir.", - ratisGroupDir, omRatisDir); + // Create Ratis storage dir + String omRatisDirectory = + OzoneManagerRatisUtils.getOMRatisDirectory(conf); + if (omRatisDirectory == null || omRatisDirectory.isEmpty()) { + throw new IllegalArgumentException(HddsConfigKeys.OZONE_METADATA_DIRS + + " must be defined."); + } + OmUtils.createOMDir(omRatisDirectory); + + String scmStorageDir = SCMHAUtils.getRatisStorageDir(conf); + if (!Strings.isNullOrEmpty(omRatisDirectory) && !Strings + .isNullOrEmpty(scmStorageDir) && omRatisDirectory + .equals(scmStorageDir)) { + throw new IOException( + "Path of " + OMConfigKeys.OZONE_OM_RATIS_STORAGE_DIR + " and " + + ScmConfigKeys.OZONE_SCM_HA_RATIS_STORAGE_DIR + + " should not be co located. Please change atleast one path."); + } + + // Create Ratis snapshot dir + omRatisSnapshotDir = OmUtils.createOMDir( + OzoneManagerRatisUtils.getOMRatisSnapshotDirectory(conf)); + + // Before starting ratis server, check if previous installation has + // snapshot directory in Ratis storage directory. if yes, move it to + // new snapshot directory. + + File snapshotDir = new File(omRatisDirectory, OZONE_RATIS_SNAPSHOT_DIR); + + if (snapshotDir.isDirectory()) { + FileUtils.moveDirectory(snapshotDir.toPath(), + omRatisSnapshotDir.toPath()); + } + + File omRatisDir = new File(omRatisDirectory); + String groupIDfromServiceID = RaftGroupId.valueOf( + getRaftGroupIdFromOmServiceId(getOMServiceId())).getUuid().toString(); + + // If a directory exists in ratis storage dir + // Check the Ratis group Dir is same as the one generated from + // om service id. + + // This will help to catch if some one has changed service id later on. + File[] ratisDirFiles = omRatisDir.listFiles(); + if (ratisDirFiles != null) { + for (File ratisGroupDir : ratisDirFiles) { + if (ratisGroupDir.isDirectory()) { + if (!ratisGroupDir.getName().equals(groupIDfromServiceID)) { + throw new IOException("Ratis group Dir on disk " + + ratisGroupDir.getName() + " does not match with RaftGroupID" + + groupIDfromServiceID + " generated from service id " + + getOMServiceId() + ". Looks like there is a change to " + + OMConfigKeys.OZONE_OM_SERVICE_IDS_KEY + " value after the " + + "cluster is setup. Currently change to this value is not " + + "supported."); } + } else { + LOG.warn("Unknown file {} exists in ratis storage dir {}." + + " It is recommended not to share the ratis storage dir.", + ratisGroupDir, omRatisDir); } } + } - if (peerNodesMap != null && !peerNodesMap.isEmpty()) { - this.omRatisSnapshotProvider = new OmRatisSnapshotProvider( - configuration, omRatisSnapshotDir, peerNodesMap); - } + if (peerNodesMap != null && !peerNodesMap.isEmpty()) { + this.omRatisSnapshotProvider = new OmRatisSnapshotProvider( + configuration, omRatisSnapshotDir, peerNodesMap); } } @@ -1750,7 +1737,6 @@ public void start() throws IOException { } registerMXBean(); - startJVMPauseMonitor(); setStartTime(); if (omState == State.BOOTSTRAPPING) { @@ -1832,7 +1818,6 @@ public void restart() throws IOException { omS3gGrpcServer.start(); isOmGrpcServerRunning = true; } - startJVMPauseMonitor(); setStartTime(); omState = State.RUNNING; auditMap.put("NewOmState", omState.name()); @@ -1941,23 +1926,17 @@ private void checkRemoteOMConfig(String remoteNodeId, @Override public void bootstrap(OMNodeDetails newOMNode) throws IOException { // Create InterOmServiceProtocol client to send request to other OMs - if (isRatisEnabled) { - try (OMInterServiceProtocolClientSideImpl omInterServiceProtocol = - new OMInterServiceProtocolClientSideImpl(configuration, - getRemoteUser(), getOMServiceId())) { + try (OMInterServiceProtocolClientSideImpl omInterServiceProtocol = + new OMInterServiceProtocolClientSideImpl(configuration, + getRemoteUser(), getOMServiceId())) { - omInterServiceProtocol.bootstrap(omNodeDetails); + omInterServiceProtocol.bootstrap(omNodeDetails); - LOG.info("Successfully bootstrapped OM {} and joined the Ratis group " + - "{}", getOMNodeId(), omRatisServer.getRaftGroup()); - } catch (Exception e) { - LOG.error("Failed to Bootstrap OM."); - throw e; - } - } else { - throw new IOException("OzoneManager can be bootstrapped only when ratis" + - " is enabled and there is atleast one OzoneManager to bootstrap" + - " from."); + LOG.info("Successfully bootstrapped OM {} and joined the Ratis group " + + "{}", getOMNodeId(), omRatisServer.getRaftGroup()); + } catch (Exception e) { + LOG.error("Failed to Bootstrap OM."); + throw e; } } @@ -2195,20 +2174,16 @@ private GrpcOzoneManagerServer getOmS3gGrpcServer(OzoneConfiguration conf) { */ private void initializeRatisServer(boolean shouldBootstrap) throws IOException { - if (isRatisEnabled) { - if (omRatisServer == null) { - // This needs to be done before initializing Ratis. - ratisReporterList = RatisDropwizardExports. - registerRatisMetricReporters(ratisMetricsMap, this::isStopped); - omRatisServer = OzoneManagerRatisServer.newOMRatisServer( - configuration, this, omNodeDetails, peerNodesMap, - secConfig, certClient, shouldBootstrap); - } - LOG.info("OzoneManager Ratis server initialized at port {}", - omRatisServer.getServerPort()); - } else { - omRatisServer = null; + if (omRatisServer == null) { + // This needs to be done before initializing Ratis. + ratisReporterList = RatisDropwizardExports. + registerRatisMetricReporters(ratisMetricsMap, this::isStopped); + omRatisServer = OzoneManagerRatisServer.newOMRatisServer( + configuration, this, omNodeDetails, peerNodesMap, + secConfig, certClient, shouldBootstrap); } + LOG.info("OzoneManager Ratis server initialized at port {}", + omRatisServer.getServerPort()); } public long getObjectIdFromTxId(long trxnId) { @@ -2216,23 +2191,6 @@ public long getObjectIdFromTxId(long trxnId) { trxnId); } - @VisibleForTesting - long getLastTrxnIndexForNonRatis() throws IOException { - TransactionInfo transactionInfo = - TransactionInfo.readTransactionInfo(metadataManager); - // If the OMTransactionInfo does not exist in DB, return 0 so that new incoming - // requests can have transaction index starting from 1. - if (transactionInfo == null) { - return 0; - } - // If there exists a last transaction index in DB, including two cases: - // 1. transactionInfo.getTerm() == -1 corresponds to a non-Ratis cluster - // 2. transactionInfo.getTerm() != -1 indicates that the DB may be migrated from Ratis cluster - // For both cases above, the new incoming requests in non-Ratis cluster must have - // transaction index incrementally increasing from the stored transaction index onwards. - return transactionInfo.getTransactionIndex(); - } - /** * * @return Gets the stored layout version from the DB meta table. @@ -2286,9 +2244,6 @@ public boolean stop() { } // When ratis is not enabled, we need to call stop() to stop // OzoneManageDoubleBuffer in OM server protocol. - if (!isRatisEnabled) { - omServerProtocol.stop(); - } if (omRatisServer != null) { omRatisServer.stop(); OMHAMetrics.unRegister(); @@ -2316,9 +2271,6 @@ public boolean stop() { metrics.unRegister(); omClientProtocolMetrics.unregister(); unregisterMXBean(); - if (jvmPauseMonitor != null) { - jvmPauseMonitor.stop(); - } if (omRatisSnapshotProvider != null) { omRatisSnapshotProvider.close(); } @@ -3093,9 +3045,6 @@ private static List> getRatisRolesException(String exceptionString) @Override public List> getRatisRoles() { int port = omNodeDetails.getRatisPort(); - if (!isRatisEnabled) { - return getRatisRolesException("Ratis is disabled"); - } if (null == omRatisServer) { return getRatisRolesException("Server is shutting down"); } @@ -3196,35 +3145,33 @@ public List getServiceList() throws IOException { .build(); omServiceInfoBuilder.setOmRoleInfo(omRole); - if (isRatisEnabled) { - if (omRatisServer != null) { - omServiceInfoBuilder.addServicePort(ServicePort.newBuilder() - .setType(ServicePort.Type.RATIS) - .setValue(omNodeDetails.getRatisPort()) - .build()); - } + if (omRatisServer != null) { + omServiceInfoBuilder.addServicePort(ServicePort.newBuilder() + .setType(ServicePort.Type.RATIS) + .setValue(omNodeDetails.getRatisPort()) + .build()); + } - for (OMNodeDetails peerNode : peerNodesMap.values()) { - ServiceInfo.Builder peerOmServiceInfoBuilder = ServiceInfo.newBuilder() - .setNodeType(HddsProtos.NodeType.OM) - .setHostname(peerNode.getHostName()) - // For now assume peer is at the same version. - // This field needs to be fetched from peer when rolling upgrades - // are implemented. - .setOmVersion(OzoneManagerVersion.CURRENT) - .addServicePort(ServicePort.newBuilder() - .setType(ServicePort.Type.RPC) - .setValue(peerNode.getRpcPort()) - .build()); + for (OMNodeDetails peerNode : peerNodesMap.values()) { + ServiceInfo.Builder peerOmServiceInfoBuilder = ServiceInfo.newBuilder() + .setNodeType(HddsProtos.NodeType.OM) + .setHostname(peerNode.getHostName()) + // For now assume peer is at the same version. + // This field needs to be fetched from peer when rolling upgrades + // are implemented. + .setOmVersion(OzoneManagerVersion.CURRENT) + .addServicePort(ServicePort.newBuilder() + .setType(ServicePort.Type.RPC) + .setValue(peerNode.getRpcPort()) + .build()); - OMRoleInfo peerOmRole = OMRoleInfo.newBuilder() - .setNodeId(peerNode.getNodeId()) - .setServerRole(RaftPeerRole.FOLLOWER.name()) - .build(); - peerOmServiceInfoBuilder.setOmRoleInfo(peerOmRole); + OMRoleInfo peerOmRole = OMRoleInfo.newBuilder() + .setNodeId(peerNode.getNodeId()) + .setServerRole(RaftPeerRole.FOLLOWER.name()) + .build(); + peerOmServiceInfoBuilder.setOmRoleInfo(peerOmRole); - services.add(peerOmServiceInfoBuilder.build()); - } + services.add(peerOmServiceInfoBuilder.build()); } services.add(omServiceInfoBuilder.build()); @@ -3356,9 +3303,6 @@ public ListOpenFilesResult listOpenFiles(String path, public void transferLeadership(String newLeaderId) throws IOException { checkAdminUserPrivilege("transfer raft leadership."); - if (!isRatisEnabled) { - throw new IOException("OM HA not enabled."); - } boolean auditSuccess = true; Map auditMap = new LinkedHashMap<>(); auditMap.put("newLeaderId", newLeaderId); @@ -4259,9 +4203,7 @@ public long getMaxUserVolumeCount() { */ public boolean isLeaderReady() { final OzoneManagerRatisServer ratisServer = omRatisServer; - return !isRatisEnabled - || (ratisServer != null && - ratisServer.checkLeaderStatus() == LEADER_AND_READY); + return ratisServer != null && ratisServer.checkLeaderStatus() == LEADER_AND_READY; } /** @@ -4289,8 +4231,9 @@ public void checkLeaderStatus() throws OMNotLeaderException, /** * Return if Ratis is enabled or not. */ + // TODO remove in HDDS-12161 public boolean isRatisEnabled() { - return isRatisEnabled; + return true; } /** @@ -4379,12 +4322,6 @@ public boolean isRunning() { return omState == State.RUNNING; } - private void startJVMPauseMonitor() { - if (jvmPauseMonitor != null) { - jvmPauseMonitor.start(); - } - } - public ResolvedBucket resolveBucketLink(KeyArgs args, OMClientRequest omClientRequest) throws IOException { return resolveBucketLink( @@ -5067,11 +5004,7 @@ public ReconfigurationHandler getReconfigurationHandler() { * fs needs to synchronized with the rocksdb's for the snapshots. */ public void awaitDoubleBufferFlush() throws InterruptedException { - if (isRatisEnabled()) { - getOmRatisServer().getOmStateMachine().awaitDoubleBufferFlush(); - } else { - getOmServerProtocol().awaitDoubleBufferFlush(); - } + getOmRatisServer().getOmStateMachine().awaitDoubleBufferFlush(); } public void checkFeatureEnabled(OzoneManagerVersion feature) throws OMException { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/PrefixManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/PrefixManagerImpl.java index e8e930891df6..a62bfa20aa27 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/PrefixManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/PrefixManagerImpl.java @@ -64,9 +64,8 @@ public class PrefixManagerImpl implements PrefixManager { // In-memory prefix tree to optimize ACL evaluation private RadixTree prefixTree; - // TODO: This isRatisEnabled check will be removed as part of HDDS-1909, - // where we integrate both HA and Non-HA code. - private boolean isRatisEnabled; + // Ratis is disabled for snapshots + private final boolean isRatisEnabled; public PrefixManagerImpl(OzoneManager ozoneManager, OMMetadataManager metadataManager, boolean isRatisEnabled) { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/TrashOzoneFileSystem.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/TrashOzoneFileSystem.java index bd462224e9d8..0dc7edde6ad7 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/TrashOzoneFileSystem.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/TrashOzoneFileSystem.java @@ -94,11 +94,9 @@ public TrashOzoneFileSystem(OzoneManager ozoneManager) throws IOException { private void submitRequest(OzoneManagerProtocolProtos.OMRequest omRequest) throws Exception { ozoneManager.getMetrics().incNumTrashWriteRequests(); - if (ozoneManager.isRatisEnabled()) { - // perform preExecute as ratis submit do no perform preExecute - OMClientRequest omClientRequest = OzoneManagerRatisUtils.createClientRequest(omRequest, ozoneManager); - omRequest = omClientRequest.preExecute(ozoneManager); - } + // perform preExecute as ratis submit do no perform preExecute + OMClientRequest omClientRequest = OzoneManagerRatisUtils.createClientRequest(omRequest, ozoneManager); + omRequest = omClientRequest.preExecute(ozoneManager); OzoneManagerRatisUtils.submitRequest(ozoneManager, omRequest, CLIENT_ID, runCount.getAndIncrement()); } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java index dc634248c280..25971d9e95d5 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java @@ -520,10 +520,6 @@ public static GrpcTlsConfig createServerTlsConfig(SecurityConfig conf, public static OzoneManagerProtocolProtos.OMResponse submitRequest( OzoneManager om, OMRequest omRequest, ClientId clientId, long callId) throws ServiceException { - if (om.isRatisEnabled()) { - return om.getOmRatisServer().submitRequest(omRequest, clientId, callId); - } else { - return om.getOmServerProtocol().submitRequest(NULL_RPC_CONTROLLER, omRequest); - } + return om.getOmRatisServer().submitRequest(omRequest, clientId, callId); } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/AbstractKeyDeletingService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/AbstractKeyDeletingService.java index 6369d708a2ce..841575c1dbe7 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/AbstractKeyDeletingService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/AbstractKeyDeletingService.java @@ -122,15 +122,8 @@ protected int processKeyDeletes(List keyBlocksList, keyBlocksList.size(), Time.monotonicNow() - startTime); if (blockDeletionResults != null) { startTime = Time.monotonicNow(); - if (isRatisEnabled()) { - delCount = submitPurgeKeysRequest(blockDeletionResults, - keysToModify, snapTableKey, expectedPreviousSnapshotId); - } else { - // TODO: Once HA and non-HA paths are merged, we should have - // only one code path here. Purge keys should go through an - // OMRequest model. - delCount = deleteAllKeys(blockDeletionResults, manager); - } + delCount = submitPurgeKeysRequest(blockDeletionResults, + keysToModify, snapTableKey, expectedPreviousSnapshotId); int limit = ozoneManager.getConfiguration().getInt(OMConfigKeys.OZONE_KEY_DELETING_LIMIT_PER_TASK, OMConfigKeys.OZONE_KEY_DELETING_LIMIT_PER_TASK_DEFAULT); LOG.info("Blocks for {} (out of {}) keys are deleted from DB in {} ms. Limit per task is {}.", @@ -631,13 +624,6 @@ protected boolean isKeyReclaimable( return !isBlockLocationInfoSame(prevKeyInfo, deletedKeyInfo); } - public boolean isRatisEnabled() { - if (ozoneManager == null) { - return false; - } - return ozoneManager.isRatisEnabled(); - } - public OzoneManager getOzoneManager() { return ozoneManager; } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/MultipartUploadCleanupService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/MultipartUploadCleanupService.java index f1084155e98e..2b1f7bc9a3d7 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/MultipartUploadCleanupService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/MultipartUploadCleanupService.java @@ -141,10 +141,6 @@ private boolean shouldRun() { return !suspended.get() && ozoneManager.isLeaderReady(); } - private boolean isRatisEnabled() { - return ozoneManager.isRatisEnabled(); - } - private class MultipartUploadCleanupTask implements BackgroundTask { @Override diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/OMRangerBGSyncService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/OMRangerBGSyncService.java index fc6fe2b0c455..06148db50e80 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/OMRangerBGSyncService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/OMRangerBGSyncService.java @@ -258,8 +258,7 @@ private boolean shouldRun() { // OzoneManager can be null for testing return true; } - if (ozoneManager.isRatisEnabled() && - (ozoneManager.getOmRatisServer() == null)) { + if (ozoneManager.getOmRatisServer() == null) { LOG.warn("OzoneManagerRatisServer is not initialized yet"); return false; } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/OpenKeyCleanupService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/OpenKeyCleanupService.java index d185d8729083..fc0ad42ba876 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/OpenKeyCleanupService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/OpenKeyCleanupService.java @@ -162,10 +162,6 @@ private boolean shouldRun() { return !suspended.get() && ozoneManager.isLeaderReady(); } - private boolean isRatisEnabled() { - return ozoneManager.isRatisEnabled(); - } - private class OpenKeyCleanupTask implements BackgroundTask { private final BucketLayout bucketLayout; diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/QuotaRepairUpgradeAction.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/QuotaRepairUpgradeAction.java index 446c7382d50b..4157dd0a7a1a 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/QuotaRepairUpgradeAction.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/QuotaRepairUpgradeAction.java @@ -44,9 +44,7 @@ public void execute(OzoneManager arg) throws Exception { if (enabled) { // just trigger quota repair and status can be checked via CLI try { - if (arg.isRatisEnabled()) { - arg.checkLeaderStatus(); - } + arg.checkLeaderStatus(); QuotaRepairTask quotaRepairTask = new QuotaRepairTask(arg); quotaRepairTask.repair(); } catch (OMNotLeaderException | OMLeaderNotReadyException ex) { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OMAdminProtocolServerSideImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OMAdminProtocolServerSideImpl.java index 1e9786abf931..62345f674a32 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OMAdminProtocolServerSideImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OMAdminProtocolServerSideImpl.java @@ -75,13 +75,6 @@ public DecommissionOMResponse decommission(RpcController controller, if (request == null) { return null; } - if (!ozoneManager.isRatisEnabled()) { - return DecommissionOMResponse.newBuilder() - .setSuccess(false) - .setErrorMsg("OM node cannot be decommissioned as Ratis is " + - "not enabled.") - .build(); - } OzoneManagerRatisServer omRatisServer = ozoneManager.getOmRatisServer(); OzoneManagerRatisUtils.checkLeaderStatus(ozoneManager); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OMInterServiceProtocolServerSideImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OMInterServiceProtocolServerSideImpl.java index 48bc12c3c0e8..95a1027aaf49 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OMInterServiceProtocolServerSideImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OMInterServiceProtocolServerSideImpl.java @@ -39,14 +39,12 @@ public class OMInterServiceProtocolServerSideImpl implements OMInterServiceProtocolPB { private final OzoneManagerRatisServer omRatisServer; - private final boolean isRatisEnabled; private final OzoneManager ozoneManager; public OMInterServiceProtocolServerSideImpl(OzoneManager ozoneMgr, - OzoneManagerRatisServer ratisServer, boolean enableRatis) { + OzoneManagerRatisServer ratisServer) { this.ozoneManager = ozoneMgr; this.omRatisServer = ratisServer; - this.isRatisEnabled = enableRatis; } @Override @@ -55,14 +53,6 @@ public BootstrapOMResponse bootstrap(RpcController controller, if (request == null) { return null; } - if (!isRatisEnabled) { - return BootstrapOMResponse.newBuilder() - .setSuccess(false) - .setErrorCode(ErrorCode.RATIS_NOT_ENABLED) - .setErrorMsg("New OM node cannot be bootstrapped as Ratis " + - "is not enabled on existing OM") - .build(); - } OzoneManagerRatisUtils.checkLeaderStatus(ozoneManager); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java index 91d234d4d0b4..82e45d6a40da 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java @@ -23,15 +23,11 @@ import static org.apache.hadoop.ozone.util.MetricUtil.captureLatencyNs; import java.io.IOException; -import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher; -import org.apache.hadoop.hdds.tracing.TracingUtil; import org.apache.hadoop.hdds.utils.ProtocolMessageMetrics; -import org.apache.hadoop.hdds.utils.TransactionInfo; import org.apache.hadoop.ipc.ProcessingDetails.Timing; import org.apache.hadoop.ipc.Server; import org.apache.hadoop.ozone.OmUtils; @@ -41,15 +37,12 @@ import org.apache.hadoop.ozone.om.exceptions.OMLeaderNotReadyException; import org.apache.hadoop.ozone.om.helpers.OMAuditLogger; import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolPB; -import org.apache.hadoop.ozone.om.execution.flowcontrol.ExecutionContext; -import org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer; import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer; import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.RaftServerStatus; import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils; import org.apache.hadoop.ozone.om.request.OMClientRequest; import org.apache.hadoop.ozone.om.request.validation.RequestValidations; import org.apache.hadoop.ozone.om.request.validation.ValidationContext; -import org.apache.hadoop.ozone.om.response.OMClientResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse; @@ -59,8 +52,6 @@ import com.google.protobuf.ServiceException; import org.apache.hadoop.ozone.security.S3SecurityUtil; import org.apache.ratis.protocol.RaftPeerId; -import org.apache.ratis.server.protocol.TermIndex; -import org.apache.ratis.util.ExitUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -75,12 +66,6 @@ public class OzoneManagerProtocolServerSideTranslatorPB implements OzoneManagerP private final OzoneManagerRatisServer omRatisServer; private final RequestHandler handler; private final OzoneManager ozoneManager; - /** - * Only used to handle write requests when ratis is disabled. - * When ratis is enabled, write requests are handled by the state machine. - */ - private final OzoneManagerDoubleBuffer ozoneManagerDoubleBuffer; - private final AtomicLong transactionIndex; private final OzoneProtocolMessageDispatcher dispatcher; private final RequestValidations requestValidations; @@ -97,23 +82,10 @@ public class OzoneManagerProtocolServerSideTranslatorPB implements OzoneManagerP public OzoneManagerProtocolServerSideTranslatorPB( OzoneManager impl, OzoneManagerRatisServer ratisServer, - ProtocolMessageMetrics metrics, - boolean enableRatis, - long lastTransactionIndexForNonRatis) { + ProtocolMessageMetrics metrics) { this.ozoneManager = impl; this.perfMetrics = impl.getPerfMetrics(); - // Update the transactionIndex with the last TransactionIndex read from DB. - // New requests should have transactionIndex incremented from this index - // onwards to ensure unique objectIDs. - this.transactionIndex = new AtomicLong(lastTransactionIndexForNonRatis); - - // When ratis is enabled, the handler does not require a double-buffer since it only handle read requests. - this.ozoneManagerDoubleBuffer = enableRatis ? null - : OzoneManagerDoubleBuffer.newBuilder() - .setOmMetadataManager(ozoneManager.getMetadataManager()) - .enableTracing(TracingUtil.isTracingEnabled(ozoneManager.getConfiguration())) - .build() - .start(); + this.handler = new OzoneManagerRequestHandler(impl); this.omRatisServer = ratisServer; dispatcher = new OzoneProtocolMessageDispatcher<>("OzoneProtocol", @@ -126,10 +98,6 @@ public OzoneManagerProtocolServerSideTranslatorPB( .load(); } - private boolean isRatisEnabled() { - return ozoneManagerDoubleBuffer == null; - } - /** * Submit mutating requests to Ratis server in OM, and process read requests. */ @@ -190,10 +158,6 @@ private OMResponse internalProcessRequest(OMRequest request) throws ServiceExcep } } - if (!isRatisEnabled()) { - return submitRequestDirectlyToOM(request); - } - if (OmUtils.isReadOnly(request)) { return submitReadRequestToOM(request); } @@ -285,48 +249,6 @@ private ServiceException createLeaderNotReadyException() { return new ServiceException(leaderNotReadyException); } - /** - * Submits request directly to OM. - */ - private OMResponse submitRequestDirectlyToOM(OMRequest request) { - final OMClientResponse omClientResponse; - try { - if (OmUtils.isReadOnly(request)) { - return handler.handleReadRequest(request); - } else { - OMClientRequest omClientRequest = - createClientRequest(request, ozoneManager); - try { - request = omClientRequest.preExecute(ozoneManager); - } catch (IOException ex) { - // log only when audit build is complete as required - OMAuditLogger.log(omClientRequest.getAuditBuilder()); - throw ex; - } - final TermIndex termIndex = TransactionInfo.getTermIndex(transactionIndex.incrementAndGet()); - final ExecutionContext context = ExecutionContext.of(termIndex.getIndex(), termIndex); - omClientResponse = handler.handleWriteRequest(request, context, ozoneManagerDoubleBuffer); - } - } catch (IOException ex) { - // As some preExecute returns error. So handle here. - return createErrorResponse(request, ex); - } - try { - omClientResponse.getFlushFuture().get(); - if (LOG.isTraceEnabled()) { - LOG.trace("Future for {} is completed", request); - } - } catch (ExecutionException | InterruptedException ex) { - // terminate OM. As if we are in this stage means, while getting - // response from flush future, we got an exception. - String errorMessage = "Got error during waiting for flush to be " + - "completed for " + "request" + request.toString(); - ExitUtils.terminate(1, errorMessage, ex, LOG); - Thread.currentThread().interrupt(); - } - return omClientResponse.getOMResponse(); - } - /** @return an {@link OMResponse} from the given {@link OMRequest} and the given exception. */ private OMResponse createErrorResponse( OMRequest omRequest, IOException exception) { @@ -344,22 +266,7 @@ private OMResponse createErrorResponse( return omResponse.build(); } - public void stop() { - if (ozoneManagerDoubleBuffer != null) { - ozoneManagerDoubleBuffer.stop(); - } - } - public static Logger getLog() { return LOG; } - - /** - * Wait until both buffers are flushed. This is used in cases like - * "follower bootstrap tarball creation" where the rocksDb for the active - * fs needs to synchronized with the rocksdb's for the snapshots. - */ - public void awaitDoubleBufferFlush() throws InterruptedException { - ozoneManagerDoubleBuffer.awaitFlush(); - } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/security/OzoneDelegationTokenSecretManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/security/OzoneDelegationTokenSecretManager.java index c496da41eb43..e4303318a00b 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/security/OzoneDelegationTokenSecretManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/security/OzoneDelegationTokenSecretManager.java @@ -87,8 +87,6 @@ public class OzoneDelegationTokenSecretManager */ private final Object noInterruptsLock = new Object(); - private final boolean isRatisEnabled; - /** * Create a secret manager with a builder object. * @@ -104,7 +102,6 @@ public OzoneDelegationTokenSecretManager(Builder b) throws IOException { this.ozoneManager = b.ozoneManager; this.store = new OzoneSecretStore(b.ozoneConf, this.ozoneManager.getMetadataManager()); - isRatisEnabled = true; this.secretKeyClient = b.secretKeyClient; loadTokenSecretState(store.loadState()); } @@ -217,12 +214,6 @@ public Token createToken(Text owner, Text renewer, } long expiryTime = identifier.getIssueDate() + getTokenRenewInterval(); - // For HA ratis will take care of updating. - // This will be removed, when HA/Non-HA code is merged. - if (!isRatisEnabled) { - addToTokenStore(identifier, password, expiryTime); - } - Token token = new Token<>(identifier.getBytes(), password, identifier.getKind(), getService()); if (LOG.isDebugEnabled()) { @@ -315,18 +306,7 @@ public synchronized long renewToken(Token token, + " with non-matching renewer " + id.getRenewer()); } - long renewTime = Math.min(id.getMaxDate(), now + getTokenRenewInterval()); - - // For HA ratis will take care of updating. - // This will be removed, when HA/Non-HA code is merged. - if (!isRatisEnabled) { - try { - addToTokenStore(id, token.getPassword(), renewTime); - } catch (IOException e) { - LOG.error("Unable to update token " + id.getSequenceNumber(), e); - } - } - return renewTime; + return Math.min(id.getMaxDate(), now + getTokenRenewInterval()); } public void updateRenewToken(Token token, @@ -372,26 +352,12 @@ public OzoneTokenIdentifier cancelToken(Token token, + " is not authorized to cancel the token " + formatTokenId(id)); } - // For HA ratis will take care of removal. - // This check will be removed, when HA/Non-HA code is merged. - if (!isRatisEnabled) { - try { - store.removeToken(id); - } catch (IOException e) { - LOG.error("Unable to remove token " + id.getSequenceNumber(), e); - } - TokenInfo info = currentTokens.remove(id); - if (info == null) { - throw new InvalidToken("Token not found " + formatTokenId(id)); - } - } else { - // Check whether token is there in-memory map of tokens or not on the - // OM leader. - TokenInfo info = currentTokens.get(id); - if (info == null) { - throw new InvalidToken("Token not found in-memory map of tokens" + - formatTokenId(id)); - } + // Check whether token is there in-memory map of tokens or not on the + // OM leader. + TokenInfo info = currentTokens.get(id); + if (info == null) { + throw new InvalidToken("Token not found in-memory map of tokens" + + formatTokenId(id)); } return id; } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestQuotaRepairTask.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestQuotaRepairTask.java index 18625fe5c440..93d02f1ce4e9 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestQuotaRepairTask.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestQuotaRepairTask.java @@ -58,7 +58,6 @@ public class TestQuotaRepairTask extends TestOMKeyRequest { @Test public void testQuotaRepair() throws Exception { - when(ozoneManager.isRatisEnabled()).thenReturn(true); OzoneManagerProtocolProtos.OMResponse respMock = mock(OzoneManagerProtocolProtos.OMResponse.class); when(respMock.getSuccess()).thenReturn(true); OzoneManagerRatisServer ratisServerMock = mock(OzoneManagerRatisServer.class); @@ -136,7 +135,6 @@ public void testQuotaRepair() throws Exception { @Test public void testQuotaRepairForOldVersionVolumeBucket() throws Exception { - when(ozoneManager.isRatisEnabled()).thenReturn(true); OzoneManagerProtocolProtos.OMResponse respMock = mock(OzoneManagerProtocolProtos.OMResponse.class); when(respMock.getSuccess()).thenReturn(true); OzoneManagerRatisServer ratisServerMock = mock(OzoneManagerRatisServer.class);