|
@@ -25,6 +25,7 @@ import com.google.common.base.Preconditions;
|
|
import com.google.protobuf.BlockingService;
|
|
import com.google.protobuf.BlockingService;
|
|
|
|
|
|
import java.net.InetAddress;
|
|
import java.net.InetAddress;
|
|
|
|
+import java.nio.file.Path;
|
|
import java.security.PrivateKey;
|
|
import java.security.PrivateKey;
|
|
import java.security.PublicKey;
|
|
import java.security.PublicKey;
|
|
import java.security.KeyPair;
|
|
import java.security.KeyPair;
|
|
@@ -143,6 +144,10 @@ import org.apache.hadoop.util.KMSUtil;
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
import org.apache.hadoop.util.ShutdownHookManager;
|
|
import org.apache.hadoop.util.ShutdownHookManager;
|
|
import org.apache.hadoop.utils.RetriableTask;
|
|
import org.apache.hadoop.utils.RetriableTask;
|
|
|
|
+import org.apache.hadoop.utils.db.DBCheckpoint;
|
|
|
|
+import org.apache.hadoop.utils.db.DBStore;
|
|
|
|
+import org.apache.ratis.server.protocol.TermIndex;
|
|
|
|
+import org.apache.ratis.util.FileUtils;
|
|
import org.apache.ratis.util.LifeCycle;
|
|
import org.apache.ratis.util.LifeCycle;
|
|
import org.bouncycastle.pkcs.PKCS10CertificationRequest;
|
|
import org.bouncycastle.pkcs.PKCS10CertificationRequest;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
@@ -236,18 +241,20 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
private RPC.Server omRpcServer;
|
|
private RPC.Server omRpcServer;
|
|
private InetSocketAddress omRpcAddress;
|
|
private InetSocketAddress omRpcAddress;
|
|
private String omId;
|
|
private String omId;
|
|
- private final OMMetadataManager metadataManager;
|
|
|
|
- private final VolumeManager volumeManager;
|
|
|
|
- private final BucketManager bucketManager;
|
|
|
|
- private final KeyManager keyManager;
|
|
|
|
- private final PrefixManagerImpl prefixManager;
|
|
|
|
|
|
+
|
|
|
|
+ private OMMetadataManager metadataManager;
|
|
|
|
+ private VolumeManager volumeManager;
|
|
|
|
+ private BucketManager bucketManager;
|
|
|
|
+ private KeyManager keyManager;
|
|
|
|
+ private PrefixManagerImpl prefixManager;
|
|
|
|
+ private S3BucketManager s3BucketManager;
|
|
|
|
+
|
|
private final OMMetrics metrics;
|
|
private final OMMetrics metrics;
|
|
private OzoneManagerHttpServer httpServer;
|
|
private OzoneManagerHttpServer httpServer;
|
|
private final OMStorage omStorage;
|
|
private final OMStorage omStorage;
|
|
private final ScmBlockLocationProtocol scmBlockClient;
|
|
private final ScmBlockLocationProtocol scmBlockClient;
|
|
private final StorageContainerLocationProtocol scmContainerClient;
|
|
private final StorageContainerLocationProtocol scmContainerClient;
|
|
private ObjectName omInfoBeanName;
|
|
private ObjectName omInfoBeanName;
|
|
- private final S3BucketManager s3BucketManager;
|
|
|
|
private Timer metricsTimer;
|
|
private Timer metricsTimer;
|
|
private ScheduleOMMetricsWriteTask scheduleOMMetricsWriteTask;
|
|
private ScheduleOMMetricsWriteTask scheduleOMMetricsWriteTask;
|
|
private static final ObjectWriter WRITER =
|
|
private static final ObjectWriter WRITER =
|
|
@@ -258,7 +265,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
private final Runnable shutdownHook;
|
|
private final Runnable shutdownHook;
|
|
private final File omMetaDir;
|
|
private final File omMetaDir;
|
|
private final boolean isAclEnabled;
|
|
private final boolean isAclEnabled;
|
|
- private final IAccessAuthorizer accessAuthorizer;
|
|
|
|
|
|
+ private IAccessAuthorizer accessAuthorizer;
|
|
private JvmPauseMonitor jvmPauseMonitor;
|
|
private JvmPauseMonitor jvmPauseMonitor;
|
|
private final SecurityConfig secConfig;
|
|
private final SecurityConfig secConfig;
|
|
private S3SecretManager s3SecretManager;
|
|
private S3SecretManager s3SecretManager;
|
|
@@ -308,12 +315,37 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
throw new OMException("OM not initialized.",
|
|
throw new OMException("OM not initialized.",
|
|
ResultCodes.OM_NOT_INITIALIZED);
|
|
ResultCodes.OM_NOT_INITIALIZED);
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ // Read configuration and set values.
|
|
|
|
+ ozAdmins = conf.getTrimmedStringCollection(OZONE_ADMINISTRATORS);
|
|
|
|
+ omMetaDir = OmUtils.getOmDbDir(configuration);
|
|
|
|
+ this.isAclEnabled = conf.getBoolean(OZONE_ACL_ENABLED,
|
|
|
|
+ OZONE_ACL_ENABLED_DEFAULT);
|
|
|
|
+ this.scmBlockSize = (long) conf.getStorageSize(OZONE_SCM_BLOCK_SIZE,
|
|
|
|
+ OZONE_SCM_BLOCK_SIZE_DEFAULT, StorageUnit.BYTES);
|
|
|
|
+ this.preallocateBlocksMax = conf.getInt(
|
|
|
|
+ OZONE_KEY_PREALLOCATION_BLOCKS_MAX,
|
|
|
|
+ OZONE_KEY_PREALLOCATION_BLOCKS_MAX_DEFAULT);
|
|
|
|
+ this.grpcBlockTokenEnabled = conf.getBoolean(HDDS_BLOCK_TOKEN_ENABLED,
|
|
|
|
+ HDDS_BLOCK_TOKEN_ENABLED_DEFAULT);
|
|
|
|
+ this.useRatisForReplication = conf.getBoolean(
|
|
|
|
+ DFS_CONTAINER_RATIS_ENABLED_KEY, DFS_CONTAINER_RATIS_ENABLED_DEFAULT);
|
|
|
|
+ // TODO: This is a temporary check. Once fully implemented, all OM state
|
|
|
|
+ // change should go through Ratis - be it standalone (for non-HA) or
|
|
|
|
+ // replicated (for HA).
|
|
|
|
+ isRatisEnabled = configuration.getBoolean(
|
|
|
|
+ OMConfigKeys.OZONE_OM_RATIS_ENABLE_KEY,
|
|
|
|
+ OMConfigKeys.OZONE_OM_RATIS_ENABLE_DEFAULT);
|
|
|
|
+
|
|
// Load HA related configurations
|
|
// Load HA related configurations
|
|
loadOMHAConfigs(configuration);
|
|
loadOMHAConfigs(configuration);
|
|
|
|
+ InetSocketAddress omNodeRpcAddr = omNodeDetails.getRpcAddress();
|
|
|
|
+ omRpcAddressTxt = new Text(omNodeDetails.getRpcAddressString());
|
|
|
|
|
|
scmContainerClient = getScmContainerClient(configuration);
|
|
scmContainerClient = getScmContainerClient(configuration);
|
|
// verifies that the SCM info in the OM Version file is correct.
|
|
// verifies that the SCM info in the OM Version file is correct.
|
|
scmBlockClient = getScmBlockClient(configuration);
|
|
scmBlockClient = getScmBlockClient(configuration);
|
|
|
|
+ this.scmClient = new ScmClient(scmBlockClient, scmContainerClient);
|
|
|
|
|
|
// For testing purpose only, not hit scm from om as Hadoop UGI can't login
|
|
// For testing purpose only, not hit scm from om as Hadoop UGI can't login
|
|
// two principals in the same JVM.
|
|
// two principals in the same JVM.
|
|
@@ -329,16 +361,32 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
RPC.setProtocolEngine(configuration, OzoneManagerProtocolPB.class,
|
|
RPC.setProtocolEngine(configuration, OzoneManagerProtocolPB.class,
|
|
ProtobufRpcEngine.class);
|
|
ProtobufRpcEngine.class);
|
|
|
|
|
|
- metadataManager = new OmMetadataManagerImpl(configuration);
|
|
|
|
|
|
+ secConfig = new SecurityConfig(configuration);
|
|
|
|
+ // Create the KMS Key Provider
|
|
|
|
+ try {
|
|
|
|
+ kmsProvider = createKeyProviderExt(configuration);
|
|
|
|
+ } catch (IOException ioe) {
|
|
|
|
+ kmsProvider = null;
|
|
|
|
+ LOG.error("Fail to create Key Provider");
|
|
|
|
+ }
|
|
|
|
+ if (secConfig.isSecurityEnabled()) {
|
|
|
|
+ omComponent = OM_DAEMON + "-" + omId;
|
|
|
|
+ if(omStorage.getOmCertSerialId() == null) {
|
|
|
|
+ throw new RuntimeException("OzoneManager started in secure mode but " +
|
|
|
|
+ "doesn't have SCM signed certificate.");
|
|
|
|
+ }
|
|
|
|
+ certClient = new OMCertificateClient(new SecurityConfig(conf),
|
|
|
|
+ omStorage.getOmCertSerialId());
|
|
|
|
+ }
|
|
|
|
+ if (secConfig.isBlockTokenEnabled()) {
|
|
|
|
+ blockTokenMgr = createBlockTokenSecretManager(configuration);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ instantiateServices();
|
|
|
|
+
|
|
|
|
+ initializeRatisServer();
|
|
|
|
+ initializeRatisClient();
|
|
|
|
|
|
- // This is a temporary check. Once fully implemented, all OM state change
|
|
|
|
- // should go through Ratis - be it standalone (for non-HA) or replicated
|
|
|
|
- // (for HA).
|
|
|
|
- isRatisEnabled = configuration.getBoolean(
|
|
|
|
- OMConfigKeys.OZONE_OM_RATIS_ENABLE_KEY,
|
|
|
|
- OMConfigKeys.OZONE_OM_RATIS_ENABLE_DEFAULT);
|
|
|
|
- startRatisServer();
|
|
|
|
- startRatisClient();
|
|
|
|
if (isRatisEnabled) {
|
|
if (isRatisEnabled) {
|
|
// Create Ratis storage dir
|
|
// Create Ratis storage dir
|
|
String omRatisDirectory = OmUtils.getOMRatisDirectory(configuration);
|
|
String omRatisDirectory = OmUtils.getOMRatisDirectory(configuration);
|
|
@@ -361,59 +409,44 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
OM_RATIS_SNAPSHOT_INDEX);
|
|
OM_RATIS_SNAPSHOT_INDEX);
|
|
this.snapshotIndex = loadRatisSnapshotIndex();
|
|
this.snapshotIndex = loadRatisSnapshotIndex();
|
|
|
|
|
|
- InetSocketAddress omNodeRpcAddr = omNodeDetails.getRpcAddress();
|
|
|
|
- omRpcAddressTxt = new Text(omNodeDetails.getRpcAddressString());
|
|
|
|
- secConfig = new SecurityConfig(configuration);
|
|
|
|
- volumeManager = new VolumeManagerImpl(metadataManager, configuration);
|
|
|
|
|
|
+ metrics = OMMetrics.create();
|
|
|
|
|
|
- // Create the KMS Key Provider
|
|
|
|
- try {
|
|
|
|
- kmsProvider = createKeyProviderExt(configuration);
|
|
|
|
- } catch (IOException ioe) {
|
|
|
|
- kmsProvider = null;
|
|
|
|
- LOG.error("Fail to create Key Provider");
|
|
|
|
- }
|
|
|
|
|
|
+ // Start Om Rpc Server.
|
|
|
|
+ omRpcServer = getRpcServer(conf);
|
|
|
|
+ omRpcAddress = updateRPCListenAddress(configuration,
|
|
|
|
+ OZONE_OM_ADDRESS_KEY, omNodeRpcAddr, omRpcServer);
|
|
|
|
+
|
|
|
|
+ shutdownHook = () -> {
|
|
|
|
+ saveOmMetrics();
|
|
|
|
+ };
|
|
|
|
+ ShutdownHookManager.get().addShutdownHook(shutdownHook,
|
|
|
|
+ SHUTDOWN_HOOK_PRIORITY);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Instantiate services which are dependent on the OM DB state.
|
|
|
|
+ * When OM state is reloaded, these services are re-initialized with the
|
|
|
|
+ * new OM state.
|
|
|
|
+ */
|
|
|
|
+ private void instantiateServices() throws IOException {
|
|
|
|
|
|
|
|
+ metadataManager = new OmMetadataManagerImpl(configuration);
|
|
|
|
+ volumeManager = new VolumeManagerImpl(metadataManager, configuration);
|
|
bucketManager = new BucketManagerImpl(metadataManager, getKmsProvider(),
|
|
bucketManager = new BucketManagerImpl(metadataManager, getKmsProvider(),
|
|
isRatisEnabled);
|
|
isRatisEnabled);
|
|
- metrics = OMMetrics.create();
|
|
|
|
-
|
|
|
|
s3BucketManager = new S3BucketManagerImpl(configuration, metadataManager,
|
|
s3BucketManager = new S3BucketManagerImpl(configuration, metadataManager,
|
|
volumeManager, bucketManager);
|
|
volumeManager, bucketManager);
|
|
if (secConfig.isSecurityEnabled()) {
|
|
if (secConfig.isSecurityEnabled()) {
|
|
- omComponent = OM_DAEMON + "-" + omId;
|
|
|
|
- if(omStorage.getOmCertSerialId() == null) {
|
|
|
|
- throw new RuntimeException("OzoneManager started in secure mode but " +
|
|
|
|
- "doesn't have SCM signed certificate.");
|
|
|
|
- }
|
|
|
|
- certClient = new OMCertificateClient(new SecurityConfig(conf),
|
|
|
|
- omStorage.getOmCertSerialId());
|
|
|
|
s3SecretManager = new S3SecretManagerImpl(configuration, metadataManager);
|
|
s3SecretManager = new S3SecretManagerImpl(configuration, metadataManager);
|
|
delegationTokenMgr = createDelegationTokenSecretManager(configuration);
|
|
delegationTokenMgr = createDelegationTokenSecretManager(configuration);
|
|
}
|
|
}
|
|
- if (secConfig.isBlockTokenEnabled()) {
|
|
|
|
- blockTokenMgr = createBlockTokenSecretManager(configuration);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- omRpcServer = getRpcServer(conf);
|
|
|
|
- omRpcAddress = updateRPCListenAddress(configuration,
|
|
|
|
- OZONE_OM_ADDRESS_KEY, omNodeRpcAddr, omRpcServer);
|
|
|
|
-
|
|
|
|
- this.scmClient = new ScmClient(scmBlockClient, scmContainerClient);
|
|
|
|
|
|
|
|
prefixManager = new PrefixManagerImpl(metadataManager);
|
|
prefixManager = new PrefixManagerImpl(metadataManager);
|
|
keyManager = new KeyManagerImpl(this, scmClient, configuration,
|
|
keyManager = new KeyManagerImpl(this, scmClient, configuration,
|
|
omStorage.getOmId());
|
|
omStorage.getOmId());
|
|
|
|
|
|
- shutdownHook = () -> {
|
|
|
|
- saveOmMetrics();
|
|
|
|
- };
|
|
|
|
- ShutdownHookManager.get().addShutdownHook(shutdownHook,
|
|
|
|
- SHUTDOWN_HOOK_PRIORITY);
|
|
|
|
- isAclEnabled = conf.getBoolean(OZONE_ACL_ENABLED,
|
|
|
|
- OZONE_ACL_ENABLED_DEFAULT);
|
|
|
|
if (isAclEnabled) {
|
|
if (isAclEnabled) {
|
|
- accessAuthorizer = getACLAuthorizerInstance(conf);
|
|
|
|
|
|
+ accessAuthorizer = getACLAuthorizerInstance(configuration);
|
|
if (accessAuthorizer instanceof OzoneNativeAuthorizer) {
|
|
if (accessAuthorizer instanceof OzoneNativeAuthorizer) {
|
|
OzoneNativeAuthorizer authorizer =
|
|
OzoneNativeAuthorizer authorizer =
|
|
(OzoneNativeAuthorizer) accessAuthorizer;
|
|
(OzoneNativeAuthorizer) accessAuthorizer;
|
|
@@ -425,17 +458,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
} else {
|
|
} else {
|
|
accessAuthorizer = null;
|
|
accessAuthorizer = null;
|
|
}
|
|
}
|
|
- ozAdmins = conf.getTrimmedStringCollection(OZONE_ADMINISTRATORS);
|
|
|
|
- omMetaDir = OmUtils.getOmDbDir(configuration);
|
|
|
|
- this.scmBlockSize = (long) conf.getStorageSize(OZONE_SCM_BLOCK_SIZE,
|
|
|
|
- OZONE_SCM_BLOCK_SIZE_DEFAULT, StorageUnit.BYTES);
|
|
|
|
- this.preallocateBlocksMax = conf.getInt(
|
|
|
|
- OZONE_KEY_PREALLOCATION_BLOCKS_MAX,
|
|
|
|
- OZONE_KEY_PREALLOCATION_BLOCKS_MAX_DEFAULT);
|
|
|
|
- this.grpcBlockTokenEnabled = conf.getBoolean(HDDS_BLOCK_TOKEN_ENABLED,
|
|
|
|
- HDDS_BLOCK_TOKEN_ENABLED_DEFAULT);
|
|
|
|
- this.useRatisForReplication = conf.getBoolean(
|
|
|
|
- DFS_CONTAINER_RATIS_ENABLED_KEY, DFS_CONTAINER_RATIS_ENABLED_DEFAULT);
|
|
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1235,6 +1257,14 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
|
|
|
DefaultMetricsSystem.initialize("OzoneManager");
|
|
DefaultMetricsSystem.initialize("OzoneManager");
|
|
|
|
|
|
|
|
+ // Start Ratis services
|
|
|
|
+ if (omRatisServer != null) {
|
|
|
|
+ omRatisServer.start();
|
|
|
|
+ }
|
|
|
|
+ if (omRatisClient != null) {
|
|
|
|
+ omRatisClient.connect();
|
|
|
|
+ }
|
|
|
|
+
|
|
metadataManager.start(configuration);
|
|
metadataManager.start(configuration);
|
|
startSecretManagerIfNecessary();
|
|
startSecretManagerIfNecessary();
|
|
|
|
|
|
@@ -1305,8 +1335,14 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
omRpcServer.start();
|
|
omRpcServer.start();
|
|
isOmRpcServerRunning = true;
|
|
isOmRpcServerRunning = true;
|
|
|
|
|
|
- startRatisServer();
|
|
|
|
- startRatisClient();
|
|
|
|
|
|
+ initializeRatisServer();
|
|
|
|
+ if (omRatisServer != null) {
|
|
|
|
+ omRatisServer.start();
|
|
|
|
+ }
|
|
|
|
+ initializeRatisClient();
|
|
|
|
+ if (omRatisClient != null) {
|
|
|
|
+ omRatisClient.connect();
|
|
|
|
+ }
|
|
|
|
|
|
try {
|
|
try {
|
|
httpServer = new OzoneManagerHttpServer(configuration, this);
|
|
httpServer = new OzoneManagerHttpServer(configuration, this);
|
|
@@ -1353,15 +1389,13 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
/**
|
|
/**
|
|
* Creates an instance of ratis server.
|
|
* Creates an instance of ratis server.
|
|
*/
|
|
*/
|
|
- private void startRatisServer() throws IOException {
|
|
|
|
|
|
+ private void initializeRatisServer() throws IOException {
|
|
if (isRatisEnabled) {
|
|
if (isRatisEnabled) {
|
|
if (omRatisServer == null) {
|
|
if (omRatisServer == null) {
|
|
omRatisServer = OzoneManagerRatisServer.newOMRatisServer(
|
|
omRatisServer = OzoneManagerRatisServer.newOMRatisServer(
|
|
configuration, this, omNodeDetails, peerNodes);
|
|
configuration, this, omNodeDetails, peerNodes);
|
|
}
|
|
}
|
|
- omRatisServer.start();
|
|
|
|
-
|
|
|
|
- LOG.info("OzoneManager Ratis server started at port {}",
|
|
|
|
|
|
+ LOG.info("OzoneManager Ratis server initialized at port {}",
|
|
omRatisServer.getServerPort());
|
|
omRatisServer.getServerPort());
|
|
} else {
|
|
} else {
|
|
omRatisServer = null;
|
|
omRatisServer = null;
|
|
@@ -1371,14 +1405,13 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
/**
|
|
/**
|
|
* Creates an instance of ratis client.
|
|
* Creates an instance of ratis client.
|
|
*/
|
|
*/
|
|
- private void startRatisClient() throws IOException {
|
|
|
|
|
|
+ private void initializeRatisClient() throws IOException {
|
|
if (isRatisEnabled) {
|
|
if (isRatisEnabled) {
|
|
if (omRatisClient == null) {
|
|
if (omRatisClient == null) {
|
|
omRatisClient = OzoneManagerRatisClient.newOzoneManagerRatisClient(
|
|
omRatisClient = OzoneManagerRatisClient.newOzoneManagerRatisClient(
|
|
omNodeDetails.getOMNodeId(), omRatisServer.getRaftGroup(),
|
|
omNodeDetails.getOMNodeId(), omRatisServer.getRaftGroup(),
|
|
configuration);
|
|
configuration);
|
|
}
|
|
}
|
|
- omRatisClient.connect();
|
|
|
|
} else {
|
|
} else {
|
|
omRatisClient = null;
|
|
omRatisClient = null;
|
|
}
|
|
}
|
|
@@ -1398,11 +1431,13 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
- public long saveRatisSnapshot() throws IOException {
|
|
|
|
|
|
+ public long saveRatisSnapshot(boolean flush) throws IOException {
|
|
snapshotIndex = omRatisServer.getStateMachineLastAppliedIndex();
|
|
snapshotIndex = omRatisServer.getStateMachineLastAppliedIndex();
|
|
|
|
|
|
- // Flush the OM state to disk
|
|
|
|
- getMetadataManager().getStore().flush();
|
|
|
|
|
|
+ if (flush) {
|
|
|
|
+ // Flush the OM state to disk
|
|
|
|
+ metadataManager.getStore().flush();
|
|
|
|
+ }
|
|
|
|
|
|
PersistentLongFile.writeFile(ratisSnapshotFile, snapshotIndex);
|
|
PersistentLongFile.writeFile(ratisSnapshotFile, snapshotIndex);
|
|
LOG.info("Saved Ratis Snapshot on the OM with snapshotIndex {}",
|
|
LOG.info("Saved Ratis Snapshot on the OM with snapshotIndex {}",
|
|
@@ -2697,7 +2732,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
@Override
|
|
@Override
|
|
public OmMultipartInfo initiateMultipartUpload(OmKeyArgs keyArgs) throws
|
|
public OmMultipartInfo initiateMultipartUpload(OmKeyArgs keyArgs) throws
|
|
IOException {
|
|
IOException {
|
|
@@ -3069,6 +3103,179 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Download and install latest checkpoint from leader OM.
|
|
|
|
+ * If the download checkpoints snapshot index is greater than this OM's
|
|
|
|
+ * last applied transaction index, then re-initialize the OM state via this
|
|
|
|
+ * checkpoint. Before re-initializing OM state, the OM Ratis server should
|
|
|
|
+ * be stopped so that no new transactions can be applied.
|
|
|
|
+ * @param leaderId peerNodeID of the leader OM
|
|
|
|
+ * @return If checkpoint is installed, return the corresponding termIndex.
|
|
|
|
+ * Otherwise, return null.
|
|
|
|
+ */
|
|
|
|
+ public TermIndex installSnapshot(String leaderId) {
|
|
|
|
+ if (omSnapshotProvider == null) {
|
|
|
|
+ LOG.error("OM Snapshot Provider is not configured as there are no peer " +
|
|
|
|
+ "nodes.");
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ DBCheckpoint omDBcheckpoint = getDBCheckpointFromLeader(leaderId);
|
|
|
|
+ Path newDBlocation = omDBcheckpoint.getCheckpointLocation();
|
|
|
|
+
|
|
|
|
+ // Check if current ratis log index is smaller than the downloaded
|
|
|
|
+ // snapshot index. If yes, proceed by stopping the ratis server so that
|
|
|
|
+ // the OM state can be re-initialized. If no, then do not proceed with
|
|
|
|
+ // installSnapshot.
|
|
|
|
+ long lastAppliedIndex = omRatisServer.getStateMachineLastAppliedIndex();
|
|
|
|
+ long checkpointSnapshotIndex = omDBcheckpoint.getRatisSnapshotIndex();
|
|
|
|
+ if (checkpointSnapshotIndex <= lastAppliedIndex) {
|
|
|
|
+ LOG.error("Failed to install checkpoint from OM leader: {}. The last " +
|
|
|
|
+ "applied index: {} is greater than or equal to the checkpoint's " +
|
|
|
|
+ "snapshot index: {}. Deleting the downloaded checkpoint {}", leaderId,
|
|
|
|
+ lastAppliedIndex, checkpointSnapshotIndex,
|
|
|
|
+ newDBlocation);
|
|
|
|
+ try {
|
|
|
|
+ FileUtils.deleteFully(newDBlocation);
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.error("Failed to fully delete the downloaded DB checkpoint {} " +
|
|
|
|
+ "from OM leader {}.", newDBlocation,
|
|
|
|
+ leaderId, e);
|
|
|
|
+ }
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // Pause the State Machine so that no new transactions can be applied.
|
|
|
|
+ // This action also clears the OM Double Buffer so that if there are any
|
|
|
|
+ // pending transactions in the buffer, they are discarded.
|
|
|
|
+ // TODO: The Ratis server should also be paused here. This is required
|
|
|
|
+ // because a leader election might happen while the snapshot
|
|
|
|
+ // installation is in progress and the new leader might start sending
|
|
|
|
+ // append log entries to the ratis server.
|
|
|
|
+ omRatisServer.getOmStateMachine().pause();
|
|
|
|
+
|
|
|
|
+ File dbBackup;
|
|
|
|
+ try {
|
|
|
|
+ dbBackup = replaceOMDBWithCheckpoint(lastAppliedIndex, newDBlocation);
|
|
|
|
+ } catch (Exception e) {
|
|
|
|
+ LOG.error("OM DB checkpoint replacement with new downloaded checkpoint " +
|
|
|
|
+ "failed.", e);
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // Reload the OM DB store with the new checkpoint.
|
|
|
|
+ // Restart (unpause) the state machine and update its last applied index
|
|
|
|
+ // to the installed checkpoint's snapshot index.
|
|
|
|
+ try {
|
|
|
|
+ reloadOMState(checkpointSnapshotIndex);
|
|
|
|
+ omRatisServer.getOmStateMachine().unpause(checkpointSnapshotIndex);
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.error("Failed to reload OM state with new DB checkpoint.", e);
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // Delete the backup DB
|
|
|
|
+ try {
|
|
|
|
+ FileUtils.deleteFully(dbBackup);
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.error("Failed to delete the backup of the original DB {}", dbBackup);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // TODO: We should only return the snpashotIndex to the leader.
|
|
|
|
+ // Should be fixed after RATIS-586
|
|
|
|
+ TermIndex newTermIndex = TermIndex.newTermIndex(0,
|
|
|
|
+ checkpointSnapshotIndex);
|
|
|
|
+
|
|
|
|
+ return newTermIndex;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Download the latest OM DB checkpoint from the leader OM.
|
|
|
|
+ * @param leaderId OMNodeID of the leader OM node.
|
|
|
|
+ * @return latest DB checkpoint from leader OM.
|
|
|
|
+ */
|
|
|
|
+ private DBCheckpoint getDBCheckpointFromLeader(String leaderId) {
|
|
|
|
+ LOG.info("Downloading checkpoint from leader OM {} and reloading state " +
|
|
|
|
+ "from the checkpoint.", leaderId);
|
|
|
|
+
|
|
|
|
+ try {
|
|
|
|
+ return omSnapshotProvider.getOzoneManagerDBSnapshot(leaderId);
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.error("Failed to download checkpoint from OM leader {}", leaderId, e);
|
|
|
|
+ }
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Replace the current OM DB with the new DB checkpoint.
|
|
|
|
+ * @param lastAppliedIndex the last applied index in the current OM DB.
|
|
|
|
+ * @param checkpointPath path to the new DB checkpoint
|
|
|
|
+ * @return location of the backup of the original DB
|
|
|
|
+ * @throws Exception
|
|
|
|
+ */
|
|
|
|
+ File replaceOMDBWithCheckpoint(long lastAppliedIndex, Path checkpointPath)
|
|
|
|
+ throws Exception {
|
|
|
|
+ // Stop the DB first
|
|
|
|
+ DBStore store = metadataManager.getStore();
|
|
|
|
+ store.close();
|
|
|
|
+
|
|
|
|
+ // Take a backup of the current DB
|
|
|
|
+ File db = store.getDbLocation();
|
|
|
|
+ String dbBackupName = OzoneConsts.OM_DB_BACKUP_PREFIX +
|
|
|
|
+ lastAppliedIndex + "_" + System.currentTimeMillis();
|
|
|
|
+ File dbBackup = new File(db.getParentFile(), dbBackupName);
|
|
|
|
+
|
|
|
|
+ try {
|
|
|
|
+ Files.move(db.toPath(), dbBackup.toPath());
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.error("Failed to create a backup of the current DB. Aborting " +
|
|
|
|
+ "snapshot installation.");
|
|
|
|
+ throw e;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // Move the new DB checkpoint into the om metadata dir
|
|
|
|
+ try {
|
|
|
|
+ Files.move(checkpointPath, db.toPath());
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.error("Failed to move downloaded DB checkpoint {} to metadata " +
|
|
|
|
+ "directory {}. Resetting to original DB.", checkpointPath,
|
|
|
|
+ db.toPath());
|
|
|
|
+ Files.move(dbBackup.toPath(), db.toPath());
|
|
|
|
+ throw e;
|
|
|
|
+ }
|
|
|
|
+ return dbBackup;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Re-instantiate MetadataManager with new DB checkpoint.
|
|
|
|
+ * All the classes which use/ store MetadataManager should also be updated
|
|
|
|
+ * with the new MetadataManager instance.
|
|
|
|
+ */
|
|
|
|
+ void reloadOMState(long newSnapshotIndex) throws IOException {
|
|
|
|
+
|
|
|
|
+ instantiateServices();
|
|
|
|
+
|
|
|
|
+ // Restart required services
|
|
|
|
+ metadataManager.start(configuration);
|
|
|
|
+ keyManager.start(configuration);
|
|
|
|
+
|
|
|
|
+ // Set metrics and start metrics back ground thread
|
|
|
|
+ metrics.setNumVolumes(metadataManager.countRowsInTable(metadataManager
|
|
|
|
+ .getVolumeTable()));
|
|
|
|
+ metrics.setNumBuckets(metadataManager.countRowsInTable(metadataManager
|
|
|
|
+ .getBucketTable()));
|
|
|
|
+
|
|
|
|
+ // Delete the omMetrics file if it exists and save the a new metrics file
|
|
|
|
+ // with new data
|
|
|
|
+ Files.deleteIfExists(getMetricsStorageFile().toPath());
|
|
|
|
+ saveOmMetrics();
|
|
|
|
+
|
|
|
|
+ // Update OM snapshot index with the new snapshot index (from the new OM
|
|
|
|
+ // DB state) and save the snapshot index to disk
|
|
|
|
+ this.snapshotIndex = newSnapshotIndex;
|
|
|
|
+ saveRatisSnapshot(false);
|
|
|
|
+ }
|
|
|
|
+
|
|
public static Logger getLogger() {
|
|
public static Logger getLogger() {
|
|
return LOG;
|
|
return LOG;
|
|
}
|
|
}
|