|
@@ -78,6 +78,7 @@ import org.apache.hadoop.ozone.om.helpers.OmDeleteVolumeResponse;
|
|
|
import org.apache.hadoop.ozone.om.helpers.OmVolumeOwnerChangeResponse;
|
|
|
import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
|
|
|
import org.apache.hadoop.ozone.om.protocol.OzoneManagerServerProtocol;
|
|
|
+import org.apache.hadoop.ozone.om.snapshot.OzoneManagerSnapshotProvider;
|
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
|
.KeyArgs;
|
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
@@ -184,22 +185,17 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_ENABLED;
|
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_ENABLED_DEFAULT;
|
|
|
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_RATIS_SNAPSHOT_INDEX;
|
|
|
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_HANDLER_COUNT_DEFAULT;
|
|
|
-import static org.apache.hadoop.ozone.om.OMConfigKeys
|
|
|
- .OZONE_OM_HANDLER_COUNT_KEY;
|
|
|
-import static org.apache.hadoop.ozone.om.OMConfigKeys
|
|
|
- .OZONE_OM_METRICS_SAVE_INTERVAL;
|
|
|
-import static org.apache.hadoop.ozone.om.OMConfigKeys
|
|
|
- .OZONE_OM_METRICS_SAVE_INTERVAL_DEFAULT;
|
|
|
-
|
|
|
+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_KEY;
|
|
|
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_KERBEROS_KEYTAB_FILE_KEY;
|
|
|
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_KERBEROS_PRINCIPAL_KEY;
|
|
|
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_METRICS_SAVE_INTERVAL;
|
|
|
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_METRICS_SAVE_INTERVAL_DEFAULT;
|
|
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_NODE_ID_KEY;
|
|
|
-import static org.apache.hadoop.ozone.om.OMConfigKeys
|
|
|
- .OZONE_OM_RATIS_PORT_DEFAULT;
|
|
|
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_RATIS_PORT_DEFAULT;
|
|
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SERVICE_IDS_KEY;
|
|
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_RATIS_PORT_KEY;
|
|
|
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_AUTH_METHOD;
|
|
@@ -208,10 +204,6 @@ import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TOKE
|
|
|
import static org.apache.hadoop.ozone.protocol.proto
|
|
|
.OzoneManagerProtocolProtos.OzoneManagerService
|
|
|
.newReflectiveBlockingService;
|
|
|
-import static org.apache.hadoop.ozone.om.OMConfigKeys
|
|
|
- .OZONE_OM_KERBEROS_KEYTAB_FILE_KEY;
|
|
|
-import static org.apache.hadoop.ozone.om.OMConfigKeys
|
|
|
- .OZONE_OM_KERBEROS_PRINCIPAL_KEY;
|
|
|
import static org.apache.hadoop.util.ExitUtil.terminate;
|
|
|
|
|
|
/**
|
|
@@ -241,7 +233,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
private RPC.Server omRpcServer;
|
|
|
private InetSocketAddress omRpcAddress;
|
|
|
private String omId;
|
|
|
- private List<OMNodeDetails> peerNodes;
|
|
|
private final OMMetadataManager metadataManager;
|
|
|
private final VolumeManager volumeManager;
|
|
|
private final BucketManager bucketManager;
|
|
@@ -273,7 +264,10 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
private boolean isRatisEnabled;
|
|
|
private OzoneManagerRatisServer omRatisServer;
|
|
|
private OzoneManagerRatisClient omRatisClient;
|
|
|
+ private OzoneManagerSnapshotProvider omSnapshotProvider;
|
|
|
private OMNodeDetails omNodeDetails;
|
|
|
+ private List<OMNodeDetails> peerNodes;
|
|
|
+ private File omRatisSnapshotDir;
|
|
|
private final File ratisSnapshotFile;
|
|
|
private long snapshotIndex;
|
|
|
|
|
@@ -319,6 +313,11 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
startRatisServer();
|
|
|
startRatisClient();
|
|
|
|
|
|
+ if (peerNodes != null && !peerNodes.isEmpty()) {
|
|
|
+ this.omSnapshotProvider = new OzoneManagerSnapshotProvider(configuration,
|
|
|
+ omRatisSnapshotDir, peerNodes);
|
|
|
+ }
|
|
|
+
|
|
|
this.ratisSnapshotFile = new File(omStorage.getCurrentDir(),
|
|
|
OM_RATIS_SNAPSHOT_INDEX);
|
|
|
this.snapshotIndex = loadRatisSnapshotIndex();
|
|
@@ -449,11 +448,17 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
} else {
|
|
|
// This OMNode belongs to same OM service as the current OMNode.
|
|
|
// Add it to peerNodes list.
|
|
|
+ String httpAddr = OmUtils.getHttpAddressForOMPeerNode(conf,
|
|
|
+ serviceId, nodeId, addr.getHostName());
|
|
|
+ String httpsAddr = OmUtils.getHttpsAddressForOMPeerNode(conf,
|
|
|
+ serviceId, nodeId, addr.getHostName());
|
|
|
OMNodeDetails peerNodeInfo = new OMNodeDetails.Builder()
|
|
|
.setOMServiceId(serviceId)
|
|
|
.setOMNodeId(nodeId)
|
|
|
.setRpcAddress(addr)
|
|
|
.setRatisPort(ratisPort)
|
|
|
+ .setHttpAddress(httpAddr)
|
|
|
+ .setHttpsAddress(httpsAddr)
|
|
|
.build();
|
|
|
peerNodesList.add(peerNodeInfo);
|
|
|
}
|
|
@@ -465,6 +470,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
|
|
|
setOMNodeDetails(localOMServiceId, localOMNodeId, localRpcAddress,
|
|
|
localRatisPort);
|
|
|
+
|
|
|
this.peerNodes = peerNodesList;
|
|
|
|
|
|
LOG.info("Found matching OM address with OMServiceId: {}, " +
|
|
@@ -530,6 +536,49 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
// Set this nodes OZONE_OM_ADDRESS_KEY to the discovered address.
|
|
|
configuration.set(OZONE_OM_ADDRESS_KEY,
|
|
|
NetUtils.getHostPortString(rpcAddress));
|
|
|
+
|
|
|
+ // Create Ratis storage dir
|
|
|
+ String omRatisDirectory = OmUtils.getOMRatisDirectory(configuration);
|
|
|
+ if (omRatisDirectory == null || omRatisDirectory.isEmpty()) {
|
|
|
+ throw new IllegalArgumentException(HddsConfigKeys.OZONE_METADATA_DIRS +
|
|
|
+ " must be defined.");
|
|
|
+ }
|
|
|
+ OmUtils.createOMDir(omRatisDirectory);
|
|
|
+
|
|
|
+ // Create Ratis snapshot dir
|
|
|
+ omRatisSnapshotDir = OmUtils.createOMDir(
|
|
|
+ OmUtils.getOMRatisSnapshotDirectory(configuration));
|
|
|
+
|
|
|
+ // Get and set Http(s) address of local node. If base config keys are
|
|
|
+ // not set, check for keys suffixed with OM serivce ID and node ID.
|
|
|
+ setOMNodeSpecificConfigs(serviceId, nodeId);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Check if any of the following configuration keys have been set using OM
|
|
|
+ * Node ID suffixed to the key. If yes, then set the base key with the
|
|
|
+ * configured valued.
|
|
|
+ * 1. {@link OMConfigKeys#OZONE_OM_HTTP_ADDRESS_KEY}
|
|
|
+ * 2. {@link OMConfigKeys#OZONE_OM_HTTPS_ADDRESS_KEY}
|
|
|
+ * 3. {@link OMConfigKeys#OZONE_OM_HTTP_BIND_HOST_KEY}
|
|
|
+ * 4. {@link OMConfigKeys#OZONE_OM_HTTPS_BIND_HOST_KEY}
|
|
|
+ */
|
|
|
+ private void setOMNodeSpecificConfigs(String omServiceId, String omNodeId) {
|
|
|
+ String[] confKeys = new String[] {
|
|
|
+ OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY,
|
|
|
+ OMConfigKeys.OZONE_OM_HTTPS_ADDRESS_KEY,
|
|
|
+ OMConfigKeys.OZONE_OM_HTTP_BIND_HOST_KEY,
|
|
|
+ OMConfigKeys.OZONE_OM_HTTPS_BIND_HOST_KEY};
|
|
|
+
|
|
|
+ for (String confKey : confKeys) {
|
|
|
+ String confValue = OmUtils.getConfSuffixedWithOMNodeId(
|
|
|
+ configuration, confKey, omServiceId, omNodeId);
|
|
|
+ if (confValue != null) {
|
|
|
+ LOG.info("Setting configuration key {} with value of key {}: {}",
|
|
|
+ confKey, OmUtils.addKeySuffixes(confKey, omNodeId), confValue);
|
|
|
+ configuration.set(confKey, confValue);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
private KeyProviderCryptoExtension createKeyProviderExt(
|
|
@@ -1124,6 +1173,11 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
return omRatisServer;
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ public OzoneManagerSnapshotProvider getOmSnapshotProvider() {
|
|
|
+ return omSnapshotProvider;
|
|
|
+ }
|
|
|
+
|
|
|
@VisibleForTesting
|
|
|
public InetSocketAddress getOmRpcServerAddr() {
|
|
|
return omRpcAddress;
|