summaryrefslogtreecommitdiffstats
path: root/configserver
diff options
context:
space:
mode:
authorHarald Musum <musum@oath.com>2018-06-28 15:11:28 +0200
committerHarald Musum <musum@oath.com>2018-06-28 15:11:28 +0200
commit4df5067eb19e230de04f53a6484e85c61ff356c3 (patch)
tree18e2b29cf28228601cfc3f7090ef5bf1f7110554 /configserver
parentd9be160582d3488c944907f6958e09b9ab1555b8 (diff)
Support deleting an application on any config server
Support application delete on any config server, not just the one the application was deployed to. Legacy method that does it the old way kept and used in most places until we have got some expericence with the new one.
Diffstat (limited to 'configserver')
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/ApplicationRepository.java85
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandler.java4
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/session/LocalSession.java10
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/session/LocalSessionRepo.java43
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/session/LocalSessionStateWatcher.java76
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/session/RemoteSession.java17
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/session/RemoteSessionRepo.java23
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/session/RemoteSessionStateWatcher.java (renamed from configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionStateWatcher.java)22
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/session/Session.java2
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionZooKeeperClient.java14
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/session/SilentDeployLogger.java2
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/tenant/TenantBuilder.java4
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/ApplicationRepositoryTest.java64
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/http/SessionHandlerTest.java3
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandlerTest.java23
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/SessionActiveHandlerTest.java2
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/SessionCreateHandlerTest.java3
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/SessionPrepareHandlerTest.java7
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/session/LocalSessionRepoTest.java2
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/session/LocalSessionTest.java6
20 files changed, 310 insertions, 102 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 581c6e0394e..599ff1eb067 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
@@ -41,6 +41,7 @@ import com.yahoo.vespa.config.server.session.LocalSession;
import com.yahoo.vespa.config.server.session.LocalSessionRepo;
import com.yahoo.vespa.config.server.session.PrepareParams;
import com.yahoo.vespa.config.server.session.RemoteSession;
+import com.yahoo.vespa.config.server.session.RemoteSessionRepo;
import com.yahoo.vespa.config.server.session.Session;
import com.yahoo.vespa.config.server.session.SessionFactory;
import com.yahoo.vespa.config.server.session.SilentDeployLogger;
@@ -252,17 +253,77 @@ public class ApplicationRepository implements com.yahoo.config.provision.Deploye
// ---------------- Application operations ----------------------------------------------------------------
/**
- * Removes a previously deployed application
+ * Deletes an application
*
- * @return true if the application was found and removed, false if it was not present
- * @throws RuntimeException if the remove transaction fails. This method is exception safe.
+ * @return true if the application was found and deleted, false if it was not present
+ * @throws RuntimeException if the delete transaction fails. This method is exception safe.
*/
- public boolean remove(ApplicationId applicationId) {
+ public boolean delete(ApplicationId applicationId) {
+ // TODO: Use deleteApplication() in all zones, for now use it only in non-hosted
+ if (configserverConfig.hostedVespa()) {
+ return deleteApplicationLegacy(applicationId);
+ } else {
+ return deleteApplication(applicationId);
+ }
+ }
+
+ /**
+ * Deletes an application
+ *
+ * @return true if the application was found and deleted, false if it was not present
+ * @throws RuntimeException if the delete transaction fails. This method is exception safe.
+ */
+ public boolean deleteApplication(ApplicationId applicationId) {
+ Tenant tenant = tenantRepository.getTenant(applicationId.tenant());
+ if (tenant == null) return false;
+
+ TenantApplications tenantApplications = tenant.getApplicationRepo();
+ if (!tenantApplications.listApplications().contains(applicationId)) return false;
+
+ // Deleting an application is done by deleting the remote session and waiting
+ // until the config server where the deployment happened picks it up and deletes
+ // the local session
+ long sessionId = tenantApplications.getSessionIdForApplication(applicationId);
+ RemoteSession remoteSession = getRemoteSession(tenant, sessionId);
+ remoteSession.createDeleteTransaction().commit();
+
+ log.log(LogLevel.INFO, TenantRepository.logPre(applicationId) + "Waiting for session " + sessionId + " to be deleted");
+ // TODO: Add support for timeout in request
+ Duration waitTime = Duration.ofSeconds(60);
+ if (localSessionHasBeenDeleted(applicationId, sessionId, waitTime)) {
+ log.log(LogLevel.INFO, TenantRepository.logPre(applicationId) + "Session " + sessionId + " deleted");
+ } else {
+ log.log(LogLevel.ERROR, TenantRepository.logPre(applicationId) + "Session " + sessionId + " was not deleted (waited " + waitTime + ")");
+ return false;
+ }
+
+ NestedTransaction transaction = new NestedTransaction();
+ transaction.add(new Rotations(tenant.getCurator(), tenant.getPath()).delete(applicationId)); // TODO: Not unit tested
+ // (When rotations are updated in zk, we need to redeploy the zone app, on the right config server
+ // this is done asynchronously in application maintenance by the node repository)
+ transaction.add(tenantApplications.deleteApplication(applicationId));
+
+ hostProvisioner.ifPresent(provisioner -> provisioner.remove(transaction, applicationId));
+ transaction.onCommitted(() -> log.log(LogLevel.INFO, "Deleted " + applicationId));
+ transaction.commit();
+
+ return true;
+ }
+
+ /**
+ * Deletes an application the legacy way (if there is more than one config server, the call needs to be done
+ * on the config server the application was deployed)
+ *
+ * @return true if the application was found and deleted, false if it was not present
+ * @throws RuntimeException if the delete transaction fails. This method is exception safe.
+ */
+ // TODO: Remove this method, use delete(ApplicationId) instead
+ boolean deleteApplicationLegacy(ApplicationId applicationId) {
Optional<Tenant> owner = Optional.ofNullable(tenantRepository.getTenant(applicationId.tenant()));
- if ( ! owner.isPresent()) return false;
+ if (!owner.isPresent()) return false;
TenantApplications tenantApplications = owner.get().getApplicationRepo();
- if ( ! tenantApplications.listApplications().contains(applicationId)) return false;
+ if (!tenantApplications.listApplications().contains(applicationId)) return false;
// TODO: Push lookup logic down
long sessionId = tenantApplications.getSessionIdForApplication(applicationId);
@@ -277,7 +338,6 @@ public class ApplicationRepository implements com.yahoo.config.provision.Deploye
transaction.add(new Rotations(owner.get().getCurator(), owner.get().getPath()).delete(applicationId)); // TODO: Not unit tested
// (When rotations are updated in zk, we need to redeploy the zone app, on the right config server
// this is done asynchronously in application maintenance by the node repository)
-
transaction.add(tenantApplications.deleteApplication(applicationId));
hostProvisioner.ifPresent(provisioner -> provisioner.remove(transaction, applicationId));
@@ -378,6 +438,17 @@ public class ApplicationRepository implements com.yahoo.config.provision.Deploye
}
}
+ private boolean localSessionHasBeenDeleted(ApplicationId applicationId, long sessionId, Duration waitTime) {
+ RemoteSessionRepo remoteSessionRepo = tenantRepository.getTenant(applicationId.tenant()).getRemoteSessionRepo();
+ Instant end = Instant.now().plus(waitTime);
+ do {
+ if (remoteSessionRepo.getSession(sessionId) == null) return true;
+ try { Thread.sleep(10); } catch (InterruptedException e) { /* ignored */}
+ } while (Instant.now().isBefore(end));
+
+ return false;
+ }
+
// ---------------- Convergence ----------------------------------------------------------------
public HttpResponse checkServiceForConfigConvergence(ApplicationId applicationId, String hostAndPort, URI uri, Duration timeout) {
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandler.java b/configserver/src/main/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandler.java
index a734c3a70db..2004ab95144 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandler.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandler.java
@@ -46,8 +46,8 @@ public class ApplicationHandler extends HttpHandler {
@Override
public HttpResponse handleDELETE(HttpRequest request) {
ApplicationId applicationId = getApplicationIdFromRequest(request);
- boolean removed = applicationRepository.remove(applicationId);
- if ( ! removed)
+ boolean deleted = applicationRepository.delete(applicationId);
+ if ( ! deleted)
return HttpErrorResponse.notFoundError("Unable to delete " + applicationId + ": Not found");
return new DeleteApplicationResponse(Response.Status.OK, applicationId);
}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/session/LocalSession.java b/configserver/src/main/java/com/yahoo/vespa/config/server/session/LocalSession.java
index d294e462c44..70ea20d5b4d 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/session/LocalSession.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/session/LocalSession.java
@@ -120,14 +120,6 @@ public class LocalSession extends Session implements Comparable<LocalSession> {
return getSessionId() > sessionId;
}
- /** Delete this session */
- // TODO: Use transactional delete instead
- public void delete() {
- superModelGenerationCounter.increment();
- IOUtils.recursiveDeleteDir(serverDB);
- zooKeeperClient.delete();
- }
-
/** Add transactions to delete this session to the given nested transaction */
public void delete(NestedTransaction transaction) {
transaction.add(zooKeeperClient.deleteTransaction(), FileTransaction.class);
@@ -175,6 +167,8 @@ public class LocalSession extends Session implements Comparable<LocalSession> {
return zooKeeperClient.getAllocatedHosts();
}
+ public TenantName getTenantName() { return tenant; }
+
@Override
public String logPre() {
if (getApplicationId().equals(ApplicationId.defaultId())) {
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/session/LocalSessionRepo.java b/configserver/src/main/java/com/yahoo/vespa/config/server/session/LocalSessionRepo.java
index 184ba686f63..dbb581eb16d 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/session/LocalSessionRepo.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/session/LocalSessionRepo.java
@@ -3,14 +3,21 @@ package com.yahoo.vespa.config.server.session;
import com.yahoo.concurrent.ThreadFactoryFactory;
import com.yahoo.log.LogLevel;
+import com.yahoo.path.Path;
+import com.yahoo.transaction.NestedTransaction;
import com.yahoo.vespa.config.server.deploy.TenantFileSystemDirs;
+import com.yahoo.vespa.config.server.tenant.TenantRepository;
+import com.yahoo.vespa.config.server.zookeeper.ConfigCurator;
+import com.yahoo.vespa.curator.Curator;
import java.io.File;
import java.io.FilenameFilter;
import java.time.Clock;
import java.time.Duration;
import java.util.ArrayList;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
@@ -30,25 +37,37 @@ public class LocalSessionRepo extends SessionRepo<LocalSession> {
// One executor for all instances of this class
private static final ScheduledExecutorService purgeOldSessionsExecutor =
new ScheduledThreadPoolExecutor(1, ThreadFactoryFactory.getDaemonThreadFactory("purge-old-sessions"));
+ private final Map<Long, LocalSessionStateWatcher> sessionStateWatchers = new HashMap<>();
private final long sessionLifetime; // in seconds
private final Clock clock;
+ private final Curator curator;
public LocalSessionRepo(TenantFileSystemDirs tenantFileSystemDirs, LocalSessionLoader loader,
- Clock clock, long sessionLifeTime) {
- this(clock, sessionLifeTime);
+ Clock clock, long sessionLifeTime, Curator curator) {
+ this(clock, curator, sessionLifeTime);
loadSessions(tenantFileSystemDirs.sessionsPath(), loader);
purgeOldSessionsExecutor.scheduleWithFixedDelay(this::purgeOldSessions, delay.getSeconds(), delay.getSeconds(), TimeUnit.SECONDS);
}
// Constructor public only for testing
- public LocalSessionRepo(Clock clock) {
- this(clock, TimeUnit.DAYS.toMillis(1));
+ public LocalSessionRepo(Clock clock, Curator curator) {
+ this(clock, curator, TimeUnit.DAYS.toMillis(1));
}
// Constructor public only for testing
- private LocalSessionRepo(Clock clock, long sessionLifetime) {
- this.sessionLifetime = sessionLifetime;
+ private LocalSessionRepo(Clock clock, Curator curator, long sessionLifetime) {
this.clock = clock;
+ this.curator = curator;
+ this.sessionLifetime = sessionLifetime;
+ }
+
+ @Override
+ public synchronized void addSession(LocalSession session) {
+ super.addSession(session);
+ Path sessionsPath = TenantRepository.getSessionsPath(session.getTenantName());
+ long sessionId = session.getSessionId();
+ Curator.FileCache fileCache = curator.createFileCache(sessionsPath.append(String.valueOf(sessionId)).append(ConfigCurator.SESSIONSTATE_ZK_SUBPATH).getAbsolute(), false);
+ sessionStateWatchers.put(sessionId, new LocalSessionStateWatcher(fileCache, session, this));
}
private void loadSessions(File applicationsDir, LocalSessionLoader loader) {
@@ -91,9 +110,15 @@ public class LocalSessionRepo extends SessionRepo<LocalSession> {
return candidate.getStatus() == Session.Status.ACTIVATE;
}
- private void deleteSession(LocalSession candidate) {
- removeSession(candidate.getSessionId());
- candidate.delete();
+ void deleteSession(LocalSession session) {
+ long sessionId = session.getSessionId();
+ log.log(LogLevel.DEBUG, "Deleting local session " + sessionId);
+ LocalSessionStateWatcher watcher = sessionStateWatchers.remove(sessionId);
+ if (watcher != null) watcher.close();
+ removeSession(sessionId);
+ NestedTransaction transaction = new NestedTransaction();
+ session.delete(transaction);
+ transaction.commit();
}
public void deleteAllSessions() {
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/session/LocalSessionStateWatcher.java b/configserver/src/main/java/com/yahoo/vespa/config/server/session/LocalSessionStateWatcher.java
new file mode 100644
index 00000000000..198f8e8e917
--- /dev/null
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/session/LocalSessionStateWatcher.java
@@ -0,0 +1,76 @@
+// Copyright 2018 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.concurrent.ThreadFactoryFactory;
+import com.yahoo.log.LogLevel;
+import com.yahoo.text.Utf8;
+import com.yahoo.vespa.curator.Curator;
+import org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.curator.framework.recipes.cache.NodeCacheListener;
+
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.logging.Logger;
+
+/**
+ * Watches one particular local session (/config/v2/tenants/&lt;tenantName&gt;/sessions/&lt;n&gt;/sessionState in ZooKeeper)
+ * to pick up when an application is deleted (the delete might be done on any config server in the cluster)
+ *
+ * @author Harald Musum
+ */
+public class LocalSessionStateWatcher implements NodeCacheListener {
+
+ private static final Logger log = Logger.getLogger(LocalSessionStateWatcher.class.getName());
+ // One thread pool for all instances of this class
+ private static final Executor executor = Executors.newCachedThreadPool(ThreadFactoryFactory.getDaemonThreadFactory(LocalSessionStateWatcher.class.getName()));
+
+ private final Curator.FileCache fileCache;
+ private final LocalSession session;
+ private final LocalSessionRepo localSessionRepo;
+
+ LocalSessionStateWatcher(Curator.FileCache fileCache, LocalSession session, LocalSessionRepo localSessionRepo) {
+ this.fileCache = fileCache;
+ this.session = session;
+ this.localSessionRepo = localSessionRepo;
+ this.fileCache.start();
+ this.fileCache.addListener(this);
+ }
+
+ // Will delete session if it exists in local session repo
+ private void sessionChanged(Session.Status status) {
+ long sessionId = session.getSessionId();
+ log.log(LogLevel.DEBUG, session.logPre() + "Session change: Local session " + sessionId + " changed status to " + status);
+
+ if (status.equals(Session.Status.DELETE) && localSessionRepo.getSession(sessionId) != null) {
+ log.log(LogLevel.DEBUG, session.logPre() + "Deleting session " + sessionId);
+ localSessionRepo.deleteSession(session);
+ }
+ }
+
+ public long getSessionId() {
+ return session.getSessionId();
+ }
+
+ public void close() {
+ try {
+ fileCache.close();
+ } catch (Exception e) {
+ log.log(LogLevel.WARNING, "Exception when closing watcher", e);
+ }
+ }
+
+ @Override
+ public void nodeChanged() {
+ executor.execute(() -> {
+ try {
+ ChildData data = fileCache.getCurrentData();
+ if (data != null) {
+ sessionChanged(Session.Status.parse(Utf8.toString(fileCache.getCurrentData().getData())));
+ }
+ } catch (Exception e) {
+ log.log(LogLevel.WARNING, session.logPre() + "Error handling session changed for session " + getSessionId(), e);
+ }
+ });
+ }
+
+}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/session/RemoteSession.java b/configserver/src/main/java/com/yahoo/vespa/config/server/session/RemoteSession.java
index bc7c428833b..c47d5791a1b 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/session/RemoteSession.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/session/RemoteSession.java
@@ -2,10 +2,13 @@
package com.yahoo.vespa.config.server.session;
import com.yahoo.config.application.api.ApplicationPackage;
-import com.yahoo.config.provision.*;
+import com.yahoo.config.provision.AllocatedHosts;
+import com.yahoo.config.provision.TenantName;
import com.yahoo.lang.SettableOptional;
-import com.yahoo.vespa.config.server.*;
+import com.yahoo.transaction.Transaction;
import com.yahoo.log.LogLevel;
+import com.yahoo.vespa.config.server.GlobalComponentRegistry;
+import com.yahoo.vespa.config.server.ReloadHandler;
import com.yahoo.vespa.config.server.application.ApplicationSet;
import com.yahoo.vespa.config.server.modelfactory.ActivatedModelsBuilder;
import com.yahoo.vespa.config.server.tenant.TenantRepository;
@@ -14,7 +17,7 @@ import org.apache.zookeeper.KeeperException;
import java.time.Clock;
import java.time.Instant;
-import java.util.*;
+import java.util.Optional;
import java.util.logging.Logger;
/**
@@ -82,9 +85,13 @@ public class RemoteSession extends Session {
applicationSet = null;
}
+ public Transaction createDeleteTransaction() {
+ return zooKeeperClient.createWriteStatusTransaction(Status.DELETE);
+ }
+
public void makeActive(ReloadHandler reloadHandler) {
Curator.CompletionWaiter waiter = zooKeeperClient.getActiveWaiter();
- log.log(LogLevel.DEBUG, logPre()+"Getting session from repo: " + getSessionId());
+ log.log(LogLevel.DEBUG, logPre() + "Getting session from repo: " + getSessionId());
ApplicationSet app = ensureApplicationLoaded();
log.log(LogLevel.DEBUG, logPre() + "Reloading config for " + app);
reloadHandler.reloadConfig(app);
@@ -106,7 +113,7 @@ public class RemoteSession extends Session {
Curator.CompletionWaiter waiter = zooKeeperClient.getUploadWaiter();
log.log(LogLevel.DEBUG, "Notifying upload waiter for session " + getSessionId());
notifyCompletion(waiter);
- log.log(LogLevel.DEBUG, "Done notifying for session " + getSessionId());
+ log.log(LogLevel.DEBUG, "Done notifying upload for session " + getSessionId());
}
private void notifyCompletion(Curator.CompletionWaiter completionWaiter) {
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/session/RemoteSessionRepo.java b/configserver/src/main/java/com/yahoo/vespa/config/server/session/RemoteSessionRepo.java
index 40b9ca82850..d16d5a17518 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/session/RemoteSessionRepo.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/session/RemoteSessionRepo.java
@@ -14,7 +14,6 @@ import com.yahoo.concurrent.ThreadFactoryFactory;
import com.yahoo.config.provision.TenantName;
import com.yahoo.log.LogLevel;
import com.yahoo.path.Path;
-import com.yahoo.transaction.NestedTransaction;
import com.yahoo.vespa.config.server.application.ApplicationSet;
import com.yahoo.vespa.config.server.tenant.TenantRepository;
import com.yahoo.vespa.curator.Curator;
@@ -47,7 +46,7 @@ public class RemoteSessionRepo extends SessionRepo<RemoteSession> implements Nod
private final Curator curator;
private final Path sessionsPath;
private final RemoteSessionFactory remoteSessionFactory;
- private final Map<Long, SessionStateWatcher> sessionStateWatchers = new HashMap<>();
+ private final Map<Long, RemoteSessionStateWatcher> sessionStateWatchers = new HashMap<>();
private final ReloadHandler reloadHandler;
private final MetricUpdater metrics;
private final Curator.DirectoryCache directoryCache;
@@ -89,16 +88,6 @@ public class RemoteSessionRepo extends SessionRepo<RemoteSession> implements Nod
this.applicationRepo = null;
}
- //---------- START overrides to keep sessions changed in sync
-
- @Override
- public void removeSession(long id, NestedTransaction transaction) {
- super.removeSession(id, transaction);
- transaction.onCommitted(() -> sessionRemoved(id));
- }
-
- //---------- END overrides to keep sessions changed in sync
-
private void loadActiveSession(RemoteSession session) {
tryReload(session.ensureApplicationLoaded(), session.logPre());
}
@@ -106,9 +95,9 @@ public class RemoteSessionRepo extends SessionRepo<RemoteSession> implements Nod
private void tryReload(ApplicationSet applicationSet, String logPre) {
try {
reloadHandler.reloadConfig(applicationSet);
- log.log(LogLevel.INFO, logPre+"Application activated successfully: " + applicationSet.getId());
+ log.log(LogLevel.INFO, logPre + "Application activated successfully: " + applicationSet.getId());
} catch (Exception e) {
- log.log(LogLevel.WARNING, logPre+"Skipping loading of application '" + applicationSet.getId() + "': " + Exceptions.toMessageString(e));
+ log.log(LogLevel.WARNING, logPre + "Skipping loading of application '" + applicationSet.getId() + "': " + Exceptions.toMessageString(e));
}
}
@@ -158,7 +147,7 @@ public class RemoteSessionRepo extends SessionRepo<RemoteSession> implements Nod
Curator.FileCache fileCache = curator.createFileCache(sessionPath.append(ConfigCurator.SESSIONSTATE_ZK_SUBPATH).getAbsolute(), false);
fileCache.addListener(this);
loadSessionIfActive(session);
- sessionStateWatchers.put(sessionId, new SessionStateWatcher(fileCache, reloadHandler, session, metrics));
+ sessionStateWatchers.put(sessionId, new RemoteSessionStateWatcher(fileCache, reloadHandler, session, metrics));
addSession(session);
metrics.incAddedSessions();
} catch (Exception e) {
@@ -167,8 +156,8 @@ public class RemoteSessionRepo extends SessionRepo<RemoteSession> implements Nod
}
private void sessionRemoved(long sessionId) {
- SessionStateWatcher watcher = sessionStateWatchers.remove(sessionId);
- watcher.close();
+ RemoteSessionStateWatcher watcher = sessionStateWatchers.remove(sessionId);
+ if (watcher != null) watcher.close();
removeSession(sessionId);
metrics.incRemovedSessions();
}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionStateWatcher.java b/configserver/src/main/java/com/yahoo/vespa/config/server/session/RemoteSessionStateWatcher.java
index 117930c6d7d..1a891c65c49 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionStateWatcher.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/session/RemoteSessionStateWatcher.java
@@ -15,16 +15,16 @@ import java.util.concurrent.Executors;
import java.util.logging.Logger;
/**
- * Watches one particular session (/vespa/config/apps/n/sessionState in ZK)
+ * Watches one particular session (/config/v2/tenants/&lt;tenantName&gt;/sessions/&lt;n&gt;/sessionState in ZooKeeper)
* The session must be in the session repo.
*
- * @author vegardh
+ * @author Vegard Havdal
*/
-public class SessionStateWatcher implements NodeCacheListener {
+public class RemoteSessionStateWatcher implements NodeCacheListener {
- private static final Logger log = Logger.getLogger(SessionStateWatcher.class.getName());
+ private static final Logger log = Logger.getLogger(RemoteSessionStateWatcher.class.getName());
// One thread pool for all instances of this class
- private static final Executor executor = Executors.newCachedThreadPool(ThreadFactoryFactory.getDaemonThreadFactory(SessionStateWatcher.class.getName()));
+ private static final Executor executor = Executors.newCachedThreadPool(ThreadFactoryFactory.getDaemonThreadFactory(RemoteSessionStateWatcher.class.getName()));
private final Curator.FileCache fileCache;
private final ReloadHandler reloadHandler;
@@ -32,10 +32,10 @@ public class SessionStateWatcher implements NodeCacheListener {
private final MetricUpdater metrics;
- public SessionStateWatcher(Curator.FileCache fileCache,
- ReloadHandler reloadHandler,
- RemoteSession session,
- MetricUpdater metrics) {
+ RemoteSessionStateWatcher(Curator.FileCache fileCache,
+ ReloadHandler reloadHandler,
+ RemoteSession session,
+ MetricUpdater metrics) {
this.fileCache = fileCache;
this.reloadHandler = reloadHandler;
this.session = session;
@@ -45,7 +45,7 @@ public class SessionStateWatcher implements NodeCacheListener {
}
private void sessionChanged(Session.Status status) {
- log.log(LogLevel.DEBUG, session.logPre()+"Session change: Session " + session.getSessionId() + " changed status to " + status);
+ log.log(LogLevel.DEBUG, session.logPre() + "Session change: Remote session " + session.getSessionId() + " changed status to " + status);
// valid for NEW -> PREPARE transitions, not ACTIVATE -> PREPARE.
if (status.equals(Session.Status.PREPARE)) {
@@ -55,6 +55,8 @@ public class SessionStateWatcher implements NodeCacheListener {
session.makeActive(reloadHandler);
} else if (status.equals(Session.Status.DEACTIVATE)) {
session.deactivate();
+ } else if (status.equals(Session.Status.DELETE)) {
+ session.deactivate();
}
}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/session/Session.java b/configserver/src/main/java/com/yahoo/vespa/config/server/session/Session.java
index 3978a1f25f8..e98931b0573 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/session/Session.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/session/Session.java
@@ -43,7 +43,7 @@ public abstract class Session {
* Represents the status of this session.
*/
public enum Status {
- NEW, PREPARE, ACTIVATE, DEACTIVATE, NONE;
+ NEW, PREPARE, ACTIVATE, DEACTIVATE, DELETE, NONE;
public static Status parse(String data) {
for (Status status : Status.values()) {
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionZooKeeperClient.java b/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionZooKeeperClient.java
index de903b7f2dd..2049a7f9eb0 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionZooKeeperClient.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionZooKeeperClient.java
@@ -7,6 +7,7 @@ import com.yahoo.config.application.api.ApplicationPackage;
import com.yahoo.config.application.api.DeployLogger;
import com.yahoo.config.provision.NodeFlavors;
import com.yahoo.config.provision.AllocatedHosts;
+import com.yahoo.transaction.NestedTransaction;
import com.yahoo.transaction.Transaction;
import com.yahoo.log.LogLevel;
import com.yahoo.path.Path;
@@ -103,9 +104,7 @@ public class SessionZooKeeperClient {
return getCompletionWaiter(getWaiterPath(ACTIVE_BARRIER));
}
- Curator.CompletionWaiter getUploadWaiter() {
- return getCompletionWaiter(getWaiterPath(UPLOAD_BARRIER));
- }
+ Curator.CompletionWaiter getUploadWaiter() { return getCompletionWaiter(getWaiterPath(UPLOAD_BARRIER)); }
private static final String PREPARE_BARRIER = "prepareBarrier";
private static final String ACTIVE_BARRIER = "activeBarrier";
@@ -128,12 +127,15 @@ public class SessionZooKeeperClient {
return curator.getCompletionWaiter(path, getNumberOfMembers(), serverId);
}
- public void delete() {
+ public void delete(NestedTransaction transaction ) {
try {
log.log(LogLevel.DEBUG, "Deleting " + sessionPath.getAbsolute());
- configCurator.deleteRecurse(sessionPath.getAbsolute());
+ CuratorTransaction curatorTransaction = new CuratorTransaction(curator);
+ CuratorOperations.deleteAll(sessionPath.getAbsolute(), curator).forEach(curatorTransaction::add);
+ transaction.add(curatorTransaction);
+ transaction.commit();
} catch (RuntimeException e) {
- log.log(LogLevel.INFO, "Error deleting session (" + sessionPath.getAbsolute() + ") from zookeeper");
+ log.log(LogLevel.INFO, "Error deleting session (" + sessionPath.getAbsolute() + ") from zookeeper", e);
}
}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/session/SilentDeployLogger.java b/configserver/src/main/java/com/yahoo/vespa/config/server/session/SilentDeployLogger.java
index 445837104d4..92ab6b3fbf5 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/session/SilentDeployLogger.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/session/SilentDeployLogger.java
@@ -7,7 +7,7 @@ import java.util.logging.Logger;
import com.yahoo.config.application.api.DeployLogger;
/**
- * The purpose of this is to mute the log messages from model and application building in {@link RemoteSession} that is triggered by {@link SessionStateWatcher}, since those messages already
+ * The purpose of this is to mute the log messages from model and application building in {@link RemoteSession} that is triggered by {@link RemoteSessionStateWatcher}, since those messages already
* have been emitted by the prepare handler, for the same prepare operation.
*
* @author vegardh
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/tenant/TenantBuilder.java b/configserver/src/main/java/com/yahoo/vespa/config/server/tenant/TenantBuilder.java
index 8c66b19653c..de44a0328f3 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/tenant/TenantBuilder.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/tenant/TenantBuilder.java
@@ -101,7 +101,9 @@ public class TenantBuilder {
private void createLocalSessionRepo() {
if (localSessionRepo == null) {
- localSessionRepo = new LocalSessionRepo(tenantFileSystemDirs, localSessionLoader, componentRegistry.getClock(), componentRegistry.getConfigserverConfig().sessionLifetime());
+ localSessionRepo = new LocalSessionRepo(tenantFileSystemDirs, localSessionLoader, componentRegistry.getClock(),
+ componentRegistry.getConfigserverConfig().sessionLifetime(),
+ componentRegistry.getCurator());
}
}
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/ApplicationRepositoryTest.java b/configserver/src/test/java/com/yahoo/vespa/config/server/ApplicationRepositoryTest.java
index 12a4ed7a4c0..e2d59fb2108 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/ApplicationRepositoryTest.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/ApplicationRepositoryTest.java
@@ -15,7 +15,9 @@ import com.yahoo.test.ManualClock;
import com.yahoo.text.Utf8;
import com.yahoo.vespa.config.server.http.SessionHandlerTest;
import com.yahoo.vespa.config.server.http.v2.PrepareResult;
+import com.yahoo.vespa.config.server.session.LocalSession;
import com.yahoo.vespa.config.server.session.PrepareParams;
+import com.yahoo.vespa.config.server.tenant.Tenant;
import com.yahoo.vespa.config.server.tenant.TenantRepository;
import com.yahoo.vespa.curator.Curator;
import com.yahoo.vespa.curator.mock.MockCurator;
@@ -32,8 +34,12 @@ import java.time.Instant;
import java.util.Collections;
import java.util.Set;
+import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
/**
@@ -52,6 +58,7 @@ public class ApplicationRepositoryTest {
private ApplicationRepository applicationRepository;
private TenantRepository tenantRepository;
+ private SessionHandlerTest.MockProvisioner provisioner;
private TimeoutBudget timeoutBudget;
@Rule
@@ -66,7 +73,7 @@ public class ApplicationRepositoryTest {
tenantRepository.addTenant(tenant1);
tenantRepository.addTenant(tenant2);
tenantRepository.addTenant(tenant3);
- Provisioner provisioner = new SessionHandlerTest.MockProvisioner();
+ provisioner = new SessionHandlerTest.MockProvisioner();
applicationRepository = new ApplicationRepository(tenantRepository, provisioner, clock);
timeoutBudget = new TimeoutBudget(clock, Duration.ofSeconds(60));
}
@@ -110,7 +117,7 @@ public class ApplicationRepositoryTest {
assertEquals(tenant3, applicationRepository.deleteUnusedTenants(ttlForUnusedTenant, now).iterator().next());
// Delete app used by tenant1, tenant2 still has an application
- applicationRepository.remove(applicationId());
+ applicationRepository.delete(applicationId());
Set<TenantName> tenantsDeleted = applicationRepository.deleteUnusedTenants(Duration.ofMillis(1), now);
assertTrue(tenantsDeleted.contains(tenant1));
assertFalse(tenantsDeleted.contains(tenant2));
@@ -172,6 +179,59 @@ public class ApplicationRepositoryTest {
return filereferenceDir;
}
+ @Test
+ public void delete() {
+ {
+ PrepareResult result = deployApp(testApp);
+ long sessionId = result.sessionId();
+ Tenant tenant = tenantRepository.getTenant(applicationId().tenant());
+ LocalSession applicationData = tenant.getLocalSessionRepo().getSession(sessionId);
+ assertNotNull(applicationData);
+ assertNotNull(applicationData.getApplicationId());
+ assertNotNull(tenant.getRemoteSessionRepo().getSession(sessionId));
+ assertNotNull(applicationRepository.getActiveSession(applicationId()));
+
+ // Delete app and verify that it has been deleted from repos and provisioner
+ assertTrue(applicationRepository.deleteApplication(applicationId()));
+ assertNull(applicationRepository.getActiveSession(applicationId()));
+ assertNull(tenant.getLocalSessionRepo().getSession(sessionId));
+ assertNull(tenant.getRemoteSessionRepo().getSession(sessionId));
+ assertTrue(provisioner.removed);
+ assertThat(provisioner.lastApplicationId.tenant(), is(tenant.getName()));
+ assertThat(provisioner.lastApplicationId, is(applicationId()));
+
+ assertFalse(applicationRepository.deleteApplication(applicationId()));
+ }
+
+ {
+ deployApp(testApp);
+ assertTrue(applicationRepository.deleteApplication(applicationId()));
+ deployApp(testApp);
+
+ // Deploy another app (with id fooId)
+ ApplicationId fooId = applicationId(tenant2);
+ PrepareParams prepareParams2 = new PrepareParams.Builder().applicationId(fooId).build();
+ deployApp(testApp, prepareParams2);
+ assertNotNull(applicationRepository.getActiveSession(fooId));
+
+ // Delete app with id fooId, should not affect original app
+ assertTrue(applicationRepository.deleteApplication(fooId));
+ assertThat(provisioner.lastApplicationId, is(fooId));
+ assertNotNull(applicationRepository.getActiveSession(applicationId()));
+
+ assertTrue(applicationRepository.deleteApplication(applicationId()));
+ }
+ }
+
+ @Test
+ public void deleteLegacy() {
+ deployApp(testApp);
+ assertNotNull(applicationRepository.getActiveSession(applicationId()));
+ assertTrue(applicationRepository.deleteApplicationLegacy(applicationId()));
+ assertNull(applicationRepository.getActiveSession(applicationId()));
+ assertFalse(applicationRepository.deleteApplicationLegacy(applicationId()));
+ }
+
private PrepareResult prepareAndActivateApp(File application) throws IOException {
FilesApplicationPackage appDir = FilesApplicationPackage.fromFile(application);
ApplicationId applicationId = applicationId();
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/http/SessionHandlerTest.java b/configserver/src/test/java/com/yahoo/vespa/config/server/http/SessionHandlerTest.java
index eb5dc7a2abf..c5faebe4a28 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/http/SessionHandlerTest.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/http/SessionHandlerTest.java
@@ -160,9 +160,6 @@ public class SessionHandlerTest {
}
@Override
- public void delete() { }
-
- @Override
public void delete(NestedTransaction transaction) { }
}
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandlerTest.java b/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandlerTest.java
index 6680907e1c0..a5564d7ccf9 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandlerTest.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandlerTest.java
@@ -19,7 +19,6 @@ import com.yahoo.vespa.config.server.http.HttpErrorResponse;
import com.yahoo.vespa.config.server.http.StaticResponse;
import com.yahoo.vespa.config.server.http.SessionHandlerTest;
import com.yahoo.vespa.config.server.provision.HostProvisionerProvider;
-import com.yahoo.vespa.config.server.session.LocalSession;
import com.yahoo.vespa.config.server.session.PrepareParams;
import com.yahoo.vespa.config.server.tenant.Tenant;
import com.yahoo.vespa.config.server.tenant.TenantBuilder;
@@ -33,12 +32,9 @@ import java.io.IOException;
import java.net.URI;
import java.time.Clock;
-import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.eq;
@@ -78,25 +74,9 @@ public class ApplicationHandlerTest {
@Test
public void testDelete() throws Exception {
{
- // This block is a real test of the interplay of (most of) the components of the config server
- // TODO: Extract it to ApplicationRepositoryTest, rewrite to bypass the HTTP layer and extend
- // as login is moved from the HTTP layer into ApplicationRepository
-
- PrepareResult result = applicationRepository.deploy(testApp, prepareParams(applicationId));
- long sessionId = result.sessionId();
+ applicationRepository.deploy(testApp, prepareParams(applicationId));
Tenant mytenant = tenantRepository.getTenant(applicationId.tenant());
- LocalSession applicationData = mytenant.getLocalSessionRepo().getSession(sessionId);
- assertNotNull(applicationData);
- assertNotNull(applicationData.getApplicationId());
- assertFalse(provisioner.removed);
-
deleteAndAssertOKResponse(mytenant, applicationId);
- assertTrue(provisioner.removed);
- assertThat(provisioner.lastApplicationId.tenant(), is(mytenantName));
- assertThat(provisioner.lastApplicationId, is(applicationId));
-
- assertNull(mytenant.getLocalSessionRepo().getSession(sessionId));
- assertNull(mytenant.getRemoteSessionRepo().getSession(sessionId));
}
{
@@ -114,7 +94,6 @@ public class ApplicationHandlerTest {
assertApplicationExists(fooId, Zone.defaultZone());
deleteAndAssertOKResponseMocked(fooId, true);
- assertThat(provisioner.lastApplicationId, is(fooId));
assertApplicationExists(applicationId, Zone.defaultZone());
deleteAndAssertOKResponseMocked(applicationId, true);
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/SessionActiveHandlerTest.java b/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/SessionActiveHandlerTest.java
index fba4e40000d..75a2d2f778d 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/SessionActiveHandlerTest.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/SessionActiveHandlerTest.java
@@ -100,7 +100,7 @@ public class SessionActiveHandlerTest extends SessionHandlerTest {
applicationRepo = new MemoryTenantApplications();
curator = new MockCurator();
configCurator = ConfigCurator.create(curator);
- localRepo = new LocalSessionRepo(clock);
+ localRepo = new LocalSessionRepo(clock, curator);
pathPrefix = "/application/v2/tenant/" + tenantName + "/session/";
hostProvisioner = new MockProvisioner();
modelFactory = new VespaModelFactory(new NullConfigModelRegistry());
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/SessionCreateHandlerTest.java b/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/SessionCreateHandlerTest.java
index 7c0e410d244..640f5dafbed 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/SessionCreateHandlerTest.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/SessionCreateHandlerTest.java
@@ -17,6 +17,7 @@ import com.yahoo.vespa.config.server.http.SessionHandlerTest;
import com.yahoo.vespa.config.server.session.LocalSessionRepo;
import com.yahoo.vespa.config.server.tenant.TenantBuilder;
import com.yahoo.vespa.config.server.tenant.TenantRepository;
+import com.yahoo.vespa.curator.mock.MockCurator;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
@@ -69,7 +70,7 @@ public class SessionCreateHandlerTest extends SessionHandlerTest {
@Before
public void setupRepo() {
applicationRepo = new MemoryTenantApplications();
- localSessionRepo = new LocalSessionRepo(Clock.systemUTC());
+ localSessionRepo = new LocalSessionRepo(Clock.systemUTC(), new MockCurator());
tenantRepository = new TenantRepository(componentRegistry, false);
sessionFactory = new MockSessionFactory();
TenantBuilder tenantBuilder = TenantBuilder.create(componentRegistry, tenant)
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/SessionPrepareHandlerTest.java b/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/SessionPrepareHandlerTest.java
index 149bec7ab79..e9b53c95c70 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/SessionPrepareHandlerTest.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/SessionPrepareHandlerTest.java
@@ -16,6 +16,7 @@ import com.yahoo.jdisc.http.HttpRequest;
import com.yahoo.path.Path;
import com.yahoo.slime.JsonDecoder;
import com.yahoo.slime.Slime;
+import com.yahoo.transaction.NestedTransaction;
import com.yahoo.transaction.Transaction;
import com.yahoo.vespa.config.server.ApplicationRepository;
import com.yahoo.vespa.config.server.TestComponentRegistry;
@@ -74,7 +75,7 @@ public class SessionPrepareHandlerTest extends SessionHandlerTest {
@Before
public void setupRepo() {
curator = new MockCurator();
- localRepo = new LocalSessionRepo(clock);
+ localRepo = new LocalSessionRepo(clock, curator);
pathPrefix = "/application/v2/tenant/" + tenant + "/session/";
preparedMessage = " for tenant '" + tenant + "' prepared.\"";
tenantMessage = ",\"tenant\":\"" + tenant + "\"";
@@ -243,7 +244,7 @@ public class SessionPrepareHandlerTest extends SessionHandlerTest {
@Test
public void require_that_preparing_with_multiple_tenants_work() throws Exception {
// Need different repo for 'test2' tenant
- LocalSessionRepo localRepoDefault = new LocalSessionRepo(clock);
+ LocalSessionRepo localRepoDefault = new LocalSessionRepo(clock, curator);
final TenantName defaultTenant = TenantName.from("test2");
TenantBuilder defaultTenantBuilder = TenantBuilder.create(componentRegistry, defaultTenant)
.withLocalSessionRepo(localRepoDefault)
@@ -444,6 +445,6 @@ public class SessionPrepareHandlerTest extends SessionHandlerTest {
}
@Override
- public void delete() { }
+ public void delete(NestedTransaction transaction) { }
}
}
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/session/LocalSessionRepoTest.java b/configserver/src/test/java/com/yahoo/vespa/config/server/session/LocalSessionRepoTest.java
index 829dfb978b2..7b9389ada9b 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/session/LocalSessionRepoTest.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/session/LocalSessionRepoTest.java
@@ -57,7 +57,7 @@ public class LocalSessionRepoTest {
new MemoryTenantApplications(),
tenantFileSystemDirs, new HostRegistry<>(),
tenantName);
- repo = new LocalSessionRepo(tenantFileSystemDirs, loader, clock, 5);
+ repo = new LocalSessionRepo(tenantFileSystemDirs, loader, clock, 5, globalComponentRegistry.getCurator());
}
@Test
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/session/LocalSessionTest.java b/configserver/src/test/java/com/yahoo/vespa/config/server/session/LocalSessionTest.java
index 316c439a3cd..b2cfe3e7575 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/session/LocalSessionTest.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/session/LocalSessionTest.java
@@ -2,12 +2,12 @@
package com.yahoo.vespa.config.server.session;
import com.google.common.io.Files;
-import com.yahoo.cloud.config.ConfigserverConfig;
import com.yahoo.config.application.api.ApplicationFile;
import com.yahoo.config.provision.*;
import com.yahoo.path.Path;
import com.yahoo.config.model.application.provider.*;
import com.yahoo.slime.Slime;
+import com.yahoo.transaction.NestedTransaction;
import com.yahoo.vespa.config.server.*;
import com.yahoo.vespa.config.server.application.MemoryTenantApplications;
import com.yahoo.vespa.config.server.deploy.DeployHandlerLogger;
@@ -115,7 +115,9 @@ public class LocalSessionTest {
assertTrue(configCurator.exists(sessionNode));
assertTrue(new File(tenantFileSystemDirs.sessionsPath(), "3").exists());
long gen = superModelGenerationCounter.get();
- session.delete();
+ NestedTransaction transaction = new NestedTransaction();
+ session.delete(transaction);
+ transaction.commit();
assertThat(superModelGenerationCounter.get(), is(gen + 1));
assertFalse(configCurator.exists(sessionNode));
assertFalse(new File(tenantFileSystemDirs.sessionsPath(), "3").exists());