Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
43 commits
Select commit Hold shift + click to select a range
785e4a8
Move a util method from KeyManagementBase to KeyManagementUtils to br…
haridsv Oct 27, 2025
c769d75
Move key management message classes from ManagedKey.proto to HBase.pr…
haridsv Oct 27, 2025
e3a5a24
Add eject and clear methods for cache
haridsv Oct 27, 2025
79d3961
Cursor implementation for eject and clear, first cut
haridsv Oct 27, 2025
eec3b8f
Cursor changes for code review round 1
haridsv Oct 27, 2025
350fe02
Misc. fixes
haridsv Oct 28, 2025
c2a52ae
Test fixes
haridsv Oct 28, 2025
676cba0
Rename methods for consistency
haridsv Oct 28, 2025
92dfcf7
More test coverage
haridsv Oct 28, 2025
32ea3c8
Refactor test code
haridsv Oct 28, 2025
05c4af9
Remove duplicate method
haridsv Oct 28, 2025
4a52653
test coverage for cache eject
haridsv Oct 28, 2025
0f49f26
Expect key metadata instead of hash to be able to clear the cache pro…
haridsv Oct 28, 2025
f3e690e
Test coverage for eject
haridsv Oct 28, 2025
96dbc43
Symbol rename for consistency
haridsv Oct 29, 2025
b45c4f7
Cleanup unused method and remove duplication
haridsv Oct 29, 2025
792e7b0
Cursor change to add new APIs, unverified
haridsv Oct 29, 2025
3b65ed4
Fixes to comppilation errors
haridsv Nov 3, 2025
c0c97ca
Ran spotify:apply
haridsv Nov 3, 2025
0213acc
Rename ejectManagedKeyDataCacheEntryOnAllServers to ejectManagedKeyDa…
haridsv Nov 4, 2025
991dc3f
spotless:apply
haridsv Nov 4, 2025
101c773
WIP changes to drop key metadata usage from client side
haridsv Nov 4, 2025
b4b4f9b
Small refactor
haridsv Nov 4, 2025
3858852
Simplify the disable flow
haridsv Nov 5, 2025
9b934cf
Misc code review fixes to Cursor changes
haridsv Nov 6, 2025
6b2e9d4
Misc. fiexes
haridsv Nov 7, 2025
39a0fda
Removed redundant RotateSTKResponse
haridsv Nov 7, 2025
d635dad
New test coverage by cursor, needs review
haridsv Nov 10, 2025
cdc60e0
Misc. manual review changes such as cleanups, refactors and addl. cov…
haridsv Nov 10, 2025
bc1a621
Bug fix for active key lookup
haridsv Nov 11, 2025
2c91e3f
Shell commands for new API and E2E test coverage
haridsv Nov 11, 2025
9ee4683
More test coverage
haridsv Nov 12, 2025
771b927
Follow existing test pattern for the change in encryption attribute
haridsv Nov 13, 2025
6ce3065
Missing Test annotation
haridsv Nov 13, 2025
9441ca6
Fix javadoc errors
haridsv Nov 16, 2025
c7875f9
WIP manual changes to address gaps in state handling
haridsv Nov 20, 2025
cd4e9b7
Stabiliation of test coverage
haridsv Nov 21, 2025
6924f45
Primarily updated the behavior of disableKeyManagement
haridsv Nov 24, 2025
26dc9fd
Additional test coverage by Cursor
haridsv Nov 24, 2025
88cdcae
Ran spotless
haridsv Nov 24, 2025
9051ada
Better exception handling and logging in refreshKey and other fixes f…
haridsv Nov 25, 2025
ac5a3eb
Address misc. CI issues
haridsv Nov 25, 2025
8302aa3
Fixed another CI issue
haridsv Nov 25, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -2669,5 +2669,23 @@ List<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType, Server
* Refresh the system key cache on all specified region servers.
* @param regionServers the list of region servers to refresh the system key cache on
*/
void refreshSystemKeyCacheOnServers(Set<ServerName> regionServers) throws IOException;
void refreshSystemKeyCacheOnServers(List<ServerName> regionServers) throws IOException;

/**
* Eject a specific managed key entry from the managed key data cache on all specified region
* servers.
* @param regionServers the list of region servers to eject the managed key entry from
* @param keyCustodian the key custodian
* @param keyNamespace the key namespace
* @param keyMetadata the key metadata
*/
void ejectManagedKeyDataCacheEntryOnServers(List<ServerName> regionServers, byte[] keyCustodian,
String keyNamespace, String keyMetadata) throws IOException;

/**
* Clear all entries in the managed key data cache on all specified region servers without having
* to restart the process.
* @param regionServers the list of region servers to clear the managed key data cache on
*/
void clearManagedKeyDataCacheOnServers(List<ServerName> regionServers) throws IOException;
}
Original file line number Diff line number Diff line change
Expand Up @@ -1148,7 +1148,19 @@ public void restoreBackupSystemTable(String snapshotName) throws IOException {
}

