Cross-site validation for lazy loading of offline sessions & Switch default offline sessions to lazy loaded

This commit is contained in:
Martin Kanis 2022-01-26 09:05:14 +01:00 committed by Hynek Mlnařík
parent ec49695a98
commit 0471ec4941
14 changed files with 273 additions and 75 deletions

View file

@ -34,8 +34,10 @@ import org.keycloak.models.ModelException;
import org.keycloak.models.OfflineUserSessionModel;
import org.keycloak.models.RealmModel;
import org.keycloak.models.UserModel;
import org.keycloak.models.UserProvider;
import org.keycloak.models.UserSessionModel;
import org.keycloak.models.UserSessionProvider;
import org.keycloak.models.UserSessionSpi;
import org.keycloak.models.session.UserSessionPersisterProvider;
import org.keycloak.models.sessions.infinispan.changes.Tasks;
import org.keycloak.models.sessions.infinispan.changes.sessions.CrossDCLastSessionRefreshStore;
@ -108,7 +110,7 @@ public class InfinispanUserSessionProvider implements UserSessionProvider {
protected final RemoteCacheInvoker remoteCacheInvoker;
protected final InfinispanKeyGenerator keyGenerator;
protected final boolean loadOfflineSessionsStatsFromDatabase;
protected final boolean loadOfflineSessionsFromDatabase;
public InfinispanUserSessionProvider(KeycloakSession session,
RemoteCacheInvoker remoteCacheInvoker,
@ -120,7 +122,7 @@ public class InfinispanUserSessionProvider implements UserSessionProvider {
Cache<String, SessionEntityWrapper<UserSessionEntity>> offlineSessionCache,
Cache<UUID, SessionEntityWrapper<AuthenticatedClientSessionEntity>> clientSessionCache,
Cache<UUID, SessionEntityWrapper<AuthenticatedClientSessionEntity>> offlineClientSessionCache,
boolean loadOfflineSessionsStatsFromDatabase) {
boolean loadOfflineSessionsFromDatabase) {
this.session = session;
this.sessionCache = sessionCache;
@ -140,7 +142,7 @@ public class InfinispanUserSessionProvider implements UserSessionProvider {
this.persisterLastSessionRefreshStore = persisterLastSessionRefreshStore;
this.remoteCacheInvoker = remoteCacheInvoker;
this.keyGenerator = keyGenerator;
this.loadOfflineSessionsStatsFromDatabase = loadOfflineSessionsStatsFromDatabase;
this.loadOfflineSessionsFromDatabase = loadOfflineSessionsFromDatabase;
session.getTransactionManager().enlistAfterCompletion(clusterEventsSenderTx);
session.getTransactionManager().enlistAfterCompletion(sessionTx);
@ -341,28 +343,39 @@ public class InfinispanUserSessionProvider implements UserSessionProvider {
protected Stream<UserSessionModel> getUserSessionsStream(RealmModel realm, UserSessionPredicate predicate, boolean offline) {
if (offline && loadOfflineSessionsStatsFromDatabase) {
if (offline && loadOfflineSessionsFromDatabase) {
// fetch the offline user-sessions from the persistence provider
UserSessionPersisterProvider persister = session.getProvider(UserSessionPersisterProvider.class);
if (predicate.getUserId() != null) {
UserModel user = session.users().getUserById(realm, predicate.getUserId());
if (user != null) {
return persister.loadUserSessionsStream(realm, user, offline, 0, null);
return persister.loadUserSessionsStream(realm, user, true, 0, null);
}
}
if (predicate.getBrokerUserId() != null) {
String[] idpAliasSessionId = predicate.getBrokerUserId().split("\\.");
Map<String, String> attributes = new HashMap<>();
attributes.put(UserModel.IDP_ALIAS, idpAliasSessionId[0]);
attributes.put(UserModel.IDP_USER_ID, idpAliasSessionId[1]);
UserProvider userProvider = session.getProvider(UserProvider.class);
UserModel userModel = userProvider.searchForUserStream(realm, attributes, 0, null).findFirst().orElse(null);
return userModel != null ?
persister.loadUserSessionsStream(realm, userModel, true, 0, null) :
Stream.empty();
}
if (predicate.getBrokerSessionId() != null) {
// TODO add support for offline user-session lookup by brokerSessionId
// currently it is not possible to access the brokerSessionId in offline user-session in a database agnostic way
throw new ModelException("Dynamic database lookup for offline user-sessions by brokerSessionId is currently only supported for preloaded sessions.");
throw new ModelException("Dynamic database lookup for offline user-sessions by broker session ID is currently only supported for preloaded sessions. " +
"Set preloadOfflineSessionsFromDatabase option to \"true\" in " + UserSessionSpi.NAME + " SPI in "
+ InfinispanUserSessionProviderFactory.PROVIDER_ID + " provider to enable the lookup.");
}
if (predicate.getBrokerUserId() != null) {
// TODO add support for offline user-session lookup by brokerUserId
// currently it is not possible to access the brokerUserId in offline user-session in a database agnostic way
throw new ModelException("Dynamic database lookup for offline user-sessions by brokerUserId is currently only supported for preloaded sessions.");
}
}
Cache<String, SessionEntityWrapper<UserSessionEntity>> cache = getCache(offline);
@ -421,10 +434,10 @@ public class InfinispanUserSessionProvider implements UserSessionProvider {
protected Stream<UserSessionModel> getUserSessionsStream(final RealmModel realm, ClientModel client, Integer firstResult, Integer maxResults, final boolean offline) {
if (offline && loadOfflineSessionsStatsFromDatabase) {
if (offline && loadOfflineSessionsFromDatabase) {
// fetch the actual offline user session count from the database
UserSessionPersisterProvider persister = session.getProvider(UserSessionPersisterProvider.class);
return persister.loadUserSessionsStream(realm, client, offline, firstResult, maxResults);
return persister.loadUserSessionsStream(realm, client, true, firstResult, maxResults);
}
final String clientUuid = client.getId();
@ -516,9 +529,9 @@ public class InfinispanUserSessionProvider implements UserSessionProvider {
@Override
public Map<String, Long> getActiveClientSessionStats(RealmModel realm, boolean offline) {
if (offline && loadOfflineSessionsStatsFromDatabase) {
if (offline && loadOfflineSessionsFromDatabase) {
UserSessionPersisterProvider persister = session.getProvider(UserSessionPersisterProvider.class);
return persister.getUserSessionsCountsByClients(realm, offline);
return persister.getUserSessionsCountsByClients(realm, true);
}
Cache<String, SessionEntityWrapper<UserSessionEntity>> cache = getCache(offline);
@ -536,10 +549,10 @@ public class InfinispanUserSessionProvider implements UserSessionProvider {
protected long getUserSessionsCount(RealmModel realm, ClientModel client, boolean offline) {
if (offline && loadOfflineSessionsStatsFromDatabase) {
if (offline && loadOfflineSessionsFromDatabase) {
// fetch the actual offline user session count from the database
UserSessionPersisterProvider persister = session.getProvider(UserSessionPersisterProvider.class);
return persister.getUserSessionsCount(realm, client, offline);
return persister.getUserSessionsCount(realm, client, true);
}
Cache<String, SessionEntityWrapper<UserSessionEntity>> cache = getCache(offline);
@ -782,7 +795,7 @@ public class InfinispanUserSessionProvider implements UserSessionProvider {
@Override
public Stream<UserSessionModel> getOfflineUserSessionsStream(RealmModel realm, UserModel user) {
if (loadOfflineSessionsStatsFromDatabase) {
if (loadOfflineSessionsFromDatabase) {
return getUserSessionsFromPersistenceProviderStream(realm, user, true);
}

View file

@ -79,6 +79,8 @@ public class InfinispanUserSessionProviderFactory implements UserSessionProvider
public static final String REMOVE_USER_SESSIONS_EVENT = "REMOVE_USER_SESSIONS_EVENT";
private boolean preloadOfflineSessionsFromDatabase;
private Config.Scope config;
private RemoteCacheInvoker remoteCacheInvoker;
@ -95,15 +97,14 @@ public class InfinispanUserSessionProviderFactory implements UserSessionProvider
Cache<UUID, SessionEntityWrapper<AuthenticatedClientSessionEntity>> clientSessionCache = connections.getCache(InfinispanConnectionProvider.CLIENT_SESSION_CACHE_NAME);
Cache<UUID, SessionEntityWrapper<AuthenticatedClientSessionEntity>> offlineClientSessionsCache = connections.getCache(InfinispanConnectionProvider.OFFLINE_CLIENT_SESSION_CACHE_NAME);
boolean loadOfflineSessionsStatsFromDatabase = !isPreloadingOfflineSessionsFromDatabaseEnabled();
return new InfinispanUserSessionProvider(session, remoteCacheInvoker, lastSessionRefreshStore, offlineLastSessionRefreshStore,
persisterLastSessionRefreshStore, keyGenerator, cache, offlineSessionsCache, clientSessionCache, offlineClientSessionsCache, loadOfflineSessionsStatsFromDatabase);
persisterLastSessionRefreshStore, keyGenerator, cache, offlineSessionsCache, clientSessionCache, offlineClientSessionsCache, !preloadOfflineSessionsFromDatabase);
}
@Override
public void init(Config.Scope config) {
this.config = config;
preloadOfflineSessionsFromDatabase = config.getBoolean("preloadOfflineSessionsFromDatabase", false);
}
@Override
@ -147,10 +148,6 @@ public class InfinispanUserSessionProviderFactory implements UserSessionProvider
});
}
private boolean isPreloadingOfflineSessionsFromDatabaseEnabled() {
return config.getBoolean("preloadOfflineSessionsFromDatabase", true);
}
// Max count of worker errors. Initialization will end with exception when this number is reached
private int getMaxErrors() {
return config.getInt("maxErrors", 20);
@ -175,7 +172,7 @@ public class InfinispanUserSessionProviderFactory implements UserSessionProvider
@Override
public void run(KeycloakSession session) {
if (isPreloadingOfflineSessionsFromDatabaseEnabled()) {
if (preloadOfflineSessionsFromDatabase) {
// only preload offline-sessions if necessary
log.debug("Start pre-loading userSessions from persistent storage");
@ -340,7 +337,6 @@ public class InfinispanUserSessionProviderFactory implements UserSessionProvider
log.debugf("Pre-loading sessions from remote cache '%s' finished", cacheName);
}
@Override
public void close() {
}

View file

@ -259,24 +259,21 @@ public class JpaUserSessionPersisterProvider implements UserSessionPersisterProv
String offlineStr = offlineToString(offline);
Query query = em.createNamedQuery("findUserSessionsCountsByClientId");
TypedQuery<Object[]> query = em.createNamedQuery("findClientSessionsClientIds", Object[].class);
query.setParameter("offline", offlineStr);
query.setParameter("realmId", realm.getId());
Map<String, Long> offlineSessionsByClient = new HashMap<>();
closing(query.getResultStream()).forEach(record -> {
Object[] row = (Object[]) record;
String clientId = String.valueOf(row[0]);
Long count = ((Number)row[1]).longValue();
offlineSessionsByClient.put(clientId, count);
});
return offlineSessionsByClient;
return closing(query.getResultStream())
.collect(Collectors.toMap(row -> {
String clientId = row[0].toString();
if (clientId.equals(PersistentClientSessionEntity.EXTERNAL)) {
final String externalClientId = row[1].toString();
final String clientStorageProvider = row[2].toString();
clientId = new StorageId(clientStorageProvider, externalClientId).getId();
}
return clientId;
}, row -> (Long) row[3]));
}
@Override
@ -319,14 +316,23 @@ public class JpaUserSessionPersisterProvider implements UserSessionPersisterProv
public Stream<UserSessionModel> loadUserSessionsStream(RealmModel realm, ClientModel client, boolean offline, Integer firstResult, Integer maxResults) {
String offlineStr = offlineToString(offline);
TypedQuery<PersistentUserSessionEntity> query = paginateQuery(
TypedQuery<PersistentUserSessionEntity> query;
StorageId clientStorageId = new StorageId(client.getId());
if (clientStorageId.isLocal()) {
query = paginateQuery(
em.createNamedQuery("findUserSessionsByClientId", PersistentUserSessionEntity.class),
firstResult, maxResults);
query.setParameter("clientId", client.getId());
} else {
query = paginateQuery(
em.createNamedQuery("findUserSessionsByExternalClientId", PersistentUserSessionEntity.class),
firstResult, maxResults);
query.setParameter("clientStorageProvider", clientStorageId.getProviderId());
query.setParameter("externalClientId", clientStorageId.getExternalId());
}
query.setParameter("offline", offlineStr);
query.setParameter("realmId", realm.getId());
query.setParameter("clientId", client.getId());
return loadUserSessionsWithClientSessions(query, offlineStr);
}
@ -461,11 +467,20 @@ public class JpaUserSessionPersisterProvider implements UserSessionPersisterProv
public int getUserSessionsCount(RealmModel realm, ClientModel clientModel, boolean offline) {
String offlineStr = offlineToString(offline);
Query query = em.createNamedQuery("findClientSessionsCountByClient");
// Note, that realm is unused here, since the clientModel id already determines the offline user-sessions bound to a owning realm.
query.setParameter("offline", offlineStr);
Query query;
StorageId clientStorageId = new StorageId(clientModel.getId());
if (clientStorageId.isLocal()) {
query = em.createNamedQuery("findClientSessionsCountByClient");
query.setParameter("clientId", clientModel.getId());
} else {
query = em.createNamedQuery("findClientSessionsCountByExternalClient");
query.setParameter("clientStorageProvider", clientStorageId.getProviderId());
query.setParameter("externalClientId", clientStorageId.getExternalId());
}
// Note, that realm is unused here, since the clientModel id already determines the offline user-sessions bound to an owning realm.
query.setParameter("offline", offlineStr);
Number n = (Number) query.getSingleResult();
return n.intValue();
}

View file

@ -41,7 +41,8 @@ import java.io.Serializable;
//@NamedQuery(name="deleteExpiredClientSessions", query="delete from PersistentClientSessionEntity sess where sess.userSessionId IN (select u.userSessionId from PersistentUserSessionEntity u where u.realmId = :realmId AND u.offline = :offline AND u.lastSessionRefresh < :lastSessionRefresh)"),
@NamedQuery(name="findClientSessionsByUserSession", query="select sess from PersistentClientSessionEntity sess where sess.userSessionId=:userSessionId and sess.offline = :offline"),
@NamedQuery(name="findClientSessionsOrderedById", query="select sess from PersistentClientSessionEntity sess where sess.offline = :offline and sess.userSessionId >= :fromSessionId and sess.userSessionId <= :toSessionId order by sess.userSessionId"),
@NamedQuery(name="findClientSessionsCountByClient", query="select count(sess) from PersistentClientSessionEntity sess where sess.offline = :offline and sess.clientId = :clientId")
@NamedQuery(name="findClientSessionsCountByClient", query="select count(sess) from PersistentClientSessionEntity sess where sess.offline = :offline and sess.clientId = :clientId"),
@NamedQuery(name="findClientSessionsCountByExternalClient", query="select count(sess) from PersistentClientSessionEntity sess where sess.offline = :offline and sess.clientStorageProvider = :clientStorageProvider and sess.externalClientId = :externalClientId")
})
@Table(name="OFFLINE_CLIENT_SESSION")
@Entity

View file

@ -47,14 +47,14 @@ import java.io.Serializable;
" AND sess.realmId = :realmId AND sess.userId = :userId ORDER BY sess.userSessionId"),
@NamedQuery(name="findUserSessionsByClientId", query="SELECT sess FROM PersistentUserSessionEntity sess INNER JOIN PersistentClientSessionEntity clientSess " +
" ON sess.userSessionId = clientSess.userSessionId AND clientSess.clientId = :clientId WHERE sess.offline = :offline " +
" AND sess.userSessionId = clientSess.userSessionId AND sess.realmId = :realmId ORDER BY sess.userSessionId"),
@NamedQuery(name="findUserSessionsCountsByClientId", query="SELECT clientSess.clientId, count(clientSess) " +
" FROM PersistentUserSessionEntity sess INNER JOIN PersistentClientSessionEntity clientSess " +
" ON sess.userSessionId = clientSess.userSessionId " +
// find all available offline user-session for all clients in a realm
" WHERE sess.offline = :offline " +
" AND sess.userSessionId = clientSess.userSessionId AND sess.realmId = :realmId " +
" GROUP BY clientSess.clientId")
" AND sess.realmId = :realmId ORDER BY sess.userSessionId"),
@NamedQuery(name="findUserSessionsByExternalClientId", query="SELECT sess FROM PersistentUserSessionEntity sess INNER JOIN PersistentClientSessionEntity clientSess " +
" ON sess.userSessionId = clientSess.userSessionId AND clientSess.clientStorageProvider = :clientStorageProvider AND clientSess.externalClientId = :externalClientId WHERE sess.offline = :offline " +
" AND sess.realmId = :realmId ORDER BY sess.userSessionId"),
@NamedQuery(name="findClientSessionsClientIds", query="SELECT clientSess.clientId, clientSess.externalClientId, clientSess.clientStorageProvider, count(clientSess)" +
" FROM PersistentClientSessionEntity clientSess INNER JOIN PersistentUserSessionEntity sess ON clientSess.userSessionId = sess.userSessionId " +
" WHERE sess.offline = :offline AND sess.realmId = :realmId " +
" GROUP BY clientSess.clientId, clientSess.externalClientId, clientSess.clientStorageProvider")
})
@Table(name="OFFLINE_USER_SESSION")

View file

@ -20,6 +20,7 @@ package org.keycloak.protocol.oidc.endpoints;
import org.jboss.logging.Logger;
import org.jboss.resteasy.annotations.cache.NoCache;
import org.jboss.resteasy.spi.HttpRequest;
import org.keycloak.Config;
import org.keycloak.OAuth2Constants;
import org.keycloak.OAuthErrorException;
import org.keycloak.TokenVerifier;
@ -96,12 +97,16 @@ public class LogoutEndpoint {
private RealmModel realm;
private EventBuilder event;
// When enabled we cannot search offline sessions by brokerSessionId. We need to search by federated userId and then filter by brokerSessionId.
private boolean offlineSessionsLazyLoadingEnabled;
private Cors cors;
public LogoutEndpoint(TokenManager tokenManager, RealmModel realm, EventBuilder event) {
this.tokenManager = tokenManager;
this.realm = realm;
this.event = event;
this.offlineSessionsLazyLoadingEnabled = !Config.scope("userSessions").scope("infinispan").getBoolean("preloadOfflineSessionsFromDatabase", false);
}
@Path("/")
@ -318,7 +323,7 @@ public class LogoutEndpoint {
if (logoutToken.getSid() != null) {
backchannelLogoutResponse = backchannelLogoutWithSessionId(logoutToken.getSid(), identityProviderAliases,
logoutOfflineSessions);
logoutOfflineSessions, logoutToken.getSubject());
} else {
backchannelLogoutResponse = backchannelLogoutFederatedUserId(logoutToken.getSubject(),
identityProviderAliases, logoutOfflineSessions);
@ -349,7 +354,7 @@ public class LogoutEndpoint {
}
private BackchannelLogoutResponse backchannelLogoutWithSessionId(String sessionId,
Stream<String> identityProviderAliases, boolean logoutOfflineSessions) {
Stream<String> identityProviderAliases, boolean logoutOfflineSessions, String federatedUserId) {
AtomicReference<BackchannelLogoutResponse> backchannelLogoutResponse = new AtomicReference<>(new BackchannelLogoutResponse());
backchannelLogoutResponse.get().setLocalLogoutSucceeded(true);
identityProviderAliases.forEach(identityProviderAlias -> {
@ -357,8 +362,12 @@ public class LogoutEndpoint {
identityProviderAlias + "." + sessionId);
if (logoutOfflineSessions) {
if (offlineSessionsLazyLoadingEnabled) {
logoutOfflineUserSessionByBrokerUserId(identityProviderAlias + "." + federatedUserId, identityProviderAlias + "." + sessionId);
} else {
logoutOfflineUserSession(identityProviderAlias + "." + sessionId);
}
}
if (userSession != null) {
backchannelLogoutResponse.set(logoutUserSession(userSession));
@ -407,6 +416,15 @@ public class LogoutEndpoint {
.forEach(userSessionManager::revokeOfflineUserSession);
}
private void logoutOfflineUserSessionByBrokerUserId(String brokerUserId, String brokerSessionId) {
UserSessionManager userSessionManager = new UserSessionManager(session);
if (brokerUserId != null && brokerSessionId != null) {
session.sessions().getOfflineUserSessionByBrokerUserIdStream(realm, brokerUserId)
.filter(userSession -> brokerSessionId.equals(userSession.getBrokerSessionId()))
.forEach(userSessionManager::revokeOfflineUserSession);
}
}
private BackchannelLogoutResponse logoutUserSession(UserSessionModel userSession) {
BackchannelLogoutResponse backchannelLogoutResponse = AuthenticationManager.backchannelLogout(session, realm,
userSession, session.getContext().getUri(), clientConnection, headers, false);

View file

@ -231,6 +231,7 @@
</resources>
<filterset>
<filter token="HOTROD_SASL_MECHANISM" value="${hotrod.sasl.mechanism}"/>
<filter token="PRELOADING_ENABLED" value="${keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase}"/>
</filterset>
</copy>
<copy todir="${auth.server.home}/standalone/configuration">

View file

@ -145,3 +145,7 @@ echo *** Update undertow subsystem ***
echo *** Update keycloak-server subsystem, infinispan remoteStoreSecurity ***
/subsystem=keycloak-server/spi=connectionsInfinispan/provider=default:map-put(name=properties,key=remoteStoreSecurityEnabled,value=${keycloak.connectionsInfinispan.default.remoteStoreSecurityEnabled:true})
echo *** Enable offline user session preloading ***
/subsystem=keycloak-server/spi=userSessions:add(default-provider=infinispan)
/subsystem=keycloak-server/spi=userSessions/provider=infinispan:add(properties={preloadOfflineSessionsFromDatabase => @PRELOADING_ENABLED@},enabled=true)

View file

@ -534,6 +534,8 @@
<ant.scenario>scenario-crossdc</ant.scenario>
<h2.jdbc.url>jdbc:h2:tcp://localhost:9092/mem:keycloak-dc-shared;DB_CLOSE_DELAY=-1;DB_CLOSE_ON_EXIT=FALSE</h2.jdbc.url>
<keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase>true</keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase>
</properties>
<build>
<plugins>

View file

@ -25,7 +25,6 @@ import org.junit.Test;
import org.keycloak.OAuth2Constants;
import org.keycloak.common.util.Retry;
import org.keycloak.connections.infinispan.InfinispanConnectionProvider;
import org.keycloak.representations.idm.RealmRepresentation;
import org.keycloak.testsuite.Assert;
import org.keycloak.testsuite.admin.ApiUtil;
import org.keycloak.testsuite.arquillian.CrossDCTestEnricher;

View file

@ -194,23 +194,48 @@ public class ClientStorageTest extends AbstractTestRealmKeycloakTest {
public void testClientStats() throws Exception {
testDirectGrant("hardcoded-client");
testDirectGrant("hardcoded-client");
testDirectGrant("direct-grant");
testBrowser("test-app");
offlineTokenDirectGrantFlowNoRefresh();
offlineTokenDirectGrantFlowNoRefresh("hardcoded-client");
offlineTokenDirectGrantFlowNoRefresh("hardcoded-client");
offlineTokenDirectGrantFlowNoRefresh("direct-grant");
offlineTokenDirectGrantFlowNoRefresh("direct-grant");
List<Map<String, String>> list = adminClient.realm("test").getClientSessionStats();
boolean hardTested = false;
boolean testAppTested = false;
boolean directTested = false;
for (Map<String, String> entry : list) {
if (entry.get("clientId").equals("hardcoded-client")) {
Assert.assertEquals("3", entry.get("active"));
Assert.assertEquals("1", entry.get("offline"));
Assert.assertEquals("4", entry.get("active"));
Assert.assertEquals("2", entry.get("offline"));
hardTested = true;
} else if (entry.get("clientId").equals("test-app")) {
Assert.assertEquals("1", entry.get("active"));
Assert.assertEquals("0", entry.get("offline"));
testAppTested = true;
} else if (entry.get("clientId").equals("direct-grant")) {
Assert.assertEquals("3", entry.get("active"));
Assert.assertEquals("2", entry.get("offline"));
directTested = true;
}
}
Assert.assertTrue(hardTested && testAppTested);
Assert.assertTrue(hardTested && testAppTested && directTested);
testingClient.server().run(session -> {
RealmModel realm = session.realms().getRealmByName("test");
ClientModel hardcoded = realm.getClientByClientId("hardcoded-client");
long activeUserSessions = session.sessions().getActiveUserSessions(realm, hardcoded);
long offlineSessionsCount = session.sessions().getOfflineSessionsCount(realm, hardcoded);
Assert.assertEquals(4, activeUserSessions);
Assert.assertEquals(2, offlineSessionsCount);
ClientModel direct = realm.getClientByClientId("direct-grant");
activeUserSessions = session.sessions().getActiveUserSessions(realm, direct);
offlineSessionsCount = session.sessions().getOfflineSessionsCount(realm, direct);
Assert.assertEquals(3, activeUserSessions);
Assert.assertEquals(2, offlineSessionsCount);
});
}
@ -460,9 +485,9 @@ public class ClientStorageTest extends AbstractTestRealmKeycloakTest {
// Assert same token can be refreshed again
testRefreshWithOfflineToken(token, offlineToken, offlineTokenString, token.getSessionState(), userId);
}
public void offlineTokenDirectGrantFlowNoRefresh() throws Exception {
public void offlineTokenDirectGrantFlowNoRefresh(String clientId) throws Exception {
oauth.scope(OAuth2Constants.OFFLINE_ACCESS);
oauth.clientId("hardcoded-client");
oauth.clientId(clientId);
OAuthClient.AccessTokenResponse tokenResponse = oauth.doGrantAccessTokenRequest("password", "test-user@localhost", "password");
Assert.assertNull(tokenResponse.getErrorDescription());
AccessToken token = oauth.verifyToken(tokenResponse.getAccessToken());

View file

@ -425,6 +425,7 @@
"keycloak.connectionsInfinispan.remoteStorePort": "${keycloak.connectionsInfinispan.remoteStorePort:11222}",
"keycloak.connectionsInfinispan.remoteStoreEnabled": "${keycloak.connectionsInfinispan.remoteStoreEnabled:true}",
"keycloak.connectionsInfinispan.hotrodProtocolVersion": "${keycloak.connectionsInfinispan.hotrodProtocolVersion}",
"keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase": "${keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase:true}",
"keycloak.connectionsJpa.url": "${keycloak.connectionsJpa.url.crossdc:jdbc:h2:mem:test-dc-shared}",
"keycloak.connectionsJpa.driver": "${keycloak.connectionsJpa.driver.crossdc:org.h2.Driver}",
"keycloak.connectionsJpa.driverDialect": "${keycloak.connectionsJpa.driverDialect.crossdc:}"
@ -451,6 +452,7 @@
"keycloak.connectionsInfinispan.remoteStorePort": "${keycloak.connectionsInfinispan.remoteStorePort:11222}",
"keycloak.connectionsInfinispan.remoteStoreEnabled": "${keycloak.connectionsInfinispan.remoteStoreEnabled:true}",
"keycloak.connectionsInfinispan.hotrodProtocolVersion": "${keycloak.connectionsInfinispan.hotrodProtocolVersion}",
"keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase": "${keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase:true}",
"keycloak.connectionsJpa.url": "${keycloak.connectionsJpa.url.crossdc:jdbc:h2:mem:test-dc-shared}",
"keycloak.connectionsJpa.driver": "${keycloak.connectionsJpa.driver.crossdc:org.h2.Driver}",
"keycloak.connectionsJpa.driverDialect": "${keycloak.connectionsJpa.driverDialect.crossdc:}"
@ -478,6 +480,7 @@
"keycloak.connectionsInfinispan.remoteStorePort": "${keycloak.connectionsInfinispan.remoteStorePort.2:11222}",
"keycloak.connectionsInfinispan.remoteStoreEnabled": "${keycloak.connectionsInfinispan.remoteStoreEnabled:true}",
"keycloak.connectionsInfinispan.hotrodProtocolVersion": "${keycloak.connectionsInfinispan.hotrodProtocolVersion}",
"keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase": "${keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase:true}",
"keycloak.connectionsJpa.url": "${keycloak.connectionsJpa.url.crossdc:jdbc:h2:mem:test-dc-shared}",
"keycloak.connectionsJpa.driver": "${keycloak.connectionsJpa.driver.crossdc:org.h2.Driver}",
"keycloak.connectionsJpa.driverDialect": "${keycloak.connectionsJpa.driverDialect.crossdc:}"
@ -504,6 +507,7 @@
"keycloak.connectionsInfinispan.remoteStorePort": "${keycloak.connectionsInfinispan.remoteStorePort.2:11222}",
"keycloak.connectionsInfinispan.remoteStoreEnabled": "${keycloak.connectionsInfinispan.remoteStoreEnabled:true}",
"keycloak.connectionsInfinispan.hotrodProtocolVersion": "${keycloak.connectionsInfinispan.hotrodProtocolVersion}",
"keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase": "${keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase:true}",
"keycloak.connectionsJpa.url": "${keycloak.connectionsJpa.url.crossdc:jdbc:h2:mem:test-dc-shared}",
"keycloak.connectionsJpa.driver": "${keycloak.connectionsJpa.driver.crossdc:org.h2.Driver}",
"keycloak.connectionsJpa.driverDialect": "${keycloak.connectionsJpa.driverDialect.crossdc:}"

View file

@ -26,7 +26,7 @@
<log4j.configuration>file:${project.build.directory}/dependency/log4j.properties</log4j.configuration>
<jacoco.skip>true</jacoco.skip>
<keycloak.profile.feature.map_storage>disabled</keycloak.profile.feature.map_storage>
<keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase>true</keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase>
<keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase>false</keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase>
</properties>
<dependencies>
@ -195,10 +195,18 @@
</profile>
<profile>
<id>jpa+infinispan-sessions-preloading-disabled</id>
<id>jpa+cross-dc-infinispan-offline-sessions-preloading</id>
<properties>
<keycloak.model.parameters>CrossDCInfinispan,Jpa</keycloak.model.parameters>
<keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase>true</keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase>
</properties>
</profile>
<profile>
<id>jpa+infinispan-offline-sessions-preloading</id>
<properties>
<keycloak.model.parameters>Infinispan,Jpa</keycloak.model.parameters>
<keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase>false</keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase>
<keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase>true</keycloak.userSessions.infinispan.preloadOfflineSessionsFromDatabase>
</properties>
</profile>

View file

@ -17,9 +17,11 @@
package org.keycloak.testsuite.model.session;
import org.infinispan.Cache;
import org.junit.Assert;
import org.junit.Test;
import org.keycloak.common.util.Time;
import org.keycloak.connections.infinispan.InfinispanConnectionProvider;
import org.keycloak.models.AuthenticatedClientSessionModel;
import org.keycloak.models.ClientModel;
import org.keycloak.models.Constants;
@ -40,10 +42,17 @@ import org.keycloak.timer.TimerProvider;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.keycloak.testsuite.model.KeycloakModelTest;
import org.keycloak.testsuite.model.RequireProvider;
@ -297,6 +306,73 @@ public class UserSessionProviderOfflineModelTest extends KeycloakModelTest {
}
}
@Test
public void testOfflineSessionLazyLoading() throws InterruptedException {
AtomicReference<List<UserSessionModel>> offlineUserSessions = new AtomicReference<>(new LinkedList<>());
AtomicReference<List<AuthenticatedClientSessionModel>> offlineClientSessions = new AtomicReference<>(new LinkedList<>());
createOfflineSessions("user1", 10, offlineUserSessions, offlineClientSessions);
closeKeycloakSessionFactory();
AtomicBoolean result = new AtomicBoolean(true);
CountDownLatch latch = new CountDownLatch(4);
inIndependentFactories(4, 300, () -> {
withRealm(realmId, (session, realm) -> {
final UserModel user = session.users().getUserByUsername(realm, "user1");
result.set(result.get() && assertOfflineSession(offlineUserSessions, session.sessions().getOfflineUserSessionsStream(realm, user).collect(Collectors.toList())));
return null;
});
latch.countDown();
awaitLatch(latch);
});
Assert.assertTrue(result.get());
}
@Test
public void testOfflineSessionLazyLoadingPropagationBetweenNodes() throws InterruptedException {
AtomicReference<List<UserSessionModel>> offlineUserSessions = new AtomicReference<>(new LinkedList<>());
AtomicReference<List<AuthenticatedClientSessionModel>> offlineClientSessions = new AtomicReference<>(new LinkedList<>());
AtomicBoolean result = new AtomicBoolean(true);
AtomicInteger index = new AtomicInteger();
CountDownLatch latch = new CountDownLatch(4);
CountDownLatch afterFirstNodeLatch = new CountDownLatch(1);
inIndependentFactories(4, 300, () -> {
if (index.incrementAndGet() == 1) {
createOfflineSessions("user1", 10, offlineUserSessions, offlineClientSessions);
afterFirstNodeLatch.countDown();
}
awaitLatch(afterFirstNodeLatch);
log.debug("Joining the cluster");
inComittedTransaction(session -> {
InfinispanConnectionProvider provider = session.getProvider(InfinispanConnectionProvider.class);
Cache<String, Object> cache = provider.getCache(InfinispanConnectionProvider.WORK_CACHE_NAME);
do {
try { Thread.sleep(1000); } catch (InterruptedException ex) { Thread.currentThread().interrupt(); throw new RuntimeException(ex); }
} while (! cache.getAdvancedCache().getDistributionManager().isJoinComplete());
cache.keySet().forEach(s -> {});
});
log.debug("Cluster joined");
withRealm(realmId, (session, realm) -> {
final UserModel user = session.users().getUserByUsername(realm, "user1");
result.set(result.get() && assertOfflineSession(offlineUserSessions, session.sessions().getOfflineUserSessionsStream(realm, user).collect(Collectors.toList())));
return null;
});
latch.countDown();
awaitLatch(latch);
});
Assert.assertTrue(result.get());
}
private static Set<String> createOfflineSessionIncludeClientSessions(KeycloakSession session, UserSessionModel
userSession) {
Set<String> offlineSessions = new HashSet<>();
@ -308,4 +384,40 @@ public class UserSessionProviderOfflineModelTest extends KeycloakModelTest {
return offlineSessions;
}
private void createOfflineSessions(String username, int sessionsPerUser, AtomicReference<List<UserSessionModel>> offlineUserSessions, AtomicReference<List<AuthenticatedClientSessionModel>> offlineClientSessions) {
withRealm(realmId, (session, realm) -> {
final UserModel user = session.users().getUserByUsername(realm, username);
ClientModel testAppClient = realm.getClientByClientId("test-app");
ClientModel thirdPartyClient = realm.getClientByClientId("third-party");
IntStream.range(0, sessionsPerUser)
.mapToObj(index -> session.sessions().createUserSession(realm, user, username + index, "ip" + index, "auth", false, null, null))
.forEach(userSession -> {
AuthenticatedClientSessionModel testAppClientSession = session.sessions().createClientSession(realm, testAppClient, userSession);
AuthenticatedClientSessionModel thirdPartyClientSession = session.sessions().createClientSession(realm, thirdPartyClient, userSession);
UserSessionModel offlineUserSession = session.sessions().createOfflineUserSession(userSession);
offlineUserSessions.get().add(offlineUserSession);
offlineClientSessions.get().add(session.sessions().createOfflineClientSession(testAppClientSession, offlineUserSession));
offlineClientSessions.get().add(session.sessions().createOfflineClientSession(thirdPartyClientSession, offlineUserSession));
});
return null;
});
}
private boolean assertOfflineSession(AtomicReference<List<UserSessionModel>> expectedUserSessions, List<UserSessionModel> actualUserSessions) {
boolean result = expectedUserSessions.get().size() == actualUserSessions.size();
for (UserSessionModel userSession: expectedUserSessions.get()) {
result = result && actualUserSessions.contains(userSession);
}
return result;
}
private void awaitLatch(CountDownLatch latch) {
try {
latch.await();
} catch (InterruptedException e) {
}
}
}