-
Notifications
You must be signed in to change notification settings - Fork 526
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
HDDS-12486. Warmup KMS encrypted keys when OM starts #8081
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -77,6 +77,12 @@ | |
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SERVER_DEFAULT_REPLICATION_KEY; | ||
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SERVER_DEFAULT_REPLICATION_TYPE_DEFAULT; | ||
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SERVER_DEFAULT_REPLICATION_TYPE_KEY; | ||
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_EDEKCACHELOADER_INITIAL_DELAY_MS_KEY; | ||
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_EDEKCACHELOADER_INITIAL_DELAY_MS_DEFAULT; | ||
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_EDEKCACHELOADER_INTERVAL_MS_KEY; | ||
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_EDEKCACHELOADER_INTERVAL_MS_DEFAULT; | ||
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_EDEKCACHELOADER_MAX_RETRIES_KEY; | ||
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_EDEKCACHELOADER_MAX_RETRIES_DEFAULT; | ||
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.DETECTED_LOOP_IN_BUCKET_LINKS; | ||
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FEATURE_NOT_ENABLED; | ||
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INTERNAL_ERROR; | ||
|
@@ -97,6 +103,7 @@ | |
import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PrepareStatusResponse.PrepareStatus; | ||
import static org.apache.hadoop.security.UserGroupInformation.getCurrentUser; | ||
import static org.apache.hadoop.util.ExitUtil.terminate; | ||
import static org.apache.hadoop.util.Time.monotonicNow; | ||
import static org.apache.ozone.graph.PrintableGraph.GraphType.FILE_NAME; | ||
|
||
import com.fasterxml.jackson.databind.ObjectMapper; | ||
|
@@ -137,6 +144,8 @@ | |
import java.util.TimerTask; | ||
import java.util.UUID; | ||
import java.util.concurrent.ConcurrentHashMap; | ||
import java.util.concurrent.ExecutorService; | ||
import java.util.concurrent.Executors; | ||
import java.util.concurrent.TimeUnit; | ||
import java.util.concurrent.atomic.AtomicReference; | ||
import java.util.stream.Collectors; | ||
|
@@ -318,6 +327,7 @@ | |
import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; | ||
import org.apache.hadoop.security.authentication.client.AuthenticationException; | ||
import org.apache.hadoop.security.token.Token; | ||
import com.google.common.util.concurrent.ThreadFactoryBuilder; | ||
import org.apache.hadoop.util.KMSUtil; | ||
import org.apache.hadoop.util.Time; | ||
import org.apache.ozone.graph.PrintableGraph; | ||
|
@@ -386,6 +396,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl | |
private KeyManager keyManager; | ||
private PrefixManagerImpl prefixManager; | ||
private final UpgradeFinalizer<OzoneManager> upgradeFinalizer; | ||
private ExecutorService edekCacheLoader = null; | ||
|
||
/** | ||
* OM super user / admin list. | ||
|
@@ -644,6 +655,7 @@ private OzoneManager(OzoneConfiguration conf, StartupOption startupOption) | |
kmsProvider = null; | ||
LOG.error("Fail to create Key Provider"); | ||
} | ||
initializeEdekCache(conf); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. it should only run if it's a leader node. |
||
if (secConfig.isSecurityEnabled()) { | ||
omComponent = OM_DAEMON + "-" + omId; | ||
HddsProtos.OzoneManagerDetailsProto omInfo = | ||
|
@@ -727,6 +739,110 @@ private OzoneManager(OzoneConfiguration conf, StartupOption startupOption) | |
omHostName = HddsUtils.getHostName(conf); | ||
} | ||
|
||
private void initializeEdekCache(OzoneConfiguration conf) { | ||
int edekCacheLoaderDelay = | ||
conf.getInt(OZONE_OM_EDEKCACHELOADER_INITIAL_DELAY_MS_KEY, OZONE_OM_EDEKCACHELOADER_INITIAL_DELAY_MS_DEFAULT); | ||
int edekCacheLoaderInterval = | ||
conf.getInt(OZONE_OM_EDEKCACHELOADER_INTERVAL_MS_KEY, OZONE_OM_EDEKCACHELOADER_INTERVAL_MS_DEFAULT); | ||
int edekCacheLoaderMaxRetries = | ||
conf.getInt(OZONE_OM_EDEKCACHELOADER_MAX_RETRIES_KEY, OZONE_OM_EDEKCACHELOADER_MAX_RETRIES_DEFAULT); | ||
if (kmsProvider != null) { | ||
edekCacheLoader = Executors.newSingleThreadExecutor( | ||
new ThreadFactoryBuilder().setDaemon(true) | ||
.setNameFormat("Warm Up EDEK Cache Thread #%d") | ||
.build()); | ||
warmUpEdekCache(edekCacheLoader, edekCacheLoaderDelay, edekCacheLoaderInterval, edekCacheLoaderMaxRetries); | ||
} | ||
} | ||
|
||
static class EDEKCacheLoader implements Runnable { | ||
private final String[] keyNames; | ||
private final KeyProviderCryptoExtension kp; | ||
private int initialDelay; | ||
private int retryInterval; | ||
private int maxRetries; | ||
|
||
EDEKCacheLoader(final String[] names, final KeyProviderCryptoExtension kp, | ||
final int delay, final int interval, final int maxRetries) { | ||
this.keyNames = names; | ||
this.kp = kp; | ||
this.initialDelay = delay; | ||
this.retryInterval = interval; | ||
this.maxRetries = maxRetries; | ||
} | ||
|
||
@Override | ||
public void run() { | ||
LOG.info("Warming up {} EDEKs... (initialDelay={}, " | ||
+ "retryInterval={}, maxRetries={})", keyNames.length, initialDelay, retryInterval, | ||
maxRetries); | ||
try { | ||
Thread.sleep(initialDelay); | ||
} catch (InterruptedException ie) { | ||
LOG.info("EDEKCacheLoader interrupted before warming up."); | ||
return; | ||
} | ||
|
||
boolean success = false; | ||
int retryCount = 0; | ||
IOException lastSeenIOE = null; | ||
long warmUpEDEKStartTime = monotonicNow(); | ||
|
||
while (!success && retryCount < maxRetries) { | ||
try { | ||
kp.warmUpEncryptedKeys(keyNames); | ||
LOG.info("Successfully warmed up {} EDEKs.", keyNames.length); | ||
success = true; | ||
} catch (IOException ioe) { | ||
lastSeenIOE = ioe; | ||
LOG.info("Failed to warm up EDEKs.", ioe); | ||
} catch (Exception e) { | ||
LOG.error("Cannot warm up EDEKs.", e); | ||
throw e; | ||
} | ||
|
||
if (!success) { | ||
try { | ||
Thread.sleep(retryInterval); | ||
} catch (InterruptedException ie) { | ||
LOG.info("EDEKCacheLoader interrupted during retry."); | ||
break; | ||
} | ||
retryCount++; | ||
} | ||
} | ||
|
||
long warmUpEDEKTime = monotonicNow() - warmUpEDEKStartTime; | ||
LOG.debug("Time taken to load EDEK keys to the cache: {}", warmUpEDEKTime); | ||
if (!success) { | ||
LOG.warn("Max retry {} reached, unable to warm up EDEKs.", maxRetries); | ||
if (lastSeenIOE != null) { | ||
LOG.warn("Last seen exception:", lastSeenIOE); | ||
} | ||
} | ||
} | ||
} | ||
|
||
public void warmUpEdekCache(final ExecutorService executor, final int delay, final int interval, int maxRetries) { | ||
List<String> keys = new ArrayList<>(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Multiple buckets could use the same encryption key. Doing so could warm up the same encryption key multiple times, causing unnecessary delays. Let's use HashSet to eliminate duplicates. |
||
try ( | ||
TableIterator<String, ? extends Table.KeyValue<String, OmBucketInfo>> iterator = | ||
metadataManager.getBucketTable().iterator()) { | ||
while (iterator.hasNext()) { | ||
Table.KeyValue<String, OmBucketInfo> entry = iterator.next(); | ||
if (entry.getValue().getEncryptionKeyInfo() != null) { | ||
String encKey = entry.getValue().getEncryptionKeyInfo().getKeyName(); | ||
keys.add(encKey); | ||
} | ||
} | ||
} catch (IOException ex) { | ||
LOG.error("Error while retrieving encryption keys for warming up EDEK cache", ex); | ||
} | ||
String[] edeks = new String[keys.size()]; | ||
edeks = keys.toArray(edeks); | ||
executor.execute(new EDEKCacheLoader(edeks, getKmsProvider(), delay, interval, maxRetries)); | ||
} | ||
|
||
public boolean isStopped() { | ||
return omState == State.STOPPED; | ||
} | ||
|
@@ -2299,6 +2415,10 @@ public boolean stop() { | |
if (versionManager != null) { | ||
versionManager.close(); | ||
} | ||
|
||
if (edekCacheLoader != null) { | ||
edekCacheLoader.shutdown(); | ||
} | ||
return true; | ||
} catch (Exception e) { | ||
LOG.error("OzoneManager stop failed.", e); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Let's add these configuration properties into ozone-default.xml?