summaryrefslogtreecommitdiffstats
path: root/configserver
diff options
context:
space:
mode:
authorHenning Baldersheim <balder@yahoo-inc.com>2020-08-04 09:51:26 +0200
committerGitHub <noreply@github.com>2020-08-04 09:51:26 +0200
commit6c6a10377d7d9ef5597b534076cdee9ee72bc653 (patch)
treecb283dd487c82fb247f920bc2fc9f1544031c56f /configserver
parenta0a8d7019cfb035b000c616c22c0589ce4ee0986 (diff)
parenteb039ff41f1e612f7012af81950255933f2671b8 (diff)
Merge branch 'master' into bjorncs/container-thread-pool
Diffstat (limited to 'configserver')
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/ApplicationRepository.java25
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/ConfigServerBootstrap.java19
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/application/TenantApplications.java6
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/deploy/Deployment.java12
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/deploy/ZooKeeperClient.java18
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/filedistribution/FileDistributionFactory.java2
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/filedistribution/FileDistributionUtil.java23
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/filedistribution/MockFileDistributionProvider.java24
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandler.java25
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/ApplicationPackageMaintainer.java32
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/ConfigServerMaintainer.java15
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/ConfigServerMaintenance.java21
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/FileDistributionMaintainer.java11
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/SessionsMaintainer.java18
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/TenantsMaintainer.java3
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/monitoring/ZKMetricUpdater.java10
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/rpc/RpcServer.java6
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/session/LocalSession.java12
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/session/RemoteSession.java9
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionPreparer.java14
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionRepository.java22
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionZooKeeperClient.java6
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/zookeeper/ZKApplicationPackage.java3
-rw-r--r--configserver/src/test/apps/content/schemas/music.sd50
-rw-r--r--configserver/src/test/apps/content/services.xml38
-rw-r--r--configserver/src/test/apps/content2/schemas/music.sd50
-rw-r--r--configserver/src/test/apps/content2/services.xml38
-rw-r--r--configserver/src/test/apps/zkapp/deployment.xml2
-rw-r--r--configserver/src/test/apps/zkapp/services.xml4
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/ApplicationRepositoryTest.java33
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/TestComponentRegistry.java8
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/configdefs/a.def2
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/deploy/ZooKeeperClientTest.java8
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/deploy/ZooKeeperDeployerTest.java2
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/filedistribution/MockFileDistributionFactory.java5
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/filedistribution/MockFileDistributionProvider.java22
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/filedistribution/MockFileRegistry.java49
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/http/SessionHandlerTest.java56
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/ApplicationContentHandlerTest.java64
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/ApplicationHandlerTest.java4
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/HostHandlerTest.java100
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/SessionContentHandlerTest.java64
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/TenantHandlerTest.java19
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/session/LocalSessionTest.java19
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/session/SessionPreparerTest.java41
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/session/SessionTest.java15
-rw-r--r--configserver/src/test/java/com/yahoo/vespa/config/server/session/SessionZooKeeperClientTest.java5
47 files changed, 626 insertions, 408 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 41ec7626049..62d5184d7a9 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
@@ -206,6 +206,14 @@ public class ApplicationRepository implements com.yahoo.config.provision.Deploye
this.metric = metric;
}
+ public Clock clock() {
+ return clock;
+ }
+
+ public Metric metric() {
+ return metric;
+ }
+
// ---------------- Deploying ----------------------------------------------------------------
public PrepareResult prepare(Tenant tenant, long sessionId, PrepareParams prepareParams, Instant now) {
@@ -499,9 +507,6 @@ public class ApplicationRepository implements com.yahoo.config.provision.Deploye
long sessionId = getSessionIdForApplication(tenant, applicationId);
RemoteSession session = getRemoteSession(tenant, sessionId);
return session.ensureApplicationLoaded().getForVersionOrLatest(version, clock.instant());
- } catch (NotFoundException e) {
- log.log(Level.WARNING, "Failed getting application for '" + applicationId + "': " + e.getMessage());
- throw e;
} catch (Exception e) {
log.log(Level.WARNING, "Failed getting application for '" + applicationId + "'", e);
throw e;
@@ -677,7 +682,11 @@ public class ApplicationRepository implements com.yahoo.config.provision.Deploye
tenantRepository.getAllTenants().forEach(tenant -> sessionsPerTenant.put(tenant, tenant.getSessionRepository().getLocalSessions()));
Set<ApplicationId> applicationIds = new HashSet<>();
- sessionsPerTenant.values().forEach(sessionList -> sessionList.forEach(s -> applicationIds.add(s.getApplicationId())));
+ sessionsPerTenant.values()
+ .forEach(sessionList -> sessionList.stream()
+ .map(Session::getApplicationId)
+ .filter(Objects::nonNull)
+ .forEach(applicationIds::add));
Map<ApplicationId, Long> activeSessions = new HashMap<>();
applicationIds.forEach(applicationId -> {
@@ -862,12 +871,8 @@ public class ApplicationRepository implements com.yahoo.config.provision.Deploye
// We make no validation that the hostname is actually allocated to the given application since
// most applications under hosted-vespa are not known to the model and it's OK for a user to get
// logs for any host if they are authorized for the hosted-vespa tenant.
- if (hostname.isPresent()) {
- if (HOSTED_VESPA_TENANT.equals(applicationId.tenant()))
- return "http://" + hostname.get() + ":8080/logs";
- else
- throw new IllegalArgumentException("Using hostname parameter when getting logs is not supported for application "
- + applicationId);
+ if (hostname.isPresent() && HOSTED_VESPA_TENANT.equals(applicationId.tenant())) {
+ return "http://" + hostname.get() + ":8080/logs";
}
Application application = getApplication(applicationId);
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/ConfigServerBootstrap.java b/configserver/src/main/java/com/yahoo/vespa/config/server/ConfigServerBootstrap.java
index d628702758b..2d296b0a691 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/ConfigServerBootstrap.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/ConfigServerBootstrap.java
@@ -41,7 +41,7 @@ import static com.yahoo.vespa.config.server.ConfigServerBootstrap.RedeployingApp
* applications. If that is done successfully the RPC server will start and the health status code will change from
* 'initializing' to 'up'. If VIP status mode is VIP_STATUS_PROGRAMMATICALLY the config server
* will be put into rotation (start serving status.html with 200 OK), if the mode is VIP_STATUS_FILE a VIP status
- * file is created or removed ny some external program based on the health status code.
+ * file is created or removed by some external program based on the health status code.
*
* @author Ulf Lilleengen
* @author hmusum
@@ -51,9 +51,9 @@ public class ConfigServerBootstrap extends AbstractComponent implements Runnable
private static final Logger log = Logger.getLogger(ConfigServerBootstrap.class.getName());
// INITIALIZE_ONLY is for testing only
- enum Mode {BOOTSTRAP_IN_CONSTRUCTOR, BOOTSTRAP_IN_SEPARATE_THREAD, INITIALIZE_ONLY}
- enum RedeployingApplicationsFails {EXIT_JVM, CONTINUE}
- enum VipStatusMode {VIP_STATUS_FILE, VIP_STATUS_PROGRAMMATICALLY}
+ enum Mode { BOOTSTRAP_IN_CONSTRUCTOR, BOOTSTRAP_IN_SEPARATE_THREAD, INITIALIZE_ONLY }
+ enum RedeployingApplicationsFails { EXIT_JVM, CONTINUE }
+ enum VipStatusMode { VIP_STATUS_FILE, VIP_STATUS_PROGRAMMATICALLY }
private final ApplicationRepository applicationRepository;
private final RpcServer server;
@@ -208,12 +208,17 @@ public class ConfigServerBootstrap extends AbstractComponent implements Runnable
private boolean redeployAllApplications() throws InterruptedException {
Instant end = Instant.now().plus(maxDurationOfRedeployment);
Set<ApplicationId> applicationsNotRedeployed = applicationRepository.listApplications();
+ long failCount = 0;
do {
applicationsNotRedeployed = redeployApplications(applicationsNotRedeployed);
- if ( ! applicationsNotRedeployed.isEmpty()) {
+ if ( ! applicationsNotRedeployed.isEmpty() && ! sleepTimeWhenRedeployingFails.isZero()) {
+ failCount++;
+ Duration sleepTime = sleepTimeWhenRedeployingFails.multipliedBy(failCount);
+ if (sleepTime.compareTo(Duration.ofMinutes(10)) > 0)
+ sleepTime = Duration.ofMinutes(10);
log.log(Level.INFO, "Redeployment of " + applicationsNotRedeployed +
- " failed, will retry in " + sleepTimeWhenRedeployingFails);
- Thread.sleep(sleepTimeWhenRedeployingFails.toMillis());
+ " failed (" + failCount + " failures), will retry in " + sleepTime);
+ Thread.sleep(sleepTime.toMillis());
}
} while ( ! applicationsNotRedeployed.isEmpty() && Instant.now().isBefore(end));
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/application/TenantApplications.java b/configserver/src/main/java/com/yahoo/vespa/config/server/application/TenantApplications.java
index 5567de6dadc..ba3b6f71b4b 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/application/TenantApplications.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/application/TenantApplications.java
@@ -395,11 +395,7 @@ public class TenantApplications implements RequestHandler, HostValidator<Applica
@Override
public ApplicationId resolveApplicationId(String hostName) {
- ApplicationId applicationId = hostRegistry.getKeyForHost(hostName);
- if (applicationId == null) {
- applicationId = ApplicationId.defaultId();
- }
- return applicationId;
+ return hostRegistry.getKeyForHost(hostName);
}
@Override
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/deploy/Deployment.java b/configserver/src/main/java/com/yahoo/vespa/config/server/deploy/Deployment.java
index def629f738c..6e571fec72b 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/deploy/Deployment.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/deploy/Deployment.java
@@ -30,6 +30,8 @@ import java.time.Duration;
import java.util.Optional;
import java.util.logging.Logger;
+import static com.yahoo.vespa.curator.Curator.CompletionWaiter;
+
/**
* The process of deploying an application.
* Deployments are created by an {@link ApplicationRepository}.
@@ -134,15 +136,17 @@ public class Deployment implements com.yahoo.config.provision.Deployment {
try (ActionTimer timer = applicationRepository.timerFor(session.getApplicationId(), "deployment.activateMillis")) {
TimeoutBudget timeoutBudget = new TimeoutBudget(clock, timeout);
+ validateSessionStatus(session);
ApplicationId applicationId = session.getApplicationId();
if ( ! timeoutBudget.hasTimeLeft()) throw new RuntimeException("Timeout exceeded when trying to activate '" + applicationId + "'");
RemoteSession previousActiveSession;
+ CompletionWaiter waiter;
try (Lock lock = tenant.getApplicationRepo().lock(applicationId)) {
- validateSessionStatus(session);
- NestedTransaction transaction = new NestedTransaction();
previousActiveSession = applicationRepository.getActiveSession(applicationId);
+ waiter = session.createActiveWaiter();
+ NestedTransaction transaction = new NestedTransaction();
transaction.add(deactivateCurrentActivateNew(previousActiveSession, session, ignoreSessionStaleFailure));
hostProvisioner.ifPresent(provisioner -> provisioner.activate(transaction, applicationId, session.getAllocatedHosts().getHosts()));
transaction.commit();
@@ -154,11 +158,11 @@ public class Deployment implements com.yahoo.config.provision.Deployment {
throw new InternalServerException("Error activating application", e);
}
- session.waitUntilActivated(timeoutBudget);
+ waiter.awaitCompletion(timeoutBudget.timeLeft());
log.log(Level.INFO, session.logPre() + "Session " + session.getSessionId() + " activated successfully using " +
hostProvisioner.map(provisioner -> provisioner.getClass().getSimpleName()).orElse("no host provisioner") +
". Config generation " + session.getMetaData().getGeneration() +
- (previousActiveSession != null ? ". Based on previous active session " + previousActiveSession.getSessionId() : "") +
+ (previousActiveSession != null ? ". Based on session " + previousActiveSession.getSessionId() : "") +
". File references: " + applicationRepository.getFileReferences(applicationId));
}
}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/deploy/ZooKeeperClient.java b/configserver/src/main/java/com/yahoo/vespa/config/server/deploy/ZooKeeperClient.java
index 7e83d7013e0..c634d82010e 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/deploy/ZooKeeperClient.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/deploy/ZooKeeperClient.java
@@ -36,16 +36,14 @@ public class ZooKeeperClient {
private final ConfigCurator configCurator;
private final DeployLogger logger;
- private final boolean logFine;
/* This is the generation that will be used for reading and writing application data. (1 more than last deployed application) */
private final Path rootPath;
private static final ApplicationFile.PathFilter xmlFilter = path -> path.getName().endsWith(".xml");
- public ZooKeeperClient(ConfigCurator configCurator, DeployLogger logger, boolean logFine, Path rootPath) {
+ public ZooKeeperClient(ConfigCurator configCurator, DeployLogger logger, Path rootPath) {
this.configCurator = configCurator;
this.logger = logger;
- this.logFine = logFine;
this.rootPath = rootPath;
}
@@ -62,7 +60,6 @@ public class ZooKeeperClient {
try {
while (retries > 0) {
try {
- logFine("Setting up ZooKeeper nodes for this application");
createZooKeeperNodes();
break;
} catch (RuntimeException e) {
@@ -105,16 +102,11 @@ public class ZooKeeperClient {
* @param app the application package to feed to zookeeper
*/
void write(ApplicationPackage app) {
- logFine("Feeding application config into ZooKeeper");
try {
- logFine("Feeding user def files into ZooKeeper");
writeUserDefs(app);
- logFine("Feeding application package into ZooKeeper");
writeSomeOf(app);
writeSearchDefinitions(app);
writeUserIncludeDirs(app, app.getUserIncludeDirs());
- logFine("Feeding sd from docproc bundle into ZooKeeper");
- logFine("Write application metadata into ZooKeeper");
write(app.getMetaData());
} catch (Exception e) {
throw new IllegalStateException("Unable to write vespa model to config server(s) " + System.getProperty("configsources") + "\n" +
@@ -269,7 +261,6 @@ public class ZooKeeperClient {
}
private void write(Version vespaVersion, FileRegistry fileRegistry) {
- logFine("Feeding file registry data into ZooKeeper");
String exportedRegistry = PreGeneratedFileRegistry.exportRegistry(fileRegistry);
configCurator.putData(getZooKeeperAppPath(null).append(ZKApplicationPackage.fileRegistryNode).getAbsolute(),
@@ -288,7 +279,6 @@ public class ZooKeeperClient {
}
void cleanupZooKeeper() {
- logFine("Exception occurred. Cleaning up ZooKeeper");
try {
for (String subPath : Arrays.asList(
ConfigCurator.DEFCONFIGS_ZK_SUBPATH,
@@ -317,12 +307,6 @@ public class ZooKeeperClient {
}
}
- private void logFine(String msg) {
- if (logFine) {
- logger.log(Level.FINE, msg);
- }
- }
-
public void write(AllocatedHosts hosts) throws IOException {
configCurator.putData(rootPath.append(ZKApplicationPackage.allocatedHostsNode).getAbsolute(),
AllocatedHostsSerializer.toJson(hosts));
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/filedistribution/FileDistributionFactory.java b/configserver/src/main/java/com/yahoo/vespa/config/server/filedistribution/FileDistributionFactory.java
index 1cfe30270c3..ae18c3e6e95 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/filedistribution/FileDistributionFactory.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/filedistribution/FileDistributionFactory.java
@@ -16,7 +16,7 @@ import java.io.File;
@SuppressWarnings("WeakerAccess")
public class FileDistributionFactory {
- private final ConfigserverConfig configserverConfig;
+ protected final ConfigserverConfig configserverConfig;
private final Supervisor supervisor = new Supervisor(new Transport());
@Inject
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/filedistribution/FileDistributionUtil.java b/configserver/src/main/java/com/yahoo/vespa/config/server/filedistribution/FileDistributionUtil.java
index c06e4da2b7b..305bc71e66a 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/filedistribution/FileDistributionUtil.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/filedistribution/FileDistributionUtil.java
@@ -1,6 +1,7 @@
package com.yahoo.vespa.config.server.filedistribution;
import com.yahoo.cloud.config.ConfigserverConfig;
+import com.yahoo.config.FileReference;
import com.yahoo.config.subscription.ConfigSourceSet;
import com.yahoo.jrt.Supervisor;
import com.yahoo.jrt.Transport;
@@ -50,14 +51,25 @@ public class FileDistributionUtil {
return configServers.size() > 0 ? new JRTConnectionPool(new ConfigSourceSet(configServers)) : emptyConnectionPool();
}
+ public static boolean fileReferenceExistsOnDisk(File downloadDirectory, FileReference applicationPackageReference) {
+ return getFileReferencesOnDisk(downloadDirectory).contains(applicationPackageReference.value());
+ }
+
static ConnectionPool emptyConnectionPool() {
return new EmptyConnectionPool();
}
private static class EmptyConnectionPool implements ConnectionPool {
+ private Supervisor supervisor;
@Override
- public void close() {}
+ public void close() {
+ synchronized (this) {
+ if (supervisor != null) {
+ supervisor.transport().shutdown().join();
+ }
+ }
+ }
@Override
public void setError(Connection connection, int i) {}
@@ -72,7 +84,14 @@ public class FileDistributionUtil {
public int getSize() { return 0; }
@Override
- public Supervisor getSupervisor() { return new Supervisor(new Transport()); }
+ public Supervisor getSupervisor() {
+ synchronized (this) {
+ if (supervisor == null) {
+ supervisor = new Supervisor(new Transport());
+ }
+ }
+ return supervisor;
+ }
}
}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/filedistribution/MockFileDistributionProvider.java b/configserver/src/main/java/com/yahoo/vespa/config/server/filedistribution/MockFileDistributionProvider.java
deleted file mode 100644
index db70a51b2b4..00000000000
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/filedistribution/MockFileDistributionProvider.java
+++ /dev/null
@@ -1,24 +0,0 @@
-// 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.filedistribution;
-
-import com.yahoo.config.model.api.FileDistribution;
-import com.yahoo.config.model.application.provider.MockFileRegistry;
-
-import java.io.File;
-
-/**
- * @author Ulf Lilleengen
- */
-public class MockFileDistributionProvider extends FileDistributionProvider {
- public int timesCalled = 0;
-
- public MockFileDistributionProvider(File fileReferencesDir) {
- super(new MockFileRegistry(), new MockFileDistribution(fileReferencesDir));
- }
-
- public FileDistribution getFileDistribution() {
- timesCalled++;
- return super.getFileDistribution();
- }
-
-}
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 d1861184c24..749f57b3104 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
@@ -3,6 +3,7 @@ package com.yahoo.vespa.config.server.http.v2;
import com.google.inject.Inject;
import com.yahoo.component.Version;
+import com.yahoo.config.FileReference;
import com.yahoo.config.application.api.ApplicationFile;
import com.yahoo.config.provision.ApplicationId;
import com.yahoo.config.provision.ApplicationName;
@@ -24,8 +25,11 @@ import com.yahoo.vespa.config.server.http.HttpErrorResponse;
import com.yahoo.vespa.config.server.http.HttpHandler;
import com.yahoo.vespa.config.server.http.JSONResponse;
import com.yahoo.vespa.config.server.http.NotFoundException;
+import com.yahoo.vespa.config.server.session.RemoteSession;
import com.yahoo.vespa.config.server.tenant.Tenant;
+import com.yahoo.vespa.defaults.Defaults;
+import java.io.File;
import java.io.IOException;
import java.time.Duration;
import java.util.List;
@@ -34,6 +38,8 @@ import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.Stream;
+import static com.yahoo.vespa.config.server.filedistribution.FileDistributionUtil.fileReferenceExistsOnDisk;
+
/**
* Operations on applications (delete, wait for config convergence, restart, application content etc.)
*
@@ -154,11 +160,25 @@ public class ApplicationHandler extends HttpHandler {
}
}
+ return getApplicationResponse(applicationId);
+ }
+
+ GetApplicationResponse getApplicationResponse(ApplicationId applicationId) {
Tenant tenant = applicationRepository.getTenant(applicationId);
Optional<ApplicationSet> applicationSet = applicationRepository.getCurrentActiveApplicationSet(tenant, applicationId);
+ String applicationPackage = "";
+ RemoteSession session = applicationRepository.getActiveSession(applicationId);
+ if (session != null) {
+ FileReference applicationPackageReference = session.getApplicationPackageReference();
+ File downloadDirectory = new File(Defaults.getDefaults().underVespaHome(applicationRepository.configserverConfig().fileReferencesDir()));
+ if (applicationPackageReference != null && ! fileReferenceExistsOnDisk(downloadDirectory, applicationPackageReference))
+ applicationPackage = applicationPackageReference.value();
+ }
+
return new GetApplicationResponse(Response.Status.OK,
applicationRepository.getApplicationGeneration(applicationId),
- applicationSet.get().getAllVersions(applicationId));
+ applicationSet.get().getAllVersions(applicationId),
+ applicationPackage);
}
@Override
@@ -316,9 +336,10 @@ public class ApplicationHandler extends HttpHandler {
}
private static class GetApplicationResponse extends JSONResponse {
- GetApplicationResponse(int status, long generation, List<Version> modelVersions) {
+ GetApplicationResponse(int status, long generation, List<Version> modelVersions, String applicationPackageReference) {
super(status);
object.setLong("generation", generation);
+ object.setString("applicationPackageFileReference", applicationPackageReference);
Cursor modelVersionArray = object.setArray("modelVersions");
modelVersions.forEach(version -> modelVersionArray.addString(version.toFullString()));
}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/ApplicationPackageMaintainer.java b/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/ApplicationPackageMaintainer.java
index 7c983ab48a0..e9687000b3c 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/ApplicationPackageMaintainer.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/ApplicationPackageMaintainer.java
@@ -3,6 +3,7 @@ package com.yahoo.vespa.config.server.maintenance;
import com.yahoo.cloud.config.ConfigserverConfig;
import com.yahoo.config.FileReference;
import com.yahoo.config.provision.ApplicationId;
+import com.yahoo.vespa.config.ConnectionPool;
import com.yahoo.vespa.config.server.ApplicationRepository;
import com.yahoo.vespa.config.server.session.RemoteSession;
import com.yahoo.vespa.config.server.session.SessionRepository;
@@ -16,11 +17,10 @@ import com.yahoo.vespa.flags.Flags;
import java.io.File;
import java.time.Duration;
-import java.util.Set;
import java.util.logging.Logger;
-import static com.yahoo.vespa.config.server.filedistribution.FileDistributionUtil.getFileReferencesOnDisk;
import static com.yahoo.vespa.config.server.filedistribution.FileDistributionUtil.createConnectionPool;
+import static com.yahoo.vespa.config.server.filedistribution.FileDistributionUtil.fileReferenceExistsOnDisk;
/**
* Verifies that all active sessions has an application package on local disk.
@@ -33,28 +33,29 @@ public class ApplicationPackageMaintainer extends ConfigServerMaintainer {
private static final Logger log = Logger.getLogger(ApplicationPackageMaintainer.class.getName());
private final ApplicationRepository applicationRepository;
- private final ConfigserverConfig configserverConfig;
+ private final ConnectionPool connectionPool;
private final File downloadDirectory;
private final BooleanFlag distributeApplicationPackage;
ApplicationPackageMaintainer(ApplicationRepository applicationRepository,
Curator curator,
Duration interval,
- ConfigserverConfig configserverConfig,
FlagSource flagSource) {
super(applicationRepository, curator, flagSource, interval, interval);
this.applicationRepository = applicationRepository;
- this.configserverConfig = configserverConfig;
+ ConfigserverConfig configserverConfig = applicationRepository.configserverConfig();
+ connectionPool = createConnectionPool(configserverConfig);
distributeApplicationPackage = Flags.CONFIGSERVER_DISTRIBUTE_APPLICATION_PACKAGE.bindTo(flagSource);
downloadDirectory = new File(Defaults.getDefaults().underVespaHome(configserverConfig.fileReferencesDir()));
}
@Override
- protected void maintain() {
- if (! distributeApplicationPackage.value()) return;
+ protected boolean maintain() {
+ boolean success = true;
+ if (! distributeApplicationPackage.value()) return success;
- try (var fileDownloader = new FileDownloader(createConnectionPool(configserverConfig), downloadDirectory)) {
+ try (var fileDownloader = new FileDownloader(connectionPool, downloadDirectory)) {
for (var applicationId : applicationRepository.listApplications()) {
log.fine(() -> "Verifying application package for " + applicationId);
RemoteSession session = applicationRepository.getActiveSession(applicationId);
@@ -65,10 +66,11 @@ public class ApplicationPackageMaintainer extends ConfigServerMaintainer {
log.fine(() -> "Verifying application package file reference " + applicationPackage + " for session " + sessionId);
if (applicationPackage != null) {
- if (missingOnDisk(applicationPackage)) {
+ if (! fileReferenceExistsOnDisk(downloadDirectory, applicationPackage)) {
log.fine(() -> "Downloading missing application package for application " + applicationId + " - session " + sessionId);
if (fileDownloader.getFile(applicationPackage).isEmpty()) {
+ success = false;
log.warning("Failed to download application package for application " + applicationId + " - session " + sessionId);
continue;
}
@@ -77,6 +79,13 @@ public class ApplicationPackageMaintainer extends ConfigServerMaintainer {
}
}
}
+ return success;
+ }
+
+ @Override
+ public void close() {
+ connectionPool.close();
+ super.close();
}
private void createLocalSessionIfMissing(ApplicationId applicationId, long sessionId) {
@@ -86,9 +95,4 @@ public class ApplicationPackageMaintainer extends ConfigServerMaintainer {
sessionRepository.createLocalSessionUsingDistributedApplicationPackage(sessionId);
}
- private boolean missingOnDisk(FileReference applicationPackageReference) {
- Set<String> fileReferencesOnDisk = getFileReferencesOnDisk(downloadDirectory);
- return ! fileReferencesOnDisk.contains(applicationPackageReference.value());
- }
-
}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/ConfigServerMaintainer.java b/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/ConfigServerMaintainer.java
index 5369bbef366..5854b1d85da 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/ConfigServerMaintainer.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/ConfigServerMaintainer.java
@@ -3,7 +3,9 @@ package com.yahoo.vespa.config.server.maintenance;
import com.yahoo.concurrent.maintenance.JobControl;
import com.yahoo.concurrent.maintenance.JobControlState;
+import com.yahoo.concurrent.maintenance.JobMetrics;
import com.yahoo.concurrent.maintenance.Maintainer;
+import com.yahoo.jdisc.Metric;
import com.yahoo.path.Path;
import com.yahoo.transaction.Mutex;
import com.yahoo.vespa.config.server.ApplicationRepository;
@@ -13,6 +15,7 @@ import com.yahoo.vespa.flags.Flags;
import com.yahoo.vespa.flags.ListFlag;
import java.time.Duration;
+import java.util.Map;
import java.util.Set;
/**
@@ -26,16 +29,24 @@ public abstract class ConfigServerMaintainer extends Maintainer {
ConfigServerMaintainer(ApplicationRepository applicationRepository, Curator curator, FlagSource flagSource,
Duration initialDelay, Duration interval) {
- super(null, interval, initialDelay, new JobControl(new JobControlFlags(curator, flagSource)));
+ super(null, interval, initialDelay, new JobControl(new JobControlFlags(curator, flagSource)),
+ jobMetrics(applicationRepository.metric()));
this.applicationRepository = applicationRepository;
}
+ private static JobMetrics jobMetrics(Metric metric) {
+ return new JobMetrics((job, consecutiveFailures) -> {
+ metric.set("maintenance.consecutiveFailures", consecutiveFailures, metric.createContext(Map.of("job", job)));
+ });
+ }
+
private static class JobControlFlags implements JobControlState {
private static final Path root = Path.fromString("/configserver/v1/");
- private static final Path lockRoot = root.append("locks");
+ private static final Path lockRoot = root.append("locks");
private final Curator curator;
+
private final ListFlag<String> inactiveJobsFlag;
public JobControlFlags(Curator curator, FlagSource flagSource) {
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/ConfigServerMaintenance.java b/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/ConfigServerMaintenance.java
index a6585be391c..ecdca39dc72 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/ConfigServerMaintenance.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/ConfigServerMaintenance.java
@@ -4,9 +4,8 @@ package com.yahoo.vespa.config.server.maintenance;
import com.google.inject.Inject;
import com.yahoo.cloud.config.ConfigserverConfig;
import com.yahoo.component.AbstractComponent;
-import com.yahoo.config.provision.SystemName;
+import com.yahoo.jdisc.Metric;
import com.yahoo.vespa.config.server.ApplicationRepository;
-import com.yahoo.vespa.config.server.filedistribution.FileDistributionFactory;
import com.yahoo.vespa.curator.Curator;
import com.yahoo.vespa.flags.FlagSource;
@@ -30,19 +29,18 @@ public class ConfigServerMaintenance extends AbstractComponent {
public ConfigServerMaintenance(ConfigserverConfig configserverConfig,
ApplicationRepository applicationRepository,
Curator curator,
- FileDistributionFactory fileDistributionFactory,
- FlagSource flagSource) {
+ FlagSource flagSource,
+ Metric metric) {
DefaultTimes defaults = new DefaultTimes(configserverConfig);
- // TODO: Disabled until we have application metadata
+ // TODO: Disabled until we have application metadata per tenant
//tenantsMaintainer = new TenantsMaintainer(applicationRepository, curator, defaults.tenantsMaintainerInterval);
- fileDistributionMaintainer = new FileDistributionMaintainer(applicationRepository, curator, defaults.defaultInterval, configserverConfig, flagSource);
+ fileDistributionMaintainer = new FileDistributionMaintainer(applicationRepository, curator, defaults.defaultInterval, flagSource);
sessionsMaintainer = new SessionsMaintainer(applicationRepository, curator, Duration.ofMinutes(1), flagSource);
- applicationPackageMaintainer = new ApplicationPackageMaintainer(applicationRepository, curator, Duration.ofMinutes(1), configserverConfig, flagSource);
+ applicationPackageMaintainer = new ApplicationPackageMaintainer(applicationRepository, curator, Duration.ofMinutes(1), flagSource);
}
@Override
public void deconstruct() {
- //tenantsMaintainer.close();
fileDistributionMaintainer.close();
sessionsMaintainer.close();
applicationPackageMaintainer.close();
@@ -55,16 +53,9 @@ public class ConfigServerMaintenance extends AbstractComponent {
private static class DefaultTimes {
private final Duration defaultInterval;
- private final Duration tenantsMaintainerInterval;
DefaultTimes(ConfigserverConfig configserverConfig) {
this.defaultInterval = Duration.ofMinutes(configserverConfig.maintainerIntervalMinutes());
- boolean isCd = configserverConfig.system().equals(SystemName.cd.value());
- // TODO: Want job control or feature flag to control when to run this, for now use a very
- // long interval to avoid running the maintainer except in CD
- this.tenantsMaintainerInterval = isCd
- ? defaultInterval
- : Duration.ofMinutes(configserverConfig.tenantsMaintainerIntervalMinutes());
}
}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/FileDistributionMaintainer.java b/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/FileDistributionMaintainer.java
index ed57be799c7..8e5069abfed 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/FileDistributionMaintainer.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/FileDistributionMaintainer.java
@@ -1,7 +1,6 @@
// 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.maintenance;
-import com.yahoo.cloud.config.ConfigserverConfig;
import com.yahoo.vespa.config.server.ApplicationRepository;
import com.yahoo.vespa.curator.Curator;
import com.yahoo.vespa.defaults.Defaults;
@@ -26,17 +25,17 @@ public class FileDistributionMaintainer extends ConfigServerMaintainer {
FileDistributionMaintainer(ApplicationRepository applicationRepository,
Curator curator,
Duration interval,
- ConfigserverConfig configserverConfig,
FlagSource flagSource) {
super(applicationRepository, curator, flagSource, interval, interval);
this.applicationRepository = applicationRepository;
- this.maxUnusedFileReferenceAge = Duration.ofHours(configserverConfig.keepUnusedFileReferencesHours());
- this.fileReferencesDir = new File(Defaults.getDefaults().underVespaHome(configserverConfig.fileReferencesDir()));
+ this.maxUnusedFileReferenceAge = Duration.ofHours(applicationRepository.configserverConfig().keepUnusedFileReferencesHours());
+ this.fileReferencesDir = new File(Defaults.getDefaults().underVespaHome(applicationRepository.configserverConfig().fileReferencesDir()));
}
@Override
- protected void maintain() {
+ protected boolean maintain() {
applicationRepository.deleteUnusedFiledistributionReferences(fileReferencesDir, maxUnusedFileReferenceAge);
-
+ return true;
}
+
}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/SessionsMaintainer.java b/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/SessionsMaintainer.java
index 4adf287448d..911b31077ea 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/SessionsMaintainer.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/SessionsMaintainer.java
@@ -9,7 +9,7 @@ import com.yahoo.vespa.flags.FlagSource;
import java.time.Duration;
/**
- * Removes inactive sessions
+ * Removes expired sessions and locks
* <p>
* Note: Unit test is in ApplicationRepositoryTest
*
@@ -26,20 +26,20 @@ public class SessionsMaintainer extends ConfigServerMaintainer {
}
@Override
- protected void maintain() {
+ protected boolean maintain() {
applicationRepository.deleteExpiredLocalSessions();
- // Expired remote sessions are sessions that belong to an application that have external deployments that
- // are no longer active
if (hostedVespa) {
- Duration expiryTime = Duration.ofDays(1);
+ Duration expiryTime = Duration.ofHours(6);
int deleted = applicationRepository.deleteExpiredRemoteSessions(expiryTime);
- log.log(LogLevel.FINE, "Deleted " + deleted + " expired remote sessions, expiry time " + expiryTime);
+ log.log(LogLevel.FINE, () -> "Deleted " + deleted + " expired remote sessions older than " + expiryTime);
}
- Duration lockExpiryTime = Duration.ofDays(1);
+ Duration lockExpiryTime = Duration.ofHours(12);
int deleted = applicationRepository.deleteExpiredLocks(lockExpiryTime);
- if (deleted > 0)
- log.log(LogLevel.INFO, "Deleted " + deleted + " locks older than " + lockExpiryTime);
+ log.log(LogLevel.FINE, () -> "Deleted " + deleted + " locks older than " + lockExpiryTime);
+
+ return true;
}
+
}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/TenantsMaintainer.java b/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/TenantsMaintainer.java
index 9a81d9f7547..d29eea842f5 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/TenantsMaintainer.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/maintenance/TenantsMaintainer.java
@@ -28,8 +28,9 @@ public class TenantsMaintainer extends ConfigServerMaintainer {
}
@Override
- protected void maintain() {
+ protected boolean maintain() {
applicationRepository.deleteUnusedTenants(ttlForUnusedTenant, clock.instant());
+ return true;
}
}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/monitoring/ZKMetricUpdater.java b/configserver/src/main/java/com/yahoo/vespa/config/server/monitoring/ZKMetricUpdater.java
index 52ca73c68b9..401588f478c 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/monitoring/ZKMetricUpdater.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/monitoring/ZKMetricUpdater.java
@@ -36,11 +36,11 @@ public class ZKMetricUpdater implements Runnable {
public static final String METRIC_ZK_CONNECTIONS = getMetricName("zkConnections");
public static final String METRIC_ZK_OUTSTANDING_REQUESTS = getMetricName("zkOutstandingRequests");
- private static final int CONNECTION_TIMEOUT_MS = 500;
- private static final int WRITE_TIMEOUT_MS = 250;
- private static final int READ_TIMEOUT_MS = 500;
+ private static final int CONNECTION_TIMEOUT_MS = 1000;
+ private static final int WRITE_TIMEOUT_MS = 500;
+ private static final int READ_TIMEOUT_MS = 1000;
- private AtomicReference<Map<String, Long>> zkMetrics = new AtomicReference<>(new HashMap<>());
+ private final AtomicReference<Map<String, Long>> zkMetrics = new AtomicReference<>(new HashMap<>());
private final ScheduledExecutorService executorService;
private final int zkPort;
@@ -103,7 +103,7 @@ public class ZKMetricUpdater implements Runnable {
return Optional.of(baos.toString(StandardCharsets.UTF_8));
} catch (IOException | InterruptedException | ExecutionException | TimeoutException e) {
- log.warning("Failure in retrieving monitoring data: (" + e.getClass().getName() + ") " + e.getMessage());
+ log.warning("Failure in retrieving monitoring data: (" + e.getClass().getSimpleName() + ") " + e.getMessage());
return Optional.empty();
}
}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/rpc/RpcServer.java b/configserver/src/main/java/com/yahoo/vespa/config/server/rpc/RpcServer.java
index a6c12f49a8e..388ddb4048c 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/rpc/RpcServer.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/rpc/RpcServer.java
@@ -392,7 +392,8 @@ public class RpcServer implements Runnable, ReloadListener, TenantListener {
if ("*".equals(request.getConfigKey().getConfigId())) {
return GetConfigContext.create(ApplicationId.global(), superModelRequestHandler, trace);
}
- TenantName tenant = optionalTenant.orElse(TenantName.defaultName()); // perhaps needed for non-hosted?
+ // TODO: Look into if this fallback really is needed
+ TenantName tenant = optionalTenant.orElse(TenantName.defaultName());
Optional<RequestHandler> requestHandler = getRequestHandler(tenant);
if (requestHandler.isEmpty()) {
String msg = TenantRepository.logPre(tenant) + "Unable to find request handler for tenant '" + tenant +
@@ -404,6 +405,9 @@ public class RpcServer implements Runnable, ReloadListener, TenantListener {
}
RequestHandler handler = requestHandler.get();
ApplicationId applicationId = handler.resolveApplicationId(request.getClientHostName());
+ // TODO: Look into if this fallback really is needed
+ if (applicationId == null && tenant.equals(TenantName.defaultName()))
+ applicationId = ApplicationId.defaultId();
if (trace.shouldTrace(TRACELEVEL_DEBUG)) {
trace.trace(TRACELEVEL_DEBUG, "Host '" + request.getClientHostName() + "' should have config from application '" + 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 73e7b36c381..96324ea4320 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
@@ -7,9 +7,10 @@ import com.yahoo.config.application.api.ApplicationPackage;
import com.yahoo.config.provision.TenantName;
import com.yahoo.path.Path;
import com.yahoo.transaction.Transaction;
-import com.yahoo.vespa.config.server.TimeoutBudget;
import com.yahoo.vespa.config.server.application.TenantApplications;
+import static com.yahoo.vespa.curator.Curator.CompletionWaiter;
+
/**
* A LocalSession is a session that has been created locally on this configserver. A local session can be edited and
* prepared. Deleting a local session will ensure that the local filesystem state and global zookeeper state is
@@ -56,8 +57,11 @@ public class LocalSession extends Session {
sessionZooKeeperClient.writeStatus(newStatus);
}
+ public CompletionWaiter createActiveWaiter() {
+ return sessionZooKeeperClient.createActiveWaiter();
+ }
+
public Transaction createActivateTransaction() {
- sessionZooKeeperClient.createActiveWaiter();
Transaction transaction = createSetStatusTransaction(Status.ACTIVATE);
transaction.add(applicationRepo.createPutTransaction(sessionZooKeeperClient.readApplicationId(), getSessionId()).operations());
return transaction;
@@ -71,10 +75,6 @@ public class LocalSession extends Session {
return applicationPackage.getMetaData().getPreviousActiveGeneration();
}
- public void waitUntilActivated(TimeoutBudget timeoutBudget) {
- sessionZooKeeperClient.getActiveWaiter().awaitCompletion(timeoutBudget.timeLeft());
- }
-
public enum Mode {
READ, WRITE
}
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 23300239d17..9677c7cf20e 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
@@ -102,10 +102,11 @@ public class RemoteSession extends Session {
KeeperException.NodeExistsException.class);
Class<? extends Throwable> exceptionClass = e.getCause().getClass();
if (acceptedExceptions.contains(exceptionClass))
- log.log(Level.INFO, "Not able to notify completion for session: " + getSessionId() + ", node " +
- (exceptionClass.equals(KeeperException.NoNodeException.class)
- ? "has been deleted"
- : "already exists"));
+ log.log(Level.INFO, "Not able to notify completion for session " + getSessionId() +
+ " (" + completionWaiter + ")," +
+ " node " + (exceptionClass.equals(KeeperException.NoNodeException.class)
+ ? "has been deleted"
+ : "already exists"));
else
throw e;
}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionPreparer.java b/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionPreparer.java
index b6b0ac45bb5..35bbc1a8233 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionPreparer.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionPreparer.java
@@ -120,10 +120,10 @@ public class SessionPreparer {
* @param tenantPath Zookeeper path for the tenant for this session
* @return the config change actions that must be done to handle the activation of the models prepared.
*/
- public ConfigChangeActions prepare(HostValidator<ApplicationId> hostValidator, DeployLogger logger, PrepareParams params,
- Optional<ApplicationSet> currentActiveApplicationSet, Path tenantPath,
- Instant now, File serverDbSessionDir, ApplicationPackage applicationPackage,
- SessionZooKeeperClient sessionZooKeeperClient) {
+ public PrepareResult prepare(HostValidator<ApplicationId> hostValidator, DeployLogger logger, PrepareParams params,
+ Optional<ApplicationSet> currentActiveApplicationSet, Path tenantPath,
+ Instant now, File serverDbSessionDir, ApplicationPackage applicationPackage,
+ SessionZooKeeperClient sessionZooKeeperClient) {
Preparation preparation = new Preparation(hostValidator, logger, params, currentActiveApplicationSet,
tenantPath, serverDbSessionDir, applicationPackage, sessionZooKeeperClient);
@@ -313,8 +313,8 @@ public class SessionPreparer {
checkTimeout("distribute files");
}
- ConfigChangeActions result() {
- return prepareResult.getConfigChangeActions();
+ PrepareResult result() {
+ return prepareResult;
}
private List<ContainerEndpoint> readEndpointsIfNull(List<ContainerEndpoint> endpoints) {
@@ -352,7 +352,7 @@ public class SessionPreparer {
}
/** The result of preparation over all model versions */
- private static class PrepareResult {
+ static class PrepareResult {
private final AllocatedHosts allocatedHosts;
private final ImmutableList<PreparedModelsBuilder.PreparedModelResult> results;
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 0f6ba37cf51..9b337c97d88 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
@@ -164,7 +164,8 @@ public class SessionRepository {
ConfigChangeActions actions = sessionPreparer.prepare(applicationRepo.getHostValidator(), logger, params,
currentActiveApplicationSet, tenantPath, now,
getSessionAppDir(sessionId),
- session.getApplicationPackage(), sessionZooKeeperClient);
+ session.getApplicationPackage(), sessionZooKeeperClient)
+ .getConfigChangeActions();
session.setPrepared();
waiter.awaitCompletion(params.getTimeoutBudget().timeLeft());
return actions;
@@ -571,11 +572,18 @@ public class SessionRepository {
throw new IllegalArgumentException(sourceDir.getAbsolutePath() + " is not a directory");
// Copy app atomically: Copy to a temp dir and move to destination
- java.nio.file.Path tempDestinationDir = Files.createTempDirectory(destinationDir.getParentFile().toPath(), "app-package");
- log.log(Level.FINE, "Copying dir " + sourceDir.getAbsolutePath() + " to " + tempDestinationDir.toFile().getAbsolutePath());
- IOUtils.copyDirectory(sourceDir, tempDestinationDir.toFile());
- log.log(Level.FINE, "Moving " + tempDestinationDir + " to " + destinationDir.getAbsolutePath());
- Files.move(tempDestinationDir, destinationDir.toPath(), StandardCopyOption.ATOMIC_MOVE);
+ java.nio.file.Path tempDestinationDir = null;
+ try {
+ tempDestinationDir = Files.createTempDirectory(destinationDir.getParentFile().toPath(), "app-package");
+ log.log(Level.FINE, "Copying dir " + sourceDir.getAbsolutePath() + " to " + tempDestinationDir.toFile().getAbsolutePath());
+ IOUtils.copyDirectory(sourceDir, tempDestinationDir.toFile());
+ log.log(Level.FINE, "Moving " + tempDestinationDir + " to " + destinationDir.getAbsolutePath());
+ Files.move(tempDestinationDir, destinationDir.toPath(), StandardCopyOption.ATOMIC_MOVE);
+ } finally {
+ // In case some of the operations above fail
+ if (tempDestinationDir != null)
+ IOUtils.recursiveDeleteDir(tempDestinationDir.toFile());
+ }
}
/**
@@ -614,7 +622,7 @@ public class SessionRepository {
return Optional.empty();
}
ApplicationId applicationId = sessionZKClient.readApplicationId();
- log.log(Level.INFO, "Creating local session for session id " + sessionId);
+ log.log(Level.INFO, "Creating local session for tenant '" + tenantName + "' with session id " + sessionId);
LocalSession localSession = createLocalSession(sessionDir, applicationId, sessionId);
addLocalSession(localSession);
return Optional.of(localSession);
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 807629a2148..1b9527f4376 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
@@ -150,8 +150,8 @@ public class SessionZooKeeperClient {
}
public ApplicationId readApplicationId() {
- if ( ! configCurator.exists(applicationIdPath())) return ApplicationId.defaultId();
- return ApplicationId.fromSerializedForm(configCurator.getData(applicationIdPath()));
+ String idString = configCurator.getData(applicationIdPath());
+ return idString == null ? null : ApplicationId.fromSerializedForm(idString);
}
void writeApplicationPackageReference(FileReference applicationPackageReference) {
@@ -214,7 +214,7 @@ public class SessionZooKeeperClient {
}
public ZooKeeperDeployer createDeployer(DeployLogger logger) {
- ZooKeeperClient zkClient = new ZooKeeperClient(configCurator, logger, true, sessionPath);
+ ZooKeeperClient zkClient = new ZooKeeperClient(configCurator, logger, sessionPath);
return new ZooKeeperDeployer(zkClient);
}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/zookeeper/ZKApplicationPackage.java b/configserver/src/main/java/com/yahoo/vespa/config/server/zookeeper/ZKApplicationPackage.java
index 685856d5cf8..11cec9efd95 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/zookeeper/ZKApplicationPackage.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/zookeeper/ZKApplicationPackage.java
@@ -93,8 +93,7 @@ public class ZKApplicationPackage implements ApplicationPackage {
try {
return PreGeneratedFileRegistry.importRegistry(zkApplication.getDataReader(fileRegistryNode));
} catch (Exception e) {
- throw new RuntimeException("Could not determine which files to distribute. " +
- "Please try redeploying the application", e);
+ throw new RuntimeException("Could not determine which files to distribute", e);
}
}
diff --git a/configserver/src/test/apps/content/schemas/music.sd b/configserver/src/test/apps/content/schemas/music.sd
new file mode 100644
index 00000000000..7670e78f22b
--- /dev/null
+++ b/configserver/src/test/apps/content/schemas/music.sd
@@ -0,0 +1,50 @@
+# Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+# A basic search definition - called music, should be saved to music.sd
+search music {
+
+ # It contains one document type only - called music as well
+ document music {
+
+ field title type string {
+ indexing: summary | index # How this field should be indexed
+ # index-to: title, default # Create two indexes
+ weight: 75 # Ranking importancy of this field, used by the built in nativeRank feature
+ }
+
+ field artist type string {
+ indexing: summary | attribute | index
+ # index-to: artist, default
+
+ weight: 25
+ }
+
+ field year type int {
+ indexing: summary | attribute
+ }
+
+ # Increase query
+ field popularity type int {
+ indexing: summary | attribute
+ }
+
+ field url type uri {
+ indexing: summary | index
+ }
+
+ }
+
+ rank-profile default inherits default {
+ first-phase {
+ expression: nativeRank(title,artist) + attribute(popularity)
+ }
+
+ }
+
+ rank-profile textmatch inherits default {
+ first-phase {
+ expression: nativeRank(title,artist)
+ }
+
+ }
+
+}
diff --git a/configserver/src/test/apps/content/services.xml b/configserver/src/test/apps/content/services.xml
new file mode 100644
index 00000000000..509d7786be0
--- /dev/null
+++ b/configserver/src/test/apps/content/services.xml
@@ -0,0 +1,38 @@
+<?xml version="1.0" encoding="utf-8" ?>
+<!-- Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root. -->
+<services version="1.0">
+
+ <admin version="2.0">
+ <adminserver hostalias="node1"/>
+ <logserver hostalias="node1" />
+ </admin>
+
+ <content version="1.0">
+ <redundancy>2</redundancy>
+ <documents>
+ <document type="music" mode="index"/>
+ </documents>
+ <nodes>
+ <node hostalias="node1" distribution-key="0"/>
+ </nodes>
+
+ </content>
+
+ <container version="1.0">
+ <document-processing compressdocuments="true">
+ <chain id="ContainerWrapperTest">
+ <documentprocessor id="com.yahoo.vespa.config.AppleDocProc"/>
+ </chain>
+ </document-processing>
+
+ <config name="project.specific">
+ <value>someval</value>
+ </config>
+
+ <nodes>
+ <node hostalias="node1" />
+ </nodes>
+
+ </container>
+
+</services>
diff --git a/configserver/src/test/apps/content2/schemas/music.sd b/configserver/src/test/apps/content2/schemas/music.sd
new file mode 100644
index 00000000000..7670e78f22b
--- /dev/null
+++ b/configserver/src/test/apps/content2/schemas/music.sd
@@ -0,0 +1,50 @@
+# Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+# A basic search definition - called music, should be saved to music.sd
+search music {
+
+ # It contains one document type only - called music as well
+ document music {
+
+ field title type string {
+ indexing: summary | index # How this field should be indexed
+ # index-to: title, default # Create two indexes
+ weight: 75 # Ranking importancy of this field, used by the built in nativeRank feature
+ }
+
+ field artist type string {
+ indexing: summary | attribute | index
+ # index-to: artist, default
+
+ weight: 25
+ }
+
+ field year type int {
+ indexing: summary | attribute
+ }
+
+ # Increase query
+ field popularity type int {
+ indexing: summary | attribute
+ }
+
+ field url type uri {
+ indexing: summary | index
+ }
+
+ }
+
+ rank-profile default inherits default {
+ first-phase {
+ expression: nativeRank(title,artist) + attribute(popularity)
+ }
+
+ }
+
+ rank-profile textmatch inherits default {
+ first-phase {
+ expression: nativeRank(title,artist)
+ }
+
+ }
+
+}
diff --git a/configserver/src/test/apps/content2/services.xml b/configserver/src/test/apps/content2/services.xml
new file mode 100644
index 00000000000..509d7786be0
--- /dev/null
+++ b/configserver/src/test/apps/content2/services.xml
@@ -0,0 +1,38 @@
+<?xml version="1.0" encoding="utf-8" ?>
+<!-- Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root. -->
+<services version="1.0">
+
+ <admin version="2.0">
+ <adminserver hostalias="node1"/>
+ <logserver hostalias="node1" />
+ </admin>
+
+ <content version="1.0">
+ <redundancy>2</redundancy>
+ <documents>
+ <document type="music" mode="index"/>
+ </documents>
+ <nodes>
+ <node hostalias="node1" distribution-key="0"/>
+ </nodes>
+
+ </content>
+
+ <container version="1.0">
+ <document-processing compressdocuments="true">
+ <chain id="ContainerWrapperTest">
+ <documentprocessor id="com.yahoo.vespa.config.AppleDocProc"/>
+ </chain>
+ </document-processing>
+
+ <config name="project.specific">
+ <value>someval</value>
+ </config>
+
+ <nodes>
+ <node hostalias="node1" />
+ </nodes>
+
+ </container>
+
+</services>
diff --git a/configserver/src/test/apps/zkapp/deployment.xml b/configserver/src/test/apps/zkapp/deployment.xml
index dd47299f578..2944b8220f3 100644
--- a/configserver/src/test/apps/zkapp/deployment.xml
+++ b/configserver/src/test/apps/zkapp/deployment.xml
@@ -2,7 +2,7 @@
<!-- Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root. -->
<deployment version='1.0'>
<test/>
- <prod global-service-id='mydisc'>+
+ <prod global-service-id='mydisc'>
<region active='true'>us-east</region>
</prod>
</deployment>
diff --git a/configserver/src/test/apps/zkapp/services.xml b/configserver/src/test/apps/zkapp/services.xml
index 672f058bc0a..58ecf41707d 100644
--- a/configserver/src/test/apps/zkapp/services.xml
+++ b/configserver/src/test/apps/zkapp/services.xml
@@ -11,7 +11,7 @@
</admin>
<container version="1.0">
- <documentapi/>
+ <document-api/>
<search/>
<nodes>
<node hostalias="node1"/>
@@ -21,7 +21,7 @@
<content version="1.0">
<redundancy>1</redundancy>
<documents>
- <document name="music" mode="index"/>
+ <document type="music" mode="index"/>
</documents>
<nodes>
<node hostalias="node1" distribution-key="0"/>
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 a4b2bfb8902..50a8cac2837 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
@@ -121,14 +121,15 @@ public class ApplicationRepositoryTest {
public void setup(FlagSource flagSource) throws IOException {
Curator curator = new MockCurator();
configCurator = ConfigCurator.create(curator);
+ ConfigserverConfig configserverConfig = new ConfigserverConfig.Builder()
+ .payloadCompressionType(ConfigserverConfig.PayloadCompressionType.Enum.UNCOMPRESSED)
+ .configServerDBDir(temporaryFolder.newFolder().getAbsolutePath())
+ .configDefinitionsDir(temporaryFolder.newFolder().getAbsolutePath())
+ .fileReferencesDir(temporaryFolder.newFolder().getAbsolutePath())
+ .build();
TestComponentRegistry componentRegistry = new TestComponentRegistry.Builder()
.curator(curator)
- .configServerConfig(new ConfigserverConfig.Builder()
- .payloadCompressionType(ConfigserverConfig.PayloadCompressionType.Enum.UNCOMPRESSED)
- .configServerDBDir(temporaryFolder.newFolder().getAbsolutePath())
- .configDefinitionsDir(temporaryFolder.newFolder().getAbsolutePath())
- .fileReferencesDir(temporaryFolder.newFolder().getAbsolutePath())
- .build())
+ .configServerConfig(configserverConfig)
.flagSource(flagSource)
.clock(clock)
.build();
@@ -142,7 +143,11 @@ public class ApplicationRepositoryTest {
applicationRepository = new ApplicationRepository(tenantRepository,
provisioner,
orchestrator,
- clock);
+ configserverConfig,
+ new MockLogRetriever(),
+ clock,
+ new MockTesterClient(),
+ new NullMetric());
timeoutBudget = new TimeoutBudget(clock, Duration.ofSeconds(60));
}
@@ -234,14 +239,6 @@ public class ApplicationRepositoryTest {
assertEquals(200, response.getStatus());
}
- @Test(expected = IllegalArgumentException.class)
- public void refuseToGetLogsFromHostnameNotInApplication() {
- applicationRepository = createApplicationRepository();
- deployApp(testAppLogServerWithContainer);
- HttpResponse response = applicationRepository.getLogs(applicationId(), Optional.of("host123.fake.yahoo.com"), "");
- assertEquals(200, response.getStatus());
- }
-
@Test
public void deleteUnusedFileReferences() throws IOException {
File fileReferencesDir = temporaryFolder.newFolder();
@@ -368,7 +365,7 @@ public class ApplicationRepositoryTest {
deployment3.get().prepare(); // session 4 (not activated)
LocalSession deployment3session = ((com.yahoo.vespa.config.server.deploy.Deployment) deployment3.get()).session();
- assertNotEquals(activeSessionId, deployment3session);
+ assertNotEquals(activeSessionId, deployment3session.getSessionId());
// No change to active session id
assertEquals(activeSessionId, tester.tenant().getApplicationRepo().requireActiveSessionOf(tester.applicationId()));
SessionRepository sessionRepository = tester.tenant().getSessionRepository();
@@ -603,7 +600,7 @@ public class ApplicationRepositoryTest {
assertEquals(1330, config2.intval());
assertTrue(requestHandler.hasApplication(applicationId(), Optional.of(vespaVersion)));
- assertThat(requestHandler.resolveApplicationId("doesnotexist"), Is.is(ApplicationId.defaultId()));
+ assertNull(requestHandler.resolveApplicationId("doesnotexist"));
assertThat(requestHandler.resolveApplicationId("mytesthost"), Is.is(new ApplicationId.Builder()
.tenant(tenant1)
.applicationName("testapp").build())); // Host set in application package.
@@ -681,7 +678,7 @@ public class ApplicationRepositoryTest {
}
private ApplicationId applicationId() {
- return ApplicationId.from(tenant1, ApplicationName.from("testapp"), InstanceName.defaultName());
+ return applicationId(tenant1);
}
private ApplicationId applicationId(TenantName tenantName) {
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/TestComponentRegistry.java b/configserver/src/test/java/com/yahoo/vespa/config/server/TestComponentRegistry.java
index f03550c0a80..68dd5396cf1 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/TestComponentRegistry.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/TestComponentRegistry.java
@@ -51,7 +51,7 @@ public class TestComponentRegistry implements GlobalComponentRegistry {
private final TenantListener tenantListener;
private final PermanentApplicationPackage permanentApplicationPackage;
private final HostRegistries hostRegistries;
- private final FileDistributionFactory fileDistributionProvider;
+ private final FileDistributionFactory fileDistributionFactory;
private final ModelFactoryRegistry modelFactoryRegistry;
private final Optional<Provisioner> hostProvisioner;
private final Zone zone;
@@ -65,7 +65,7 @@ public class TestComponentRegistry implements GlobalComponentRegistry {
private TestComponentRegistry(Curator curator, ConfigCurator configCurator, Metrics metrics,
ModelFactoryRegistry modelFactoryRegistry,
PermanentApplicationPackage permanentApplicationPackage,
- FileDistributionFactory fileDistributionProvider,
+ FileDistributionFactory fileDistributionFactory,
HostRegistries hostRegistries,
ConfigserverConfig configserverConfig,
SessionPreparer sessionPreparer,
@@ -86,7 +86,7 @@ public class TestComponentRegistry implements GlobalComponentRegistry {
this.defRepo = defRepo;
this.permanentApplicationPackage = permanentApplicationPackage;
this.hostRegistries = hostRegistries;
- this.fileDistributionProvider = fileDistributionProvider;
+ this.fileDistributionFactory = fileDistributionFactory;
this.modelFactoryRegistry = modelFactoryRegistry;
this.hostProvisioner = hostProvisioner;
this.sessionPreparer = sessionPreparer;
@@ -247,6 +247,6 @@ public class TestComponentRegistry implements GlobalComponentRegistry {
return secretStore;
}
- public FileDistributionFactory getFileDistributionProvider() { return fileDistributionProvider; }
+ public FileDistributionFactory getFileDistributionFactory() { return fileDistributionFactory; }
}
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/configdefs/a.def b/configserver/src/test/java/com/yahoo/vespa/config/server/configdefs/a.def
index bf99c65ec14..e2f22a38fc3 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/configdefs/a.def
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/configdefs/a.def
@@ -29,7 +29,7 @@ config[].role string
config[].id reference
## Wether the NC should start the corresponding role using the
-## slavewrapper utility application or not.
+## wrapper utility application or not.
config[].usewrapper bool default=false
routingtable[].hop[].name string
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/deploy/ZooKeeperClientTest.java b/configserver/src/test/java/com/yahoo/vespa/config/server/deploy/ZooKeeperClientTest.java
index 8394611737e..a4fce5e37ba 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/deploy/ZooKeeperClientTest.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/deploy/ZooKeeperClientTest.java
@@ -53,7 +53,7 @@ public class ZooKeeperClientTest {
@Before
public void setupZK() throws IOException {
zk = ConfigCurator.create(new MockCurator());
- ZooKeeperClient zkc = new ZooKeeperClient(zk, new BaseDeployLogger(), true, Path.fromString(appPath));
+ ZooKeeperClient zkc = new ZooKeeperClient(zk, new BaseDeployLogger(), Path.fromString(appPath));
ApplicationPackage app = FilesApplicationPackage.fromFileWithDeployData(new File("src/test/apps/zkfeed"),
new DeployData("foo",
"/bar/baz",
@@ -85,7 +85,7 @@ public class ZooKeeperClientTest {
ConfigCurator zk = ConfigCurator.create(new MockCurator());
BaseDeployLogger logger = new BaseDeployLogger();
long generation = 1L;
- ZooKeeperClient zooKeeperClient = new ZooKeeperClient(zk, logger, true, Path.fromString("/1"));
+ ZooKeeperClient zooKeeperClient = new ZooKeeperClient(zk, logger, Path.fromString("/1"));
zooKeeperClient.setupZooKeeper();
String appPath = "/";
assertThat(zk.getChildren(appPath).size(), is(1));
@@ -120,7 +120,7 @@ public class ZooKeeperClientTest {
ConfigCurator zk = ConfigCurator.create(new MockCurator());
BaseDeployLogger logger = new BaseDeployLogger();
Path app = Path.fromString("/1");
- ZooKeeperClient zooKeeperClient = new ZooKeeperClient(zk, logger, true, app);
+ ZooKeeperClient zooKeeperClient = new ZooKeeperClient(zk, logger, app);
zooKeeperClient.setupZooKeeper();
String currentAppPath = app.getAbsolute();
@@ -191,7 +191,7 @@ public class ZooKeeperClientTest {
ConfigCurator zk = ConfigCurator.create(new MockCurator());
BaseDeployLogger logger = new BaseDeployLogger();
Path app = Path.fromString("/1");
- ZooKeeperClient zooKeeperClient = new ZooKeeperClient(zk, logger, true, app);
+ ZooKeeperClient zooKeeperClient = new ZooKeeperClient(zk, logger, app);
zooKeeperClient.setupZooKeeper();
HostSpec host1 = new HostSpec("host1.yahoo.com", Collections.emptyList(), Optional.empty());
HostSpec host2 = new HostSpec("host2.yahoo.com", Collections.emptyList(), Optional.empty());
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/deploy/ZooKeeperDeployerTest.java b/configserver/src/test/java/com/yahoo/vespa/config/server/deploy/ZooKeeperDeployerTest.java
index 4825ccc1328..641fbe5bf41 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/deploy/ZooKeeperDeployerTest.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/deploy/ZooKeeperDeployerTest.java
@@ -48,7 +48,7 @@ public class ZooKeeperDeployerTest {
public void deploy(ApplicationPackage applicationPackage, ConfigCurator configCurator, Path appPath) throws IOException {
MockDeployLogger logger = new MockDeployLogger();
- ZooKeeperClient client = new ZooKeeperClient(configCurator, logger, true, appPath);
+ ZooKeeperClient client = new ZooKeeperClient(configCurator, logger, appPath);
ZooKeeperDeployer deployer = new ZooKeeperDeployer(client);
deployer.deploy(applicationPackage, Collections.singletonMap(new Version(1, 0, 0), new MockFileRegistry()), AllocatedHosts.withHosts(Collections.emptySet()));
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/filedistribution/MockFileDistributionFactory.java b/configserver/src/test/java/com/yahoo/vespa/config/server/filedistribution/MockFileDistributionFactory.java
index 78729156b93..af55dc6a90e 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/filedistribution/MockFileDistributionFactory.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/filedistribution/MockFileDistributionFactory.java
@@ -10,16 +10,13 @@ import java.io.File;
*/
public class MockFileDistributionFactory extends FileDistributionFactory {
- public final MockFileDistributionProvider mockFileDistributionProvider;
-
public MockFileDistributionFactory(ConfigserverConfig configserverConfig) {
super(configserverConfig);
- mockFileDistributionProvider = new MockFileDistributionProvider(new File(configserverConfig.fileReferencesDir()));
}
@Override
public com.yahoo.vespa.config.server.filedistribution.FileDistributionProvider createProvider(File applicationFile) {
- return mockFileDistributionProvider;
+ return new MockFileDistributionProvider(applicationFile, new File(configserverConfig.fileReferencesDir()));
}
}
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/filedistribution/MockFileDistributionProvider.java b/configserver/src/test/java/com/yahoo/vespa/config/server/filedistribution/MockFileDistributionProvider.java
new file mode 100644
index 00000000000..45e00e2ece8
--- /dev/null
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/filedistribution/MockFileDistributionProvider.java
@@ -0,0 +1,22 @@
+// Copyright Verizon Media. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.vespa.config.server.filedistribution;
+
+import com.yahoo.config.model.api.FileDistribution;
+
+import java.io.File;
+
+/**
+ * @author hmusum
+ */
+public class MockFileDistributionProvider extends FileDistributionProvider {
+
+ public MockFileDistributionProvider(File applicationDir, File fileReferencesDir) {
+ super(new MockFileRegistry(applicationDir, fileReferencesDir.toPath()),
+ new MockFileDistribution(fileReferencesDir));
+ }
+
+ public FileDistribution getFileDistribution() {
+ return super.getFileDistribution();
+ }
+
+}
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/filedistribution/MockFileRegistry.java b/configserver/src/test/java/com/yahoo/vespa/config/server/filedistribution/MockFileRegistry.java
new file mode 100644
index 00000000000..343e0c50520
--- /dev/null
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/filedistribution/MockFileRegistry.java
@@ -0,0 +1,49 @@
+// Copyright Verizon Media. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.vespa.config.server.filedistribution;
+
+import com.yahoo.config.FileReference;
+import com.yahoo.config.application.api.FileRegistry;
+import com.yahoo.net.HostName;
+
+import java.io.File;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A file registry for config server tests
+ *
+ * @author hmusum
+ */
+public class MockFileRegistry implements FileRegistry {
+
+ private final List<Entry> entries = new ArrayList<>();
+ private final AddFileInterface addFileInterface;
+
+ public MockFileRegistry(File applicationDir, Path rootPath) {
+ FileDirectory fileDirectory = new FileDirectory(rootPath.toFile());
+ this.addFileInterface = new ApplicationFileManager(applicationDir, fileDirectory);
+ }
+
+ public FileReference addFile(String relativePath) {
+ if (relativePath.isEmpty())
+ relativePath = "./";
+ addFileInterface.addFile(relativePath);
+
+ FileReference fileReference = new FileReference(relativePath);
+ entries.add(new Entry(relativePath, fileReference));
+ return fileReference;
+ }
+
+ @Override
+ public String fileSourceHost() { return HostName.getLocalhost(); }
+
+ public List<Entry> export() { return entries; }
+
+ @Override
+ public FileReference addUri(String uri) {
+ throw new IllegalArgumentException("FileReference addUri(String uri) is not implemented for " + getClass().getCanonicalName());
+ }
+
+}
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 0b9a780d9e1..bd306880039 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
@@ -1,8 +1,6 @@
// 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.http;
-import com.yahoo.config.application.api.ApplicationFile;
-import com.yahoo.config.application.api.ApplicationPackage;
import com.yahoo.config.provision.ApplicationId;
import com.yahoo.config.provision.Capacity;
import com.yahoo.config.provision.ClusterSpec;
@@ -10,22 +8,14 @@ import com.yahoo.config.provision.HostFilter;
import com.yahoo.config.provision.HostSpec;
import com.yahoo.config.provision.ProvisionLogger;
import com.yahoo.config.provision.Provisioner;
-import com.yahoo.config.provision.TenantName;
import com.yahoo.container.jdisc.HttpRequest;
import com.yahoo.container.jdisc.HttpResponse;
-import com.yahoo.path.Path;
import com.yahoo.transaction.NestedTransaction;
-import com.yahoo.transaction.Transaction;
-import com.yahoo.vespa.config.server.session.DummyTransaction;
-import com.yahoo.vespa.config.server.session.LocalSession;
-import com.yahoo.vespa.config.server.session.MockSessionZKClient;
-import com.yahoo.vespa.config.server.session.Session;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
-import java.time.Instant;
import java.util.Collection;
import java.util.List;
import java.util.Map;
@@ -79,52 +69,6 @@ public class SessionHandlerTest {
return baos.toString(StandardCharsets.UTF_8);
}
- public static class MockLocalSession extends LocalSession {
-
- public Session.Status status;
- private Instant createTime = Instant.now();
- private ApplicationId applicationId;
-
- public MockLocalSession(long sessionId, ApplicationPackage app) {
- super(TenantName.defaultName(), sessionId, app, new MockSessionZKClient(app), null);
- }
-
- public MockLocalSession(long sessionId, ApplicationPackage app, ApplicationId applicationId) {
- this(sessionId, app);
- this.applicationId = applicationId;
- }
-
- public void setStatus(Session.Status status) {
- this.status = status;
- }
-
- @Override
- public Session.Status getStatus() {
- return this.status;
- }
-
- @Override
- public Transaction createActivateTransaction() {
- return new DummyTransaction().add((DummyTransaction.RunnableOperation) () -> status = Status.ACTIVATE);
- }
-
- @Override
- public ApplicationFile getApplicationFile(Path relativePath, Mode mode) {
- return this.applicationPackage.getFile(relativePath);
- }
-
- @Override
- public ApplicationId getApplicationId() {
- return applicationId;
- }
-
- @Override
- public Instant getCreateTime() {
- return createTime;
- }
-
- }
-
public enum Cmd {
PREPARED("prepared"),
ACTIVE("active"),
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/ApplicationContentHandlerTest.java b/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/ApplicationContentHandlerTest.java
index 97085416073..8bf5215a696 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/ApplicationContentHandlerTest.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/ApplicationContentHandlerTest.java
@@ -1,7 +1,6 @@
// 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.http.v2;
-import com.yahoo.config.model.application.provider.FilesApplicationPackage;
import com.yahoo.config.provision.ApplicationId;
import com.yahoo.config.provision.TenantName;
import com.yahoo.config.provision.Zone;
@@ -12,6 +11,8 @@ import com.yahoo.vespa.config.server.ApplicationRepository;
import com.yahoo.vespa.config.server.TestComponentRegistry;
import com.yahoo.vespa.config.server.application.OrchestratorMock;
import com.yahoo.vespa.config.server.http.ContentHandlerTestBase;
+import com.yahoo.vespa.config.server.session.LocalSession;
+import com.yahoo.vespa.config.server.session.PrepareParams;
import com.yahoo.vespa.config.server.session.Session;
import com.yahoo.vespa.config.server.tenant.Tenant;
import com.yahoo.vespa.config.server.tenant.TenantRepository;
@@ -30,47 +31,42 @@ import static org.junit.Assert.assertThat;
* @author Ulf Lilleengen
*/
public class ApplicationContentHandlerTest extends ContentHandlerTestBase {
+
+ private static final File testApp = new File("src/test/apps/content");
+ private static final File testApp2 = new File("src/test/apps/content2");
+
private final TestComponentRegistry componentRegistry = new TestComponentRegistry.Builder().build();
private final Clock clock = componentRegistry.getClock();
+ private final TenantName tenantName1 = TenantName.from("mofet");
+ private final TenantName tenantName2 = TenantName.from("bla");
+ private final String baseServer = "http://foo:1337";
+
+ private final ApplicationId appId1 = new ApplicationId.Builder().tenant(tenantName1).applicationName("foo").instanceName("quux").build();
+ private final ApplicationId appId2 = new ApplicationId.Builder().tenant(tenantName2).applicationName("foo").instanceName("quux").build();
+
+ private TenantRepository tenantRepository;
+ private ApplicationRepository applicationRepository;
private ApplicationHandler handler;
- private TenantName tenantName1 = TenantName.from("mofet");
- private TenantName tenantName2 = TenantName.from("bla");
- private String baseServer = "http://foo:1337";
-
- private ApplicationId idTenant1 = new ApplicationId.Builder()
- .tenant(tenantName1)
- .applicationName("foo").instanceName("quux").build();
- private ApplicationId idTenant2 = new ApplicationId.Builder()
- .tenant(tenantName2)
- .applicationName("foo").instanceName("quux").build();
- private MockLocalSession session2;
@Before
public void setupHandler() {
- TenantRepository tenantRepository = new TenantRepository(componentRegistry, false);
+ tenantRepository = new TenantRepository(componentRegistry, false);
tenantRepository.addTenant(tenantName1);
tenantRepository.addTenant(tenantName2);
- session2 = new MockLocalSession(2, FilesApplicationPackage.fromFile(new File("src/test/apps/content")));
- Tenant tenant1 = tenantRepository.getTenant(tenantName1);
- tenant1.getSessionRepository().addLocalSession(session2);
- tenant1.getApplicationRepo().createApplication(idTenant1);
- tenant1.getApplicationRepo().createPutTransaction(idTenant1, 2).commit();
+ applicationRepository = new ApplicationRepository(tenantRepository,
+ new MockProvisioner(),
+ new OrchestratorMock(),
+ clock);
- MockLocalSession session3 = new MockLocalSession(3, FilesApplicationPackage.fromFile(new File("src/test/apps/content2")));
- Tenant tenant2 = tenantRepository.getTenant(tenantName2);
- tenant2.getSessionRepository().addLocalSession(session3);
- tenant2.getApplicationRepo().createApplication(idTenant2);
- tenant2.getApplicationRepo().createPutTransaction(idTenant2, 3).commit();
+ applicationRepository.deploy(testApp, prepareParams(appId1));
+ applicationRepository.deploy(testApp2, prepareParams(appId2));
handler = new ApplicationHandler(ApplicationHandler.testOnlyContext(),
Zone.defaultZone(),
- new ApplicationRepository(tenantRepository,
- new MockProvisioner(),
- new OrchestratorMock(),
- clock));
- pathPrefix = createPath(idTenant1, Zone.defaultZone());
+ applicationRepository);
+ pathPrefix = createPath(appId1, Zone.defaultZone());
baseUrl = baseServer + pathPrefix;
}
@@ -103,16 +99,17 @@ public class ApplicationContentHandlerTest extends ContentHandlerTestBase {
@Test
public void require_that_multiple_tenants_are_handled() throws IOException {
assertContent("/test.txt", "foo\n");
- pathPrefix = createPath(idTenant2, Zone.defaultZone());
+ pathPrefix = createPath(appId2, Zone.defaultZone());
baseUrl = baseServer + pathPrefix;
assertContent("/test.txt", "bar\n");
}
@Test
public void require_that_get_does_not_set_write_flag() throws IOException {
- session2.status = Session.Status.PREPARE;
+ Tenant tenant1 = tenantRepository.getTenant(tenantName1);
+ LocalSession session = applicationRepository.getActiveLocalSession(tenant1, appId1);
assertContent("/test.txt", "foo\n");
- assertThat(session2.status, is(Session.Status.PREPARE));
+ assertThat(session.getStatus(), is(Session.Status.ACTIVATE));
}
private void assertNotFound(HttpRequest request) {
@@ -126,4 +123,9 @@ public class ApplicationContentHandlerTest extends ContentHandlerTestBase {
HttpRequest request = HttpRequest.createTestRequest(baseUrl + path, method);
return handler.handle(request);
}
+
+ private PrepareParams prepareParams(ApplicationId applicationId) {
+ return new PrepareParams.Builder().applicationId(applicationId).build();
+ }
+
}
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 eca32cd364a..1a558f89284 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
@@ -353,7 +353,9 @@ public class ApplicationHandlerTest {
HttpResponse response = createApplicationHandler().handle(HttpRequest.createTestRequest(url, GET));
assertEquals(200, response.getStatus());
String renderedString = SessionHandlerTest.getRenderedString(response);
- assertEquals("{\"generation\":" + expectedGeneration + ",\"modelVersions\":[\"" + expectedVersion.toFullString() + "\"]}", renderedString);
+ assertEquals("{\"generation\":" + expectedGeneration +
+ ",\"applicationPackageFileReference\":\"\"" +
+ ",\"modelVersions\":[\"" + expectedVersion.toFullString() + "\"]}", renderedString);
}
private void assertApplicationExists(ApplicationId applicationId, Zone zone) throws IOException {
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/HostHandlerTest.java b/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/HostHandlerTest.java
index 20e4ef56166..364e7372e20 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/HostHandlerTest.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/HostHandlerTest.java
@@ -1,9 +1,6 @@
// 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.http.v2;
-import com.yahoo.config.application.api.ApplicationPackage;
-import com.yahoo.config.model.NullConfigModelRegistry;
-import com.yahoo.config.model.application.provider.FilesApplicationPackage;
import com.yahoo.config.provision.ApplicationId;
import com.yahoo.config.provision.ApplicationName;
import com.yahoo.config.provision.InstanceName;
@@ -15,28 +12,25 @@ import com.yahoo.jdisc.Response;
import com.yahoo.vespa.config.server.ApplicationRepository;
import com.yahoo.vespa.config.server.TestComponentRegistry;
import com.yahoo.vespa.config.server.application.OrchestratorMock;
-import com.yahoo.vespa.config.server.host.HostRegistry;
import com.yahoo.vespa.config.server.http.HandlerTest;
import com.yahoo.vespa.config.server.http.HttpErrorResponse;
import com.yahoo.vespa.config.server.http.SessionHandlerTest;
-import com.yahoo.vespa.config.server.modelfactory.ModelFactoryRegistry;
-import com.yahoo.vespa.config.server.session.MockSessionZKClient;
-import com.yahoo.vespa.config.server.session.RemoteSession;
+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.model.VespaModelFactory;
import org.junit.Before;
import org.junit.Test;
import java.io.File;
import java.io.IOException;
import java.time.Clock;
-import java.util.Collections;
+
+import static com.yahoo.jdisc.http.HttpRequest.Method;
+import static com.yahoo.vespa.config.server.http.HandlerTest.assertHttpStatusCodeErrorCodeAndMessage;
/**
* @author hmusum
*/
-// TODO: Try to move testing to ApplicationRepositoryTest and avoid all the low-level setup code here
public class HostHandlerTest {
private static final String urlPrefix = "http://myhost:14000/application/v2/host/";
@@ -44,57 +38,39 @@ public class HostHandlerTest {
private HostHandler handler;
private final static TenantName mytenant = TenantName.from("mytenant");
- private final static String hostname = "testhost";
private final static Zone zone = Zone.defaultZone();
private TenantRepository tenantRepository;
-
- static void addMockApplication(Tenant tenant, ApplicationId applicationId, long sessionId) {
- tenant.getApplicationRepo().createApplication(applicationId);
- tenant.getApplicationRepo().createPutTransaction(applicationId, sessionId).commit();
- ApplicationPackage app = FilesApplicationPackage.fromFile(testApp);
- tenant.getSessionRepository().addLocalSession(new SessionHandlerTest.MockLocalSession(sessionId, app, applicationId));
- TestComponentRegistry componentRegistry = new TestComponentRegistry.Builder()
- .modelFactoryRegistry(new ModelFactoryRegistry(Collections.singletonList(new VespaModelFactory(new NullConfigModelRegistry()))))
- .build();
- tenant.getSessionRepository().addRemoteSession(new RemoteSession(tenant.getName(), sessionId, componentRegistry, new MockSessionZKClient(app)));
- }
+ private ApplicationRepository applicationRepository;
@Before
public void setup() {
- final HostRegistry<TenantName> hostRegistry = new HostRegistry<>();
- hostRegistry.update(mytenant, Collections.singletonList(hostname));
TestComponentRegistry componentRegistry = new TestComponentRegistry.Builder()
.zone(zone)
.build();
- tenantRepository = new TenantRepository(componentRegistry, false);
+ tenantRepository = new TenantRepository(componentRegistry);
tenantRepository.addTenant(mytenant);
- Tenant tenant = tenantRepository.getTenant(mytenant);
- HostRegistry<ApplicationId> applicationHostRegistry = tenant.getApplicationRepo().getApplicationHostRegistry();
- applicationHostRegistry.update(ApplicationId.from(mytenant, ApplicationName.defaultName(), InstanceName.defaultName()), Collections.singletonList(hostname));
- ApplicationRepository applicationRepository = new ApplicationRepository(tenantRepository,
- new SessionHandlerTest.MockProvisioner(),
- new OrchestratorMock(),
- Clock.systemUTC());
+ applicationRepository = new ApplicationRepository(tenantRepository,
+ new SessionHandlerTest.MockProvisioner(),
+ new OrchestratorMock(),
+ Clock.systemUTC());
handler = new HostHandler(HostHandler.testOnlyContext(), applicationRepository);
}
@Test
public void require_correct_tenant_and_application_for_hostname() throws Exception {
- long sessionId = 1;
- ApplicationId id = ApplicationId.from(mytenant, ApplicationName.defaultName(), InstanceName.defaultName());
- addMockApplication(tenantRepository.getTenant(mytenant), id, sessionId);
- assertApplicationForHost(hostname, mytenant, id, zone);
+ ApplicationId applicationId = applicationId();
+ applicationRepository.deploy(testApp, new PrepareParams.Builder().applicationId(applicationId).build());
+ Tenant tenant = tenantRepository.getTenant(mytenant);
+ String hostname = applicationRepository.getCurrentActiveApplicationSet(tenant, applicationId).get().getAllHosts().iterator().next();
+ assertApplicationForHost(hostname, applicationId);
}
@Test
public void require_that_handler_gives_error_for_unknown_hostname() throws Exception {
- long sessionId = 1;
- addMockApplication(tenantRepository.getTenant(mytenant), ApplicationId.defaultId(), sessionId);
- final String hostname = "unknown";
- assertErrorForHost(hostname,
- Response.Status.NOT_FOUND,
- HttpErrorResponse.errorCodes.NOT_FOUND,
- "{\"error-code\":\"NOT_FOUND\",\"message\":\"Could not find any application using host '" + hostname + "'\"}");
+ String hostname = "unknown";
+ assertErrorForUnknownHost(hostname,
+ Response.Status.NOT_FOUND,
+ "{\"error-code\":\"NOT_FOUND\",\"message\":\"Could not find any application using host '" + hostname + "'\"}");
}
@Test
@@ -104,34 +80,40 @@ public class HostHandlerTest {
assertNotAllowed(com.yahoo.jdisc.http.HttpRequest.Method.DELETE);
}
- private void assertNotAllowed(com.yahoo.jdisc.http.HttpRequest.Method method) throws IOException {
- String url = urlPrefix + hostname;
- deleteAndAssertResponse(url, Response.Status.METHOD_NOT_ALLOWED,
- HttpErrorResponse.errorCodes.METHOD_NOT_ALLOWED,
+ private void assertNotAllowed(Method method) throws IOException {
+ String url = urlPrefix + "somehostname";
+ executeAndAssertResponse(url, Response.Status.METHOD_NOT_ALLOWED,
+ HttpErrorResponse.errorCodes.METHOD_NOT_ALLOWED,
"{\"error-code\":\"METHOD_NOT_ALLOWED\",\"message\":\"Method '" + method + "' is not supported\"}",
- method);
+ method);
}
- private void assertApplicationForHost(String hostname, TenantName expectedTenantName, ApplicationId expectedApplicationId, Zone zone) throws IOException {
+ private void assertApplicationForHost(String hostname, ApplicationId expectedApplicationId) throws IOException {
String url = urlPrefix + hostname;
- HttpResponse response = handler.handle(HttpRequest.createTestRequest(url, com.yahoo.jdisc.http.HttpRequest.Method.GET));
+ HttpResponse response = handler.handle(HttpRequest.createTestRequest(url, Method.GET));
HandlerTest.assertHttpStatusCodeAndMessage(response, Response.Status.OK,
- "{\"tenant\":\"" + expectedTenantName.value() + "\"," +
- "\"application\":\"" + expectedApplicationId.application().value() + "\"," +
- "\"environment\":\"" + zone.environment().value() + "\"," +
- "\"region\":\"" + zone.region().value() + "\"," +
- "\"instance\":\"" + expectedApplicationId.instance().value() + "\"}"
+ "{\"tenant\":\"" + expectedApplicationId.tenant().value() + "\"," +
+ "\"application\":\"" + expectedApplicationId.application().value() + "\"," +
+ "\"environment\":\"" + HostHandlerTest.zone.environment().value() + "\"," +
+ "\"region\":\"" + HostHandlerTest.zone.region().value() + "\"," +
+ "\"instance\":\"" + expectedApplicationId.instance().value() + "\"}"
);
}
- private void assertErrorForHost(String hostname, int expectedStatus, HttpErrorResponse.errorCodes errorCode, String expectedResponse) throws IOException {
+ private void assertErrorForUnknownHost(String hostname, int expectedStatus, String expectedResponse) throws IOException {
String url = urlPrefix + hostname;
HttpResponse response = handler.handle(HttpRequest.createTestRequest(url, com.yahoo.jdisc.http.HttpRequest.Method.GET));
- HandlerTest.assertHttpStatusCodeErrorCodeAndMessage(response, expectedStatus, errorCode, expectedResponse);
+ assertHttpStatusCodeErrorCodeAndMessage(response, expectedStatus, HttpErrorResponse.errorCodes.NOT_FOUND, expectedResponse);
}
- private void deleteAndAssertResponse(String url, int expectedStatus, HttpErrorResponse.errorCodes errorCode, String expectedResponse, com.yahoo.jdisc.http.HttpRequest.Method method) throws IOException {
+ private void executeAndAssertResponse(String url, int expectedStatus, HttpErrorResponse.errorCodes errorCode,
+ String expectedResponse, Method method) throws IOException {
HttpResponse response = handler.handle(HttpRequest.createTestRequest(url, method));
- HandlerTest.assertHttpStatusCodeErrorCodeAndMessage(response, expectedStatus, errorCode, expectedResponse);
+ assertHttpStatusCodeErrorCodeAndMessage(response, expectedStatus, errorCode, expectedResponse);
}
+
+ private ApplicationId applicationId() {
+ return ApplicationId.from(mytenant, ApplicationName.defaultName(), InstanceName.defaultName());
+ }
+
}
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/SessionContentHandlerTest.java b/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/SessionContentHandlerTest.java
index 20d9be080e9..3ab56d3869a 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/SessionContentHandlerTest.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/SessionContentHandlerTest.java
@@ -1,10 +1,11 @@
// 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.http.v2;
-import com.yahoo.config.model.application.provider.FilesApplicationPackage;
+import com.yahoo.config.provision.ApplicationId;
+import com.yahoo.config.provision.ApplicationName;
+import com.yahoo.config.provision.InstanceName;
import com.yahoo.config.provision.TenantName;
import com.yahoo.container.jdisc.HttpResponse;
-import com.yahoo.io.IOUtils;
import com.yahoo.jdisc.Response;
import com.yahoo.jdisc.http.HttpRequest;
import com.yahoo.text.Utf8;
@@ -13,6 +14,8 @@ import com.yahoo.vespa.config.server.TestComponentRegistry;
import com.yahoo.vespa.config.server.application.OrchestratorMock;
import com.yahoo.vespa.config.server.http.ContentHandlerTestBase;
import com.yahoo.vespa.config.server.http.SessionHandlerTest;
+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 org.junit.Before;
import org.junit.Ignore;
@@ -22,7 +25,7 @@ import java.io.ByteArrayInputStream;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
-import java.nio.file.Files;
+import java.time.Clock;
import static org.hamcrest.core.Is.is;
import static org.junit.Assert.assertNotNull;
@@ -32,21 +35,31 @@ import static org.junit.Assert.assertThat;
* @author Ulf Lilleengen
*/
public class SessionContentHandlerTest extends ContentHandlerTestBase {
- private static final TenantName tenant = TenantName.from("contenttest");
+ private static final TenantName tenantName = TenantName.from("contenttest");
+ private static final File testApp = new File("src/test/apps/content");
private final TestComponentRegistry componentRegistry = new TestComponentRegistry.Builder().build();
private TenantRepository tenantRepository;
private SessionContentHandler handler = null;
-
+ private long sessionId;
+
@Before
- public void setupHandler() throws Exception {
+ public void setupHandler() {
tenantRepository = new TenantRepository(componentRegistry, false);
- tenantRepository.addTenant(tenant);
- tenantRepository.getTenant(tenant).getSessionRepository().addLocalSession(new MockLocalSession(1L, FilesApplicationPackage.fromFile(createTestApp())));
+ tenantRepository.addTenant(tenantName);
+
+ ApplicationRepository applicationRepository = new ApplicationRepository(tenantRepository,
+ new SessionHandlerTest.MockProvisioner(),
+ new OrchestratorMock(),
+ Clock.systemUTC());
+ applicationRepository.deploy(testApp, new PrepareParams.Builder().applicationId(applicationId()).build());
+ Tenant tenant = tenantRepository.getTenant(tenantName);
+ sessionId = applicationRepository.getActiveLocalSession(tenant, applicationId()).getSessionId();
+
handler = createHandler();
- pathPrefix = "/application/v2/tenant/" + tenant + "/session/";
- baseUrl = "http://foo:1337/application/v2/tenant/" + tenant + "/session/1/content/";
+ pathPrefix = "/application/v2/tenant/" + tenantName + "/session/";
+ baseUrl = "http://foo:1337/application/v2/tenant/" + tenantName + "/session/" + sessionId + "/content/";
}
@Test
@@ -69,14 +82,14 @@ public class SessionContentHandlerTest extends ContentHandlerTestBase {
@Test
public void require_that_nonexistent_session_returns_not_found() {
- HttpResponse response = doRequest(HttpRequest.Method.GET, "/test.txt", 2);
+ HttpResponse response = doRequest(HttpRequest.Method.GET, "/test.txt", 9999);
assertNotNull(response);
assertThat(response.getStatus(), is(Response.Status.NOT_FOUND));
}
protected HttpResponse put(String path, String content) {
ByteArrayInputStream data = new ByteArrayInputStream(Utf8.toBytes(content));
- return doRequest(HttpRequest.Method.PUT, path, 1, data);
+ return doRequest(HttpRequest.Method.PUT, path, sessionId, data);
}
@Test
@@ -94,13 +107,13 @@ public class SessionContentHandlerTest extends ContentHandlerTestBase {
@Test
public void require_that_nonexistent_file_returns_not_found_when_deleted() throws IOException {
- assertDeleteFile(Response.Status.NOT_FOUND, "/test2.txt", "{\"error-code\":\"NOT_FOUND\",\"message\":\"Session 1 does not contain a file 'test2.txt'\"}");
+ assertDeleteFile(Response.Status.NOT_FOUND, "/test2.txt", "{\"error-code\":\"NOT_FOUND\",\"message\":\"Session " + sessionId + " does not contain a file 'test2.txt'\"}");
}
@Test
public void require_that_files_can_be_deleted() throws IOException {
assertDeleteFile(Response.Status.OK, "/test.txt");
- assertDeleteFile(Response.Status.NOT_FOUND, "/test.txt", "{\"error-code\":\"NOT_FOUND\",\"message\":\"Session 1 does not contain a file 'test.txt'\"}");
+ assertDeleteFile(Response.Status.NOT_FOUND, "/test.txt", "{\"error-code\":\"NOT_FOUND\",\"message\":\"Session " + sessionId + " does not contain a file 'test.txt'\"}");
assertDeleteFile(Response.Status.BAD_REQUEST, "/newtest", "{\"error-code\":\"BAD_REQUEST\",\"message\":\"File 'newtest' is not an empty directory\"}");
assertDeleteFile(Response.Status.OK, "/newtest/testfile.txt");
assertDeleteFile(Response.Status.OK, "/newtest");
@@ -109,10 +122,10 @@ public class SessionContentHandlerTest extends ContentHandlerTestBase {
@Test
public void require_that_status_is_given_for_new_files() throws IOException {
assertStatus("/test.txt?return=status",
- "{\"status\":\"new\",\"md5\":\"d3b07384d113edec49eaa6238ad5ff00\",\"name\":\"http://foo:1337" + pathPrefix + "1/content/test.txt\"}");
+ "{\"status\":\"new\",\"md5\":\"d3b07384d113edec49eaa6238ad5ff00\",\"name\":\"http://foo:1337" + pathPrefix + sessionId + "/content/test.txt\"}");
assertWriteFile("/test.txt", "Mycontent");
assertStatus("/test.txt?return=status",
- "{\"status\":\"changed\",\"md5\":\"01eabd73c69d78d0009ec93cd62d7f77\",\"name\":\"http://foo:1337" + pathPrefix + "1/content/test.txt\"}");
+ "{\"status\":\"changed\",\"md5\":\"01eabd73c69d78d0009ec93cd62d7f77\",\"name\":\"http://foo:1337" + pathPrefix + sessionId + "/content/test.txt\"}");
}
private void assertWriteFile(String path, String content) throws IOException {
@@ -121,11 +134,11 @@ public class SessionContentHandlerTest extends ContentHandlerTestBase {
assertThat(response.getStatus(), is(Response.Status.OK));
assertContent(path, content);
assertThat(SessionHandlerTest.getRenderedString(response),
- is("{\"prepared\":\"http://foo:1337" + pathPrefix + "1/prepared\"}"));
+ is("{\"prepared\":\"http://foo:1337" + pathPrefix + sessionId + "/prepared\"}"));
}
private void assertDeleteFile(int statusCode, String filePath) throws IOException {
- assertDeleteFile(statusCode, filePath, "{\"prepared\":\"http://foo:1337" + pathPrefix + "1/prepared\"}");
+ assertDeleteFile(statusCode, filePath, "{\"prepared\":\"http://foo:1337" + pathPrefix + sessionId + "/prepared\"}");
}
private void assertDeleteFile(int statusCode, String filePath, String expectedResponse) throws IOException {
@@ -140,17 +153,11 @@ public class SessionContentHandlerTest extends ContentHandlerTestBase {
assertNotNull(response);
assertThat(response.getStatus(), is(Response.Status.OK));
assertThat(SessionHandlerTest.getRenderedString(response),
- is("{\"prepared\":\"http://foo:1337" + pathPrefix + "1/prepared\"}"));
- }
-
- private File createTestApp() throws IOException {
- File testApp = Files.createTempDirectory("session-content-handler-test-app").toFile();
- IOUtils.copyDirectory(new File("src/test/apps/content"), testApp);
- return testApp;
+ is("{\"prepared\":\"http://foo:1337" + pathPrefix + sessionId + "/prepared\"}"));
}
protected HttpResponse doRequest(HttpRequest.Method method, String path) {
- return doRequest(method, path, 1);
+ return doRequest(method, path, sessionId);
}
private HttpResponse doRequest(HttpRequest.Method method, String path, long sessionId) {
@@ -170,4 +177,9 @@ public class SessionContentHandlerTest extends ContentHandlerTestBase {
componentRegistry.getClock())
);
}
+
+ private ApplicationId applicationId() {
+ return ApplicationId.from(tenantName, ApplicationName.defaultName(), InstanceName.defaultName());
+ }
+
}
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/TenantHandlerTest.java b/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/TenantHandlerTest.java
index d1a69ac09e0..3d143bb8f92 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/TenantHandlerTest.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/http/v2/TenantHandlerTest.java
@@ -5,6 +5,7 @@ import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.*;
import java.io.ByteArrayOutputStream;
+import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.time.Clock;
@@ -18,6 +19,7 @@ import com.yahoo.vespa.config.server.TestComponentRegistry;
import com.yahoo.vespa.config.server.application.OrchestratorMock;
import com.yahoo.vespa.config.server.http.SessionHandlerTest;
import com.yahoo.vespa.config.server.http.SessionResponse;
+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.mock.MockCurator;
@@ -32,18 +34,20 @@ import com.yahoo.vespa.config.server.http.NotFoundException;
public class TenantHandlerTest {
+ private static final File testApp = new File("src/test/apps/app");
+
private TenantRepository tenantRepository;
+ private ApplicationRepository applicationRepository;
private TenantHandler handler;
private final TenantName a = TenantName.from("a");
@Before
public void setup() {
tenantRepository = new TenantRepository(new TestComponentRegistry.Builder().curator(new MockCurator()).build());
- ApplicationRepository applicationRepository =
- new ApplicationRepository(tenantRepository,
- new SessionHandlerTest.MockProvisioner(),
- new OrchestratorMock(),
- Clock.systemUTC());
+ applicationRepository = new ApplicationRepository(tenantRepository,
+ new SessionHandlerTest.MockProvisioner(),
+ new OrchestratorMock(),
+ Clock.systemUTC());
handler = new TenantHandler(TenantHandler.testOnlyContext(), applicationRepository);
}
@@ -111,9 +115,8 @@ public class TenantHandlerTest {
Tenant tenant = tenantRepository.getTenant(a);
assertEquals(a, tenant.getName());
- int sessionId = 1;
- ApplicationId app = ApplicationId.from(a, ApplicationName.from("foo"), InstanceName.defaultName());
- HostHandlerTest.addMockApplication(tenant, app, sessionId);
+ ApplicationId applicationId = ApplicationId.from(a, ApplicationName.from("foo"), InstanceName.defaultName());
+ applicationRepository.deploy(testApp, new PrepareParams.Builder().applicationId(applicationId).build());
try {
handler.handleDELETE(HttpRequest.createTestRequest("http://deploy.example.yahoo.com:80/application/v2/tenant/" + a, Method.DELETE));
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 c1377ae439b..a2ef6aeb578 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
@@ -62,7 +62,7 @@ public class LocalSessionTest {
.configServerDBDir(temporaryFolder.newFolder().getAbsolutePath())
.build())
.build();
- tenantRepository = new TenantRepository(componentRegistry, false);
+ tenantRepository = new TenantRepository(componentRegistry);
tenantRepository.addTenant(tenantName);
configCurator = ConfigCurator.create(curator);
}
@@ -119,19 +119,21 @@ public class LocalSessionTest {
Optional<AllocatedHosts> allocatedHosts) throws Exception {
SessionZooKeeperClient zkc = new MockSessionZKClient(curator, tenant, sessionId, allocatedHosts);
zkc.createWriteStatusTransaction(Session.Status.NEW).commit();
- ZooKeeperClient zkClient = new ZooKeeperClient(configCurator, new BaseDeployLogger(), false,
+ ZooKeeperClient zkClient = new ZooKeeperClient(configCurator, new BaseDeployLogger(),
TenantRepository.getSessionsPath(tenant).append(String.valueOf(sessionId)));
if (allocatedHosts.isPresent()) {
zkClient.write(allocatedHosts.get());
}
zkClient.write(Collections.singletonMap(new Version(0, 0, 0), new MockFileRegistry()));
TenantApplications applications = tenantRepository.getTenant(tenantName).getApplicationRepo();
- applications.createApplication(zkc.readApplicationId());
- return new LocalSession(tenant, sessionId, FilesApplicationPackage.fromFile(testApp), zkc, applications);
+ applications.createApplication(applicationId());
+ LocalSession session = new LocalSession(tenant, sessionId, FilesApplicationPackage.fromFile(testApp), zkc, applications);
+ session.setApplicationId(applicationId());
+ return session;
}
private void doPrepare(LocalSession session) {
- doPrepare(session, new PrepareParams.Builder().build());
+ doPrepare(session, new PrepareParams.Builder().applicationId(applicationId()).build());
}
private void doPrepare(LocalSession session, PrepareParams params) {
@@ -140,8 +142,11 @@ public class LocalSessionTest {
}
private DeployHandlerLogger getLogger() {
- return new DeployHandlerLogger(new Slime().get(), false,
- new ApplicationId.Builder().tenant(tenantName).applicationName("testapp").build());
+ return new DeployHandlerLogger(new Slime().get(), false, applicationId());
+ }
+
+ private ApplicationId applicationId() {
+ return new ApplicationId.Builder().tenant(tenantName).applicationName("testapp").build();
}
}
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/session/SessionPreparerTest.java b/configserver/src/test/java/com/yahoo/vespa/config/server/session/SessionPreparerTest.java
index 46b8754ebe0..7d1554c3e19 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/session/SessionPreparerTest.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/session/SessionPreparerTest.java
@@ -1,6 +1,7 @@
// 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.cloud.config.ConfigserverConfig;
import com.yahoo.component.Version;
import com.yahoo.config.application.api.DeployLogger;
import com.yahoo.config.model.api.ContainerEndpoint;
@@ -33,7 +34,6 @@ import com.yahoo.vespa.config.server.TestComponentRegistry;
import com.yahoo.vespa.config.server.TimeoutBudgetTest;
import com.yahoo.vespa.config.server.application.PermanentApplicationPackage;
import com.yahoo.vespa.config.server.deploy.DeployHandlerLogger;
-import com.yahoo.vespa.config.server.filedistribution.MockFileDistributionFactory;
import com.yahoo.vespa.config.server.host.HostRegistry;
import com.yahoo.vespa.config.server.http.InvalidApplicationException;
import com.yahoo.vespa.config.server.model.TestModelFactory;
@@ -59,7 +59,6 @@ import java.security.KeyPair;
import java.security.cert.X509Certificate;
import java.time.Instant;
import java.time.temporal.ChronoUnit;
-import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
@@ -68,6 +67,7 @@ import java.util.Set;
import java.util.logging.Level;
import static com.yahoo.vespa.config.server.session.SessionZooKeeperClient.APPLICATION_PACKAGE_REFERENCE_PATH;
+import static com.yahoo.vespa.config.server.session.SessionPreparer.PrepareResult;
import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
@@ -85,27 +85,31 @@ public class SessionPreparerTest {
private static final File invalidTestApp = new File("src/test/apps/illegalApp");
private static final Version version123 = new Version(1, 2, 3);
private static final Version version321 = new Version(3, 2, 1);
- private KeyPair keyPair = KeyUtils.generateKeypair(KeyAlgorithm.EC, 256);
- private X509Certificate certificate = X509CertificateBuilder.fromKeypair(keyPair, new X500Principal("CN=subject"),
+ private final KeyPair keyPair = KeyUtils.generateKeypair(KeyAlgorithm.EC, 256);
+ private final X509Certificate certificate = X509CertificateBuilder.fromKeypair(keyPair, new X500Principal("CN=subject"),
Instant.now(), Instant.now().plus(1, ChronoUnit.DAYS), SignatureAlgorithm.SHA512_WITH_ECDSA, BigInteger.valueOf(12345)).build();
-
private final InMemoryFlagSource flagSource = new InMemoryFlagSource();
private MockCurator curator;
private ConfigCurator configCurator;
private SessionPreparer preparer;
private TestComponentRegistry componentRegistry;
- private MockFileDistributionFactory fileDistributionFactory;
- private MockSecretStore secretStore = new MockSecretStore();
+ private final MockSecretStore secretStore = new MockSecretStore();
@Rule
public TemporaryFolder folder = new TemporaryFolder();
@Before
- public void setUp() {
+ public void setUp() throws IOException {
curator = new MockCurator();
configCurator = ConfigCurator.create(curator);
- componentRegistry = new TestComponentRegistry.Builder().curator(curator).build();
- fileDistributionFactory = (MockFileDistributionFactory)componentRegistry.getFileDistributionProvider();
+ componentRegistry = new TestComponentRegistry.Builder()
+ .curator(curator)
+ .configServerConfig(new ConfigserverConfig.Builder()
+ .fileReferencesDir(folder.newFolder().getAbsolutePath())
+ .configServerDBDir(folder.newFolder().getAbsolutePath())
+ .configDefinitionsDir(folder.newFolder().getAbsolutePath())
+ .build())
+ .build();
preparer = createPreparer();
}
@@ -115,7 +119,7 @@ public class SessionPreparerTest {
private SessionPreparer createPreparer(HostProvisionerProvider hostProvisionerProvider) {
ModelFactoryRegistry modelFactoryRegistry =
- new ModelFactoryRegistry(Arrays.asList(new TestModelFactory(version123), new TestModelFactory(version321)));
+ new ModelFactoryRegistry(List.of(new TestModelFactory(version123), new TestModelFactory(version321)));
return createPreparer(modelFactoryRegistry, hostProvisionerProvider);
}
@@ -123,7 +127,7 @@ public class SessionPreparerTest {
HostProvisionerProvider hostProvisionerProvider) {
return new SessionPreparer(
modelFactoryRegistry,
- componentRegistry.getFileDistributionProvider(),
+ componentRegistry.getFileDistributionFactory(),
hostProvisionerProvider,
new PermanentApplicationPackage(componentRegistry.getConfigserverConfig()),
componentRegistry.getConfigserverConfig(),
@@ -152,14 +156,13 @@ public class SessionPreparerTest {
@Test
public void require_that_filedistribution_is_ignored_on_dryrun() throws IOException {
- prepare(testApp, new PrepareParams.Builder().dryRun(true).timeoutBudget(TimeoutBudgetTest.day()).build());
- assertThat(fileDistributionFactory.mockFileDistributionProvider.timesCalled, is(0));
+ PrepareResult result = prepare(testApp, new PrepareParams.Builder().dryRun(true).build());
+ assertTrue(result.getFileRegistries().get(version321).export().isEmpty());
}
@Test
public void require_that_application_is_prepared() throws Exception {
prepare(testApp);
- assertThat(fileDistributionFactory.mockFileDistributionProvider.timesCalled, is(1)); // Only builds the newest version
assertTrue(configCurator.exists(sessionsPath.append(ConfigCurator.USERAPP_ZK_SUBPATH).append("services.xml").getAbsolute()));
}
@@ -327,11 +330,11 @@ public class SessionPreparerTest {
prepare(app, new PrepareParams.Builder().build());
}
- private void prepare(File app, PrepareParams params) throws IOException {
+ private PrepareResult prepare(File app, PrepareParams params) throws IOException {
FilesApplicationPackage applicationPackage = getApplicationPackage(app);
- preparer.prepare(new HostRegistry<>(), getLogger(), params,
- Optional.empty(), tenantPath, Instant.now(), applicationPackage.getAppDir(),
- applicationPackage, new SessionZooKeeperClient(curator, sessionsPath));
+ return preparer.prepare(new HostRegistry<>(), getLogger(), params,
+ Optional.empty(), tenantPath, Instant.now(), applicationPackage.getAppDir(),
+ applicationPackage, new SessionZooKeeperClient(curator, sessionsPath));
}
private FilesApplicationPackage getApplicationPackage(File testFile) throws IOException {
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/session/SessionTest.java b/configserver/src/test/java/com/yahoo/vespa/config/server/session/SessionTest.java
index 5ae5910d827..7f38083797e 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/session/SessionTest.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/session/SessionTest.java
@@ -3,17 +3,18 @@ package com.yahoo.vespa.config.server.session;
import com.yahoo.config.application.api.ApplicationPackage;
import com.yahoo.config.application.api.DeployLogger;
+import com.yahoo.config.provision.AllocatedHosts;
import com.yahoo.config.provision.ApplicationId;
import com.yahoo.path.Path;
import com.yahoo.vespa.config.server.application.ApplicationSet;
-import com.yahoo.vespa.config.server.configchange.ConfigChangeActions;
import com.yahoo.vespa.config.server.host.HostValidator;
import com.yahoo.vespa.curator.mock.MockCurator;
import java.io.File;
import java.time.Instant;
-import java.util.ArrayList;
+import java.util.List;
import java.util.Optional;
+import java.util.Set;
/**
* @author Ulf Lilleengen
@@ -28,12 +29,12 @@ public class SessionTest {
}
@Override
- public ConfigChangeActions prepare(HostValidator<ApplicationId> hostValidator, DeployLogger logger, PrepareParams params,
- Optional<ApplicationSet> currentActiveApplicationSet, Path tenantPath,
- Instant now, File serverDbSessionDir, ApplicationPackage applicationPackage,
- SessionZooKeeperClient sessionZooKeeperClient) {
+ public PrepareResult prepare(HostValidator<ApplicationId> hostValidator, DeployLogger logger, PrepareParams params,
+ Optional<ApplicationSet> currentActiveApplicationSet, Path tenantPath,
+ Instant now, File serverDbSessionDir, ApplicationPackage applicationPackage,
+ SessionZooKeeperClient sessionZooKeeperClient) {
isPrepared = true;
- return new ConfigChangeActions(new ArrayList<>());
+ return new PrepareResult(AllocatedHosts.withHosts(Set.of()), List.of());
}
}
diff --git a/configserver/src/test/java/com/yahoo/vespa/config/server/session/SessionZooKeeperClientTest.java b/configserver/src/test/java/com/yahoo/vespa/config/server/session/SessionZooKeeperClientTest.java
index 5633ec2c5f8..1d7df7acfd0 100644
--- a/configserver/src/test/java/com/yahoo/vespa/config/server/session/SessionZooKeeperClientTest.java
+++ b/configserver/src/test/java/com/yahoo/vespa/config/server/session/SessionZooKeeperClientTest.java
@@ -85,11 +85,6 @@ public class SessionZooKeeperClientTest {
}
@Test
- public void require_that_default_name_is_returned_if_node_does_not_exist() {
- assertThat(createSessionZKClient("3").readApplicationId().application().value(), is("default"));
- }
-
- @Test
public void require_that_create_time_can_be_written_and_read() {
SessionZooKeeperClient zkc = createSessionZKClient("3");
curator.delete(Path.fromString("3"));