@Override
public void refreshSystemKeyCacheOnServers(Set<ServerName> regionServers) throws IOException {
public void refreshSystemKeyCacheOnServers(List<ServerName> regionServers) throws IOException {
get(admin.refreshSystemKeyCacheOnServers(regionServers));
}

@Override
public void ejectManagedKeyDataCacheEntryOnServers(List<ServerName> regionServers,
byte[] keyCustodian, String keyNamespace, String keyMetadata) throws IOException {
get(admin.ejectManagedKeyDataCacheEntryOnServers(regionServers, keyCustodian, keyNamespace,
keyMetadata));
}

@Override
public void clearManagedKeyDataCacheOnServers(List<ServerName> regionServers) throws IOException {
get(admin.clearManagedKeyDataCacheOnServers(regionServers));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1879,5 +1879,22 @@ CompletableFuture<List<LogEntry>> getLogEntries(Set<ServerName> serverNames, Str
* Refresh the system key cache on all specified region servers.
* @param regionServers the list of region servers to refresh the system key cache on
*/
CompletableFuture<Void> refreshSystemKeyCacheOnServers(Set<ServerName> regionServers);
CompletableFuture<Void> refreshSystemKeyCacheOnServers(List<ServerName> regionServers);

/**
* Eject a specific managed key entry from the managed key data cache on all specified region
* servers.
* @param regionServers the list of region servers to eject the managed key entry from
* @param keyCustodian the key custodian
* @param keyNamespace the key namespace
* @param keyMetadata the key metadata
*/
CompletableFuture<Void> ejectManagedKeyDataCacheEntryOnServers(List<ServerName> regionServers,
byte[] keyCustodian, String keyNamespace, String keyMetadata);

/**
* Clear all entries in the managed key data cache on all specified region servers.
* @param regionServers the list of region servers to clear the managed key data cache on
*/
CompletableFuture<Void> clearManagedKeyDataCacheOnServers(List<ServerName> regionServers);
}
Original file line number Diff line number Diff line change
Expand Up @@ -687,10 +687,22 @@ public CompletableFuture<Void> updateConfiguration(String groupName) {
}

@Override
public CompletableFuture<Void> refreshSystemKeyCacheOnServers(Set<ServerName> regionServers) {
public CompletableFuture<Void> refreshSystemKeyCacheOnServers(List<ServerName> regionServers) {
return wrap(rawAdmin.refreshSystemKeyCacheOnServers(regionServers));
}

@Override
public CompletableFuture<Void> ejectManagedKeyDataCacheEntryOnServers(
List<ServerName> regionServers, byte[] keyCustodian, String keyNamespace, String keyMetadata) {
return wrap(rawAdmin.ejectManagedKeyDataCacheEntryOnServers(regionServers, keyCustodian,
keyNamespace, keyMetadata));
}

@Override
public CompletableFuture<Void> clearManagedKeyDataCacheOnServers(List<ServerName> regionServers) {
return wrap(rawAdmin.clearManagedKeyDataCacheOnServers(regionServers));
}

@Override
public CompletableFuture<Void> rollWALWriter(ServerName serverName) {
return wrap(rawAdmin.rollWALWriter(serverName));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,7 @@
import org.apache.hadoop.hbase.client.replication.TableCFs;
import org.apache.hadoop.hbase.client.security.SecurityCapability;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.io.crypto.ManagedKeyData;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.net.Address;
import org.apache.hadoop.hbase.quotas.QuotaFilter;
Expand Down Expand Up @@ -152,6 +153,8 @@
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.EmptyMsg;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LastHighestWalFilenum;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ManagedKeyEntryRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ManagedKeyRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
Expand Down Expand Up @@ -4665,7 +4668,7 @@ MasterProtos.RestoreBackupSystemTableResponse> procedureCall(request,
}

@Override
public CompletableFuture<Void> refreshSystemKeyCacheOnServers(Set<ServerName> regionServers) {
public CompletableFuture<Void> refreshSystemKeyCacheOnServers(List<ServerName> regionServers) {
CompletableFuture<Void> future = new CompletableFuture<>();
List<CompletableFuture<Void>> futures =
regionServers.stream().map(this::refreshSystemKeyCache).collect(Collectors.toList());
Expand All @@ -4687,4 +4690,62 @@ private CompletableFuture<Void> refreshSystemKeyCache(ServerName serverName) {
(s, c, req, done) -> s.refreshSystemKeyCache(controller, req, done), resp -> null))
.serverName(serverName).call();
}

@Override
public CompletableFuture<Void> ejectManagedKeyDataCacheEntryOnServers(
List<ServerName> regionServers, byte[] keyCustodian, String keyNamespace, String keyMetadata) {
CompletableFuture<Void> future = new CompletableFuture<>();
// Create the request once instead of repeatedly for each server
byte[] keyMetadataHash = ManagedKeyData.constructMetadataHash(keyMetadata);
ManagedKeyEntryRequest request = ManagedKeyEntryRequest.newBuilder()
.setKeyCustNs(ManagedKeyRequest.newBuilder().setKeyCust(ByteString.copyFrom(keyCustodian))
.setKeyNamespace(keyNamespace).build())
.setKeyMetadataHash(ByteString.copyFrom(keyMetadataHash)).build();
List<CompletableFuture<Void>> futures =
regionServers.stream().map(serverName -> ejectManagedKeyDataCacheEntry(serverName, request))
.collect(Collectors.toList());
addListener(CompletableFuture.allOf(futures.toArray(new CompletableFuture<?>[0])),
(result, err) -> {
if (err != null) {
future.completeExceptionally(err);
} else {
future.complete(result);
}
});
return future;
}

private CompletableFuture<Void> ejectManagedKeyDataCacheEntry(ServerName serverName,
ManagedKeyEntryRequest request) {
return this.<Void> newAdminCaller()
.action((controller, stub) -> this.<ManagedKeyEntryRequest, HBaseProtos.BooleanMsg,
Void> adminCall(controller, stub, request,
(s, c, req, done) -> s.ejectManagedKeyDataCacheEntry(controller, req, done),
resp -> null))
.serverName(serverName).call();
}

@Override
public CompletableFuture<Void> clearManagedKeyDataCacheOnServers(List<ServerName> regionServers) {
CompletableFuture<Void> future = new CompletableFuture<>();
List<CompletableFuture<Void>> futures =
regionServers.stream().map(this::clearManagedKeyDataCache).collect(Collectors.toList());
addListener(CompletableFuture.allOf(futures.toArray(new CompletableFuture<?>[0])),
(result, err) -> {
if (err != null) {
future.completeExceptionally(err);
} else {
future.complete(result);
}
});
return future;
}

private CompletableFuture<Void> clearManagedKeyDataCache(ServerName serverName) {
return this.<Void> newAdminCaller()
.action((controller, stub) -> this.<EmptyMsg, EmptyMsg, Void> adminCall(controller, stub,
EmptyMsg.getDefaultInstance(),
(s, c, req, done) -> s.clearManagedKeyDataCache(controller, req, done), resp -> null))
.serverName(serverName).call();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,19 +21,23 @@
import java.security.KeyException;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.lang3.NotImplementedException;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.io.crypto.ManagedKeyData;
import org.apache.hadoop.hbase.io.crypto.ManagedKeyState;
import org.apache.hadoop.hbase.protobuf.generated.ManagedKeysProtos;
import org.apache.hadoop.hbase.protobuf.generated.ManagedKeysProtos.ManagedKeyRequest;
import org.apache.hadoop.hbase.protobuf.generated.ManagedKeysProtos.ManagedKeyResponse;
import org.apache.yetus.audience.InterfaceAudience;

import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;

import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BooleanMsg;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.EmptyMsg;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.GetManagedKeysResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ManagedKeyEntryRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ManagedKeyRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ManagedKeyResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ManagedKeysProtos;

@InterfaceAudience.Public
public class KeymetaAdminClient implements KeymetaAdmin {
Expand All @@ -48,9 +52,8 @@ public KeymetaAdminClient(Connection conn) throws IOException {
public ManagedKeyData enableKeyManagement(byte[] keyCust, String keyNamespace)
throws IOException {
try {
ManagedKeysProtos.ManagedKeyResponse response =
stub.enableKeyManagement(null, ManagedKeyRequest.newBuilder()
.setKeyCust(ByteString.copyFrom(keyCust)).setKeyNamespace(keyNamespace).build());
ManagedKeyResponse response = stub.enableKeyManagement(null, ManagedKeyRequest.newBuilder()
.setKeyCust(ByteString.copyFrom(keyCust)).setKeyNamespace(keyNamespace).build());
return generateKeyData(response);
} catch (ServiceException e) {
throw ProtobufUtil.handleRemoteException(e);
Expand All @@ -61,7 +64,7 @@ public ManagedKeyData enableKeyManagement(byte[] keyCust, String keyNamespace)
public List<ManagedKeyData> getManagedKeys(byte[] keyCust, String keyNamespace)
throws IOException, KeyException {
try {
ManagedKeysProtos.GetManagedKeysResponse statusResponse =
GetManagedKeysResponse statusResponse =
stub.getManagedKeys(null, ManagedKeyRequest.newBuilder()
.setKeyCust(ByteString.copyFrom(keyCust)).setKeyNamespace(keyNamespace).build());
return generateKeyDataList(statusResponse);
Expand All @@ -73,26 +76,95 @@ public List<ManagedKeyData> getManagedKeys(byte[] keyCust, String keyNamespace)
@Override
public boolean rotateSTK() throws IOException {
try {
ManagedKeysProtos.RotateSTKResponse response =
stub.rotateSTK(null, EmptyMsg.getDefaultInstance());
return response.getRotated();
BooleanMsg response = stub.rotateSTK(null, EmptyMsg.getDefaultInstance());
return response.getBoolMsg();
} catch (ServiceException e) {
throw ProtobufUtil.handleRemoteException(e);
}
}

@Override
public void ejectManagedKeyDataCacheEntry(byte[] keyCustodian, String keyNamespace,
String keyMetadata) throws IOException {
throw new NotImplementedException(
"ejectManagedKeyDataCacheEntry not supported in KeymetaAdminClient");
}

@Override
public void clearManagedKeyDataCache() throws IOException {
throw new NotImplementedException(
"clearManagedKeyDataCache not supported in KeymetaAdminClient");
}

@Override
public ManagedKeyData disableKeyManagement(byte[] keyCust, String keyNamespace)
throws IOException, KeyException {
try {
ManagedKeyResponse response = stub.disableKeyManagement(null, ManagedKeyRequest.newBuilder()
.setKeyCust(ByteString.copyFrom(keyCust)).setKeyNamespace(keyNamespace).build());
return generateKeyData(response);
} catch (ServiceException e) {
throw ProtobufUtil.handleRemoteException(e);
}
}

@Override
public ManagedKeyData disableManagedKey(byte[] keyCust, String keyNamespace,
byte[] keyMetadataHash) throws IOException, KeyException {
try {
ManagedKeyResponse response = stub.disableManagedKey(null,
ManagedKeyEntryRequest.newBuilder()
.setKeyCustNs(ManagedKeyRequest.newBuilder().setKeyCust(ByteString.copyFrom(keyCust))
.setKeyNamespace(keyNamespace).build())
.setKeyMetadataHash(ByteString.copyFrom(keyMetadataHash)).build());
return generateKeyData(response);
} catch (ServiceException e) {
throw ProtobufUtil.handleRemoteException(e);
}
}

private static List<ManagedKeyData>
generateKeyDataList(ManagedKeysProtos.GetManagedKeysResponse stateResponse) {
@Override
public ManagedKeyData rotateManagedKey(byte[] keyCust, String keyNamespace)
throws IOException, KeyException {
try {
ManagedKeyResponse response = stub.rotateManagedKey(null, ManagedKeyRequest.newBuilder()
.setKeyCust(ByteString.copyFrom(keyCust)).setKeyNamespace(keyNamespace).build());
return generateKeyData(response);
} catch (ServiceException e) {
throw ProtobufUtil.handleRemoteException(e);
}
}

@Override
public void refreshManagedKeys(byte[] keyCust, String keyNamespace)
throws IOException, KeyException {
try {
stub.refreshManagedKeys(null, ManagedKeyRequest.newBuilder()
.setKeyCust(ByteString.copyFrom(keyCust)).setKeyNamespace(keyNamespace).build());
} catch (ServiceException e) {
throw ProtobufUtil.handleRemoteException(e);
}
}

private static List<ManagedKeyData> generateKeyDataList(GetManagedKeysResponse stateResponse) {
List<ManagedKeyData> keyStates = new ArrayList<>();
for (ManagedKeyResponse state : stateResponse.getStateList()) {
keyStates.add(generateKeyData(state));
}
return keyStates;
}

private static ManagedKeyData generateKeyData(ManagedKeysProtos.ManagedKeyResponse response) {
return new ManagedKeyData(response.getKeyCust().toByteArray(), response.getKeyNamespace(), null,
ManagedKeyState.forValue((byte) response.getKeyState().getNumber()),
response.getKeyMetadata(), response.getRefreshTimestamp());
private static ManagedKeyData generateKeyData(ManagedKeyResponse response) {
// Use hash-only constructor for client-side ManagedKeyData
byte[] keyMetadataHash =
response.hasKeyMetadataHash() ? response.getKeyMetadataHash().toByteArray() : null;
if (keyMetadataHash == null) {
return new ManagedKeyData(response.getKeyCust().toByteArray(), response.getKeyNamespace(),
ManagedKeyState.forValue((byte) response.getKeyState().getNumber()));
} else {
return new ManagedKeyData(response.getKeyCust().toByteArray(), response.getKeyNamespace(),
ManagedKeyState.forValue((byte) response.getKeyState().getNumber()), keyMetadataHash,
response.getRefreshTimestamp());
}
}
}
Loading