|
@@ -67,7 +67,6 @@ import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRe
|
|
import org.apache.hadoop.hdds.server.ServiceRuntimeInfoImpl;
|
|
import org.apache.hadoop.hdds.server.ServiceRuntimeInfoImpl;
|
|
import org.apache.hadoop.hdds.tracing.TracingUtil;
|
|
import org.apache.hadoop.hdds.tracing.TracingUtil;
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
-import org.apache.hadoop.hdfs.util.PersistentLongFile;
|
|
|
|
import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.io.retry.RetryPolicy;
|
|
import org.apache.hadoop.io.retry.RetryPolicy;
|
|
import org.apache.hadoop.ipc.Client;
|
|
import org.apache.hadoop.ipc.Client;
|
|
@@ -81,6 +80,7 @@ import org.apache.hadoop.ozone.OzoneSecurityUtil;
|
|
import org.apache.hadoop.ozone.om.ha.OMFailoverProxyProvider;
|
|
import org.apache.hadoop.ozone.om.ha.OMFailoverProxyProvider;
|
|
import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
|
|
import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
|
|
import org.apache.hadoop.ozone.om.protocol.OzoneManagerServerProtocol;
|
|
import org.apache.hadoop.ozone.om.protocol.OzoneManagerServerProtocol;
|
|
|
|
+import org.apache.hadoop.ozone.om.ratis.OMRatisSnapshotInfo;
|
|
import org.apache.hadoop.ozone.om.snapshot.OzoneManagerSnapshotProvider;
|
|
import org.apache.hadoop.ozone.om.snapshot.OzoneManagerSnapshotProvider;
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DBUpdatesRequest;
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DBUpdatesRequest;
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
@@ -196,7 +196,6 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE;
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT;
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT;
|
|
import static org.apache.hadoop.ozone.OzoneConsts.OM_METRICS_FILE;
|
|
import static org.apache.hadoop.ozone.OzoneConsts.OM_METRICS_FILE;
|
|
import static org.apache.hadoop.ozone.OzoneConsts.OM_METRICS_TEMP_FILE;
|
|
import static org.apache.hadoop.ozone.OzoneConsts.OM_METRICS_TEMP_FILE;
|
|
-import static org.apache.hadoop.ozone.OzoneConsts.OM_RATIS_SNAPSHOT_INDEX;
|
|
|
|
import static org.apache.hadoop.ozone.OzoneConsts.RPC_PORT;
|
|
import static org.apache.hadoop.ozone.OzoneConsts.RPC_PORT;
|
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
|
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
|
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HANDLER_COUNT_DEFAULT;
|
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HANDLER_COUNT_DEFAULT;
|
|
@@ -284,8 +283,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
private OMNodeDetails omNodeDetails;
|
|
private OMNodeDetails omNodeDetails;
|
|
private List<OMNodeDetails> peerNodes;
|
|
private List<OMNodeDetails> peerNodes;
|
|
private File omRatisSnapshotDir;
|
|
private File omRatisSnapshotDir;
|
|
- private final File ratisSnapshotFile;
|
|
|
|
- private long snapshotIndex;
|
|
|
|
|
|
+ private final OMRatisSnapshotInfo omRatisSnapshotInfo;
|
|
private final Collection<String> ozAdmins;
|
|
private final Collection<String> ozAdmins;
|
|
|
|
|
|
private KeyProviderCryptoExtension kmsProvider = null;
|
|
private KeyProviderCryptoExtension kmsProvider = null;
|
|
@@ -388,6 +386,9 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
|
|
|
instantiateServices();
|
|
instantiateServices();
|
|
|
|
|
|
|
|
+ this.omRatisSnapshotInfo = new OMRatisSnapshotInfo(
|
|
|
|
+ omStorage.getCurrentDir());
|
|
|
|
+
|
|
initializeRatisServer();
|
|
initializeRatisServer();
|
|
initializeRatisClient();
|
|
initializeRatisClient();
|
|
|
|
|
|
@@ -409,10 +410,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- this.ratisSnapshotFile = new File(omStorage.getCurrentDir(),
|
|
|
|
- OM_RATIS_SNAPSHOT_INDEX);
|
|
|
|
- this.snapshotIndex = loadRatisSnapshotIndex();
|
|
|
|
-
|
|
|
|
metrics = OMMetrics.create();
|
|
metrics = OMMetrics.create();
|
|
|
|
|
|
// Start Om Rpc Server.
|
|
// Start Om Rpc Server.
|
|
@@ -1313,7 +1310,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
|
|
|
HddsUtils.initializeMetrics(configuration, "OzoneManager");
|
|
HddsUtils.initializeMetrics(configuration, "OzoneManager");
|
|
|
|
|
|
- metadataManager.start(configuration);
|
|
|
|
|
|
+ instantiateServices();
|
|
|
|
+
|
|
startSecretManagerIfNecessary();
|
|
startSecretManagerIfNecessary();
|
|
|
|
|
|
// Set metrics and start metrics back ground thread
|
|
// Set metrics and start metrics back ground thread
|
|
@@ -1334,7 +1332,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
metricsTimer = new Timer();
|
|
metricsTimer = new Timer();
|
|
metricsTimer.schedule(scheduleOMMetricsWriteTask, 0, period);
|
|
metricsTimer.schedule(scheduleOMMetricsWriteTask, 0, period);
|
|
|
|
|
|
- keyManager.start(configuration);
|
|
|
|
omRpcServer = getRpcServer(configuration);
|
|
omRpcServer = getRpcServer(configuration);
|
|
omRpcServer.start();
|
|
omRpcServer.start();
|
|
isOmRpcServerRunning = true;
|
|
isOmRpcServerRunning = true;
|
|
@@ -1420,31 +1417,23 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public OMRatisSnapshotInfo getSnapshotInfo() {
|
|
|
|
+ return omRatisSnapshotInfo;
|
|
|
|
+ }
|
|
|
|
+
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
- public long loadRatisSnapshotIndex() {
|
|
|
|
- if (ratisSnapshotFile.exists()) {
|
|
|
|
- try {
|
|
|
|
- return PersistentLongFile.readFile(ratisSnapshotFile, 0);
|
|
|
|
- } catch (IOException e) {
|
|
|
|
- LOG.error("Unable to read the ratis snapshot index (last applied " +
|
|
|
|
- "transaction log index)", e);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- return 0;
|
|
|
|
|
|
+ public long getRatisSnapshotIndex() {
|
|
|
|
+ return omRatisSnapshotInfo.getIndex();
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
- public long saveRatisSnapshot(boolean flush) throws IOException {
|
|
|
|
- snapshotIndex = omRatisServer.getStateMachineLastAppliedIndex();
|
|
|
|
|
|
+ public long saveRatisSnapshot() throws IOException {
|
|
|
|
+ long snapshotIndex = omRatisServer.getStateMachineLastAppliedIndex();
|
|
|
|
|
|
- if (flush) {
|
|
|
|
- // Flush the OM state to disk
|
|
|
|
- metadataManager.getStore().flush();
|
|
|
|
- }
|
|
|
|
|
|
+ // Flush the OM state to disk
|
|
|
|
+ metadataManager.getStore().flush();
|
|
|
|
|
|
- PersistentLongFile.writeFile(ratisSnapshotFile, snapshotIndex);
|
|
|
|
- LOG.info("Saved Ratis Snapshot on the OM with snapshotIndex {}",
|
|
|
|
- snapshotIndex);
|
|
|
|
|
|
+ omRatisSnapshotInfo.saveRatisSnapshotToDisk(snapshotIndex);
|
|
|
|
|
|
return snapshotIndex;
|
|
return snapshotIndex;
|
|
}
|
|
}
|
|
@@ -1468,9 +1457,11 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
}
|
|
}
|
|
if (omRatisServer != null) {
|
|
if (omRatisServer != null) {
|
|
omRatisServer.stop();
|
|
omRatisServer.stop();
|
|
|
|
+ omRatisServer = null;
|
|
}
|
|
}
|
|
if (omRatisClient != null) {
|
|
if (omRatisClient != null) {
|
|
omRatisClient.close();
|
|
omRatisClient.close();
|
|
|
|
+ omRatisClient = null;
|
|
}
|
|
}
|
|
isOmRpcServerRunning = false;
|
|
isOmRpcServerRunning = false;
|
|
keyManager.stop();
|
|
keyManager.stop();
|
|
@@ -3349,8 +3340,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
|
|
|
// Update OM snapshot index with the new snapshot index (from the new OM
|
|
// Update OM snapshot index with the new snapshot index (from the new OM
|
|
// DB state) and save the snapshot index to disk
|
|
// DB state) and save the snapshot index to disk
|
|
- this.snapshotIndex = newSnapshotIndex;
|
|
|
|
- saveRatisSnapshot(false);
|
|
|
|
|
|
+ omRatisSnapshotInfo.saveRatisSnapshotToDisk(newSnapshotIndex);
|
|
}
|
|
}
|
|
|
|
|
|
public static Logger getLogger() {
|
|
public static Logger getLogger() {
|