aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorHarald Musum <musum@verizonmedia.com>2020-06-10 19:51:59 +0200
committerGitHub <noreply@github.com>2020-06-10 19:51:59 +0200
commit52d797113dc572be0a5fe76e5d6267a890721c61 (patch)
treeb2eb2b5a6e75eff1f6dba55c04df20632ccc5b67
parenta3745acefdfdf32e321a83cecd1ef21fff5d11d8 (diff)
parent3573453011b6673675391154661f2803f476b7e0 (diff)
Merge pull request #13543 from vespa-engine/revert-13534-musum/configserver-refactoring-8
Revert "Merge SessionFactory into SessionRepository"
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/ApplicationRepository.java12
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionFactory.java280
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionRepository.java253
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/tenant/Tenant.java12
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/tenant/TenantRepository.java10
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/session/SessionRepositoryTest.java10
6 files changed, 323 insertions, 254 deletions
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/ApplicationRepository.java b/configserver/src/main/java/com/yahoo/vespa/config/server/ApplicationRepository.java
index ce6b4587ada..af5a561aa6e 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/ApplicationRepository.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/ApplicationRepository.java
@@ -53,6 +53,7 @@ import com.yahoo.vespa.config.server.session.SessionRepository;
import com.yahoo.vespa.config.server.session.PrepareParams;
import com.yahoo.vespa.config.server.session.RemoteSession;
import com.yahoo.vespa.config.server.session.Session;
+import com.yahoo.vespa.config.server.session.SessionFactory;
import com.yahoo.vespa.config.server.session.SilentDeployLogger;
import com.yahoo.vespa.config.server.tenant.ApplicationRolesStore;
import com.yahoo.vespa.config.server.tenant.ContainerEndpointsCache;
@@ -302,7 +303,7 @@ public class ApplicationRepository implements com.yahoo.config.provision.Deploye
LocalSession activeSession = getActiveLocalSession(tenant, application);
if (activeSession == null) return Optional.empty();
TimeoutBudget timeoutBudget = new TimeoutBudget(clock, timeout);
- LocalSession newSession = tenant.getSessionRepository().createSessionFromExisting(activeSession, logger, true, timeoutBudget);
+ LocalSession newSession = tenant.getSessionFactory().createSessionFromExisting(activeSession, logger, true, timeoutBudget);
tenant.getSessionRepository().addSession(newSession);
return Optional.of(Deployment.unprepared(newSession, this, hostProvisioner, tenant, timeout, clock,
@@ -634,8 +635,9 @@ public class ApplicationRepository implements com.yahoo.config.provision.Deploye
TimeoutBudget timeoutBudget) {
Tenant tenant = tenantRepository.getTenant(applicationId.tenant());
SessionRepository sessionRepository = tenant.getSessionRepository();
+ SessionFactory sessionFactory = tenant.getSessionFactory();
RemoteSession fromSession = getExistingSession(tenant, applicationId);
- LocalSession session = sessionRepository.createSessionFromExisting(fromSession, logger, internalRedeploy, timeoutBudget);
+ LocalSession session = sessionFactory.createSessionFromExisting(fromSession, logger, internalRedeploy, timeoutBudget);
sessionRepository.addSession(session);
return session.getSessionId();
}
@@ -655,10 +657,8 @@ public class ApplicationRepository implements com.yahoo.config.provision.Deploye
Tenant tenant = tenantRepository.getTenant(applicationId.tenant());
tenant.getApplicationRepo().createApplication(applicationId);
Optional<Long> activeSessionId = tenant.getApplicationRepo().activeSessionOf(applicationId);
- LocalSession session = tenant.getSessionRepository().createSession(applicationDirectory,
- applicationId,
- timeoutBudget,
- activeSessionId);
+ LocalSession session = tenant.getSessionFactory().createSession(applicationDirectory, applicationId,
+ timeoutBudget, activeSessionId);
tenant.getSessionRepository().addSession(session);
return session.getSessionId();
}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionFactory.java b/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionFactory.java
new file mode 100644
index 00000000000..8aba9fa465d
--- /dev/null
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionFactory.java
@@ -0,0 +1,280 @@
+// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.vespa.config.server.session;
+
+import com.yahoo.config.FileReference;
+import com.yahoo.config.application.api.ApplicationPackage;
+import com.yahoo.config.application.api.DeployLogger;
+import com.yahoo.config.model.application.provider.DeployData;
+import com.yahoo.config.model.application.provider.FilesApplicationPackage;
+import com.yahoo.config.provision.ApplicationId;
+import com.yahoo.config.provision.NodeFlavors;
+import com.yahoo.config.provision.TenantName;
+import com.yahoo.io.IOUtils;
+import com.yahoo.path.Path;
+import com.yahoo.vespa.config.server.GlobalComponentRegistry;
+import com.yahoo.vespa.config.server.TimeoutBudget;
+import com.yahoo.vespa.config.server.application.TenantApplications;
+import com.yahoo.vespa.config.server.deploy.TenantFileSystemDirs;
+import com.yahoo.vespa.config.server.filedistribution.FileDirectory;
+import com.yahoo.vespa.config.server.host.HostValidator;
+import com.yahoo.vespa.config.server.tenant.TenantRepository;
+import com.yahoo.vespa.config.server.zookeeper.ConfigCurator;
+import com.yahoo.vespa.config.server.zookeeper.SessionCounter;
+import com.yahoo.vespa.curator.Curator;
+import com.yahoo.vespa.defaults.Defaults;
+import com.yahoo.vespa.flags.BooleanFlag;
+import com.yahoo.vespa.flags.Flags;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Clock;
+import java.util.List;
+import java.util.Optional;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * Serves as the factory of sessions. Takes care of copying files to the correct folder and initializing the
+ * session state. There is one SessionFactory per tenant.
+ *
+ * @author Ulf Lilleengen
+ */
+public class SessionFactory {
+
+ private static final Logger log = Logger.getLogger(SessionFactory.class.getName());
+ private static final long nonExistingActiveSession = 0;
+
+ private final SessionPreparer sessionPreparer;
+ private final Curator curator;
+ private final ConfigCurator configCurator;
+ private final TenantApplications applicationRepo;
+ private final Path sessionsPath;
+ private final GlobalComponentRegistry componentRegistry;
+ private final HostValidator<ApplicationId> hostRegistry;
+ private final TenantName tenant;
+ private final String serverId;
+ private final Optional<NodeFlavors> nodeFlavors;
+ private final Clock clock;
+ private final BooleanFlag distributeApplicationPackage;
+
+ public SessionFactory(GlobalComponentRegistry globalComponentRegistry,
+ TenantApplications applicationRepo,
+ HostValidator<ApplicationId> hostRegistry,
+ TenantName tenant) {
+ this.hostRegistry = hostRegistry;
+ this.tenant = tenant;
+ this.sessionPreparer = globalComponentRegistry.getSessionPreparer();
+ this.curator = globalComponentRegistry.getCurator();
+ this.configCurator = globalComponentRegistry.getConfigCurator();
+ this.sessionsPath = TenantRepository.getSessionsPath(tenant);
+ this.applicationRepo = applicationRepo;
+ this.componentRegistry = globalComponentRegistry;
+ this.serverId = globalComponentRegistry.getConfigserverConfig().serverId();
+ this.nodeFlavors = globalComponentRegistry.getZone().nodeFlavors();
+ this.clock = globalComponentRegistry.getClock();
+ this.distributeApplicationPackage = Flags.CONFIGSERVER_DISTRIBUTE_APPLICATION_PACKAGE
+ .bindTo(globalComponentRegistry.getFlagSource());
+ }
+
+ /**
+ * Creates a new deployment session from an application package.
+ *
+ * @param applicationDirectory a File pointing to an application.
+ * @param applicationId application id for this new session.
+ * @param timeoutBudget Timeout for creating session and waiting for other servers.
+ * @return a new session
+ */
+ public LocalSession createSession(File applicationDirectory, ApplicationId applicationId,
+ TimeoutBudget timeoutBudget, Optional<Long> activeSessionId) {
+ return create(applicationDirectory, applicationId, activeSessionId.orElse(nonExistingActiveSession), false, timeoutBudget);
+ }
+
+ public RemoteSession createRemoteSession(long sessionId) {
+ SessionZooKeeperClient sessionZKClient = createSessionZooKeeperClient(getSessionPath(sessionId));
+ return new RemoteSession(tenant, sessionId, componentRegistry, sessionZKClient);
+ }
+
+ private void ensureSessionPathDoesNotExist(long sessionId) {
+ Path sessionPath = getSessionPath(sessionId);
+ if (configCurator.exists(sessionPath.getAbsolute())) {
+ throw new IllegalArgumentException("Path " + sessionPath.getAbsolute() + " already exists in ZooKeeper");
+ }
+ }
+
+ private ApplicationPackage createApplication(File userDir,
+ File configApplicationDir,
+ ApplicationId applicationId,
+ long sessionId,
+ long currentlyActiveSessionId,
+ boolean internalRedeploy) {
+ long deployTimestamp = System.currentTimeMillis();
+ String user = System.getenv("USER");
+ if (user == null) {
+ user = "unknown";
+ }
+ DeployData deployData = new DeployData(user, userDir.getAbsolutePath(), applicationId, deployTimestamp, internalRedeploy, sessionId, currentlyActiveSessionId);
+ return FilesApplicationPackage.fromFileWithDeployData(configApplicationDir, deployData);
+ }
+
+ private LocalSession createSessionFromApplication(ApplicationPackage applicationPackage,
+ long sessionId,
+ TimeoutBudget timeoutBudget,
+ Clock clock) {
+ log.log(Level.FINE, TenantRepository.logPre(tenant) + "Creating session " + sessionId + " in ZooKeeper");
+ SessionZooKeeperClient sessionZKClient = createSessionZooKeeperClient(getSessionPath(sessionId));
+ sessionZKClient.createNewSession(clock.instant());
+ Curator.CompletionWaiter waiter = sessionZKClient.getUploadWaiter();
+ LocalSession session = new LocalSession(tenant, sessionId, sessionPreparer, applicationPackage, sessionZKClient,
+ getSessionAppDir(sessionId), applicationRepo, hostRegistry);
+ waiter.awaitCompletion(timeoutBudget.timeLeft());
+ return session;
+ }
+
+ /**
+ * Creates a new deployment session from an already existing session.
+ *
+ * @param existingSession the session to use as base
+ * @param logger a deploy logger where the deploy log will be written.
+ * @param internalRedeploy whether this session is for a system internal redeploy — not an application package change
+ * @param timeoutBudget timeout for creating session and waiting for other servers.
+ * @return a new session
+ */
+ public LocalSession createSessionFromExisting(Session existingSession,
+ DeployLogger logger,
+ boolean internalRedeploy,
+ TimeoutBudget timeoutBudget) {
+ File existingApp = getSessionAppDir(existingSession.getSessionId());
+ ApplicationId existingApplicationId = existingSession.getApplicationId();
+
+ long activeSessionId = getActiveSessionId(existingApplicationId);
+ logger.log(Level.FINE, "Create new session for application id '" + existingApplicationId + "' from existing active session " + activeSessionId);
+ LocalSession session = create(existingApp, existingApplicationId, activeSessionId, internalRedeploy, timeoutBudget);
+ // Note: Needs to be kept in sync with calls in SessionPreparer.writeStateToZooKeeper()
+ session.setApplicationId(existingApplicationId);
+ if (distributeApplicationPackage.value() && existingSession.getApplicationPackageReference() != null) {
+ session.setApplicationPackageReference(existingSession.getApplicationPackageReference());
+ }
+ session.setVespaVersion(existingSession.getVespaVersion());
+ session.setDockerImageRepository(existingSession.getDockerImageRepository());
+ session.setAthenzDomain(existingSession.getAthenzDomain());
+ return session;
+ }
+
+ private LocalSession create(File applicationFile, ApplicationId applicationId, long currentlyActiveSessionId,
+ boolean internalRedeploy, TimeoutBudget timeoutBudget) {
+ long sessionId = getNextSessionId();
+ try {
+ ensureSessionPathDoesNotExist(sessionId);
+ ApplicationPackage app = createApplicationPackage(applicationFile, applicationId,
+ sessionId, currentlyActiveSessionId, internalRedeploy);
+ return createSessionFromApplication(app, sessionId, timeoutBudget, clock);
+ } catch (Exception e) {
+ throw new RuntimeException("Error creating session " + sessionId, e);
+ }
+ }
+
+ /**
+ * This method is used when creating a session based on a remote session and the distributed application package
+ * It does not wait for session being created on other servers
+ */
+ private LocalSession createLocalSession(File applicationFile, ApplicationId applicationId,
+ long sessionId, long currentlyActiveSessionId) {
+ try {
+ ApplicationPackage applicationPackage = createApplicationPackage(applicationFile, applicationId,
+ sessionId, currentlyActiveSessionId, false);
+ SessionZooKeeperClient sessionZooKeeperClient = createSessionZooKeeperClient(getSessionPath(sessionId));
+ return new LocalSession(tenant, sessionId, sessionPreparer, applicationPackage, sessionZooKeeperClient,
+ getSessionAppDir(sessionId), applicationRepo, hostRegistry);
+ } catch (Exception e) {
+ throw new RuntimeException("Error creating session " + sessionId, e);
+ }
+ }
+
+ private ApplicationPackage createApplicationPackage(File applicationFile, ApplicationId applicationId,
+ long sessionId, long currentlyActiveSessionId,
+ boolean internalRedeploy) throws IOException {
+ File userApplicationDir = getSessionAppDir(sessionId);
+ IOUtils.copyDirectory(applicationFile, userApplicationDir);
+ ApplicationPackage applicationPackage = createApplication(applicationFile,
+ userApplicationDir,
+ applicationId,
+ sessionId,
+ currentlyActiveSessionId,
+ internalRedeploy);
+ applicationPackage.writeMetaData();
+ return applicationPackage;
+ }
+
+ /**
+ * Returns a new session instance for the given session id.
+ */
+ LocalSession createSessionFromId(long sessionId) {
+ File sessionDir = getAndValidateExistingSessionAppDir(sessionId);
+ ApplicationPackage applicationPackage = FilesApplicationPackage.fromFile(sessionDir);
+ Path sessionIdPath = sessionsPath.append(String.valueOf(sessionId));
+ SessionZooKeeperClient sessionZKClient = createSessionZooKeeperClient(sessionIdPath);
+ return new LocalSession(tenant, sessionId, sessionPreparer, applicationPackage, sessionZKClient,
+ getSessionAppDir(sessionId), applicationRepo, hostRegistry);
+ }
+
+ /**
+ * Returns a new session instance for the given session id.
+ */
+ LocalSession createLocalSessionUsingDistributedApplicationPackage(long sessionId) {
+ if (applicationRepo.hasLocalSession(sessionId)) {
+ log.log(Level.FINE, "Local session for session id " + sessionId + " already exists");
+ return createSessionFromId(sessionId);
+ }
+
+ log.log(Level.INFO, "Creating local session for session id " + sessionId);
+ SessionZooKeeperClient sessionZKClient = createSessionZooKeeperClient(getSessionPath(sessionId));
+ FileReference fileReference = sessionZKClient.readApplicationPackageReference();
+ log.log(Level.FINE, "File reference for session id " + sessionId + ": " + fileReference);
+ if (fileReference != null) {
+ File rootDir = new File(Defaults.getDefaults().underVespaHome(componentRegistry.getConfigserverConfig().fileReferencesDir()));
+ File sessionDir = new FileDirectory(rootDir).getFile(fileReference);
+ if (!sessionDir.exists())
+ throw new RuntimeException("File reference for session " + sessionId + " not found (" + sessionDir.getAbsolutePath() + ")");
+ ApplicationId applicationId = sessionZKClient.readApplicationId();
+ return createLocalSession(sessionDir,
+ applicationId,
+ sessionId,
+ applicationRepo.activeSessionOf(applicationId).orElse(nonExistingActiveSession));
+ }
+ return null;
+ }
+
+ // Return Optional instead of faking it with nonExistingActiveSession
+ private long getActiveSessionId(ApplicationId applicationId) {
+ List<ApplicationId> applicationIds = applicationRepo.activeApplications();
+ if (applicationIds.contains(applicationId)) {
+ return applicationRepo.requireActiveSessionOf(applicationId);
+ }
+ return nonExistingActiveSession;
+ }
+
+ private long getNextSessionId() {
+ return new SessionCounter(componentRegistry.getConfigCurator(), tenant).nextSessionId();
+ }
+
+ private Path getSessionPath(long sessionId) {
+ return sessionsPath.append(String.valueOf(sessionId));
+ }
+
+ private SessionZooKeeperClient createSessionZooKeeperClient(Path sessionPath) {
+ return new SessionZooKeeperClient(curator, configCurator, sessionPath, serverId, nodeFlavors);
+ }
+
+ private File getAndValidateExistingSessionAppDir(long sessionId) {
+ File appDir = getSessionAppDir(sessionId);
+ if (!appDir.exists() || !appDir.isDirectory()) {
+ throw new IllegalArgumentException("Unable to find correct application directory for session " + sessionId);
+ }
+ return appDir;
+ }
+
+ private File getSessionAppDir(long sessionId) {
+ return new TenantFileSystemDirs(componentRegistry.getConfigServerDB(), tenant).getUserApplicationDir(sessionId);
+ }
+
+}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionRepository.java b/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionRepository.java
index c0d09cfba30..9fe725453f4 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionRepository.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionRepository.java
@@ -3,31 +3,19 @@ package com.yahoo.vespa.config.server.session;
import com.google.common.collect.HashMultiset;
import com.google.common.collect.Multiset;
-import com.yahoo.config.FileReference;
-import com.yahoo.config.application.api.ApplicationPackage;
-import com.yahoo.config.application.api.DeployLogger;
-import com.yahoo.config.model.application.provider.DeployData;
-import com.yahoo.config.model.application.provider.FilesApplicationPackage;
import com.yahoo.config.provision.ApplicationId;
-import com.yahoo.config.provision.NodeFlavors;
import com.yahoo.config.provision.TenantName;
-import com.yahoo.io.IOUtils;
import com.yahoo.path.Path;
import com.yahoo.transaction.NestedTransaction;
import com.yahoo.vespa.config.server.GlobalComponentRegistry;
import com.yahoo.vespa.config.server.ReloadHandler;
-import com.yahoo.vespa.config.server.TimeoutBudget;
import com.yahoo.vespa.config.server.application.TenantApplications;
import com.yahoo.vespa.config.server.deploy.TenantFileSystemDirs;
-import com.yahoo.vespa.config.server.filedistribution.FileDirectory;
-import com.yahoo.vespa.config.server.host.HostValidator;
import com.yahoo.vespa.config.server.monitoring.MetricUpdater;
import com.yahoo.vespa.config.server.monitoring.Metrics;
import com.yahoo.vespa.config.server.tenant.TenantRepository;
import com.yahoo.vespa.config.server.zookeeper.ConfigCurator;
-import com.yahoo.vespa.config.server.zookeeper.SessionCounter;
import com.yahoo.vespa.curator.Curator;
-import com.yahoo.vespa.defaults.Defaults;
import com.yahoo.vespa.flags.BooleanFlag;
import com.yahoo.vespa.flags.FlagSource;
import com.yahoo.vespa.flags.Flags;
@@ -37,7 +25,6 @@ import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
import java.io.File;
import java.io.FilenameFilter;
-import java.io.IOException;
import java.time.Clock;
import java.time.Duration;
import java.time.Instant;
@@ -46,7 +33,6 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import java.util.Optional;
import java.util.concurrent.Executor;
import java.util.logging.Level;
import java.util.logging.Logger;
@@ -65,9 +51,9 @@ public class SessionRepository {
private static final Logger log = Logger.getLogger(SessionRepository.class.getName());
private static final FilenameFilter sessionApplicationsFilter = (dir, name) -> name.matches("\\d+");
- private static final long nonExistingActiveSession = 0;
private final SessionCache<LocalSession> localSessionCache;
+
private final SessionCache<RemoteSession> remoteSessionCache = new SessionCache<>();
private final Map<Long, LocalSessionStateWatcher> localSessionStateWatchers = new HashMap<>();
private final Map<Long, RemoteSessionStateWatcher> remoteSessionStateWatchers = new HashMap<>();
@@ -81,36 +67,28 @@ public class SessionRepository {
private final MetricUpdater metrics;
private final Curator.DirectoryCache directoryCache;
private final TenantApplications applicationRepo;
- private final HostValidator<ApplicationId> hostRegistry;
- private final SessionPreparer sessionPreparer;
private final Path sessionsPath;
+ private final SessionFactory sessionFactory;
private final TenantName tenantName;
- private final GlobalComponentRegistry componentRegistry;
-
- public SessionRepository(TenantName tenantName,
- GlobalComponentRegistry componentRegistry,
- TenantApplications applicationRepo,
- ReloadHandler reloadHandler,
- FlagSource flagSource,
- HostValidator<ApplicationId> hostRegistry,
- SessionPreparer sessionPreparer) {
+
+ public SessionRepository(TenantName tenantName, GlobalComponentRegistry componentRegistry,
+ SessionFactory sessionFactory, TenantApplications applicationRepo,
+ ReloadHandler reloadHandler, FlagSource flagSource) {
this.tenantName = tenantName;
- this.componentRegistry = componentRegistry;
- this.sessionsPath = TenantRepository.getSessionsPath(tenantName);
localSessionCache = new SessionCache<>();
this.clock = componentRegistry.getClock();
this.curator = componentRegistry.getCurator();
this.sessionLifetime = Duration.ofSeconds(componentRegistry.getConfigserverConfig().sessionLifetime());
this.zkWatcherExecutor = command -> componentRegistry.getZkWatcherExecutor().execute(tenantName, command);
this.tenantFileSystemDirs = new TenantFileSystemDirs(componentRegistry.getConfigServerDB(), tenantName);
+ this.sessionFactory = sessionFactory;
this.applicationRepo = applicationRepo;
- this.hostRegistry = hostRegistry;
- this.sessionPreparer = sessionPreparer;
+ loadLocalSessions(sessionFactory);
+
this.distributeApplicationPackage = Flags.CONFIGSERVER_DISTRIBUTE_APPLICATION_PACKAGE.bindTo(flagSource);
this.reloadHandler = reloadHandler;
+ this.sessionsPath = TenantRepository.getSessionsPath(tenantName);
this.metrics = componentRegistry.getMetrics().getOrCreateMetricUpdater(Metrics.createDimensions(tenantName));
-
- loadLocalSessions();
initializeRemoteSessions();
this.directoryCache = curator.createDirectoryCache(sessionsPath.getAbsolute(), false, false, componentRegistry.getZkCacheExecutor());
this.directoryCache.addListener(this::childEvent);
@@ -135,14 +113,14 @@ public class SessionRepository {
return localSessionCache.getSessions();
}
- private void loadLocalSessions() {
+ private void loadLocalSessions(SessionFactory sessionFactory) {
File[] sessions = tenantFileSystemDirs.sessionsPath().listFiles(sessionApplicationsFilter);
if (sessions == null) {
return;
}
for (File session : sessions) {
try {
- addSession(createSessionFromId(Long.parseLong(session.getName())));
+ addSession(sessionFactory.createSessionFromId(Long.parseLong(session.getName())));
} catch (IllegalArgumentException e) {
log.log(Level.WARNING, "Could not load session '" +
session.getAbsolutePath() + "':" + e.getMessage() + ", skipping it.");
@@ -291,7 +269,7 @@ public class SessionRepository {
*/
private void sessionAdded(long sessionId) {
log.log(Level.FINE, () -> "Adding session to SessionRepository: " + sessionId);
- RemoteSession session = createRemoteSession(sessionId);
+ RemoteSession session = sessionFactory.createRemoteSession(sessionId);
Path sessionPath = sessionsPath.append(String.valueOf(sessionId));
Curator.FileCache fileCache = curator.createFileCache(sessionPath.append(ConfigCurator.SESSIONSTATE_ZK_SUBPATH).getAbsolute(), false);
fileCache.addListener(this::nodeChanged);
@@ -299,7 +277,7 @@ public class SessionRepository {
addRemoteSession(session);
remoteSessionStateWatchers.put(sessionId, new RemoteSessionStateWatcher(fileCache, reloadHandler, session, metrics, zkWatcherExecutor));
if (distributeApplicationPackage.value())
- createLocalSessionUsingDistributedApplicationPackage(sessionId);
+ sessionFactory.createLocalSessionUsingDistributedApplicationPackage(sessionId);
}
private void sessionRemoved(long sessionId) {
@@ -361,209 +339,6 @@ public class SessionRepository {
}
}
- /**
- * Creates a new deployment session from an application package.
- *
- * @param applicationDirectory a File pointing to an application.
- * @param applicationId application id for this new session.
- * @param timeoutBudget Timeout for creating session and waiting for other servers.
- * @return a new session
- */
- public LocalSession createSession(File applicationDirectory, ApplicationId applicationId,
- TimeoutBudget timeoutBudget, Optional<Long> activeSessionId) {
- return create(applicationDirectory, applicationId, activeSessionId.orElse(nonExistingActiveSession), false, timeoutBudget);
- }
-
- public RemoteSession createRemoteSession(long sessionId) {
- SessionZooKeeperClient sessionZKClient = createSessionZooKeeperClient(getSessionPath(sessionId));
- return new RemoteSession(tenantName, sessionId, componentRegistry, sessionZKClient);
- }
-
- private void ensureSessionPathDoesNotExist(long sessionId) {
- Path sessionPath = getSessionPath(sessionId);
- if (componentRegistry.getConfigCurator().exists(sessionPath.getAbsolute())) {
- throw new IllegalArgumentException("Path " + sessionPath.getAbsolute() + " already exists in ZooKeeper");
- }
- }
-
- private ApplicationPackage createApplication(File userDir,
- File configApplicationDir,
- ApplicationId applicationId,
- long sessionId,
- long currentlyActiveSessionId,
- boolean internalRedeploy) {
- long deployTimestamp = System.currentTimeMillis();
- String user = System.getenv("USER");
- if (user == null) {
- user = "unknown";
- }
- DeployData deployData = new DeployData(user, userDir.getAbsolutePath(), applicationId, deployTimestamp, internalRedeploy, sessionId, currentlyActiveSessionId);
- return FilesApplicationPackage.fromFileWithDeployData(configApplicationDir, deployData);
- }
-
- private LocalSession createSessionFromApplication(ApplicationPackage applicationPackage,
- long sessionId,
- TimeoutBudget timeoutBudget,
- Clock clock) {
- log.log(Level.FINE, TenantRepository.logPre(tenantName) + "Creating session " + sessionId + " in ZooKeeper");
- SessionZooKeeperClient sessionZKClient = createSessionZooKeeperClient(getSessionPath(sessionId));
- sessionZKClient.createNewSession(clock.instant());
- Curator.CompletionWaiter waiter = sessionZKClient.getUploadWaiter();
- LocalSession session = new LocalSession(tenantName, sessionId, sessionPreparer, applicationPackage, sessionZKClient,
- getSessionAppDir(sessionId), applicationRepo, hostRegistry);
- waiter.awaitCompletion(timeoutBudget.timeLeft());
- return session;
- }
-
- /**
- * Creates a new deployment session from an already existing session.
- *
- * @param existingSession the session to use as base
- * @param logger a deploy logger where the deploy log will be written.
- * @param internalRedeploy whether this session is for a system internal redeploy — not an application package change
- * @param timeoutBudget timeout for creating session and waiting for other servers.
- * @return a new session
- */
- public LocalSession createSessionFromExisting(Session existingSession,
- DeployLogger logger,
- boolean internalRedeploy,
- TimeoutBudget timeoutBudget) {
- File existingApp = getSessionAppDir(existingSession.getSessionId());
- ApplicationId existingApplicationId = existingSession.getApplicationId();
-
- long activeSessionId = getActiveSessionId(existingApplicationId);
- logger.log(Level.FINE, "Create new session for application id '" + existingApplicationId + "' from existing active session " + activeSessionId);
- LocalSession session = create(existingApp, existingApplicationId, activeSessionId, internalRedeploy, timeoutBudget);
- // Note: Needs to be kept in sync with calls in SessionPreparer.writeStateToZooKeeper()
- session.setApplicationId(existingApplicationId);
- if (distributeApplicationPackage.value() && existingSession.getApplicationPackageReference() != null) {
- session.setApplicationPackageReference(existingSession.getApplicationPackageReference());
- }
- session.setVespaVersion(existingSession.getVespaVersion());
- session.setDockerImageRepository(existingSession.getDockerImageRepository());
- session.setAthenzDomain(existingSession.getAthenzDomain());
- return session;
- }
-
- private LocalSession create(File applicationFile, ApplicationId applicationId, long currentlyActiveSessionId,
- boolean internalRedeploy, TimeoutBudget timeoutBudget) {
- long sessionId = getNextSessionId();
- try {
- ensureSessionPathDoesNotExist(sessionId);
- ApplicationPackage app = createApplicationPackage(applicationFile, applicationId,
- sessionId, currentlyActiveSessionId, internalRedeploy);
- return createSessionFromApplication(app, sessionId, timeoutBudget, clock);
- } catch (Exception e) {
- throw new RuntimeException("Error creating session " + sessionId, e);
- }
- }
-
- /**
- * This method is used when creating a session based on a remote session and the distributed application package
- * It does not wait for session being created on other servers
- */
- private LocalSession createLocalSession(File applicationFile, ApplicationId applicationId,
- long sessionId, long currentlyActiveSessionId) {
- try {
- ApplicationPackage applicationPackage = createApplicationPackage(applicationFile, applicationId,
- sessionId, currentlyActiveSessionId, false);
- SessionZooKeeperClient sessionZooKeeperClient = createSessionZooKeeperClient(getSessionPath(sessionId));
- return new LocalSession(tenantName, sessionId, sessionPreparer, applicationPackage, sessionZooKeeperClient,
- getSessionAppDir(sessionId), applicationRepo, hostRegistry);
- } catch (Exception e) {
- throw new RuntimeException("Error creating session " + sessionId, e);
- }
- }
-
- private ApplicationPackage createApplicationPackage(File applicationFile, ApplicationId applicationId,
- long sessionId, long currentlyActiveSessionId,
- boolean internalRedeploy) throws IOException {
- File userApplicationDir = getSessionAppDir(sessionId);
- IOUtils.copyDirectory(applicationFile, userApplicationDir);
- ApplicationPackage applicationPackage = createApplication(applicationFile,
- userApplicationDir,
- applicationId,
- sessionId,
- currentlyActiveSessionId,
- internalRedeploy);
- applicationPackage.writeMetaData();
- return applicationPackage;
- }
-
- /**
- * Returns a new session instance for the given session id.
- */
- LocalSession createSessionFromId(long sessionId) {
- File sessionDir = getAndValidateExistingSessionAppDir(sessionId);
- ApplicationPackage applicationPackage = FilesApplicationPackage.fromFile(sessionDir);
- Path sessionIdPath = sessionsPath.append(String.valueOf(sessionId));
- SessionZooKeeperClient sessionZKClient = createSessionZooKeeperClient(sessionIdPath);
- return new LocalSession(tenantName, sessionId, sessionPreparer, applicationPackage, sessionZKClient,
- getSessionAppDir(sessionId), applicationRepo, hostRegistry);
- }
-
- /**
- * Returns a new session instance for the given session id.
- */
- LocalSession createLocalSessionUsingDistributedApplicationPackage(long sessionId) {
- if (applicationRepo.hasLocalSession(sessionId)) {
- log.log(Level.FINE, "Local session for session id " + sessionId + " already exists");
- return createSessionFromId(sessionId);
- }
-
- log.log(Level.INFO, "Creating local session for session id " + sessionId);
- SessionZooKeeperClient sessionZKClient = createSessionZooKeeperClient(getSessionPath(sessionId));
- FileReference fileReference = sessionZKClient.readApplicationPackageReference();
- log.log(Level.FINE, "File reference for session id " + sessionId + ": " + fileReference);
- if (fileReference != null) {
- File rootDir = new File(Defaults.getDefaults().underVespaHome(componentRegistry.getConfigserverConfig().fileReferencesDir()));
- File sessionDir = new FileDirectory(rootDir).getFile(fileReference);
- if (!sessionDir.exists())
- throw new RuntimeException("File reference for session " + sessionId + " not found (" + sessionDir.getAbsolutePath() + ")");
- ApplicationId applicationId = sessionZKClient.readApplicationId();
- return createLocalSession(sessionDir,
- applicationId,
- sessionId,
- applicationRepo.activeSessionOf(applicationId).orElse(nonExistingActiveSession));
- }
- return null;
- }
-
- // Return Optional instead of faking it with nonExistingActiveSession
- private long getActiveSessionId(ApplicationId applicationId) {
- List<ApplicationId> applicationIds = applicationRepo.activeApplications();
- if (applicationIds.contains(applicationId)) {
- return applicationRepo.requireActiveSessionOf(applicationId);
- }
- return nonExistingActiveSession;
- }
-
- private long getNextSessionId() {
- return new SessionCounter(componentRegistry.getConfigCurator(), tenantName).nextSessionId();
- }
-
- private Path getSessionPath(long sessionId) {
- return sessionsPath.append(String.valueOf(sessionId));
- }
-
- private SessionZooKeeperClient createSessionZooKeeperClient(Path sessionPath) {
- String serverId = componentRegistry.getConfigserverConfig().serverId();
- Optional<NodeFlavors> nodeFlavors = componentRegistry.getZone().nodeFlavors();
- return new SessionZooKeeperClient(curator, componentRegistry.getConfigCurator(), sessionPath, serverId, nodeFlavors);
- }
-
- private File getAndValidateExistingSessionAppDir(long sessionId) {
- File appDir = getSessionAppDir(sessionId);
- if (!appDir.exists() || !appDir.isDirectory()) {
- throw new IllegalArgumentException("Unable to find correct application directory for session " + sessionId);
- }
- return appDir;
- }
-
- private File getSessionAppDir(long sessionId) {
- return new TenantFileSystemDirs(componentRegistry.getConfigServerDB(), tenantName).getUserApplicationDir(sessionId);
- }
-
@Override
public String toString() {
return getSessions().toString();
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/tenant/Tenant.java b/configserver/src/main/java/com/yahoo/vespa/config/server/tenant/Tenant.java
index 06cfe704046..097c3146b81 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/tenant/Tenant.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/tenant/Tenant.java
@@ -7,6 +7,7 @@ import com.yahoo.vespa.config.server.ReloadHandler;
import com.yahoo.vespa.config.server.RequestHandler;
import com.yahoo.vespa.config.server.application.TenantApplications;
import com.yahoo.vespa.config.server.session.SessionRepository;
+import com.yahoo.vespa.config.server.session.SessionFactory;
import com.yahoo.vespa.curator.Curator;
import org.apache.zookeeper.data.Stat;
@@ -27,6 +28,7 @@ public class Tenant implements TenantHandlerProvider {
private final TenantName name;
private final Path path;
+ private final SessionFactory sessionFactory;
private final SessionRepository sessionRepository;
private final TenantApplications applicationRepo;
private final RequestHandler requestHandler;
@@ -34,6 +36,7 @@ public class Tenant implements TenantHandlerProvider {
private final Curator curator;
Tenant(TenantName name,
+ SessionFactory sessionFactory,
SessionRepository sessionRepository,
RequestHandler requestHandler,
ReloadHandler reloadHandler,
@@ -43,6 +46,7 @@ public class Tenant implements TenantHandlerProvider {
this.path = TenantRepository.getTenantPath(name);
this.requestHandler = requestHandler;
this.reloadHandler = reloadHandler;
+ this.sessionFactory = sessionFactory;
this.sessionRepository = sessionRepository;
this.applicationRepo = applicationRepo;
this.curator = curator;
@@ -78,7 +82,13 @@ public class Tenant implements TenantHandlerProvider {
return path;
}
- public SessionRepository getSessionRepository() { return sessionRepository; }
+ public SessionFactory getSessionFactory() {
+ return sessionFactory;
+ }
+
+ public SessionRepository getSessionRepository() {
+ return sessionRepository;
+ }
@Override
public String toString() {
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/tenant/TenantRepository.java b/configserver/src/main/java/com/yahoo/vespa/config/server/tenant/TenantRepository.java
index 35e6a58af9c..caf9982e034 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/tenant/TenantRepository.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/tenant/TenantRepository.java
@@ -15,6 +15,7 @@ import com.yahoo.vespa.config.server.application.TenantApplications;
import com.yahoo.vespa.config.server.deploy.TenantFileSystemDirs;
import com.yahoo.vespa.config.server.monitoring.MetricUpdater;
import com.yahoo.vespa.config.server.session.SessionRepository;
+import com.yahoo.vespa.config.server.session.SessionFactory;
import com.yahoo.vespa.curator.Curator;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
@@ -221,13 +222,12 @@ public class TenantRepository {
requestHandler = applicationRepo;
if (reloadHandler == null)
reloadHandler = applicationRepo;
- SessionRepository sessionRepository = new SessionRepository(tenantName, componentRegistry,
+ SessionFactory sessionFactory = new SessionFactory(componentRegistry, applicationRepo, applicationRepo, tenantName);
+ SessionRepository sessionRepository = new SessionRepository(tenantName, componentRegistry, sessionFactory,
applicationRepo, reloadHandler,
- componentRegistry.getFlagSource(),
- componentRegistry.getHostRegistries().createApplicationHostRegistry(tenantName),
- componentRegistry.getSessionPreparer());
+ componentRegistry.getFlagSource());
log.log(Level.INFO, "Creating tenant '" + tenantName + "'");
- Tenant tenant = new Tenant(tenantName, sessionRepository, requestHandler,
+ Tenant tenant = new Tenant(tenantName, sessionFactory, sessionRepository, requestHandler,
reloadHandler, applicationRepo, componentRegistry.getCurator());
notifyNewTenant(tenant);
tenants.putIfAbsent(tenantName, tenant);
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/session/SessionRepositoryTest.java b/configserver/src/test/java/com/yahoo/vespa/config/server/session/SessionRepositoryTest.java
index f73fb053649..3c1404b5048 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/session/SessionRepositoryTest.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/session/SessionRepositoryTest.java
@@ -9,6 +9,7 @@ import com.yahoo.text.Utf8;
import com.yahoo.vespa.config.server.GlobalComponentRegistry;
import com.yahoo.vespa.config.server.TestComponentRegistry;
import com.yahoo.vespa.config.server.application.TenantApplications;
+import com.yahoo.vespa.config.server.host.HostRegistry;
import com.yahoo.vespa.config.server.http.SessionHandlerTest;
import com.yahoo.vespa.config.server.tenant.Tenant;
import com.yahoo.vespa.config.server.tenant.TenantRepository;
@@ -74,9 +75,12 @@ public class SessionRepositoryTest {
.build();
tenantRepository = new TenantRepository(globalComponentRegistry, false);
TenantApplications applicationRepo = TenantApplications.create(globalComponentRegistry, tenantName);
- sessionRepository = new SessionRepository(tenantName, globalComponentRegistry,
- applicationRepo, applicationRepo, new InMemoryFlagSource(),
- applicationRepo, globalComponentRegistry.getSessionPreparer());
+ SessionFactory sessionFactory = new SessionFactory(globalComponentRegistry,
+ applicationRepo,
+ new HostRegistry<>(),
+ tenantName);
+ sessionRepository = new SessionRepository(tenantName, globalComponentRegistry, sessionFactory,
+ applicationRepo, applicationRepo, new InMemoryFlagSource());
}
@Test