Skip to content

Commit

Permalink
HDDS-10937. Ozone Recon - Handle startup failure and log reasons as e…
Browse files Browse the repository at this point in the history
…rror due to SCM non-HA scenario (#6752)
  • Loading branch information
devmadhuu authored Jun 4, 2024
1 parent d2ee09f commit f3a0dbd
Show file tree
Hide file tree
Showing 6 changed files with 153 additions and 48 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@
/**
* Test Recon SCM HA Snapshot Download implementation.
*/
@Timeout(100)
@Timeout(300)
public class TestReconScmHASnapshot {
private OzoneConfiguration conf;
private MiniOzoneCluster ozoneCluster = null;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.recon;

import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;

import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HA_ENABLE_KEY;

/**
* Test Recon SCM HA Snapshot Download implementation.
*/
@Timeout(300)
public class TestReconScmNonHASnapshot {
private OzoneConfiguration conf;
private MiniOzoneCluster ozoneCluster = null;

@BeforeEach
public void setup() throws Exception {
conf = new OzoneConfiguration();
conf.setBoolean(OZONE_SCM_HA_ENABLE_KEY, false);
conf.setBoolean(
ReconServerConfigKeys.OZONE_RECON_SCM_SNAPSHOT_ENABLED, true);
conf.setInt(ReconServerConfigKeys.OZONE_RECON_SCM_CONTAINER_THRESHOLD, 0);
conf.setInt(ScmConfigKeys.OZONE_DATANODE_PIPELINE_LIMIT, 5);
ozoneCluster = MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(4)
.includeRecon(true)
.build();
ozoneCluster.waitForClusterToBeReady();
}

@Test
public void testScmNonHASnapshot() throws Exception {
TestReconScmSnapshot.testSnapshot(ozoneCluster);
}

@AfterEach
public void shutdown() throws Exception {
if (ozoneCluster != null) {
ozoneCluster.shutdown();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,10 @@ public enum ErrorCode {
Arrays.asList("Recon health")),
GET_OM_DB_SNAPSHOT_FAILED(
"OM DB Snapshot sync failed !!!",
Arrays.asList("Overview (OM Data)", "OM DB Insights"));
Arrays.asList("Overview (OM Data)", "OM DB Insights")),
GET_SCM_DB_SNAPSHOT_FAILED(
"SCM DB Snapshot sync failed !!!",
Arrays.asList("Containers", "Pipelines"));

private final String message;
private final List<String> impacts;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import org.apache.hadoop.hdds.security.SecurityConfig;
import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
import org.apache.hadoop.ozone.recon.api.types.FeatureProvider;
import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade;
import org.apache.hadoop.ozone.recon.security.ReconCertificateClient;
import org.apache.hadoop.hdds.utils.HddsServerUtil;
import org.apache.hadoop.ozone.OzoneSecurityUtil;
Expand All @@ -55,6 +56,7 @@

import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.concurrent.atomic.AtomicBoolean;

import static org.apache.hadoop.hdds.ratis.RatisHelper.newJvmPauseMonitor;
import static org.apache.hadoop.hdds.recon.ReconConfig.ConfigStrings.OZONE_RECON_KERBEROS_KEYTAB_FILE_KEY;
Expand Down Expand Up @@ -139,6 +141,7 @@ public Void call() throws Exception {
injector.getInstance(ReconSchemaManager.class);
LOG.info("Creating Recon Schema.");
reconSchemaManager.createReconSchema();
LOG.debug("Recon schema creation done.");

this.reconSafeModeMgr = injector.getInstance(ReconSafeModeManager.class);
this.reconSafeModeMgr.setInSafeMode(true);
Expand All @@ -153,13 +156,20 @@ public Void call() throws Exception {

LOG.info("Initializing support of Recon Features...");
FeatureProvider.initFeatureSupport(configuration);

LOG.debug("Now starting all services of Recon...");
// Start all services
start();
isStarted = true;
LOG.info("Recon server initialized successfully!");
} catch (Exception e) {
ReconStorageContainerManagerFacade reconStorageContainerManagerFacade =
(ReconStorageContainerManagerFacade) this.getReconStorageContainerManager();
ReconContext reconContext = reconStorageContainerManagerFacade.getReconContext();
reconContext.updateHealthStatus(new AtomicBoolean(false));
reconContext.getErrors().add(ReconContext.ErrorCode.INTERNAL_ERROR);
LOG.error("Error during initializing Recon server.", e);
}
// Start all services
start();
isStarted = true;

ShutdownHookManager.get().addShutdownHook(() -> {
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -745,4 +745,8 @@ public ContainerSizeCountTask getContainerSizeCountTask() {
public ContainerCountBySizeDao getContainerCountBySizeDao() {
return containerCountBySizeDao;
}

public ReconContext getReconContext() {
return reconContext;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,8 +36,10 @@
import java.util.List;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;

import javax.inject.Inject;
import javax.validation.constraints.NotNull;

import org.apache.commons.io.FileUtils;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
Expand All @@ -47,7 +49,6 @@
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.ha.InterSCMGrpcClient;
import org.apache.hadoop.hdds.scm.ha.SCMHAUtils;
import org.apache.hadoop.hdds.scm.ha.SCMSnapshotDownloader;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
Expand All @@ -57,6 +58,7 @@
import org.apache.hadoop.hdds.utils.db.RocksDBCheckpoint;
import org.apache.hadoop.hdfs.web.URLConnectionFactory;
import org.apache.hadoop.ozone.ClientVersion;
import org.apache.hadoop.ozone.recon.ReconContext;
import org.apache.hadoop.ozone.recon.ReconUtils;
import org.apache.hadoop.ozone.recon.scm.ReconStorageConfig;
import org.apache.hadoop.ozone.recon.security.ReconCertificateClient;
Expand All @@ -82,13 +84,15 @@ public class StorageContainerServiceProviderImpl
private URLConnectionFactory connectionFactory;
private ReconUtils reconUtils;
private ReconStorageConfig reconStorage;
private ReconContext reconContext;

@Inject
public StorageContainerServiceProviderImpl(
StorageContainerLocationProtocol scmClient,
ReconUtils reconUtils,
OzoneConfiguration configuration,
ReconStorageConfig reconStorage) {
ReconStorageConfig reconStorage,
ReconContext reconContext) {

int connectionTimeout = (int) configuration.getTimeDuration(
OZONE_RECON_SCM_CONNECTION_TIMEOUT,
Expand Down Expand Up @@ -123,6 +127,7 @@ public StorageContainerServiceProviderImpl(
this.scmClient = scmClient;
this.configuration = configuration;
this.reconStorage = reconStorage;
this.reconContext = reconContext;
}

@Override
Expand Down Expand Up @@ -178,59 +183,78 @@ public DBCheckpoint getSCMDBSnapshot() {
String snapshotFileName = RECON_SCM_SNAPSHOT_DB + "_" +
System.currentTimeMillis();
File targetFile = new File(scmSnapshotDBParentDir, snapshotFileName +
".tar");

".tar");
try {
if (!SCMHAUtils.isSCMHAEnabled(configuration)) {
SecurityUtil.doAsLoginUser(() -> {
try (InputStream inputStream = reconUtils.makeHttpCall(
connectionFactory, getScmDBSnapshotUrl(),
isOmSpnegoEnabled()).getInputStream()) {
FileUtils.copyInputStreamToFile(inputStream, targetFile);
}
return null;
});
LOG.info("Downloaded SCM Snapshot from SCM");
} else {
try {
List<String> ratisRoles = scmClient.getScmInfo().getRatisPeerRoles();
for (String ratisRole: ratisRoles) {
for (String ratisRole : ratisRoles) {
String[] role = ratisRole.split(":");
if (role[2].equals(RaftProtos.RaftPeerRole.LEADER.toString())) {
String hostAddress = role[4].trim();
int grpcPort = configuration.getInt(
ScmConfigKeys.OZONE_SCM_GRPC_PORT_KEY,
ScmConfigKeys.OZONE_SCM_GRPC_PORT_DEFAULT);

SecurityConfig secConf = new SecurityConfig(configuration);
SCMSecurityProtocolClientSideTranslatorPB scmSecurityClient =
getScmSecurityClientWithMaxRetry(
configuration, getCurrentUser());
try (ReconCertificateClient certClient =
new ReconCertificateClient(
secConf, scmSecurityClient, reconStorage, null, null);
SCMSnapshotDownloader downloadClient = new InterSCMGrpcClient(
hostAddress, grpcPort, configuration, certClient)) {
downloadClient.download(targetFile.toPath()).get();
} catch (ExecutionException | InterruptedException e) {
LOG.error("Rocks DB checkpoint downloading failed", e);
throw new IOException(e);
// This explicit role length check is to support older versions where we cannot change the default value
// without breaking backward compatibility during upgrade, because if Ratis is not enabled then the roles
// command output is generated outside of Ratis. It will not have the Ratis terminologies.
if (role.length > 2) {
if (role[2].equals(RaftProtos.RaftPeerRole.LEADER.toString())) {
String hostAddress = role[4].trim();
int grpcPort = configuration.getInt(
ScmConfigKeys.OZONE_SCM_GRPC_PORT_KEY,
ScmConfigKeys.OZONE_SCM_GRPC_PORT_DEFAULT);

SecurityConfig secConf = new SecurityConfig(configuration);
SCMSecurityProtocolClientSideTranslatorPB scmSecurityClient =
getScmSecurityClientWithMaxRetry(
configuration, getCurrentUser());
try (ReconCertificateClient certClient =
new ReconCertificateClient(
secConf, scmSecurityClient, reconStorage, null, null);
SCMSnapshotDownloader downloadClient = new InterSCMGrpcClient(
hostAddress, grpcPort, configuration, certClient)) {
downloadClient.download(targetFile.toPath()).get();
} catch (ExecutionException | InterruptedException e) {
LOG.error("Rocks DB checkpoint downloading failed: {}", e);
throw new IOException(e);
}
LOG.info("Downloaded SCM Snapshot from Leader SCM");
break;
}
LOG.info("Downloaded SCM Snapshot from Leader SCM");
} else {
fetchSCMDBSnapshotUsingHttpClient(targetFile);
LOG.info("Downloaded SCM Snapshot from SCM");
break;
}
}
} catch (Throwable throwable) {
LOG.error("Unexpected runtime error while downloading SCM Rocks DB snapshot/checkpoint : {}", throwable);
throw throwable;
}
Path untarredDbDir = Paths.get(scmSnapshotDBParentDir.getAbsolutePath(),
snapshotFileName);
reconUtils.untarCheckpointFile(targetFile, untarredDbDir);
FileUtils.deleteQuietly(targetFile);
return new RocksDBCheckpoint(untarredDbDir);
} catch (IOException e) {
LOG.error("Unable to obtain SCM DB Snapshot. ", e);
return getRocksDBCheckpoint(snapshotFileName, targetFile);
} catch (Throwable e) {
reconContext.updateHealthStatus(new AtomicBoolean(false));
reconContext.getErrors().add(ReconContext.ErrorCode.GET_SCM_DB_SNAPSHOT_FAILED);
LOG.error("Unable to obtain SCM DB Snapshot: {} ", e);
}
return null;
}

private void fetchSCMDBSnapshotUsingHttpClient(File targetFile) throws IOException {
SecurityUtil.doAsLoginUser(() -> {
try (InputStream inputStream = reconUtils.makeHttpCall(
connectionFactory, getScmDBSnapshotUrl(),
isOmSpnegoEnabled()).getInputStream()) {
FileUtils.copyInputStreamToFile(inputStream, targetFile);
}
return null;
});
}

@NotNull
private RocksDBCheckpoint getRocksDBCheckpoint(String snapshotFileName, File targetFile) throws IOException {
Path untarredDbDir = Paths.get(scmSnapshotDBParentDir.getAbsolutePath(),
snapshotFileName);
reconUtils.untarCheckpointFile(targetFile, untarredDbDir);
FileUtils.deleteQuietly(targetFile);
return new RocksDBCheckpoint(untarredDbDir);
}

@Override
public List<ContainerInfo> getListOfContainers(
long startContainerID, int count, HddsProtos.LifeCycleState state)
Expand Down

0 comments on commit f3a0dbd

Please sign in to comment.