diff --git a/.vscode/settings.json b/.vscode/settings.json new file mode 100644 index 000000000000..e0f15db2eb22 --- /dev/null +++ b/.vscode/settings.json @@ -0,0 +1,3 @@ +{ + "java.configuration.updateBuildConfiguration": "automatic" +} \ No newline at end of file diff --git a/cdap-app-fabric/.gitignore b/cdap-app-fabric/.gitignore index 321af1077cac..37a691f73e9c 100644 --- a/cdap-app-fabric/.gitignore +++ b/cdap-app-fabric/.gitignore @@ -17,6 +17,9 @@ lib/ .idea data/ +# VSCode Files & Dir +.vscode/ + # Gradle Files & Dir # build/ .gradle/ diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/app/guice/AppFabricServiceRuntimeModule.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/app/guice/AppFabricServiceRuntimeModule.java index 4dfdaf0f9873..27408054e803 100644 --- a/cdap-app-fabric/src/main/java/io/cdap/cdap/app/guice/AppFabricServiceRuntimeModule.java +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/app/guice/AppFabricServiceRuntimeModule.java @@ -68,6 +68,7 @@ import io.cdap.cdap.gateway.handlers.InstanceOperationHttpHandler; import io.cdap.cdap.gateway.handlers.NamespaceHttpHandler; import io.cdap.cdap.gateway.handlers.OperationalStatsHttpHandler; +import io.cdap.cdap.gateway.handlers.OperationHttpHandler; import io.cdap.cdap.gateway.handlers.OperationsDashboardHttpHandler; import io.cdap.cdap.gateway.handlers.PreferencesHttpHandler; import io.cdap.cdap.gateway.handlers.PreferencesHttpHandlerInternal; @@ -136,6 +137,7 @@ import io.cdap.cdap.internal.events.StartProgramEventSubscriber; import io.cdap.cdap.internal.namespace.credential.handler.GcpWorkloadIdentityHttpHandler; import io.cdap.cdap.internal.namespace.credential.handler.GcpWorkloadIdentityHttpHandlerInternal; +import io.cdap.cdap.internal.operation.guice.OperationModule; import io.cdap.cdap.internal.pipeline.SynchronousPipelineFactory; import io.cdap.cdap.internal.profile.ProfileService; import io.cdap.cdap.internal.provision.ProvisionerModule; @@ -198,6 +200,7 @@ public Module getInMemoryModules() { new SourceControlModule(), new EntityVerifierModule(), new MasterCredentialProviderModule(), + new OperationModule(), BootstrapModules.getInMemoryModule(), new AbstractModule() { @Override @@ -240,6 +243,7 @@ public Module getStandaloneModules() { new EntityVerifierModule(), new ProvisionerModule(), new MasterCredentialProviderModule(), + new OperationModule(), BootstrapModules.getFileBasedModule(), new AbstractModule() { @Override @@ -294,6 +298,7 @@ public Module getDistributedModules() { new EntityVerifierModule(), new ProvisionerModule(), new MasterCredentialProviderModule(), + new OperationModule(), BootstrapModules.getFileBasedModule(), new AbstractModule() { @Override @@ -453,6 +458,7 @@ protected void configure() { handlerBinder.addBinding().to(SecureStoreHandler.class); handlerBinder.addBinding().to(RemotePrivilegesHandler.class); handlerBinder.addBinding().to(OperationalStatsHttpHandler.class); + handlerBinder.addBinding().to(OperationHttpHandler.class); handlerBinder.addBinding().to(ProfileHttpHandler.class); handlerBinder.addBinding().to(ProvisionerHttpHandler.class); handlerBinder.addBinding().to(BootstrapHttpHandler.class); diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/app/store/Store.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/app/store/Store.java index d8a00979076b..a979c3ac1098 100644 --- a/cdap-app-fabric/src/main/java/io/cdap/cdap/app/store/Store.java +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/app/store/Store.java @@ -359,6 +359,7 @@ List getRuns(Collection programs, ProgramRunSt /** * Creates new application if it doesn't exist. Updates existing one otherwise. + * Always marks the added application as latest. * * @param id application id * @param meta application metadata to store @@ -366,7 +367,20 @@ List getRuns(Collection programs, ProgramRunSt * @throws ConflictException if the app cannot be deployed when the user provided parent-version doesn't match the * current latest version */ - int addApplication(ApplicationId id, ApplicationMeta meta) throws ConflictException; + int addLatestApplication(ApplicationId id, ApplicationMeta meta) throws ConflictException; + + /** + * Creates new application if it doesn't exist. Updates existing one otherwise. + * Marks the application as latest based on the isLatest param. + * + * @param id application id + * @param meta application metadata to store + * @param isLatest boolean, indicating if the application should be marked latest + * @return the number of edits to the application. A new application will return 0. + * @throws ConflictException if the app cannot be deployed when the user provided parent-version doesn't match the + * current latest version + */ + int addApplication(ApplicationId id, ApplicationMeta meta, boolean isLatest) throws ConflictException; /** * Marks existing applications as latest. diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/gateway/handlers/OperationHttpHandler.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/gateway/handlers/OperationHttpHandler.java index 6e1673b883de..964f84cd3406 100644 --- a/cdap-app-fabric/src/main/java/io/cdap/cdap/gateway/handlers/OperationHttpHandler.java +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/gateway/handlers/OperationHttpHandler.java @@ -17,53 +17,111 @@ package io.cdap.cdap.gateway.handlers; import com.google.gson.Gson; +import com.google.inject.Inject; +import io.cdap.cdap.api.feature.FeatureFlagsProvider; +import io.cdap.cdap.common.BadRequestException; +import io.cdap.cdap.common.ForbiddenException; +import io.cdap.cdap.common.conf.CConfiguration; import io.cdap.cdap.common.conf.Constants; +import io.cdap.cdap.common.conf.Constants.AppFabric; +import io.cdap.cdap.common.feature.DefaultFeatureFlagsProvider; +import io.cdap.cdap.features.Feature; import io.cdap.cdap.gateway.handlers.util.AbstractAppFabricHttpHandler; +import io.cdap.cdap.internal.operation.OperationLifecycleManager; +import io.cdap.cdap.internal.operation.OperationRunFilter; +import io.cdap.cdap.internal.operation.OperationRunNotFoundException; +import io.cdap.cdap.internal.operation.ScanOperationRunsRequest; +import io.cdap.cdap.proto.id.NamespaceId; +import io.cdap.cdap.proto.id.OperationRunId; import io.cdap.cdap.proto.operation.OperationRun; +import io.cdap.cdap.proto.operation.OperationRunStatus; +import io.cdap.cdap.proto.operation.OperationType; import io.cdap.http.HttpHandler; import io.cdap.http.HttpResponder; import io.netty.handler.codec.http.FullHttpRequest; import io.netty.handler.codec.http.HttpRequest; import io.netty.handler.codec.http.HttpResponseStatus; -import java.util.ArrayList; -import java.util.List; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import javax.ws.rs.GET; import javax.ws.rs.POST; import javax.ws.rs.Path; import javax.ws.rs.PathParam; import javax.ws.rs.QueryParam; -/** - * The {@link HttpHandler} for handling REST calls to operation endpoints. - */ +/** The {@link HttpHandler} for handling REST calls to operation endpoints. */ @Path(Constants.Gateway.API_VERSION_3 + "/namespaces/{namespace-id}/operations") public class OperationHttpHandler extends AbstractAppFabricHttpHandler { - + private final CConfiguration cConf; + private static final Pattern KEY_VALUE_PATTERN = Pattern.compile("(\"?)(\\w+)=(\\w+)(\"?)"); + private static final String FILTER_SPLITTER = "AND"; + private final FeatureFlagsProvider featureFlagsProvider; private static final Gson GSON = new Gson(); - - OperationHttpHandler() { + private final OperationLifecycleManager operationLifecycleManager; + private final int batchSize; + public static final String OPERATIONS_LIST_PAGINATED_KEY = "operations"; + + @Inject + OperationHttpHandler(CConfiguration cConf, OperationLifecycleManager operationLifecycleManager) + throws Exception { + this.cConf = cConf; + this.batchSize = this.cConf.getInt(AppFabric.STREAMING_BATCH_SIZE); + this.operationLifecycleManager = operationLifecycleManager; + this.featureFlagsProvider = new DefaultFeatureFlagsProvider(cConf); } + // TODO[CDAP-20881] : Add RBAC check /** * API to fetch all running operations in a namespace. * * @param namespaceId Namespace to fetch runs from - * @param pageToken the token identifier for the current page requested in a paginated - * request + * @param pageToken the token identifier for the current page requested in a paginated request * @param pageSize the number of application details returned in a paginated request - * @param filter optional filters in EBNF grammar. Currently Only one status and one type - * filter is supported with AND expression. + * @param filter optional filters in EBNF grammar. Currently Only one status and one type filter + * is supported with AND expression. */ @GET @Path("/") - public void scanOperations(HttpRequest request, HttpResponder responder, + public void scanOperations( + HttpRequest request, + HttpResponder responder, @PathParam("namespace-id") String namespaceId, @QueryParam("pageToken") String pageToken, @QueryParam("pageSize") Integer pageSize, - @QueryParam("filter") String filter) { - // TODO(samik, CDAP-20812) fetch the operation runs from store - List runs = new ArrayList<>(); - responder.sendJson(HttpResponseStatus.OK, GSON.toJson(runs)); + @QueryParam("filter") String filter) + throws BadRequestException, IOException, ForbiddenException { + checkSourceControlMultiAppFeatureFlag(); + validateNamespaceId(namespaceId); + JsonPaginatedListResponder.respond( + GSON, + responder, + OPERATIONS_LIST_PAGINATED_KEY, + jsonListResponder -> { + AtomicReference lastRun = new AtomicReference<>(); + ScanOperationRunsRequest scanRequest = getScanRequest(namespaceId, pageToken, pageSize, filter); + boolean pageLimitReached = false; + try { + pageLimitReached = + operationLifecycleManager.scanOperations( + scanRequest, + batchSize, + runDetail -> { + OperationRun run = runDetail.getRun(); + jsonListResponder.send(run); + lastRun.set(run); + }); + } catch (IOException e) { + responder.sendString(HttpResponseStatus.INTERNAL_SERVER_ERROR, e.getMessage()); + } catch (OperationRunNotFoundException e) { + responder.sendString(HttpResponseStatus.BAD_REQUEST, e.getMessage()); + } + OperationRun run = lastRun.get(); + return !pageLimitReached || run == null ? null : run.getId(); + }); } /** @@ -74,12 +132,23 @@ public void scanOperations(HttpRequest request, HttpResponder responder, */ @GET @Path("/{id}") - public void getOperationRun(HttpRequest request, HttpResponder responder, + public void getOperationRun( + HttpRequest request, + HttpResponder responder, @PathParam("namespace-id") String namespaceId, - @PathParam("id") String runId) { - // // TODO(samik, CDAP-20813) fetch the operation runs from store - OperationRun run = null; - responder.sendJson(HttpResponseStatus.OK, GSON.toJson(run)); + @PathParam("id") String runId) + throws BadRequestException, OperationRunNotFoundException, IOException, ForbiddenException { + checkSourceControlMultiAppFeatureFlag(); + validateNamespaceId(namespaceId); + if (runId == null || runId.isEmpty()) { + throw new BadRequestException("Path parameter runId cannot be empty"); + } + responder.sendJson( + HttpResponseStatus.OK, + GSON.toJson( + operationLifecycleManager + .getOperationRun(new OperationRunId(namespaceId, runId)) + .getRun())); } /** @@ -98,4 +167,96 @@ public void failOperation(FullHttpRequest request, HttpResponder responder, String.format("Updated status for operation run %s in namespace '%s'.", runId, namespaceId)); } + + private ScanOperationRunsRequest getScanRequest( + String namespaceId, String pageToken, Integer pageSize, String filterStr) + throws IllegalArgumentException { + ScanOperationRunsRequest.Builder builder = ScanOperationRunsRequest.builder(); + builder.setNamespace(namespaceId); + if (pageSize != null) { + builder.setLimit(pageSize); + } + if (pageToken != null) { + builder.setScanAfter(pageToken); + } + if (filterStr != null && !filterStr.isEmpty()) { + OperationRunFilter operationRunFilter = getFilter(filterStr); + builder.setFilter(operationRunFilter); + } + return builder.build(); + } + + // TODO[CDAP-20895] : Add unit tests for extracting OperationRunFilter from filter string + private OperationRunFilter getFilter(String filterStr) throws IllegalArgumentException { + Map filterKeyValMap = parseKeyValStr(filterStr, FILTER_SPLITTER); + OperationType operationType = null; + OperationRunStatus operationStatus = null; + + for (Map.Entry entry : filterKeyValMap.entrySet()) { + String filterValue = entry.getValue(); + OperationFilterKey filterKey = OperationFilterKey.valueOf(entry.getKey()); + + try { + switch (filterKey) { + case TYPE: + operationType = OperationType.valueOf(filterValue); + break; + case STATUS: + operationStatus = OperationRunStatus.valueOf(filterValue); + break; + default: + throw new IllegalArgumentException("Unknown filter key: " + filterKey); + } + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException("Invalid " + filterKey.name() + ": " + filterValue, e); + } + } + return new OperationRunFilter(operationType, operationStatus); + } + + /** + * Parses a string containing key-value pairs separated by a specified splitter. The string is + * enclosed within quotes. + * + * @param input The input string containing key-value pairs. + * @param splitter The string used to split key-value pairs. + * @return A {@code Map} containing the parsed key-value pairs. + * @throws IllegalArgumentException If the input does not match the expected key=val pair pattern. + */ + private static Map parseKeyValStr(String input, String splitter) { + Map keyValMap = new HashMap<>(); + String[] keyValPairs = input.split(splitter); + + for (String keyValPair : keyValPairs) { + Matcher matcher = KEY_VALUE_PATTERN.matcher(keyValPair.trim()); + + if (matcher.matches()) { + keyValMap.put(matcher.group(2).trim().toUpperCase(), matcher.group(3).trim().toUpperCase()); + } else { + throw new IllegalArgumentException("Invalid filter key=val pair: " + keyValPair); + } + } + return keyValMap; + } + + private NamespaceId validateNamespaceId(String namespaceId) throws BadRequestException { + try { + return new NamespaceId(namespaceId); + } catch (IllegalArgumentException e) { + throw new BadRequestException(e.getMessage(), e); + } + } + + /** throws {@link ForbiddenException} if the feature is disabled */ + private void checkSourceControlMultiAppFeatureFlag() throws ForbiddenException { + if (!Feature.SOURCE_CONTROL_MANAGEMENT_MULTI_APP.isEnabled(featureFlagsProvider)) { + throw new ForbiddenException( + "Source Control Management Multiple Apps feature is not enabled."); + } + } + + private enum OperationFilterKey { + TYPE, + STATUS + } } diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/deploy/pipeline/ApplicationRegistrationStage.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/deploy/pipeline/ApplicationRegistrationStage.java index c58b32f7b3f2..2b4f8021d70d 100644 --- a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/deploy/pipeline/ApplicationRegistrationStage.java +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/deploy/pipeline/ApplicationRegistrationStage.java @@ -67,9 +67,9 @@ public void process(ApplicationWithPrograms input) throws Exception { boolean ownerAdded = addOwnerIfRequired(input, allAppVersionsAppIds); ApplicationMeta appMeta = new ApplicationMeta(applicationSpecification.getName(), input.getSpecification(), - input.getChangeDetail(), input.getSourceControlMeta(), !input.isSkipMarkingLatest()); + input.getChangeDetail(), input.getSourceControlMeta()); try { - int editCount = store.addApplication(input.getApplicationId(), appMeta); + int editCount = store.addApplication(input.getApplicationId(), appMeta, !input.isSkipMarkingLatest()); if (input.isSkipMarkingLatest()) { // TODO [CDAP-20848] diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/preview/DistributedPreviewManager.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/preview/DistributedPreviewManager.java index 12a6701a7729..08be41066108 100644 --- a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/preview/DistributedPreviewManager.java +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/preview/DistributedPreviewManager.java @@ -235,6 +235,8 @@ public void run() { String.format("%s:%s", localhost, cConf.getInt(Constants.ArtifactLocalizer.PORT)) )); + twillPreparer = ((SecureTwillPreparer) twillPreparer) + .withNamespacedWorkloadIdentity(PreviewRunnerTwillRunnable.class.getSimpleName()); } String priorityClass = cConf.get(Constants.Preview.CONTAINER_PRIORITY_CLASS_NAME); diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/sourcecontrol/PullAppsOperation.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/sourcecontrol/PullAppsOperation.java index 394425128f96..d768f965bdcc 100644 --- a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/sourcecontrol/PullAppsOperation.java +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/sourcecontrol/PullAppsOperation.java @@ -57,7 +57,7 @@ public class PullAppsOperation implements LongRunningOperation { /** * Only request is passed using AssistedInject. See {@link PullAppsOperationFactory} * - * @param request contains apps to push + * @param request contains apps to pull * @param runner runs git operations. The reason we do not use * {@link io.cdap.cdap.sourcecontrol.operationrunner.SourceControlOperationRunner} rather than * concrete implementation is because the git operations should always run inMemory. @@ -88,7 +88,7 @@ public ListenableFuture> run(LongRunningOperationContext ); // pull and deploy applications one at a time - scmOpRunner.pull(pullReq, response -> { + scmOpRunner.multiPull(pullReq, response -> { appTobeDeployed.set(new ApplicationReference(context.getRunId().getNamespace(), response.getApplicationName())); try { @@ -106,7 +106,8 @@ public ListenableFuture> run(LongRunningOperationContext "Failed to deploy applications", appTobeDeployed.get() != null ? ImmutableList.of( new OperationResourceScopedError(appTobeDeployed.get().toString(), e.getMessage())) - : Collections.emptyList() + : Collections.emptyList(), + e ); } diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/sourcecontrol/PushAppsOperation.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/sourcecontrol/PushAppsOperation.java new file mode 100644 index 000000000000..0f7d9eec8300 --- /dev/null +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/sourcecontrol/PushAppsOperation.java @@ -0,0 +1,123 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.cdap.internal.app.sourcecontrol; + +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.inject.Inject; +import com.google.inject.assistedinject.Assisted; +import io.cdap.cdap.common.BadRequestException; +import io.cdap.cdap.common.NotFoundException; +import io.cdap.cdap.internal.operation.LongRunningOperation; +import io.cdap.cdap.internal.operation.LongRunningOperationContext; +import io.cdap.cdap.internal.operation.OperationException; +import io.cdap.cdap.proto.app.UpdateMultiSourceControlMetaReqeust; +import io.cdap.cdap.proto.app.UpdateSourceControlMetaRequest; +import io.cdap.cdap.proto.id.NamespaceId; +import io.cdap.cdap.proto.operation.OperationResource; +import io.cdap.cdap.proto.sourcecontrol.RepositoryConfig; +import io.cdap.cdap.sourcecontrol.ApplicationManager; +import io.cdap.cdap.sourcecontrol.NoChangesToPushException; +import io.cdap.cdap.sourcecontrol.SourceControlException; +import io.cdap.cdap.sourcecontrol.operationrunner.InMemorySourceControlOperationRunner; +import io.cdap.cdap.sourcecontrol.operationrunner.MultiPushAppOperationRequest; +import io.cdap.cdap.sourcecontrol.operationrunner.PushAppResponse; +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Defines operation for doing SCM Push for connected repositories. + **/ +public class PushAppsOperation implements LongRunningOperation { + + private final PushAppsRequest request; + + private final InMemorySourceControlOperationRunner scmOpRunner; + private final ApplicationManager applicationManager; + + /** + * Only request is passed using AssistedInject. See {@link PushAppsOperationFactory} + * + * @param request contains apps to push + * @param runner runs git operations. The reason we do not use + * {@link io.cdap.cdap.sourcecontrol.operationrunner.SourceControlOperationRunner} rather than + * concrete implementation is because the git operations should always run inMemory. + * @param applicationManager provides utilities to provide app-fabric exposed + * functionalities. + */ + @Inject + PushAppsOperation(@Assisted PushAppsRequest request, + InMemorySourceControlOperationRunner runner, + ApplicationManager applicationManager) { + this.request = request; + this.applicationManager = applicationManager; + this.scmOpRunner = runner; + } + + @Override + public ListenableFuture> run(LongRunningOperationContext context) + throws OperationException { + RepositoryConfig repositoryConfig = request.getConfig(); + NamespaceId namespaceId = context.getRunId().getNamespaceId(); + MultiPushAppOperationRequest pushReq = new MultiPushAppOperationRequest( + namespaceId, + repositoryConfig, + request.getApps(), + request.getCommitDetails() + ); + + Collection responses; + + try { + responses = scmOpRunner.multiPush(pushReq, applicationManager); + context.updateOperationResources(getResources(namespaceId, responses)); + } catch (SourceControlException | NoChangesToPushException e) { + throw new OperationException("Failed to push applications.", Collections.emptyList(), e); + } + + try { + // update git metadata for the pushed application + applicationManager.updateSourceControlMeta(namespaceId, getUpdateMetaRequest(responses)); + } catch (NotFoundException | BadRequestException | IOException | SourceControlException e) { + throw new OperationException("Failed to update git metadata.", Collections.emptySet(), e); + } + + // TODO(samik) Update this after along with the runner implementation + return Futures.immediateFuture(getResources(namespaceId, responses)); + } + + private UpdateMultiSourceControlMetaReqeust getUpdateMetaRequest( + Collection responses) { + List reqs = responses.stream() + .map(response -> new UpdateSourceControlMetaRequest( + response.getName(), response.getVersion(), response.getFileHash())) + .collect(Collectors.toList()); + return new UpdateMultiSourceControlMetaReqeust(reqs); + } + + private Set getResources(NamespaceId namespaceId, + Collection responses) { + return responses.stream() + .map(response -> new OperationResource( + namespaceId.app(response.getName(), response.getVersion()).toString())) + .collect(Collectors.toSet()); + } +} diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/sourcecontrol/PushAppsOperationFactory.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/sourcecontrol/PushAppsOperationFactory.java new file mode 100644 index 000000000000..6a3ce0f3e53d --- /dev/null +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/sourcecontrol/PushAppsOperationFactory.java @@ -0,0 +1,34 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.cdap.internal.app.sourcecontrol; + + +/** + * Factory interface for creating {@link PushAppsOperation}. + * This interface is for Guice assisted binding, hence there will be no concrete implementation of it. + */ +public interface PushAppsOperationFactory { + + /** + * Returns an implementation of {@link PushAppsOperation} that operates on the given {@link + * PushAppsRequest}. + * + * @param request contains list of apps to pull + * @return a new instance of {@link PushAppsOperation}. + */ + PushAppsOperation create(PushAppsRequest request); +} diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/sourcecontrol/PushAppsRequest.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/sourcecontrol/PushAppsRequest.java new file mode 100644 index 000000000000..65ffcb48a0a7 --- /dev/null +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/sourcecontrol/PushAppsRequest.java @@ -0,0 +1,76 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.cdap.internal.app.sourcecontrol; + +import com.google.common.base.Objects; +import io.cdap.cdap.proto.sourcecontrol.RepositoryConfig; +import io.cdap.cdap.sourcecontrol.CommitMeta; +import java.util.Set; + +/** + * Request type for {@link PullAppsOperation}. + */ +public class PushAppsRequest { + + private final Set apps; + private final RepositoryConfig config; + + private final CommitMeta commitDetails; + + /** + * Default Constructor. + * + * @param apps Set of apps to push. + */ + public PushAppsRequest(Set apps, RepositoryConfig config, CommitMeta commitDetails) { + this.apps = apps; + this.config = config; + this.commitDetails = commitDetails; + } + + public Set getApps() { + return apps; + } + + public RepositoryConfig getConfig() { + return config; + } + + public CommitMeta getCommitDetails() { + return commitDetails; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + PushAppsRequest that = (PushAppsRequest) o; + return Objects.equal(this.getApps(), that.getApps()) + && Objects.equal(this.getConfig(), that.getConfig()) + && Objects.equal(this.getCommitDetails(), that.getCommitDetails()); + } + + @Override + public int hashCode() { + return Objects.hashCode(getApps(), getConfig(), getCommitDetails()); + } +} diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/store/AppMetadataStore.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/store/AppMetadataStore.java index a84d54c838ea..5b93e1fb0c74 100644 --- a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/store/AppMetadataStore.java +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/store/AppMetadataStore.java @@ -59,7 +59,6 @@ import io.cdap.cdap.proto.id.ProgramReference; import io.cdap.cdap.proto.id.ProgramRunId; import io.cdap.cdap.proto.sourcecontrol.SourceControlMeta; -import io.cdap.cdap.spi.data.InvalidFieldException; import io.cdap.cdap.spi.data.SortOrder; import io.cdap.cdap.spi.data.StructuredRow; import io.cdap.cdap.spi.data.StructuredTable; @@ -630,6 +629,7 @@ public void markAsLatest(ApplicationId id) /** * Persisting a new application version in the table. + * Marks the created application version as latest (always). * * @param id the application id * @param appMeta the application metadata to be written @@ -638,12 +638,28 @@ public void markAsLatest(ApplicationId id) * @throws ConflictException if parent-version provided in the request doesn't match the * latest version, do not allow app to be created */ - public int createApplicationVersion(ApplicationId id, ApplicationMeta appMeta) + public int createLatestApplicationVersion(ApplicationId id, ApplicationMeta appMeta) + throws IOException, ConflictException { + return createApplicationVersion(id, appMeta, true); + } + + /** + * Persisting a new application version in the table. + * Marks the created application version as latest based on the value of markAsLatest. + * + * @param id the application id + * @param appMeta the application metadata to be written + * @param markAsLatest boolean, indicating if the application should be marked as latest + * @return the number of edits to the application. A new application will return 0. + * @throws IOException if failed to write app + * @throws ConflictException if parent-version provided in the request doesn't match the + * latest version, do not allow app to be created + */ + public int createApplicationVersion(ApplicationId id, ApplicationMeta appMeta, boolean markAsLatest) throws IOException, ConflictException { String parentVersion = Optional.ofNullable(appMeta.getChange()) .map(ChangeDetail::getParentVersion).orElse(null); - boolean markAsLatest = appMeta.getIsLatest(); // Fetch the latest version ApplicationMeta latest = getLatest(id.getAppReference()); String latestVersion = latest == null ? null : latest.getSpec().getAppVersion(); @@ -695,11 +711,21 @@ void writeApplication(String namespaceId, String appId, String versionId, @Nullable SourceControlMeta sourceControlMeta, boolean markAsLatest) throws IOException { writeApplicationSerialized(namespaceId, appId, versionId, GSON.toJson( - new ApplicationMeta(appId, spec, null, null, markAsLatest)), + new ApplicationMeta(appId, spec, null, null)), change, sourceControlMeta, markAsLatest); updateApplicationEdit(namespaceId, appId); } + @VisibleForTesting + List> getApplicationPrimaryKeys(String namespaceId, String appId, + String versionId) { + List> fields = new ArrayList<>(); + fields.add(Fields.stringField(StoreDefinition.AppMetadataStore.NAMESPACE_FIELD, namespaceId)); + fields.add(Fields.stringField(StoreDefinition.AppMetadataStore.APPLICATION_FIELD, appId)); + fields.add(Fields.stringField(StoreDefinition.AppMetadataStore.VERSION_FIELD, versionId)); + return fields; + } + /** * Get the edit number of an application. * @@ -2478,15 +2504,6 @@ private List> getNamespaceApplicationKeys(ApplicationReference appRef) return fields; } - private List> getApplicationPrimaryKeys(String namespaceId, String appId, - String versionId) { - List> fields = new ArrayList<>(); - fields.add(Fields.stringField(StoreDefinition.AppMetadataStore.NAMESPACE_FIELD, namespaceId)); - fields.add(Fields.stringField(StoreDefinition.AppMetadataStore.APPLICATION_FIELD, appId)); - fields.add(Fields.stringField(StoreDefinition.AppMetadataStore.VERSION_FIELD, versionId)); - return fields; - } - private List> getApplicationNamespaceAppCreationKeys(ApplicationId appId) throws IOException { List> fields = new ArrayList<>(); @@ -2644,7 +2661,8 @@ private ApplicationMeta decodeRow(StructuredRow row) { } else { changeDetail = new ChangeDetail(changeSummary, null, author, creationTimeMillis, latest); } - return new ApplicationMeta(id, spec, changeDetail, sourceControl, latest); + + return new ApplicationMeta(id, spec, changeDetail, sourceControl); } private void writeToStructuredTableWithPrimaryKeys( diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/store/ApplicationMeta.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/store/ApplicationMeta.java index 56e8a0cac078..0586c2df2179 100644 --- a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/store/ApplicationMeta.java +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/store/ApplicationMeta.java @@ -36,23 +36,13 @@ public class ApplicationMeta { private final ChangeDetail change; @Nullable private final SourceControlMeta sourceControlMeta; - // the isLatest field does not need to be serialized in the ApplicationMetadata object - // as it's stored as a separate column in the app spec table. - private final transient boolean isLatest; public ApplicationMeta(String id, ApplicationSpecification spec, - @Nullable ChangeDetail change, @Nullable SourceControlMeta sourceControlMeta, - boolean isLatest) { + @Nullable ChangeDetail change, @Nullable SourceControlMeta sourceControlMeta) { this.id = id; this.spec = spec; this.change = change; this.sourceControlMeta = sourceControlMeta; - this.isLatest = isLatest; - } - - public ApplicationMeta(String id, ApplicationSpecification spec, - @Nullable ChangeDetail change, @Nullable SourceControlMeta sourceControlMeta) { - this(id, spec, change, sourceControlMeta, true); } public ApplicationMeta(String id, ApplicationSpecification spec, @Nullable ChangeDetail change) { @@ -77,10 +67,6 @@ public SourceControlMeta getSourceControlMeta() { return sourceControlMeta; } - public boolean getIsLatest() { - return isLatest; - } - @Override public String toString() { return Objects.toStringHelper(this) @@ -88,7 +74,6 @@ public String toString() { .add("spec", ADAPTER.toJson(spec)) .add("change", change) .add("sourceControlMeta", sourceControlMeta) - .add("isLatest", isLatest) .toString(); } } diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/store/DefaultStore.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/store/DefaultStore.java index 3b059a1c97e5..a35340f26445 100644 --- a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/store/DefaultStore.java +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/store/DefaultStore.java @@ -594,9 +594,16 @@ public void markApplicationsLatest(Collection appIds) } @Override - public int addApplication(ApplicationId id, ApplicationMeta meta) throws ConflictException { + public int addLatestApplication(ApplicationId id, ApplicationMeta meta) throws ConflictException { return TransactionRunners.run(transactionRunner, context -> { - return getAppMetadataStore(context).createApplicationVersion(id, meta); + return getAppMetadataStore(context).createLatestApplicationVersion(id, meta); + }, ConflictException.class); + } + + @Override + public int addApplication(ApplicationId id, ApplicationMeta meta, boolean isLatest) throws ConflictException { + return TransactionRunners.run(transactionRunner, context -> { + return getAppMetadataStore(context).createApplicationVersion(id, meta, isLatest); }, ConflictException.class); } diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/worker/TaskWorkerServiceLauncher.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/worker/TaskWorkerServiceLauncher.java index 7156bac11787..a776ee09f688 100644 --- a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/worker/TaskWorkerServiceLauncher.java +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/worker/TaskWorkerServiceLauncher.java @@ -213,6 +213,8 @@ public void run() { String.format("%s:%s", localhost, cConf.getInt(Constants.ArtifactLocalizer.PORT)) )); + twillPreparer = ((SecureTwillPreparer) twillPreparer) + .withNamespacedWorkloadIdentity(TaskWorkerTwillRunnable.class.getSimpleName()); } String priorityClass = cConf.get(Constants.TaskWorker.CONTAINER_PRIORITY_CLASS_NAME); diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/worker/sidecar/ArtifactLocalizerService.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/worker/sidecar/ArtifactLocalizerService.java index 2de2ac698e78..4205700ee570 100644 --- a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/worker/sidecar/ArtifactLocalizerService.java +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/worker/sidecar/ArtifactLocalizerService.java @@ -23,6 +23,7 @@ import io.cdap.cdap.common.conf.Constants; import io.cdap.cdap.common.http.CommonNettyHttpServiceFactory; import io.cdap.cdap.common.internal.remote.RemoteClientFactory; +import io.cdap.cdap.security.spi.authenticator.RemoteAuthenticator; import io.cdap.http.NettyHttpService; import java.net.InetAddress; import java.nio.file.Paths; @@ -52,7 +53,7 @@ public class ArtifactLocalizerService extends AbstractIdleService { ArtifactLocalizerService(CConfiguration cConf, ArtifactLocalizer artifactLocalizer, CommonNettyHttpServiceFactory commonNettyHttpServiceFactory, - RemoteClientFactory remoteClientFactory) { + RemoteClientFactory remoteClientFactory, RemoteAuthenticator remoteAuthenticator) { this.cConf = cConf; this.artifactLocalizer = artifactLocalizer; this.httpService = commonNettyHttpServiceFactory.builder(Constants.Service.TASK_WORKER) @@ -61,7 +62,7 @@ public class ArtifactLocalizerService extends AbstractIdleService { .setBossThreadPoolSize(cConf.getInt(Constants.ArtifactLocalizer.BOSS_THREADS)) .setWorkerThreadPoolSize(cConf.getInt(Constants.ArtifactLocalizer.WORKER_THREADS)) .setHttpHandlers(new ArtifactLocalizerHttpHandlerInternal(artifactLocalizer), - new GcpMetadataHttpHandlerInternal(cConf, remoteClientFactory)) + new GcpMetadataHttpHandlerInternal(cConf, remoteClientFactory, remoteAuthenticator)) .build(); this.cacheCleanupInterval = cConf.getInt( diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/worker/sidecar/ArtifactLocalizerTwillRunnable.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/worker/sidecar/ArtifactLocalizerTwillRunnable.java index b9e8d676dbef..584b0c9943c3 100644 --- a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/worker/sidecar/ArtifactLocalizerTwillRunnable.java +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/worker/sidecar/ArtifactLocalizerTwillRunnable.java @@ -25,9 +25,11 @@ import com.google.inject.Guice; import com.google.inject.Injector; import com.google.inject.Module; +import io.cdap.cdap.api.feature.FeatureFlagsProvider; import io.cdap.cdap.app.guice.DistributedArtifactManagerModule; import io.cdap.cdap.common.conf.CConfiguration; import io.cdap.cdap.common.conf.Constants; +import io.cdap.cdap.common.feature.DefaultFeatureFlagsProvider; import io.cdap.cdap.common.guice.ConfigModule; import io.cdap.cdap.common.guice.DFSLocationModule; import io.cdap.cdap.common.guice.IOModule; @@ -40,6 +42,7 @@ import io.cdap.cdap.common.logging.LoggingContext; import io.cdap.cdap.common.logging.LoggingContextAccessor; import io.cdap.cdap.common.logging.ServiceLoggingContext; +import io.cdap.cdap.features.Feature; import io.cdap.cdap.logging.appender.LogAppenderInitializer; import io.cdap.cdap.logging.guice.KafkaLogAppenderModule; import io.cdap.cdap.logging.guice.RemoteLogAppenderModule; @@ -100,7 +103,13 @@ public static Injector createInjector(CConfiguration cConf, Configuration hConf) modules.add(new ConfigModule(cConf, hConf)); modules.add(new IOModule()); - modules.add(RemoteAuthenticatorModules.getDefaultModule()); + FeatureFlagsProvider featureFlagsProvider = new DefaultFeatureFlagsProvider(cConf); + if (Feature.NAMESPACED_SERVICE_ACCOUNTS.isEnabled(featureFlagsProvider)) { + modules.add(RemoteAuthenticatorModules.getDefaultModule( + Constants.ArtifactLocalizer.REMOTE_AUTHENTICATOR_NAME)); + } else { + modules.add(RemoteAuthenticatorModules.getDefaultModule()); + } modules.add(new AuthenticationContextModules().getMasterModule()); modules.add(coreSecurityModule); modules.add(new MessagingServiceModule(cConf)); diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/worker/sidecar/GcpMetadataHttpHandlerInternal.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/worker/sidecar/GcpMetadataHttpHandlerInternal.java index cb860ffeb750..6a6cd2028c25 100644 --- a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/worker/sidecar/GcpMetadataHttpHandlerInternal.java +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/app/worker/sidecar/GcpMetadataHttpHandlerInternal.java @@ -16,6 +16,7 @@ package io.cdap.cdap.internal.app.worker.sidecar; +import com.google.common.base.Strings; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; @@ -37,17 +38,15 @@ import io.cdap.cdap.proto.credential.NamespaceCredentialProvider; import io.cdap.cdap.proto.credential.NotFoundException; import io.cdap.cdap.proto.credential.ProvisionedCredential; +import io.cdap.cdap.proto.security.Credential; import io.cdap.cdap.proto.security.GcpMetadataTaskContext; -import io.cdap.common.http.HttpRequests; -import io.cdap.common.http.HttpResponse; +import io.cdap.cdap.security.spi.authenticator.RemoteAuthenticator; import io.cdap.http.HttpHandler; import io.cdap.http.HttpResponder; import io.netty.handler.codec.http.DefaultHttpHeaders; import io.netty.handler.codec.http.FullHttpRequest; import io.netty.handler.codec.http.HttpRequest; import io.netty.handler.codec.http.HttpResponseStatus; -import java.io.IOException; -import java.net.URL; import java.time.Duration; import java.time.Instant; import java.util.concurrent.ExecutionException; @@ -57,7 +56,6 @@ import javax.ws.rs.PUT; import javax.ws.rs.Path; import javax.ws.rs.QueryParam; -import joptsimple.internal.Strings; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -73,8 +71,8 @@ public class GcpMetadataHttpHandlerInternal extends AbstractAppFabricHttpHandler private static final Gson GSON = new GsonBuilder().registerTypeAdapter(BasicThrowable.class, new BasicThrowableCodec()).create(); private final CConfiguration cConf; - private final String metadataServiceTokenEndpoint; private final NamespaceCredentialProvider credentialProvider; + private final RemoteAuthenticator remoteAuthenticator; private final GcpWorkloadIdentityInternalAuthenticator gcpWorkloadIdentityInternalAuthenticator; private GcpMetadataTaskContext gcpMetadataTaskContext; private final LoadingCache completionFuture = SettableFuture.create(); + private static final Logger LOG = LoggerFactory.getLogger(InMemoryOperationController.class); + + InMemoryOperationController(OperationRunId runId, OperationStatePublisher statePublisher, + OperationDriver driver) { + this.runId = runId; + this.driver = driver; + this.statePublisher = statePublisher; + startListen(driver); + } + + + @Override + public ListenableFuture stop() { + LOG.trace("Stopping operation {}", runId); + driver.stop(); + return completionFuture; + } + + @Override + public ListenableFuture complete() { + return completionFuture; + } + + private void startListen(Service service) { + service.addListener(new ServiceListenerAdapter() { + @Override + public void running() { + statePublisher.publishRunning(runId); + } + + @Override + public void terminated(Service.State from) { + if (from.equals(State.STOPPING)) { + statePublisher.publishKilled(runId); + } else { + statePublisher.publishSuccess(runId); + } + markComplete(); + } + + @Override + public void failed(Service.State from, Throwable failure) { + if (failure instanceof OperationException) { + statePublisher.publishFailed(runId, ((OperationException) failure).toOperationError()); + } else { + statePublisher.publishFailed(runId, getOperationErrorFromThrowable(failure)); + } + markComplete(); + } + }, Threads.SAME_THREAD_EXECUTOR); + } + + private void markComplete() { + completionFuture.set(this); + } + + private OperationError getOperationErrorFromThrowable(Throwable t) { + LOG.debug("Operation {} of namespace {} failed", runId.getRun(), runId.getParent(), t); + return new OperationError(t.getMessage(), Collections.emptyList()); + } +} diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/InMemoryOperationRunner.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/InMemoryOperationRunner.java new file mode 100644 index 000000000000..8b6c4697849f --- /dev/null +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/InMemoryOperationRunner.java @@ -0,0 +1,50 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.cdap.internal.operation; + +import com.google.inject.Inject; +import io.cdap.cdap.internal.app.sourcecontrol.PullAppsOperationFactory; + +/** + * Implementation of {@link OperationRunner} to run an operation in the same service. + */ +public class InMemoryOperationRunner extends AbstractOperationRunner { + + private final OperationStatePublisher statePublisher; + + /** + * Default constructor. + * + * @param statePublisher Publishes the current operation state. + */ + @Inject + public InMemoryOperationRunner(OperationStatePublisher statePublisher, + PullAppsOperationFactory pullOperationFactory) { + super(pullOperationFactory); + this.statePublisher = statePublisher; + } + + @Override + public OperationController run(OperationRunDetail detail) throws IllegalStateException { + LongRunningOperationContext context = new LongRunningOperationContext(detail.getRunId(), statePublisher); + OperationDriver driver = new OperationDriver(createOperation(detail), context); + OperationController controller = new InMemoryOperationController(context.getRunId(), + statePublisher, driver); + driver.start(); + return controller; + } +} diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/LongRunningOperationContext.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/LongRunningOperationContext.java index 112a571ed694..4de07fdc38c7 100644 --- a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/LongRunningOperationContext.java +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/LongRunningOperationContext.java @@ -18,27 +18,36 @@ import io.cdap.cdap.proto.id.OperationRunId; import io.cdap.cdap.proto.operation.OperationResource; -import io.cdap.cdap.proto.operation.OperationType; import java.util.Set; /** * Provides the context for the current operation run. */ -public interface LongRunningOperationContext { +public class LongRunningOperationContext { + + private final OperationRunId runId; + private final OperationStatePublisher statePublisher; /** - * Get the {@link OperationRunId} for the current run. + * Default constructor. * - * @return the current runid + * @param runId id of the current operation + * @param type type of the current operation + * @param statePublisher to publish the operation metadata */ - OperationRunId getRunId(); + public LongRunningOperationContext(OperationRunId runId, OperationStatePublisher statePublisher) { + this.runId = runId; + this.statePublisher = statePublisher; + } /** - * Get the {@link OperationType} to be used by the runner for loading the right operation class. + * Get the {@link OperationRunId} for the current run. * - * @return the type of the current operation + * @return the current runid */ - OperationType getType(); + public OperationRunId getRunId() { + return runId; + } /** * Used by the {@link LongRunningOperation} to update the resources operated on in the @@ -46,8 +55,8 @@ public interface LongRunningOperationContext { * resources to be unique * * @param resources A set of resources to be updated. - * */ - // TODO Add exceptions based on implementations. - void updateOperationResources(Set resources); + public void updateOperationResources(Set resources) { + statePublisher.publishResources(runId, resources); + } } diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/MessagingOperationStatePublisher.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/MessagingOperationStatePublisher.java new file mode 100644 index 000000000000..4c98c3caf79e --- /dev/null +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/MessagingOperationStatePublisher.java @@ -0,0 +1,220 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.cdap.internal.operation; + + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.google.gson.Gson; +import com.google.inject.Inject; +import io.cdap.cdap.api.messaging.TopicNotFoundException; +import io.cdap.cdap.api.security.AccessException; +import io.cdap.cdap.api.service.ServiceUnavailableException; +import io.cdap.cdap.common.conf.CConfiguration; +import io.cdap.cdap.common.conf.Constants.Operation; +import io.cdap.cdap.common.service.RetryStrategies; +import io.cdap.cdap.common.service.RetryStrategy; +import io.cdap.cdap.messaging.client.StoreRequestBuilder; +import io.cdap.cdap.messaging.spi.MessagingService; +import io.cdap.cdap.proto.Notification; +import io.cdap.cdap.proto.id.NamespaceId; +import io.cdap.cdap.proto.id.OperationRunId; +import io.cdap.cdap.proto.id.TopicId; +import io.cdap.cdap.proto.operation.OperationError; +import io.cdap.cdap.proto.operation.OperationResource; +import io.cdap.cdap.proto.operation.OperationRunStatus; +import java.io.IOException; +import java.time.Instant; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Provides capabilities to send operation lifecycle specific messages. + */ +public class MessagingOperationStatePublisher implements OperationStatePublisher { + + private final MessagingService messagingService; + private final RetryStrategy retryStrategy; + private final List topicIds; + + private static final Gson GSON = new Gson(); + private static final Logger LOG = LoggerFactory.getLogger( + MessagingOperationStatePublisher.class); + + /** + * Create a publisher that writes to MessagingService topics depending on the message content. + * + * @param cConf configuration containing the topic prefix and number of partitions + * @param messagingService messaging service to write messages to + */ + @Inject + public MessagingOperationStatePublisher(CConfiguration cConf, MessagingService messagingService) { + this( + messagingService, + cConf.get(Operation.STATUS_EVENT_TOPIC), + cConf.getInt(Operation.STATUS_EVENT_NUM_PARTITIONS), + RetryStrategies.fromConfiguration( + cConf, Operation.STATUS_RETRY_STRATEGY_PREFIX) + ); + } + + /** + * Create a publisher that writes to MessagingService topics depending on the message content. + * + * @param messagingService messaging service to write messages to + * @param topicPrefix prefix of the topic(s) to write to. If there is one topic, the prefix + * will be the topic name. If there is more than one topic, the topic name will be the prefix + * followed by the topic number + * @param numTopics number of topics to write to + * @param retryStrategy retry strategy to use for failures + */ + @VisibleForTesting + public MessagingOperationStatePublisher(MessagingService messagingService, + String topicPrefix, int numTopics, RetryStrategy retryStrategy) { + this.messagingService = messagingService; + this.topicIds = Collections.unmodifiableList(IntStream + .range(0, numTopics) + .mapToObj(i -> NamespaceId.SYSTEM.topic(topicPrefix + i)) + .collect(Collectors.toList())); + this.retryStrategy = retryStrategy; + } + + @Override + public void publishResources(OperationRunId runId, Set resources) { + ImmutableMap.Builder propertiesBuilder = ImmutableMap.builder() + .put(Operation.RUN_ID_NOTIFICATION_KEY, GSON.toJson(runId)) + .put(Operation.STATUS_NOTIFICATION_KEY, OperationRunStatus.RUNNING.name()) + .put(Operation.RESOURCES_NOTIFICATION_KEY, GSON.toJson(resources)); + + publish(runId, propertiesBuilder.build()); + } + + @Override + public void publishRunning(OperationRunId runId) { + ImmutableMap.Builder propertiesBuilder = ImmutableMap.builder() + .put(Operation.RUN_ID_NOTIFICATION_KEY, GSON.toJson(runId)) + .put(Operation.STATUS_NOTIFICATION_KEY, OperationRunStatus.RUNNING.name()); + publish(runId, propertiesBuilder.build()); + } + + @Override + public void publishFailed(OperationRunId runId, OperationError error) { + ImmutableMap.Builder propertiesBuilder = ImmutableMap.builder() + .put(Operation.RUN_ID_NOTIFICATION_KEY, GSON.toJson(runId)) + .put(Operation.STATUS_NOTIFICATION_KEY, OperationRunStatus.FAILED.name()) + .put(Operation.ERROR_NOTIFICATION_KEY, GSON.toJson(error)) + .put(Operation.ENDTIME_NOTIFICATION_KEY, Instant.now().toString()); + publish(runId, propertiesBuilder.build()); + } + + @Override + public void publishSuccess(OperationRunId runId) { + ImmutableMap.Builder propertiesBuilder = ImmutableMap.builder() + .put(Operation.RUN_ID_NOTIFICATION_KEY, GSON.toJson(runId)) + .put(Operation.STATUS_NOTIFICATION_KEY, OperationRunStatus.SUCCEEDED.name()) + .put(Operation.ENDTIME_NOTIFICATION_KEY, Instant.now().toString()); + publish(runId, propertiesBuilder.build()); + } + + @Override + public void publishKilled(OperationRunId runId) { + ImmutableMap.Builder propertiesBuilder = ImmutableMap.builder() + .put(Operation.RUN_ID_NOTIFICATION_KEY, GSON.toJson(runId)) + .put(Operation.STATUS_NOTIFICATION_KEY, OperationRunStatus.KILLED.name()) + .put(Operation.ENDTIME_NOTIFICATION_KEY, Instant.now().toString()); + publish(runId, propertiesBuilder.build()); + } + + @Override + public void publishStopping(OperationRunId runId) { + ImmutableMap.Builder propertiesBuilder = ImmutableMap.builder() + .put(Operation.RUN_ID_NOTIFICATION_KEY, GSON.toJson(runId)) + .put(Operation.STATUS_NOTIFICATION_KEY, OperationRunStatus.STOPPING.name()); + publish(runId, propertiesBuilder.build()); + } + + @Override + public void publishStarting(OperationRunId runId) { + ImmutableMap.Builder propertiesBuilder = ImmutableMap.builder() + .put(Operation.RUN_ID_NOTIFICATION_KEY, GSON.toJson(runId)) + .put(Operation.STATUS_NOTIFICATION_KEY, OperationRunStatus.STARTING.name()); + publish(runId, propertiesBuilder.build()); + } + + /** + * Publish a notification to a topic. + * + * @param runId {@link OperationRunId} for the notification + * @param properties properties of the message to publish, assumed to contain the operation + * run id + */ + public void publish(OperationRunId runId, Map properties) { + // OperationRunId is always required in a notification + Notification notification = new Notification(Notification.Type.OPERATION_STATUS, properties); + + int failureCount = 0; + long startTime = -1L; + boolean done = false; + // TODO CDAP-12255 This should be refactored into a common class for publishing to TMS with a retry strategy + while (!done) { + try { + messagingService.publish(StoreRequestBuilder.of(getTopic(runId)) + .addPayload(GSON.toJson(notification)) + .build()); + LOG.trace("Published operation status notification: {}", notification); + done = true; + } catch (IOException | AccessException e) { + throw Throwables.propagate(e); + } catch (TopicNotFoundException | ServiceUnavailableException e) { + // These exceptions are retry-able due to TMS not completely started + if (startTime < 0) { + startTime = System.currentTimeMillis(); + } + long retryMillis = retryStrategy.nextRetry(++failureCount, startTime); + if (retryMillis < 0) { + LOG.error("Failed to publish messages to TMS and exceeded retry limit.", e); + throw Throwables.propagate(e); + } + LOG.debug("Failed to publish messages to TMS due to {}. Will be retried in {} ms.", + e.getMessage(), retryMillis); + try { + TimeUnit.MILLISECONDS.sleep(retryMillis); + } catch (InterruptedException e1) { + // Something explicitly stopping this thread. Simply just break and reset the interrupt flag. + LOG.warn("Publishing message to TMS interrupted."); + Thread.currentThread().interrupt(); + done = true; + } + } + } + } + + private TopicId getTopic(OperationRunId runId) { + if (topicIds.size() == 1) { + return topicIds.get(0); + } + return topicIds.get(Math.abs(runId.getRun().hashCode()) % topicIds.size()); + } +} diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/AbstractLongRunningOperationContext.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationController.java similarity index 54% rename from cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/AbstractLongRunningOperationContext.java rename to cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationController.java index de054860b75a..9522468d460d 100644 --- a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/AbstractLongRunningOperationContext.java +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationController.java @@ -16,22 +16,21 @@ package io.cdap.cdap.internal.operation; -import io.cdap.cdap.proto.id.OperationRunId; -import io.cdap.cdap.proto.operation.OperationType; +import com.google.common.util.concurrent.ListenableFuture; /** - * Abstract implementation of {@link LongRunningOperationContext} providing shared functionalities. + * Provides lifecycle hooks for managing the state of an operation. */ -public abstract class AbstractLongRunningOperationContext implements LongRunningOperationContext { +public interface OperationController { + + /** + * Attempt to stop the operation. + **/ + ListenableFuture stop(); - private final OperationRunId runId; - private final OperationType type; /** - * Default constructor. + * Returns a future which can be used to block till the operation is completed. */ - protected AbstractLongRunningOperationContext(OperationRunId runid, OperationType operationType) { - this.runId = runid; - this.type = operationType; - } + ListenableFuture complete(); } diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationDriver.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationDriver.java new file mode 100644 index 000000000000..8e6e4504a19b --- /dev/null +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationDriver.java @@ -0,0 +1,61 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.cdap.internal.operation; + +import com.google.common.util.concurrent.AbstractExecutionThreadService; +import io.cdap.cdap.proto.operation.OperationResource; +import java.util.Set; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import org.apache.twill.common.Threads; + +/** + * A Service for executing {@link LongRunningOperation}. + */ +class OperationDriver extends AbstractExecutionThreadService { + + private final LongRunningOperation operation; + private final LongRunningOperationContext context; + private ExecutorService executor; + + OperationDriver(LongRunningOperation operation, LongRunningOperationContext context) { + this.operation = operation; + this.context = context; + } + + @Override + protected void run() throws Exception { + Set resources = operation.run(context).get(); + context.updateOperationResources(resources); + } + + @Override + protected void triggerShutdown() { + if (executor != null) { + executor.shutdownNow(); + } + } + + @Override + protected Executor executor() { + executor = Executors.newSingleThreadExecutor( + Threads.createDaemonThreadFactory("operation-runner" + context.getRunId() + "-%d") + ); + return executor; + } +} diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationException.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationException.java index 283b68df1202..f5e3f1c18dd5 100644 --- a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationException.java +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationException.java @@ -32,6 +32,11 @@ public OperationException(String message, Collection errors, Throwable cause) { + super(message, cause); + this.errors = errors; + } + public OperationError toOperationError() { return new OperationError(getMessage(), errors); } diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationLifecycleManager.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationLifecycleManager.java index 1bede510c5a8..41384fe1681c 100644 --- a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationLifecycleManager.java +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationLifecycleManager.java @@ -17,6 +17,9 @@ package io.cdap.cdap.internal.operation; import com.google.inject.Inject; +import io.cdap.cdap.proto.operation.OperationRunStatus; +import io.cdap.cdap.spi.data.InvalidFieldException; +import io.cdap.cdap.proto.id.OperationRunId; import io.cdap.cdap.spi.data.StructuredTableContext; import io.cdap.cdap.spi.data.transaction.TransactionRunner; import io.cdap.cdap.spi.data.transaction.TransactionRunners; @@ -70,6 +73,40 @@ public boolean scanOperations(ScanOperationRunsRequest request, int txBatchSize, return currentLimit == 0; } + + /** + * Scan all pending operations. Needed for try running all pending operation during startup. + * + * @param consumer {@link Consumer} to process each scanned run + */ + public void scanPendingOperations(Consumer consumer) + throws IOException, InvalidFieldException { + TransactionRunners.run(transactionRunner, context -> { + getOperationRunStore(context).scanOperationByStatus(OperationRunStatus.PENDING, consumer); + }, IOException.class, InvalidFieldException.class); + } + + /** + * Retrieves details of an operation run identified by the provided {@code OperationRunId}. + * + * @param runId The unique identifier for the operation run. + * @return An {@code OperationRunDetail} object containing information about the specified operation run. + * @throws OperationRunNotFoundException If the specified operation run is not found. + */ + public OperationRunDetail getOperationRun(OperationRunId runId) + throws IOException, OperationRunNotFoundException { + OperationRunDetail operationRunDetail = + TransactionRunners.run( + transactionRunner, + context -> { + return getOperationRunStore(context).getOperation(runId); + }, + IOException.class, + OperationRunNotFoundException.class); + + return operationRunDetail; + } + private OperationRunStore getOperationRunStore(StructuredTableContext context) { return new OperationRunStore(context); } diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationNotification.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationNotification.java new file mode 100644 index 000000000000..c4ae62ad7c81 --- /dev/null +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationNotification.java @@ -0,0 +1,106 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.cdap.internal.operation; + +import com.google.gson.Gson; +import com.google.gson.reflect.TypeToken; +import io.cdap.cdap.common.conf.Constants.Operation; +import io.cdap.cdap.proto.Notification; +import io.cdap.cdap.proto.id.OperationRunId; +import io.cdap.cdap.proto.operation.OperationError; +import io.cdap.cdap.proto.operation.OperationResource; +import io.cdap.cdap.proto.operation.OperationRunStatus; +import java.lang.reflect.Type; +import java.time.Instant; +import java.util.Map; +import java.util.Set; +import javax.annotation.Nullable; + +/** + * Encapsulates an operation notification sent through TMS. + */ +public class OperationNotification { + + private final OperationRunId runId; + private final OperationRunStatus status; + @Nullable + private final Set resources; + @Nullable + private final Instant endTime; + @Nullable + private final OperationError error; + + private static final Gson GSON = new Gson(); + private static final Type resourcesType = new TypeToken>() { + }.getType(); + + /** + * Default constructor. + */ + public OperationNotification(OperationRunId runId, OperationRunStatus status, + @Nullable Set resources, Instant endTime, @Nullable OperationError error) { + this.runId = runId; + this.status = status; + this.resources = resources; + this.endTime = endTime; + this.error = error; + } + + /** + * Parse {@link Notification} to generate {@link OperationNotification}. + * + * @param notification notification to parse + */ + public static OperationNotification fromNotification(Notification notification) { + Map properties = notification.getProperties(); + + OperationRunId runId = GSON.fromJson(properties.get(Operation.RUN_ID_NOTIFICATION_KEY), + OperationRunId.class); + OperationRunStatus status = OperationRunStatus.valueOf( + properties.get(Operation.STATUS_NOTIFICATION_KEY)); + OperationError error = GSON.fromJson(properties.get(Operation.ERROR_NOTIFICATION_KEY), + OperationError.class); + Set resources = GSON.fromJson( + properties.get(Operation.RESOURCES_NOTIFICATION_KEY), resourcesType); + Instant endTime = Instant.parse(properties.get(Operation.ENDTIME_NOTIFICATION_KEY)); + + return new OperationNotification(runId, status, resources, endTime, error); + } + + public OperationRunId getRunId() { + return runId; + } + + public OperationRunStatus getStatus() { + return status; + } + + @Nullable + public Set getResources() { + return resources; + } + + @Nullable + public Instant getEndTime() { + return endTime; + } + + @Nullable + public OperationError getError() { + return error; + } +} diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationRunDetail.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationRunDetail.java index 4da517858bc1..e2780790bd24 100644 --- a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationRunDetail.java +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationRunDetail.java @@ -100,7 +100,8 @@ public boolean equals(Object o) { } OperationRunDetail that = (OperationRunDetail) o; - return Objects.equal(this.getRun(), that.getRun()) + return Objects.equal(this.getRunId(), that.getRunId()) + && Objects.equal(this.getRun(), that.getRun()) && Arrays.equals(this.getSourceId(), that.getSourceId()) && Objects.equal(this.getPullAppsRequest(), that.getPullAppsRequest()) && Objects.equal(this.getPrincipal(), that.getPrincipal()); @@ -108,8 +109,7 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hashCode(super.hashCode(), Arrays.hashCode(getSourceId()), getPrincipal(), - getPullAppsRequest()); + return Objects.hashCode(runId, run, Arrays.hashCode(sourceId), principal, pullAppsRequest); } /** diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationRunStore.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationRunStore.java index 28a7139e4558..666fa583c023 100644 --- a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationRunStore.java +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationRunStore.java @@ -24,6 +24,7 @@ import io.cdap.cdap.proto.id.OperationRunId; import io.cdap.cdap.proto.operation.OperationError; import io.cdap.cdap.proto.operation.OperationMeta; +import io.cdap.cdap.proto.operation.OperationResource; import io.cdap.cdap.proto.operation.OperationRun; import io.cdap.cdap.proto.operation.OperationRunStatus; import io.cdap.cdap.spi.data.SortOrder; @@ -34,12 +35,15 @@ import io.cdap.cdap.spi.data.table.field.Fields; import io.cdap.cdap.spi.data.table.field.Range; import io.cdap.cdap.store.StoreDefinition; +import io.cdap.cdap.store.StoreDefinition.OperationRunsStore; import java.io.IOException; import java.time.Clock; +import java.time.Instant; import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Optional; +import java.util.Set; import java.util.function.Consumer; import javax.annotation.Nullable; @@ -71,7 +75,7 @@ public OperationRunStore(StructuredTableContext context) { * @param detail the run details of the operation * @throws OperationRunAlreadyExistsException when a run with same id exist in namespace */ - public void createOperationRun(OperationRunId runId, OperationRunDetail detail) + public OperationRunDetail createOperationRun(OperationRunId runId, OperationRunDetail detail) throws OperationRunAlreadyExistsException, IOException { Optional row = getOperationRunInternal(runId); if (row.isPresent()) { @@ -80,22 +84,23 @@ public void createOperationRun(OperationRunId runId, OperationRunDetail detail) throw new OperationRunAlreadyExistsException(runId.getRun(), status); } writeOperationRun(runId, detail); + return detail; } /** - * Update the metadata of an operation run. + * Update the resources of an operation run. * * @param runId {@link OperationRunId} for the run - * @param metadata new metdata of the run + * @param resources updated resources for the run * @param sourceId the message id which is responsible for the update * @throws OperationRunNotFoundException run with id does not exist in namespace */ - public void updateOperationMeta(OperationRunId runId, OperationMeta metadata, + public void updateOperationResources(OperationRunId runId, Set resources, @Nullable byte[] sourceId) throws OperationRunNotFoundException, IOException { OperationRunDetail currentDetail = getRunDetail(runId); OperationRun currentRun = currentDetail.getRun(); - OperationRun updatedRun = OperationRun.builder(currentRun) - .setMetadata(metadata).build(); + OperationRun updatedRun = OperationRun.builder(currentRun).setMetadata( + OperationMeta.builder(currentRun.getMetadata()).setResources(resources).build()).build(); OperationRunDetail updatedDetail = OperationRunDetail.builder(currentDetail) .setRun(updatedRun).setSourceId(sourceId).build(); @@ -145,12 +150,14 @@ public void updateOperationStatus(OperationRunId runId, OperationRunStatus statu * @param sourceId the message id which is responsible for the update * @throws OperationRunNotFoundException run with id does not exist in namespace */ - public void failOperationRun(OperationRunId runId, OperationError error, + public void failOperationRun(OperationRunId runId, OperationError error, Instant endTime, @Nullable byte[] sourceId) throws OperationRunNotFoundException, IOException { OperationRunDetail currentDetail = getRunDetail(runId); OperationRun currentRun = currentDetail.getRun(); + OperationMeta updatedMeta = OperationMeta.builder(currentDetail.getRun().getMetadata()) + .setEndTime(endTime).build(); OperationRun updatedRun = OperationRun.builder(currentRun) - .setStatus(OperationRunStatus.FAILED).setError(error).build(); + .setStatus(OperationRunStatus.FAILED).setError(error).setMetadata(updatedMeta).build(); OperationRunDetail updatedDetail = OperationRunDetail.builder(currentDetail) .setRun(updatedRun).setSourceId(sourceId).build(); @@ -174,14 +181,8 @@ public void failOperationRun(OperationRunId runId, OperationError error, */ public OperationRunDetail getOperation(OperationRunId runId) throws OperationRunNotFoundException, IOException { - Optional row = getOperationRunInternal(runId); - if (!row.isPresent()) { - throw new OperationRunNotFoundException(runId.getNamespace(), runId.getRun()); - } - return GSON.fromJson( - row.get().getString(StoreDefinition.OperationRunsStore.DETAILS_FIELD), - OperationRunDetail.class - ); + return getOperationRunInternal(runId).map(this::rowToRunDetail) + .orElseThrow(() -> new OperationRunNotFoundException(runId.getNamespace(), runId.getRun())); } @@ -222,8 +223,8 @@ public String scanOperations(ScanOperationRunsRequest request, if (request.getScanAfter() != null) { startBound = Range.Bound.EXCLUSIVE; - startFields = getRangeFields( - new OperationRunId(request.getNamespace(), request.getScanAfter())); + startFields.addAll(getRangeFields( + new OperationRunId(request.getNamespace(), request.getScanAfter()))); } Range range = Range.create(endFields, endBound, startFields, startBound); @@ -235,10 +236,7 @@ public String scanOperations(ScanOperationRunsRequest request, while (iterator.hasNext()) { StructuredRow row = iterator.next(); lastKey = row.getString(StoreDefinition.OperationRunsStore.ID_FIELD); - OperationRunDetail detail = GSON.fromJson( - row.getString(StoreDefinition.OperationRunsStore.DETAILS_FIELD), - OperationRunDetail.class - ); + OperationRunDetail detail = rowToRunDetail(row); consumer.accept(detail); } } @@ -246,6 +244,22 @@ public String scanOperations(ScanOperationRunsRequest request, return lastKey; } + /** + * Returns runs with a given status for all namespaces. + */ + public void scanOperationByStatus(OperationRunStatus status, + Consumer consumer) throws IOException { + try (CloseableIterator iterator = getOperationRunsTable(context).scan( + Fields.stringField(OperationRunsStore.STATUS_FIELD, status.name()) + )) { + while (iterator.hasNext()) { + StructuredRow row = iterator.next(); + OperationRunDetail detail = rowToRunDetail(row); + consumer.accept(detail); + } + } + } + private List> getRangeFields(OperationRunId runId) throws IOException, OperationRunNotFoundException { List> fields = new ArrayList<>(); @@ -338,4 +352,4 @@ void clearData() throws IOException { SMALLEST_POSSIBLE_STRING)), Range.Bound.INCLUSIVE)); } -} +} \ No newline at end of file diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationRunner.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationRunner.java new file mode 100644 index 000000000000..848d973325a1 --- /dev/null +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationRunner.java @@ -0,0 +1,31 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.cdap.internal.operation; + +/** + * Interface representing runner for LRO. + * A runner initiates the run and returns a {@link OperationController} for lifecycle management. + */ +public interface OperationRunner { + + /** + * Run an operation in asynchronous mode. + * + * @param runDetail {@link OperationRunDetail} for the operation to be run + */ + OperationController run(OperationRunDetail runDetail) throws IllegalStateException; +} diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationStatePublisher.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationStatePublisher.java new file mode 100644 index 000000000000..f8970ede3cf7 --- /dev/null +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/OperationStatePublisher.java @@ -0,0 +1,60 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.cdap.internal.operation; + + +import io.cdap.cdap.proto.id.OperationRunId; +import io.cdap.cdap.proto.operation.OperationError; +import io.cdap.cdap.proto.operation.OperationResource; +import java.util.Set; + +/** + * Publishes operation state messages. + */ +public interface OperationStatePublisher { + + /** + * Publishes message with the current resources. The operation status should be RUNNING + * + * @param resources Current resources for the operation. + */ + void publishResources(OperationRunId runId, Set resources); + + /** + * Publishes the current operation status as RUNNING. + */ + void publishRunning(OperationRunId runId); + + /** + * Publishes the current operation status as FAILED. + */ + void publishFailed(OperationRunId runId, OperationError error); + + /** + * Publishes the current operation status as SUCCEEDED. + */ + void publishSuccess(OperationRunId runId); + + /** + * Publishes the current operation status as KILLED. + */ + void publishKilled(OperationRunId runId); + + void publishStopping(OperationRunId runId); + + void publishStarting(OperationRunId runId); +} diff --git a/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/guice/OperationModule.java b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/guice/OperationModule.java new file mode 100644 index 000000000000..dcc694ad58d8 --- /dev/null +++ b/cdap-app-fabric/src/main/java/io/cdap/cdap/internal/operation/guice/OperationModule.java @@ -0,0 +1,48 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.cdap.internal.operation.guice; + +import com.google.inject.AbstractModule; +import com.google.inject.assistedinject.FactoryModuleBuilder; +import io.cdap.cdap.internal.app.sourcecontrol.LocalApplicationManager; +import io.cdap.cdap.internal.app.sourcecontrol.PullAppsOperation; +import io.cdap.cdap.internal.app.sourcecontrol.PullAppsOperationFactory; +import io.cdap.cdap.internal.operation.InMemoryOperationRunner; +import io.cdap.cdap.internal.operation.LongRunningOperation; +import io.cdap.cdap.internal.operation.MessagingOperationStatePublisher; +import io.cdap.cdap.internal.operation.OperationRunner; +import io.cdap.cdap.internal.operation.OperationStatePublisher; +import io.cdap.cdap.sourcecontrol.ApplicationManager; + + +/** + * Guice module for operation classes. + */ +public class OperationModule extends AbstractModule { + + @Override + protected void configure() { + install(new FactoryModuleBuilder() + .implement(LongRunningOperation.class, PullAppsOperation.class) + .build(PullAppsOperationFactory.class)); + // TODO(samik) change based on worker enabled on not + bind(ApplicationManager.class).to(LocalApplicationManager.class); + bind(OperationRunner.class).to(InMemoryOperationRunner.class); + bind(OperationStatePublisher.class).to(MessagingOperationStatePublisher.class); + } +} + diff --git a/cdap-app-fabric/src/test/java/io/cdap/cdap/gateway/handlers/OperationsDashboardHttpHandlerTest.java b/cdap-app-fabric/src/test/java/io/cdap/cdap/gateway/handlers/OperationsDashboardHttpHandlerTest.java index 5f3105da0951..0544abe726d8 100644 --- a/cdap-app-fabric/src/test/java/io/cdap/cdap/gateway/handlers/OperationsDashboardHttpHandlerTest.java +++ b/cdap-app-fabric/src/test/java/io/cdap/cdap/gateway/handlers/OperationsDashboardHttpHandlerTest.java @@ -301,7 +301,7 @@ private void addAppSpecs() throws ConflictException { ApplicationMeta meta = new ApplicationMeta(dummyAppSpec1.getName(), dummyAppSpec1, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(APP1_ID, meta); + store.addLatestApplication(APP1_ID, meta); WorkflowSpecification scheduledWorfklow2 = new WorkflowSpecification("DummyClass", SCHEDULED_PROG2_ID.getProgram(), "scheduled workflow", Collections.emptyMap(), Collections.emptyList(), Collections.emptyMap(), @@ -319,7 +319,7 @@ private void addAppSpecs() throws ConflictException { meta = new ApplicationMeta(dummyAppSpec2.getName(), dummyAppSpec2, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(APP2_ID, meta); + store.addLatestApplication(APP2_ID, meta); } /** diff --git a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/deploy/RemoteConfiguratorTest.java b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/deploy/RemoteConfiguratorTest.java index 6566cde8eb83..4e93db8b031b 100644 --- a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/deploy/RemoteConfiguratorTest.java +++ b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/deploy/RemoteConfiguratorTest.java @@ -39,6 +39,7 @@ import io.cdap.cdap.common.http.CommonNettyHttpServiceBuilder; import io.cdap.cdap.common.id.Id; import io.cdap.cdap.common.internal.remote.DefaultInternalAuthenticator; +import io.cdap.cdap.common.internal.remote.NoOpRemoteAuthenticator; import io.cdap.cdap.common.internal.remote.RemoteClientFactory; import io.cdap.cdap.common.internal.remote.TaskWorkerHttpHandlerInternal; import io.cdap.cdap.common.metrics.NoOpMetricsCollectionService; @@ -126,7 +127,8 @@ public static void init() throws Exception { new ArtifactLocalizer(cConf, remoteClientFactory, ((namespaceId, retryStrategy) -> new NoOpArtifactManager())) ), - new GcpMetadataHttpHandlerInternal(cConf, remoteClientFactory) + new GcpMetadataHttpHandlerInternal(cConf, remoteClientFactory, + new NoOpRemoteAuthenticator()) ) .setChannelPipelineModifier(new ChannelPipelineModifier() { @Override diff --git a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/services/SourceControlManagementServiceTest.java b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/services/SourceControlManagementServiceTest.java index c560e66eb13f..0ce6f99b9b6c 100644 --- a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/services/SourceControlManagementServiceTest.java +++ b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/services/SourceControlManagementServiceTest.java @@ -45,11 +45,13 @@ import io.cdap.cdap.proto.sourcecontrol.SourceControlMeta; import io.cdap.cdap.security.impersonation.CurrentUGIProvider; import io.cdap.cdap.security.impersonation.UGIProvider; +import io.cdap.cdap.sourcecontrol.ApplicationManager; import io.cdap.cdap.sourcecontrol.AuthenticationConfigException; import io.cdap.cdap.sourcecontrol.NoChangesToPullException; import io.cdap.cdap.sourcecontrol.NoChangesToPushException; import io.cdap.cdap.sourcecontrol.SourceControlException; import io.cdap.cdap.sourcecontrol.operationrunner.MultiPullAppOperationRequest; +import io.cdap.cdap.sourcecontrol.operationrunner.MultiPushAppOperationRequest; import io.cdap.cdap.sourcecontrol.operationrunner.NamespaceRepository; import io.cdap.cdap.sourcecontrol.operationrunner.PullAppOperationRequest; import io.cdap.cdap.sourcecontrol.operationrunner.PullAppResponse; @@ -495,6 +497,13 @@ public PushAppResponse push(PushAppOperationRequest pushAppOperationRequest) return null; } + @Override + public List multiPush(MultiPushAppOperationRequest pushRequest, + ApplicationManager appManager) + throws NoChangesToPushException, AuthenticationConfigException { + return null; + } + @Override public PullAppResponse pull(PullAppOperationRequest pullRequest) throws NotFoundException, AuthenticationConfigException { @@ -502,7 +511,7 @@ public PullAppResponse pull(PullAppOperationRequest pullRequest) } @Override - public void pull(MultiPullAppOperationRequest pullRequest, Consumer> consumer) + public void multiPull(MultiPullAppOperationRequest pullRequest, Consumer> consumer) throws NotFoundException, AuthenticationConfigException { } diff --git a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/services/http/handlers/PreferencesHttpHandlerInternalTest.java b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/services/http/handlers/PreferencesHttpHandlerInternalTest.java index 18c92143d9c0..13cc80c823fa 100644 --- a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/services/http/handlers/PreferencesHttpHandlerInternalTest.java +++ b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/services/http/handlers/PreferencesHttpHandlerInternalTest.java @@ -54,7 +54,7 @@ private void addApplication(String namespace, Application app) throws ConflictEx ApplicationMeta meta = new ApplicationMeta(appSpec.getName(), appSpec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(new ApplicationId(namespace, appSpec.getName()), meta); + store.addLatestApplication(new ApplicationId(namespace, appSpec.getName()), meta); } @Test diff --git a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/services/http/handlers/PreferencesHttpHandlerTest.java b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/services/http/handlers/PreferencesHttpHandlerTest.java index 0da55d73ffcc..13d4e68254a1 100644 --- a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/services/http/handlers/PreferencesHttpHandlerTest.java +++ b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/services/http/handlers/PreferencesHttpHandlerTest.java @@ -55,7 +55,7 @@ private void addApplication(String namespace, Application app) throws ConflictEx ApplicationMeta meta = new ApplicationMeta(appSpec.getName(), appSpec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(new ApplicationId(namespace, appSpec.getName()), meta); + store.addLatestApplication(new ApplicationId(namespace, appSpec.getName()), meta); } @Test diff --git a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/sourcecontrol/PullAppsOperationTest.java b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/sourcecontrol/PullAppsOperationTest.java index 168e7e66daf5..411727d131c8 100644 --- a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/sourcecontrol/PullAppsOperationTest.java +++ b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/sourcecontrol/PullAppsOperationTest.java @@ -39,7 +39,6 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; -import java.nio.file.Paths; import java.util.Collection; import java.util.HashSet; import java.util.Set; @@ -75,10 +74,11 @@ public void setUp() throws Exception { RepositoryManager mockRepositoryManager = Mockito.mock(RepositoryManager.class); Mockito.doReturn(mockRepositoryManager).when(mockRepositoryManagerFactory) .create(Mockito.any(), Mockito.any()); - Mockito.doReturn(Paths.get("")).when(mockRepositoryManager).getRepositoryRoot(); - Mockito.doReturn(repositoryBase.getRoot().toPath()).when(mockRepositoryManager) - .getRepositoryRoot(); - Mockito.doReturn(repositoryBase.getRoot().toPath()).when(mockRepositoryManager).getBasePath(); + + Path rootPath = repositoryBase.getRoot().toPath(); + Mockito.doReturn(rootPath).when(mockRepositoryManager).getRepositoryRoot(); + Mockito.doReturn(rootPath).when(mockRepositoryManager).getBasePath(); + Mockito.doReturn("testHash") .when(mockRepositoryManager) .getFileHash(Mockito.any(), Mockito.any()); @@ -127,7 +127,6 @@ public void testRunSuccess() throws Exception { @Test(expected = OperationException.class) public void testRunFailedAtFirstApp() throws Exception { ApplicationManager mockManager = Mockito.mock(ApplicationManager.class); - LongRunningOperationContext mockContext = Mockito.mock(LongRunningOperationContext.class); PullAppsOperation operation = new PullAppsOperation(this.req, opRunner, mockManager); Mockito.doThrow(new SourceControlException("")).when(mockManager) diff --git a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/sourcecontrol/PushAppsOperationTest.java b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/sourcecontrol/PushAppsOperationTest.java new file mode 100644 index 000000000000..78aeee93bf34 --- /dev/null +++ b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/sourcecontrol/PushAppsOperationTest.java @@ -0,0 +1,228 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.cdap.internal.app.sourcecontrol; + + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.gson.Gson; +import com.google.inject.AbstractModule; +import com.google.inject.Injector; +import io.cdap.cdap.common.NotFoundException; +import io.cdap.cdap.common.conf.CConfiguration; +import io.cdap.cdap.internal.AppFabricTestHelper; +import io.cdap.cdap.internal.operation.LongRunningOperationContext; +import io.cdap.cdap.internal.operation.OperationException; +import io.cdap.cdap.proto.ApplicationDetail; +import io.cdap.cdap.proto.id.ApplicationId; +import io.cdap.cdap.proto.id.ApplicationReference; +import io.cdap.cdap.proto.id.NamespaceId; +import io.cdap.cdap.proto.id.OperationRunId; +import io.cdap.cdap.proto.operation.OperationResource; +import io.cdap.cdap.proto.sourcecontrol.AuthConfig; +import io.cdap.cdap.proto.sourcecontrol.AuthType; +import io.cdap.cdap.proto.sourcecontrol.PatConfig; +import io.cdap.cdap.proto.sourcecontrol.Provider; +import io.cdap.cdap.proto.sourcecontrol.RepositoryConfig; +import io.cdap.cdap.sourcecontrol.ApplicationManager; +import io.cdap.cdap.sourcecontrol.CommitMeta; +import io.cdap.cdap.sourcecontrol.RepositoryManager; +import io.cdap.cdap.sourcecontrol.RepositoryManagerFactory; +import io.cdap.cdap.sourcecontrol.operationrunner.InMemorySourceControlOperationRunner; +import io.cdap.cdap.sourcecontrol.operationrunner.PushAppResponse; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.Set; +import java.util.function.BiFunction; +import java.util.stream.Collectors; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.mockito.Mockito; + +public class PushAppsOperationTest { + + private static final ApplicationDetail testApp1Details = new ApplicationDetail( + "testApp", "v1", "description1", null, null, "conf1", new ArrayList<>(), + new ArrayList<>(), new ArrayList<>(), null, null); + private static final ApplicationDetail testApp2Details = new ApplicationDetail( + "testApp2", "v2", "description2", null, null, "conf2", new ArrayList<>(), + new ArrayList<>(), new ArrayList<>(), null, null); + + private static final Gson GSON = new Gson(); + + private final PushAppsRequest req = new PushAppsRequest( + ImmutableSet.of(testApp1Details.getName(), testApp2Details.getName()), + new RepositoryConfig.Builder() + .setProvider(Provider.GITHUB) + .setLink("test") + .setDefaultBranch("") + .setAuth(new AuthConfig(AuthType.PAT, new PatConfig("test", "test"))) + .build(), + new CommitMeta("", "", 0, "test_commit") + ); + + private InMemorySourceControlOperationRunner opRunner; + private LongRunningOperationContext context; + + private final Path path1 = Paths.get(testApp1Details.getName() + ".json"); + private final Path path2 = Paths.get(testApp2Details.getName() + ".json"); + + private static final RepositoryManager mockRepositoryManager = Mockito.mock( + RepositoryManager.class); + + + @ClassRule + public static TemporaryFolder repositoryBase = new TemporaryFolder(); + + @Before + public void setUp() throws Exception { + RepositoryManagerFactory mockRepositoryManagerFactory = Mockito.mock( + RepositoryManagerFactory.class); + Mockito.doReturn(mockRepositoryManager).when(mockRepositoryManagerFactory) + .create(Mockito.any(), Mockito.any()); + Mockito.doNothing().when(mockRepositoryManager).close(); + + Path rootPath = repositoryBase.getRoot().toPath(); + Mockito.doReturn(rootPath).when(mockRepositoryManager).getRepositoryRoot(); + Mockito.doReturn(rootPath).when(mockRepositoryManager).getBasePath(); + Mockito.doReturn(path1.getFileName()).when(mockRepositoryManager) + .getFileRelativePath(path1.getFileName().toString()); + Mockito.doReturn(path2.getFileName()).when(mockRepositoryManager) + .getFileRelativePath(path2.getFileName().toString()); + + this.context = Mockito.mock(LongRunningOperationContext.class); + Mockito.doReturn(new OperationRunId(NamespaceId.DEFAULT.getNamespace(), "id")) + .when(this.context).getRunId(); + + Injector injector = AppFabricTestHelper.getInjector(CConfiguration.create(), + new AbstractModule() { + @Override + protected void configure() { + bind(RepositoryManagerFactory.class).toInstance(mockRepositoryManagerFactory); + } + }); + + this.opRunner = injector.getInstance(InMemorySourceControlOperationRunner.class); + } + + @Test + public void testRunSuccess() throws Exception { + ApplicationManager mockManager = Mockito.mock(ApplicationManager.class); + PushAppsOperation operation = new PushAppsOperation(req, opRunner, mockManager); + + Mockito.doReturn(testApp1Details).when(mockManager).get( + new ApplicationReference(NamespaceId.DEFAULT.getNamespace(), testApp1Details.getName())); + + Mockito.doReturn(testApp2Details).when(mockManager).get( + new ApplicationReference(NamespaceId.DEFAULT.getNamespace(), testApp2Details.getName())); + + PushAppResponse mockTestApp1Response = new PushAppResponse( + testApp1Details.getName(), testApp1Details.getAppVersion(), "file-hash1"); + PushAppResponse mockTestApp2Response = new PushAppResponse( + testApp2Details.getName(), testApp2Details.getAppVersion(), "file-hash2"); + + Mockito.doReturn(ImmutableList.of(mockTestApp1Response, mockTestApp2Response)) + .when(mockRepositoryManager) + .commitAndPush(Mockito.any(CommitMeta.class), Mockito.anyCollection(), Mockito.any(BiFunction.class)); + + Set gotResources = operation.run(context).get(); + + verifyCreatedResources(gotResources); + } + + @Test(expected = OperationException.class) + public void testRunFailedAtFirstApp() throws Exception { + ApplicationManager mockManager = Mockito.mock(ApplicationManager.class); + LongRunningOperationContext mockContext = Mockito.mock(LongRunningOperationContext.class); + PushAppsOperation operation = new PushAppsOperation(req, opRunner, mockManager); + + Mockito.doThrow(new NotFoundException("")).when(mockManager) + .get(Mockito.any()); + + operation.run(context).get(); + } + + @Test(expected = OperationException.class) + public void testRunFailedAtSecondApp() throws Exception { + ApplicationManager mockManager = Mockito.mock(ApplicationManager.class); + LongRunningOperationContext mockContext = Mockito.mock(LongRunningOperationContext.class); + PushAppsOperation operation = new PushAppsOperation(req, opRunner, mockManager); + + Mockito.doReturn(testApp1Details).doThrow(new NotFoundException("")).when(mockManager) + .get(Mockito.any()); + + operation.run(context).get(); + } + + @Test + public void testRunFailedWhenMarkingLatest() throws Exception { + ApplicationManager mockManager = Mockito.mock(ApplicationManager.class); + PushAppsOperation operation = new PushAppsOperation(req, opRunner, mockManager); + + Mockito.doReturn(testApp1Details).when(mockManager).get( + new ApplicationReference(NamespaceId.DEFAULT.getNamespace(), testApp1Details.getName())); + + Mockito.doReturn(testApp2Details).when(mockManager).get( + new ApplicationReference(NamespaceId.DEFAULT.getNamespace(), testApp2Details.getName())); + + PushAppResponse mockTestApp1Response = new PushAppResponse( + testApp1Details.getName(), testApp1Details.getAppVersion(), "file-hash1"); + PushAppResponse mockTestApp2Response = new PushAppResponse( + testApp2Details.getName(), testApp2Details.getAppVersion(), "file-hash2"); + + Mockito.doReturn(ImmutableList.of(mockTestApp1Response, mockTestApp2Response)) + .when(mockRepositoryManager) + .commitAndPush(Mockito.any(CommitMeta.class), Mockito.anyCollection(), Mockito.any(BiFunction.class)); + + Mockito.doThrow(new NotFoundException("")).when(mockManager) + .updateSourceControlMeta(Mockito.any(), Mockito.any()); + + Set gotResources = new HashSet<>(); + Mockito.doAnswer(i -> { + gotResources.addAll( + (Collection) i.getArguments()[0]); + return null; + }).when(context).updateOperationResources(Mockito.any()); + try { + operation.run(context).get(); + Assert.fail(); + } catch (OperationException e) { + // expected + } + + verifyCreatedResources(gotResources); + } + + private void verifyCreatedResources(Set gotResources) { + Set expectedAppIds = ImmutableSet.of( + new ApplicationId(NamespaceId.DEFAULT.getNamespace(), testApp1Details.getName(), + testApp1Details.getAppVersion()), + new ApplicationId(NamespaceId.DEFAULT.getNamespace(), testApp2Details.getName(), + testApp2Details.getAppVersion()) + ); + Set createdAppIds = gotResources.stream() + .map(r -> ApplicationId.fromString(r.getResourceUri())).collect(Collectors.toSet()); + Assert.assertEquals(expectedAppIds, createdAppIds); + } + +} diff --git a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/store/AppMetadataStoreTest.java b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/store/AppMetadataStoreTest.java index 6854dea6e9c5..7fe5232ee566 100644 --- a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/store/AppMetadataStoreTest.java +++ b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/store/AppMetadataStoreTest.java @@ -19,6 +19,8 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; import io.cdap.cdap.AllProgramsApp; import io.cdap.cdap.api.app.ApplicationSpecification; import io.cdap.cdap.api.artifact.ArtifactId; @@ -26,6 +28,7 @@ import io.cdap.cdap.common.app.RunIds; import io.cdap.cdap.common.utils.ProjectInfo; import io.cdap.cdap.internal.AppFabricTestHelper; +import io.cdap.cdap.internal.app.ApplicationSpecificationAdapter; import io.cdap.cdap.internal.app.DefaultApplicationSpecification; import io.cdap.cdap.internal.app.deploy.Specifications; import io.cdap.cdap.internal.app.runtime.SystemArguments; @@ -40,8 +43,12 @@ import io.cdap.cdap.proto.id.ProgramReference; import io.cdap.cdap.proto.id.ProgramRunId; import io.cdap.cdap.spi.data.SortOrder; +import io.cdap.cdap.spi.data.StructuredTable; +import io.cdap.cdap.spi.data.table.field.Field; +import io.cdap.cdap.spi.data.table.field.Fields; import io.cdap.cdap.spi.data.transaction.TransactionRunner; import io.cdap.cdap.spi.data.transaction.TransactionRunners; +import io.cdap.cdap.store.StoreDefinition; import java.io.IOException; import java.time.Instant; import java.util.ArrayList; @@ -1365,7 +1372,7 @@ public void testConcurrentCreateAppFirstVersion() throws Exception { String appName = "application1"; ArtifactId artifactId = NamespaceId.DEFAULT.artifact("testArtifact", "1.0").toApiArtifactId(); ApplicationReference appRef = new ApplicationReference(NamespaceId.DEFAULT, appName); - + // Concurrently deploy different fist version of the same application int numThreads = 10; AtomicInteger idGenerator = new AtomicInteger(); @@ -1378,7 +1385,7 @@ public void testConcurrentCreateAppFirstVersion() throws Exception { ApplicationMeta meta = new ApplicationMeta(spec.getName(), spec, new ChangeDetail(null, null, null, creationTimeMillis + id)); - metaStore.createApplicationVersion(appId, meta); + metaStore.createLatestApplicationVersion(appId, meta); }) ); @@ -1409,7 +1416,7 @@ public void testConcurrentCreateAppFirstVersion() throws Exception { latestVersionCount.set(latestVersions.size()); appEditNumber.set(metaStore.getApplicationEditNumber(appRef)); }); - + // There can only be one latest version Assert.assertEquals(1, latestVersionCount.get()); Assert.assertEquals(numThreads, allVersionsCount.get()); @@ -1432,7 +1439,7 @@ public void testConcurrentCreateAppAfterTheFirstVersion() throws Exception { ApplicationMeta meta = new ApplicationMeta(spec.getName(), spec, new ChangeDetail(null, null, null, creationTimeMillis + id)); - metaStore.createApplicationVersion(appId, meta); + metaStore.createLatestApplicationVersion(appId, meta); }); // Concurrently deploy different versions of the same application @@ -1446,7 +1453,7 @@ public void testConcurrentCreateAppAfterTheFirstVersion() throws Exception { ApplicationMeta meta = new ApplicationMeta(spec.getName(), spec, new ChangeDetail(null, null, null, creationTimeMillis + id)); - metaStore.createApplicationVersion(appId, meta); + metaStore.createLatestApplicationVersion(appId, meta); }) ); @@ -1528,6 +1535,46 @@ public void testDeleteCompletedRunsStartedBefore() throws Exception { }); } + /** + * Testcase for getting the latest application, where the application was deployed + * before 6.8.0 (where the latest column is not set). + * In this case, first insert a row in app spec table with the latest column set to null. + * This step is expected to fail in the NoSql implementation. + */ + @Test + public void testGetLatestOnLegacyRows() throws Exception { + Gson GSON = ApplicationSpecificationAdapter.addTypeAdapters(new GsonBuilder()).create(); + // insert a row in appspec table with latest column set to null + String appName = "legacy_app_without_latest"; + String appVersion = ApplicationId.DEFAULT_VERSION; + ApplicationReference appRef = new ApplicationReference(NamespaceId.DEFAULT, appName); + + ArtifactId artifactId = NamespaceId.DEFAULT.artifact("testArtifact", "1.0").toApiArtifactId(); + ApplicationId appId = appRef.app(appVersion); + ApplicationSpecification spec = createDummyAppSpec(appId.getApplication(), appId.getVersion(), artifactId); + ApplicationMeta appMeta = new ApplicationMeta(appName, spec, null, null); + + TransactionRunners.run(transactionRunner, context -> { + AppMetadataStore metaStore = AppMetadataStore.create(context); + metaStore.createLatestApplicationVersion(appId, appMeta); + StructuredTable appSpecTable = context.getTable( + StoreDefinition.AppMetadataStore.APPLICATION_SPECIFICATIONS); + + List> fields = metaStore.getApplicationPrimaryKeys( + NamespaceId.DEFAULT.getNamespace(), appName, appVersion); + fields.add(Fields.booleanField(StoreDefinition.AppMetadataStore.LATEST_FIELD, null)); + appSpecTable.upsert(fields); + }); + + ApplicationMeta latestAppMeta = TransactionRunners.run(transactionRunner, context -> { + AppMetadataStore metaStore = AppMetadataStore.create(context); + return metaStore.getLatest(appRef); + }); + + Assert.assertEquals(appName, latestAppMeta.getId()); + Assert.assertEquals(appVersion, latestAppMeta.getSpec().getAppVersion()); + } + /** * Creates a new run of {@code programRunId} in the completed state with a starting time of {@code * startingTime} and returns its corresponding run id. diff --git a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/store/DefaultStoreTest.java b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/store/DefaultStoreTest.java index de72260447c1..ec8f8574bd64 100644 --- a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/store/DefaultStoreTest.java +++ b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/store/DefaultStoreTest.java @@ -159,7 +159,7 @@ public void testLoadingProgram() throws Exception { ApplicationMeta appMeta = new ApplicationMeta(appSpec.getName(), appSpec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(appId, appMeta); + store.addLatestApplication(appId, appMeta); ProgramDescriptor descriptor = store.loadProgram(appId.mr("mrJob1")); Assert.assertNotNull(descriptor); @@ -494,7 +494,7 @@ public void testAddApplication() throws ConflictException { ApplicationMeta appMeta = new ApplicationMeta("application1", spec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(appId, appMeta); + store.addLatestApplication(appId, appMeta); spec = store.getApplication(appId); Assert.assertNotNull(spec); @@ -507,12 +507,12 @@ public void testUpdateChangedApplication() throws ConflictException { ApplicationMeta appMeta = new ApplicationMeta("application1", Specifications.from(new FooApp()), new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(id, appMeta); + store.addLatestApplication(id, appMeta); // update ApplicationMeta appMetaUpdate = new ApplicationMeta("application1", Specifications.from(new ChangedFooApp()), new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(id, appMetaUpdate); + store.addLatestApplication(id, appMetaUpdate); ApplicationSpecification spec = store.getApplication(id); Assert.assertNotNull(spec); @@ -523,15 +523,18 @@ public void testUpdateChangedApplication() throws ConflictException { public void testAddApplicationWithoutMarkingLatest() throws ConflictException { long creationTime = System.currentTimeMillis(); - ApplicationId appId = new ApplicationId("account1", "app1"); - ApplicationMeta appMeta = new ApplicationMeta("app1", Specifications.from(new FooApp()), - new ChangeDetail(null, null, null, creationTime), null, false); - store.addApplication(appId, appMeta); + String appName = "notLatestApp1"; + ApplicationId appId = new ApplicationId("account1", appName, "v1"); + ApplicationMeta appMeta = new ApplicationMeta(appName, Specifications.from(new FooApp()), + new ChangeDetail(null, null, null, creationTime), null); + store.addApplication(appId, appMeta, false); ApplicationMeta storedMeta = store.getApplicationMetadata(appId); - Assert.assertEquals("app1", storedMeta.getId()); + Assert.assertEquals(appName, storedMeta.getId()); Assert.assertEquals(creationTime, storedMeta.getChange().getCreationTimeMillis()); - Assert.assertFalse(storedMeta.getIsLatest()); + + ApplicationMeta latestMeta = store.getLatest(appId.getAppReference()); + Assert.assertNull(latestMeta); } @Test @@ -541,26 +544,24 @@ public void testMarkApplicationsLatestWithNewApps() // Add 2 new applications without marking them latest ApplicationId appId1 = new ApplicationId("account1", "newApp1"); ApplicationMeta appMeta1 = new ApplicationMeta("newApp1", Specifications.from(new FooApp()), - new ChangeDetail(null, null, null, creationTime), null, false); - store.addApplication(appId1, appMeta1); + new ChangeDetail(null, null, null, creationTime), null); + store.addApplication(appId1, appMeta1, false); ApplicationId appId2 = new ApplicationId("account1", "newApp2"); ApplicationMeta appMeta2 = new ApplicationMeta("newApp2", Specifications.from(new FooApp()), - new ChangeDetail(null, null, null, creationTime), null, false); - store.addApplication(appId2, appMeta2); + new ChangeDetail(null, null, null, creationTime), null); + store.addApplication(appId2, appMeta2, false); // Now mark them as latest in bulk store.markApplicationsLatest(Arrays.asList(appId1, appId2)); - ApplicationMeta storedMeta1 = store.getApplicationMetadata(appId1); + ApplicationMeta storedMeta1 = store.getLatest(appId1.getAppReference()); Assert.assertEquals("newApp1", storedMeta1.getId()); Assert.assertEquals(creationTime, storedMeta1.getChange().getCreationTimeMillis()); - Assert.assertTrue(storedMeta1.getIsLatest()); - ApplicationMeta storedMeta2 = store.getApplicationMetadata(appId2); + ApplicationMeta storedMeta2 = store.getLatest(appId2.getAppReference()); Assert.assertEquals("newApp2", storedMeta2.getId()); Assert.assertEquals(creationTime, storedMeta2.getChange().getCreationTimeMillis()); - Assert.assertTrue(storedMeta2.getIsLatest()); } @Test @@ -576,28 +577,29 @@ public void testMarkApplicationsLatestWithExistingLatest() ApplicationId appIdV1 = new ApplicationId("account1", appName, oldVersion); ApplicationMeta appMetaV1 = new ApplicationMeta(appName, Specifications.from(new FooApp(), appName, oldVersion), new ChangeDetail(null, null, null, creationTime)); - store.addApplication(appIdV1, appMetaV1); + store.addLatestApplication(appIdV1, appMetaV1); // Add a new version of the application without marking latest ApplicationId appIdV2 = new ApplicationId("account1", appName, newVersion); ApplicationMeta appMetaV2 = new ApplicationMeta(appName, Specifications.from(new FooApp(), appName, newVersion), - new ChangeDetail(null, null, null, v2CreationTime), null, false); - store.addApplication(appIdV2, appMetaV2); + new ChangeDetail(null, null, null, v2CreationTime), null); + store.addApplication(appIdV2, appMetaV2, false); // Now mark the new version as latest store.markApplicationsLatest(Collections.singletonList(appIdV2)); + ApplicationMeta latestMeta = store.getLatest(appIdV1.getAppReference()); ApplicationMeta storedMetaV1 = store.getApplicationMetadata(appIdV1); Assert.assertEquals(appName, storedMetaV1.getId()); Assert.assertEquals(oldVersion, storedMetaV1.getSpec().getAppVersion()); Assert.assertEquals(creationTime, storedMetaV1.getChange().getCreationTimeMillis()); - Assert.assertFalse(storedMetaV1.getIsLatest()); + Assert.assertNotEquals(latestMeta.getSpec().getAppVersion(), storedMetaV1.getSpec().getAppVersion()); ApplicationMeta storedMetaV2 = store.getApplicationMetadata(appIdV2); Assert.assertEquals(appName, storedMetaV2.getId()); Assert.assertEquals(newVersion, storedMetaV2.getSpec().getAppVersion()); Assert.assertEquals(v2CreationTime, storedMetaV2.getChange().getCreationTimeMillis()); - Assert.assertTrue(storedMetaV2.getIsLatest()); + Assert.assertEquals(latestMeta.getSpec().getAppVersion(), storedMetaV2.getSpec().getAppVersion()); } @Test(expected = ApplicationNotFoundException.class) @@ -620,7 +622,7 @@ public void testUpdateApplicationScmMeta() new ChangeDetail(null, null, null, System.currentTimeMillis()), null); - store.addApplication(appId, appMeta); + store.addLatestApplication(appId, appMeta); Map updateRequests = new HashMap<>(); updateRequests.put(appId, new SourceControlMeta("updated-file-hash")); store.updateApplicationSourceControlMeta(updateRequests); @@ -640,7 +642,7 @@ public void testUpdateApplicationScmMetaWithNonExistingAppIds() new ChangeDetail(null, null, null, System.currentTimeMillis()), new SourceControlMeta("initial-file-hash")); - store.addApplication(appId, appMeta); + store.addLatestApplication(appId, appMeta); // The following appId is not added to the store ApplicationId appId2 = new ApplicationId("account1", "application2"); @@ -708,7 +710,7 @@ public void testServiceDeletion() throws ConflictException { ApplicationMeta appMeta = new ApplicationMeta(appSpec.getName(), appSpec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(appId, appMeta); + store.addLatestApplication(appId, appMeta); AbstractApplication newApp = new AppWithNoServices(); @@ -728,7 +730,7 @@ public void testServiceInstances() throws ConflictException { ApplicationMeta appMeta = new ApplicationMeta(appSpec.getName(), appSpec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(appId, appMeta); + store.addLatestApplication(appId, appMeta); // Test setting of service instances ProgramId programId = appId.program(ProgramType.SERVICE, "NoOpService"); @@ -755,7 +757,7 @@ public void testWorkerInstances() throws ConflictException { ApplicationMeta appMeta = new ApplicationMeta(spec.getName(), spec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(appId, appMeta); + store.addLatestApplication(appId, appMeta); ProgramId programId = appId.worker(AppWithWorker.WORKER); int instancesFromSpec = spec.getWorkers().get(AppWithWorker.WORKER).getInstances(); @@ -776,7 +778,7 @@ public void testRemoveAll() throws ConflictException { ApplicationMeta appMeta = new ApplicationMeta(spec.getName(), spec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(appId, appMeta); + store.addLatestApplication(appId, appMeta); Assert.assertNotNull(store.getApplication(appId)); @@ -794,7 +796,7 @@ public void testRemoveApplication() throws ConflictException { ApplicationMeta appMeta = new ApplicationMeta(spec.getName(), spec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(appId, appMeta); + store.addLatestApplication(appId, appMeta); Assert.assertNotNull(store.getApplication(appId)); @@ -818,7 +820,7 @@ public void testProgramRunCount() throws ConflictException { ApplicationMeta appMeta = new ApplicationMeta(spec.getName(), spec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(appId, appMeta); + store.addLatestApplication(appId, appMeta); // add some run records to workflow and service for (int i = 0; i < 5; i++) { @@ -864,7 +866,7 @@ public void testRuntimeArgsDeletion() throws ConflictException { ApplicationMeta appMeta = new ApplicationMeta(spec.getName(), spec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(appId, appMeta); + store.addLatestApplication(appId, appMeta); Assert.assertNotNull(store.getApplication(appId)); @@ -916,14 +918,14 @@ public void testHistoryDeletion() throws ConflictException { ApplicationMeta appMeta = new ApplicationMeta(spec.getName(), spec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(appId1, appMeta); + store.addLatestApplication(appId1, appMeta); spec = Specifications.from(new AppWithServices()); ApplicationId appId2 = namespaceId.app(spec.getName()); appMeta = new ApplicationMeta(spec.getName(), spec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(appId2, appMeta); + store.addLatestApplication(appId2, appMeta); ArtifactId artifactId = appId1.getNamespaceId().artifact("testArtifact", "1.0").toApiArtifactId(); @@ -986,7 +988,7 @@ public void testRunsLimit() throws ConflictException { ApplicationMeta appMeta = new ApplicationMeta(spec.getName(), spec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(appId, appMeta); + store.addLatestApplication(appId, appMeta); ProgramId mapreduceProgramId = new ApplicationId("testRunsLimit", spec.getName()) .mr(AllProgramsApp.NoOpMR.class.getSimpleName()); @@ -1017,7 +1019,7 @@ public void testCheckDeletedProgramSpecs() throws ConflictException { ApplicationMeta appMeta = new ApplicationMeta(spec.getName(), spec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(appId, appMeta); + store.addLatestApplication(appId, appMeta); Set specsToBeVerified = Sets.newHashSet(); specsToBeVerified.addAll(spec.getMapReduce().keySet()); @@ -1063,7 +1065,7 @@ protected void testScanApplications(Store store) throws ConflictException { ApplicationMeta appMeta = new ApplicationMeta(appName, appSpec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(new ApplicationId(NamespaceId.DEFAULT.getNamespace(), appName), appMeta); + store.addLatestApplication(new ApplicationId(NamespaceId.DEFAULT.getNamespace(), appName), appMeta); } // Mimicking editing the first count / 2 apps @@ -1073,7 +1075,7 @@ protected void testScanApplications(Store store) throws ConflictException { ApplicationMeta appMeta = new ApplicationMeta(appName, appSpec, new ChangeDetail("edited" + i, null, null, System.currentTimeMillis())); - store.addApplication(new ApplicationId(NamespaceId.DEFAULT.getNamespace(), appName, version), appMeta); + store.addLatestApplication(new ApplicationId(NamespaceId.DEFAULT.getNamespace(), appName, version), appMeta); } List allAppsVersion = new ArrayList<>(); @@ -1118,9 +1120,9 @@ public void testScanApplicationsWithNamespace(Store store) throws ConflictExcept ApplicationMeta appMeta = new ApplicationMeta(appName, appSpec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(new ApplicationId(NamespaceId.DEFAULT.getNamespace(), appName), appMeta); + store.addLatestApplication(new ApplicationId(NamespaceId.DEFAULT.getNamespace(), appName), appMeta); appName = "test" + (2 * i + 1); - store.addApplication(new ApplicationId(NamespaceId.CDAP.getNamespace(), appName), appMeta); + store.addLatestApplication(new ApplicationId(NamespaceId.CDAP.getNamespace(), appName), appMeta); } List apps = new ArrayList(); @@ -1185,7 +1187,7 @@ public void testCheckDeletedWorkflow() throws ConflictException { ApplicationMeta appMeta = new ApplicationMeta(spec.getName(), spec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(appId, appMeta); + store.addLatestApplication(appId, appMeta); Set specsToBeDeleted = Sets.newHashSet(); specsToBeDeleted.addAll(spec.getWorkflows().keySet()); @@ -1330,7 +1332,7 @@ public void testStateRemovedOnRemoveApplication() throws ApplicationNotFoundExce ApplicationMeta appMeta = new ApplicationMeta(spec.getName(), spec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(appId, appMeta); + store.addLatestApplication(appId, appMeta); store.saveState(new AppStateKeyValue(namespaceId, spec.getName(), stateKey, stateValue)); Assert.assertNotNull(store.getApplication(appId)); @@ -1357,7 +1359,7 @@ public void testStateRemovedOnRemoveAll() throws ApplicationNotFoundException, C ApplicationMeta appMeta = new ApplicationMeta(spec.getName(), spec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(appId, appMeta); + store.addLatestApplication(appId, appMeta); store.saveState(new AppStateKeyValue(namespaceId, appName, stateKey, stateValue)); Assert.assertNotNull(store.getApplication(appId)); @@ -1388,7 +1390,7 @@ public void testListRunsWithLegacyRows() throws ConflictException { List expectedApps = new ArrayList<>(); // Insert a row that is null for changeDetail - store.addApplication(appId, appMeta); + store.addLatestApplication(appId, appMeta); expectedApps.add(appId); ApplicationId newVersionAppId = appId.getAppReference().app("new_version"); @@ -1399,7 +1401,7 @@ public void testListRunsWithLegacyRows() throws ConflictException { new ChangeDetail(null, null, null, currentTime)); // Insert a second version - store.addApplication(newVersionAppId, newAppMeta); + store.addLatestApplication(newVersionAppId, newAppMeta); expectedApps.add(newVersionAppId); // Insert a third version @@ -1408,7 +1410,7 @@ public void testListRunsWithLegacyRows() throws ConflictException { ApplicationMeta anotherAppMeta = new ApplicationMeta(anotherVersionAppId.getApplication(), spec, new ChangeDetail(null, null, null, currentTime + 1000)); - store.addApplication(anotherVersionAppId, anotherAppMeta); + store.addLatestApplication(anotherVersionAppId, anotherAppMeta); expectedApps.add(anotherVersionAppId); // Reverse it because we want DESC order diff --git a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/store/NoSqlAppMetadataStoreTest.java b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/store/NoSqlAppMetadataStoreTest.java index c278f5ccff6e..a0c0d6b4571a 100644 --- a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/store/NoSqlAppMetadataStoreTest.java +++ b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/store/NoSqlAppMetadataStoreTest.java @@ -50,4 +50,14 @@ public void testScanApplicationsReverse() { public void testScanApplicationsWithNamespaceReverse() { super.testScanApplicationsWithNamespaceReverse(); } + + /** + * This testcase tries to create a row in the app spec table, with the latest column set + * to null (To test the getLatest method for applications deployed before v6.8.0). Setting + * a Boolean column to null in the NoSqlStructuredTable is expected to throw a NullPointerException. + */ + @Override @Test(expected = NullPointerException.class) + public void testGetLatestOnLegacyRows() throws Exception { + super.testGetLatestOnLegacyRows(); + } } diff --git a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/worker/sidecar/ArtifactLocalizerServiceTest.java b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/worker/sidecar/ArtifactLocalizerServiceTest.java index 61ffaf5c998b..ea9f953f718e 100644 --- a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/worker/sidecar/ArtifactLocalizerServiceTest.java +++ b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/worker/sidecar/ArtifactLocalizerServiceTest.java @@ -27,6 +27,7 @@ import io.cdap.cdap.common.id.Id; import io.cdap.cdap.common.internal.remote.DefaultInternalAuthenticator; import io.cdap.cdap.common.internal.remote.NoOpInternalAuthenticator; +import io.cdap.cdap.common.internal.remote.NoOpRemoteAuthenticator; import io.cdap.cdap.common.internal.remote.RemoteClientFactory; import io.cdap.cdap.common.io.Locations; import io.cdap.cdap.common.metrics.NoOpMetricsCollectionService; @@ -85,7 +86,7 @@ private ArtifactLocalizerService setupArtifactLocalizerService(CConfiguration cC cConf, new ArtifactLocalizer(cConf, remoteClientFactory, (namespaceId, retryStrategy) -> { return new NoOpArtifactManager(); }), new CommonNettyHttpServiceFactory(cConf, new NoOpMetricsCollectionService()), - remoteClientFactory); + remoteClientFactory, new NoOpRemoteAuthenticator()); // start the service artifactLocalizerService.startAndWait(); diff --git a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/worker/sidecar/GcpMetadataHttpHandlerInternalTest.java b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/worker/sidecar/GcpMetadataHttpHandlerInternalTest.java index b33c23294972..0e7eac4d507c 100644 --- a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/worker/sidecar/GcpMetadataHttpHandlerInternalTest.java +++ b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/app/worker/sidecar/GcpMetadataHttpHandlerInternalTest.java @@ -21,6 +21,7 @@ import io.cdap.cdap.common.conf.CConfiguration; import io.cdap.cdap.common.conf.Constants; import io.cdap.cdap.common.http.CommonNettyHttpServiceBuilder; +import io.cdap.cdap.common.internal.remote.NoOpRemoteAuthenticator; import io.cdap.cdap.common.internal.remote.RemoteClientFactory; import io.cdap.cdap.common.metrics.NoOpMetricsCollectionService; import io.cdap.cdap.common.namespace.InMemoryNamespaceAdmin; @@ -69,7 +70,8 @@ public static void init() throws Exception { httpService = new CommonNettyHttpServiceBuilder(cConf, "test", new NoOpMetricsCollectionService()) .setHttpHandlers( - new GcpMetadataHttpHandlerInternal(cConf, remoteClientFactory) + new GcpMetadataHttpHandlerInternal(cConf, remoteClientFactory, + new NoOpRemoteAuthenticator()) ) .setChannelPipelineModifier(new ChannelPipelineModifier() { @Override diff --git a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/operation/InMemoryOperationRunnerTest.java b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/operation/InMemoryOperationRunnerTest.java new file mode 100644 index 000000000000..19c0ebe6cfb5 --- /dev/null +++ b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/operation/InMemoryOperationRunnerTest.java @@ -0,0 +1,108 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.cdap.internal.operation; + + +import com.google.common.collect.ImmutableSet; +import com.google.inject.AbstractModule; +import com.google.inject.Injector; +import io.cdap.cdap.common.conf.CConfiguration; +import io.cdap.cdap.internal.AppFabricTestHelper; +import io.cdap.cdap.internal.app.sourcecontrol.PullAppsRequest; +import io.cdap.cdap.proto.id.OperationRunId; +import io.cdap.cdap.proto.operation.OperationMeta; +import io.cdap.cdap.proto.operation.OperationRun; +import io.cdap.cdap.proto.operation.OperationRunStatus; +import io.cdap.cdap.proto.operation.OperationType; +import io.cdap.cdap.sourcecontrol.ApplicationManager; +import io.cdap.cdap.sourcecontrol.operationrunner.InMemorySourceControlOperationRunner; +import java.time.Instant; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.mockito.Mockito; + +public class InMemoryOperationRunnerTest { + + private InMemoryOperationRunner runner; + + private static Injector injector; + private static final OperationStatePublisher mockPublisher = Mockito.mock( + OperationStatePublisher.class); + private static final InMemorySourceControlOperationRunner mockScmRunner = Mockito.mock( + InMemorySourceControlOperationRunner.class); + private static final ApplicationManager mockManager = Mockito.mock(ApplicationManager.class); + private static final OperationRunId runId = new OperationRunId("namespace", "run"); + private static final OperationRun run = OperationRun.builder() + .setRunId(runId.getRun()) + .setStatus(OperationRunStatus.PENDING) + .setType(OperationType.PULL_APPS) + .setMetadata( + OperationMeta.builder().setCreateTime(Instant.now()).build()) + .build(); + private static final OperationRunDetail detail = OperationRunDetail.builder() + .setSourceId(AppFabricTestHelper.createSourceId(0)) + .setRunId(runId) + .setRun(run) + .setPullAppsRequest(new PullAppsRequest(ImmutableSet.of("1", "2", "3", "4"), null)) + .build(); + + @BeforeClass + public static void setupClass() { + CConfiguration cconf = CConfiguration.create(); + injector = AppFabricTestHelper.getInjector(cconf, new AbstractModule() { + @Override + protected void configure() { + bind(OperationStatePublisher.class).toInstance(mockPublisher); + bind(InMemorySourceControlOperationRunner.class).toInstance(mockScmRunner); + bind(ApplicationManager.class).toInstance(mockManager); + } + }); + } + + @Before + public void setupTest() { + runner = injector.getInstance(InMemoryOperationRunner.class); + Mockito.reset(mockPublisher); + } + + @Test + public void testRun() throws Exception { + + OperationController controller = runner.run(detail); + controller.complete().get(); + + Mockito.verify(mockPublisher, Mockito.times(1)).publishRunning(runId); + Mockito.verify(mockPublisher, Mockito.times(1)).publishSuccess(runId); + } + + @Test + public void testRunWithError() throws Exception { + + Mockito.doThrow(new RuntimeException("test")).when(mockScmRunner).multiPull( + Mockito.any(), Mockito.any() + ); + + OperationController controller = runner.run(detail); + controller.complete().get(); + + Mockito.verify(mockPublisher, Mockito.times(1)).publishRunning(runId); + Mockito.verify(mockPublisher, Mockito.times(1)).publishFailed(Mockito.eq(runId), Mockito.any()); + Mockito.verify(mockPublisher, Mockito.never()).publishSuccess(runId); + } +} + diff --git a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/operation/OperationLifecycleManagerTest.java b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/operation/OperationLifecycleManagerTest.java new file mode 100644 index 000000000000..41a271615e77 --- /dev/null +++ b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/operation/OperationLifecycleManagerTest.java @@ -0,0 +1,190 @@ +package io.cdap.cdap.internal.operation; + +import com.google.common.io.Closeables; +import com.google.inject.AbstractModule; +import com.google.inject.Guice; +import com.google.inject.Injector; +import com.google.inject.Scopes; +import io.cdap.cdap.api.metrics.MetricsCollectionService; +import io.cdap.cdap.common.conf.CConfiguration; +import io.cdap.cdap.common.conf.Constants.AppFabric; +import io.cdap.cdap.common.guice.ConfigModule; +import io.cdap.cdap.common.guice.LocalLocationModule; +import io.cdap.cdap.common.id.Id.Namespace; +import io.cdap.cdap.common.metrics.NoOpMetricsCollectionService; +import io.cdap.cdap.data.runtime.StorageModule; +import io.cdap.cdap.data.runtime.SystemDatasetRuntimeModule; +import io.cdap.cdap.proto.id.OperationRunId; +import io.cdap.cdap.proto.operation.OperationRunStatus; +import io.cdap.cdap.proto.operation.OperationType; +import io.cdap.cdap.spi.data.StructuredTableAdmin; +import io.cdap.cdap.spi.data.sql.PostgresInstantiator; +import io.cdap.cdap.spi.data.transaction.TransactionRunner; +import io.cdap.cdap.spi.data.transaction.TransactionRunners; +import io.cdap.cdap.store.StoreDefinition; +import io.zonky.test.db.postgres.embedded.EmbeddedPostgres; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class OperationLifecycleManagerTest extends OperationTestBase { + protected static TransactionRunner transactionRunner; + private static final String testNamespace = "test"; + private static OperationLifecycleManager operationLifecycleManager; + private static int batchSize; + + @ClassRule public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder(); + + private static EmbeddedPostgres postgres; + + @Before + public void before() throws Exception { + TransactionRunners.run( + transactionRunner, + context -> { + OperationRunStore operationRunsStore = new OperationRunStore(context); + operationRunsStore.clearData(); + }); + } + + @BeforeClass + public static void beforeClass() throws Exception { + CConfiguration cConf = CConfiguration.create(); + postgres = PostgresInstantiator.createAndStart(cConf, TEMP_FOLDER.newFolder()); + Injector injector = + Guice.createInjector( + new ConfigModule(cConf), + new LocalLocationModule(), + new SystemDatasetRuntimeModule().getInMemoryModules(), + new StorageModule(), + new AbstractModule() { + @Override + protected void configure() { + bind(MetricsCollectionService.class) + .to(NoOpMetricsCollectionService.class) + .in(Scopes.SINGLETON); + } + }); + + transactionRunner = injector.getInstance(TransactionRunner.class); + operationLifecycleManager = injector.getInstance(OperationLifecycleManager.class); + StoreDefinition.OperationRunsStore.create(injector.getInstance(StructuredTableAdmin.class)); + batchSize = cConf.getInt(AppFabric.STREAMING_BATCH_SIZE); + } + + @AfterClass + public static void afterClass() { + Closeables.closeQuietly(postgres); + } + + @Test + public void testScanOperations() throws Exception { + List insertedRuns = insertTestRuns(transactionRunner); + // get a filtered list of testNamespace runs + List testNamespaceRuns = + insertedRuns.stream() + .filter(detail -> detail.getRunId().getNamespace().equals(testNamespace)) + .collect(Collectors.toList()); + + TransactionRunners.run( + transactionRunner, + context -> { + List gotRuns = new ArrayList<>(); + List expectedRuns; + ScanOperationRunsRequest request; + + // verify the scan without filters picks all runs for testNamespace + request = ScanOperationRunsRequest.builder().setNamespace(testNamespace).build(); + operationLifecycleManager.scanOperations(request, batchSize, d -> gotRuns.add(d)); + expectedRuns = testNamespaceRuns; + Assert.assertArrayEquals(expectedRuns.toArray(), gotRuns.toArray()); + + // verify limit + gotRuns.clear(); + request = + ScanOperationRunsRequest.builder().setNamespace(testNamespace).setLimit(2).build(); + operationLifecycleManager.scanOperations(request, batchSize, d -> gotRuns.add(d)); + expectedRuns = testNamespaceRuns.stream().limit(2).collect(Collectors.toList()); + Assert.assertArrayEquals(expectedRuns.toArray(), gotRuns.toArray()); + + // verify the scan with type filter + gotRuns.clear(); + request = + ScanOperationRunsRequest.builder() + .setNamespace(testNamespace) + .setFilter(new OperationRunFilter(OperationType.PUSH_APPS, null)) + .build(); + operationLifecycleManager.scanOperations(request, batchSize, d -> gotRuns.add(d)); + expectedRuns = + testNamespaceRuns.stream() + .filter(detail -> detail.getRun().getType().equals(OperationType.PUSH_APPS)) + .collect(Collectors.toList()); + Assert.assertArrayEquals(expectedRuns.toArray(), gotRuns.toArray()); + + // verify the scan with status filter and limit + gotRuns.clear(); + request = + ScanOperationRunsRequest.builder() + .setNamespace(testNamespace) + .setLimit(2) + .setFilter( + new OperationRunFilter(OperationType.PULL_APPS, OperationRunStatus.FAILED)) + .build(); + operationLifecycleManager.scanOperations(request, batchSize, d -> gotRuns.add(d)); + expectedRuns = + testNamespaceRuns.stream() + .filter(detail -> detail.getRun().getType().equals(OperationType.PULL_APPS)) + .filter(detail -> detail.getRun().getStatus().equals(OperationRunStatus.FAILED)) + .limit(2) + .collect(Collectors.toList()); + Assert.assertArrayEquals(expectedRuns.toArray(), gotRuns.toArray()); + + // verify the scan with status filter + gotRuns.clear(); + request = + ScanOperationRunsRequest.builder() + .setNamespace(testNamespace) + .setFilter( + new OperationRunFilter(OperationType.PULL_APPS, OperationRunStatus.FAILED)) + .build(); + operationLifecycleManager.scanOperations(request, batchSize, d -> gotRuns.add(d)); + expectedRuns = + testNamespaceRuns.stream() + .filter(detail -> detail.getRun().getType().equals(OperationType.PULL_APPS)) + .filter(detail -> detail.getRun().getStatus().equals(OperationRunStatus.FAILED)) + .collect(Collectors.toList()); + Assert.assertArrayEquals(expectedRuns.toArray(), gotRuns.toArray()); + }); + } + + @Test + public void testGetOperation() throws Exception { + OperationRunDetail expectedDetail = + insertRun( + testNamespace, OperationType.PUSH_APPS, OperationRunStatus.RUNNING, transactionRunner); + String testId = expectedDetail.getRun().getId(); + OperationRunId runId = new OperationRunId(testNamespace, testId); + + TransactionRunners.run( + transactionRunner, + context -> { + OperationRunDetail gotDetail = operationLifecycleManager.getOperationRun(runId); + Assert.assertEquals(expectedDetail, gotDetail); + try { + operationLifecycleManager.getOperationRun( + new OperationRunId(Namespace.DEFAULT.getId(), testId)); + Assert.fail("Found unexpected run in default namespace"); + } catch (OperationRunNotFoundException e) { + // expected + } + }, + Exception.class); + } +} diff --git a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/operation/OperationRunStoreTest.java b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/operation/OperationRunStoreTest.java index a983b815db54..4bc1af5cb798 100644 --- a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/operation/OperationRunStoreTest.java +++ b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/operation/OperationRunStoreTest.java @@ -16,23 +16,26 @@ package io.cdap.cdap.internal.operation; -import io.cdap.cdap.common.app.RunIds; +import com.google.common.collect.ImmutableSet; import io.cdap.cdap.common.id.Id.Namespace; import io.cdap.cdap.internal.AppFabricTestHelper; -import io.cdap.cdap.internal.app.sourcecontrol.PullAppsRequest; import io.cdap.cdap.proto.id.OperationRunId; import io.cdap.cdap.proto.operation.OperationError; import io.cdap.cdap.proto.operation.OperationMeta; +import io.cdap.cdap.proto.operation.OperationResource; import io.cdap.cdap.proto.operation.OperationRun; import io.cdap.cdap.proto.operation.OperationRunStatus; import io.cdap.cdap.proto.operation.OperationType; +import io.cdap.cdap.spi.data.InvalidFieldException; import io.cdap.cdap.spi.data.transaction.TransactionRunner; import io.cdap.cdap.spi.data.transaction.TransactionRunners; import java.io.IOException; import java.time.Instant; import java.util.ArrayList; import java.util.Collections; +import java.util.HashSet; import java.util.List; +import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; @@ -40,15 +43,11 @@ import org.junit.Before; import org.junit.Test; -public abstract class OperationRunStoreTest { +public abstract class OperationRunStoreTest extends OperationTestBase { protected static TransactionRunner transactionRunner; private final AtomicInteger sourceId = new AtomicInteger(); private final AtomicLong runIdTime = new AtomicLong(System.currentTimeMillis()); - private final String testNamespace = "test"; - - private final PullAppsRequest input = new PullAppsRequest(Collections.emptySet(), null); - @Before public void before() throws Exception { @@ -61,7 +60,7 @@ public void before() throws Exception { @Test public void testGetOperation() throws Exception { OperationRunDetail expectedDetail = insertRun(testNamespace, OperationType.PUSH_APPS, - OperationRunStatus.RUNNING); + OperationRunStatus.RUNNING, transactionRunner); String testId = expectedDetail.getRun().getId(); OperationRunId runId = new OperationRunId(testNamespace, testId); @@ -79,9 +78,9 @@ public void testGetOperation() throws Exception { } @Test - public void testUpdateMetadata() throws Exception { + public void testUpdateResources() throws Exception { OperationRunDetail expectedDetail = insertRun(testNamespace, OperationType.PUSH_APPS, - OperationRunStatus.RUNNING); + OperationRunStatus.RUNNING, transactionRunner); String testId = expectedDetail.getRun().getId(); OperationRunId runId = new OperationRunId(testNamespace, testId); @@ -90,23 +89,23 @@ public void testUpdateMetadata() throws Exception { OperationRunDetail gotDetail = store.getOperation(runId); Assert.assertEquals(expectedDetail, gotDetail); - OperationMeta updatedMeta = new OperationMeta(Collections.emptySet(), - Instant.ofEpochMilli(runIdTime.incrementAndGet()), - Instant.ofEpochMilli(runIdTime.incrementAndGet())); + OperationMeta updatedMeta = OperationMeta.builder(expectedDetail.getRun().getMetadata()) + .setResources(ImmutableSet.of(new OperationResource("test"))).build(); OperationRun updatedRun = OperationRun.builder(expectedDetail.getRun()) .setMetadata(updatedMeta).build(); OperationRunDetail updatedDetail = OperationRunDetail.builder(expectedDetail) .setRun(updatedRun) .setSourceId(AppFabricTestHelper.createSourceId(sourceId.incrementAndGet())) .build(); - store.updateOperationMeta(runId, updatedMeta, updatedDetail.getSourceId()); + store.updateOperationResources(runId, updatedMeta.getResources(), + updatedDetail.getSourceId()); gotDetail = store.getOperation(runId); Assert.assertEquals(updatedDetail, gotDetail); try { - store.updateOperationMeta( + store.updateOperationResources( new OperationRunId(Namespace.DEFAULT.getId(), testId), - updatedMeta, + updatedMeta.getResources(), updatedDetail.getSourceId()); Assert.fail("Found unexpected run in default namespace"); } catch (OperationRunNotFoundException e) { @@ -118,7 +117,7 @@ public void testUpdateMetadata() throws Exception { @Test public void testUpdateStatus() throws Exception { OperationRunDetail expectedDetail = insertRun(testNamespace, OperationType.PUSH_APPS, - OperationRunStatus.RUNNING); + OperationRunStatus.RUNNING, transactionRunner); String testId = expectedDetail.getRun().getId(); OperationRunId runId = new OperationRunId(testNamespace, testId); @@ -155,7 +154,7 @@ public void testUpdateStatus() throws Exception { @Test public void testFailOperation() throws Exception { OperationRunDetail expectedDetail = insertRun(testNamespace, OperationType.PUSH_APPS, - OperationRunStatus.RUNNING); + OperationRunStatus.RUNNING, transactionRunner); String testId = expectedDetail.getRun().getId(); OperationRunId runId = new OperationRunId(testNamespace, testId); @@ -168,19 +167,24 @@ public void testFailOperation() throws Exception { OperationRun updatedRun = OperationRun.builder(expectedDetail.getRun()) .setStatus(OperationRunStatus.FAILED) .setError(error) + .setMetadata(OperationMeta.builder(expectedDetail.getRun().getMetadata()) + .setEndTime(Instant.ofEpochMilli(runIdTime.incrementAndGet())) + .build()) .build(); OperationRunDetail updatedDetail = OperationRunDetail.builder(expectedDetail) .setRun(updatedRun) .setSourceId(AppFabricTestHelper.createSourceId(sourceId.incrementAndGet())) .build(); - store.failOperationRun(runId, error, updatedDetail.getSourceId()); - gotDetail = (OperationRunDetail) store.getOperation(runId); + store.failOperationRun(runId, error, updatedRun.getMetadata().getEndTime(), + updatedDetail.getSourceId()); + gotDetail = store.getOperation(runId); Assert.assertEquals(updatedDetail, gotDetail); try { store.failOperationRun( new OperationRunId(Namespace.DEFAULT.getId(), testId), error, + Instant.now(), // no need to verify this updatedDetail.getSourceId() ); Assert.fail("Found unexpected run in default namespace"); @@ -192,7 +196,7 @@ public void testFailOperation() throws Exception { @Test public void testScanOperation() throws Exception { - List insertedRuns = insertTestRuns(); + List insertedRuns = insertTestRuns(transactionRunner); // get a filtered list of testNamespace runs List testNamespaceRuns = insertedRuns.stream() .filter(detail -> detail.getRunId().getNamespace().equals(testNamespace)) @@ -245,46 +249,16 @@ public void testScanOperation() throws Exception { }, Exception.class); } - private OperationRunDetail insertRun(String namespace, OperationType type, - OperationRunStatus status) - throws IOException, OperationRunAlreadyExistsException { - long startTime = runIdTime.incrementAndGet(); - String id = RunIds.generate(startTime).getId(); - OperationRun run = OperationRun.builder() - .setRunId(id) - .setStatus(status) - .setType(type) - .setMetadata( - new OperationMeta(Collections.emptySet(), Instant.ofEpochMilli(startTime), null)) - .build(); - OperationRunId runId = new OperationRunId(namespace, id); - OperationRunDetail detail = OperationRunDetail.builder() - .setSourceId(AppFabricTestHelper.createSourceId(sourceId.incrementAndGet())) - .setRunId(runId) - .setRun(run) - .setPullAppsRequest(input) - .build(); + @Test + public void testScanOperationByStatus() throws Exception { TransactionRunners.run(transactionRunner, context -> { - OperationRunStore operationRunStore = new OperationRunStore(context); - operationRunStore.createOperationRun(runId, detail); - }, IOException.class, OperationRunAlreadyExistsException.class); - return detail; - } - - private List insertTestRuns() throws Exception { - List details = new ArrayList<>(); - // insert 10 runs with increasing start time in two namespaces - // 5 would be in running state 5 in Failed - // 5 would be of type PUSH 5 would be of type PULL - for (int i = 0; i < 5; i++) { - details.add(insertRun(testNamespace, OperationType.PUSH_APPS, OperationRunStatus.RUNNING)); - details.add(insertRun(Namespace.DEFAULT.getId(), OperationType.PUSH_APPS, OperationRunStatus.RUNNING)); - details.add(insertRun(testNamespace, OperationType.PULL_APPS, OperationRunStatus.FAILED)); - details.add(insertRun(Namespace.DEFAULT.getId(), OperationType.PULL_APPS, OperationRunStatus.RUNNING)); - } - // The runs are added in increasing start time, hence reversing the List - Collections.reverse(details); - return details; + Set expectedRuns = insertTestRuns(transactionRunner).stream().filter( + d -> d.getRun().getStatus().equals(OperationRunStatus.PENDING) + ).collect(Collectors.toSet()); + Set gotRuns = new HashSet<>(); + OperationRunStore store = new OperationRunStore(context); + store.scanOperationByStatus(OperationRunStatus.PENDING, gotRuns::add); + Assert.assertTrue(expectedRuns.containsAll(gotRuns)); + }, InvalidFieldException.class, IOException.class); } - } diff --git a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/operation/OperationTestBase.java b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/operation/OperationTestBase.java new file mode 100644 index 000000000000..24aa72d01075 --- /dev/null +++ b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/operation/OperationTestBase.java @@ -0,0 +1,131 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.cdap.internal.operation; + +import io.cdap.cdap.common.app.RunIds; +import io.cdap.cdap.common.id.Id.Namespace; +import io.cdap.cdap.internal.AppFabricTestHelper; +import io.cdap.cdap.internal.app.sourcecontrol.PullAppsRequest; +import io.cdap.cdap.proto.id.OperationRunId; +import io.cdap.cdap.proto.operation.OperationMeta; +import io.cdap.cdap.proto.operation.OperationRun; +import io.cdap.cdap.proto.operation.OperationRunStatus; +import io.cdap.cdap.proto.operation.OperationType; +import io.cdap.cdap.spi.data.transaction.TransactionRunner; +import io.cdap.cdap.spi.data.transaction.TransactionRunners; +import java.io.IOException; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +public class OperationTestBase { + + private static final AtomicInteger sourceId = new AtomicInteger(); + private static final AtomicLong runIdTime = new AtomicLong(System.currentTimeMillis()); + protected static final String testNamespace = "test"; + private static final PullAppsRequest input = new PullAppsRequest(Collections.emptySet(), null); + + protected static OperationRunDetail insertRun( + String namespace, + OperationType type, + OperationRunStatus status, + TransactionRunner transactionRunner) + throws IOException, OperationRunAlreadyExistsException { + long startTime = runIdTime.incrementAndGet(); + String id = RunIds.generate(startTime).getId(); + OperationRun run = + OperationRun.builder() + .setRunId(id) + .setStatus(status) + .setType(type) + .setMetadata( + OperationMeta.builder() + .setCreateTime(Instant.ofEpochMilli(startTime)) + .build() + ) + .build(); + OperationRunId runId = new OperationRunId(namespace, id); + OperationRunDetail detail = + OperationRunDetail.builder() + .setSourceId(AppFabricTestHelper.createSourceId(sourceId.incrementAndGet())) + .setRunId(runId) + .setRun(run) + .setPullAppsRequest(input) + .build(); + TransactionRunners.run( + transactionRunner, + context -> { + OperationRunStore operationRunStore = new OperationRunStore(context); + operationRunStore.createOperationRun(runId, detail); + }, + IOException.class, + OperationRunAlreadyExistsException.class); + return detail; + } + + protected static List insertTestRuns(TransactionRunner transactionRunner) + throws Exception { + List details = new ArrayList<>(); + // insert 10 runs with increasing start time in two namespaces + // 5 would be in running state 5 in Failed + // 5 would be of type PUSH 5 would be of type PULL + for (int i = 0; i < 5; i++) { + details.add( + insertRun( + testNamespace, + OperationType.PUSH_APPS, + OperationRunStatus.PENDING, + transactionRunner)); + details.add( + insertRun( + Namespace.DEFAULT.getId(), + OperationType.PUSH_APPS, + OperationRunStatus.RUNNING, + transactionRunner)); + details.add( + insertRun( + Namespace.DEFAULT.getId(), + OperationType.PUSH_APPS, + OperationRunStatus.FAILED, + transactionRunner)); + details.add( + insertRun( + testNamespace, + OperationType.PULL_APPS, + OperationRunStatus.PENDING, + transactionRunner)); + details.add( + insertRun( + Namespace.DEFAULT.getId(), + OperationType.PULL_APPS, + OperationRunStatus.RUNNING, + transactionRunner)); + details.add( + insertRun( + Namespace.DEFAULT.getId(), + OperationType.PUSH_APPS, + OperationRunStatus.FAILED, + transactionRunner)); + } + // The runs are added in increasing start time, hence reversing the List + Collections.reverse(details); + return details; + } +} \ No newline at end of file diff --git a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/operation/SqlOperationRunsStoreTest.java b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/operation/SqlOperationRunsStoreTest.java index e2c54d7288b0..5700fcb66d9a 100644 --- a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/operation/SqlOperationRunsStoreTest.java +++ b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/operation/SqlOperationRunsStoreTest.java @@ -16,6 +16,7 @@ package io.cdap.cdap.internal.operation; +import com.google.common.io.Closeables; import com.google.inject.AbstractModule; import com.google.inject.Guice; import com.google.inject.Injector; @@ -69,7 +70,7 @@ protected void configure() { } @AfterClass - public static void afterClass() throws IOException { - pg.close(); + public static void afterClass(){ + Closeables.closeQuietly(pg); } } diff --git a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/profile/ProfileServiceTest.java b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/profile/ProfileServiceTest.java index a9b62aabbef1..31853154e8fe 100644 --- a/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/profile/ProfileServiceTest.java +++ b/cdap-app-fabric/src/test/java/io/cdap/cdap/internal/profile/ProfileServiceTest.java @@ -354,7 +354,7 @@ public void testProfileDeletion() throws Exception { ApplicationMeta appMeta = new ApplicationMeta(appSpec.getName(), appSpec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(appId, appMeta); + store.addLatestApplication(appId, appMeta); ProgramId programId = NamespaceId.DEFAULT.app(appSpec.getName()).workflow(SampleWorkflow.NAME); ArtifactId artifactId = NamespaceId.DEFAULT.artifact("testArtifact", "1.0").toApiArtifactId(); diff --git a/cdap-app-fabric/src/test/java/io/cdap/cdap/metadata/LineageAdminTest.java b/cdap-app-fabric/src/test/java/io/cdap/cdap/metadata/LineageAdminTest.java index dcb9f1c8e7a6..3c08eb81beca 100644 --- a/cdap-app-fabric/src/test/java/io/cdap/cdap/metadata/LineageAdminTest.java +++ b/cdap-app-fabric/src/test/java/io/cdap/cdap/metadata/LineageAdminTest.java @@ -495,7 +495,7 @@ public void testWorkflowLineage() throws ConflictException { ApplicationMeta meta = new ApplicationMeta(appSpec.getName(), appSpec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(testApp, meta); + store.addLatestApplication(testApp, meta); LineageAdmin lineageAdmin = new LineageAdmin(lineageReader, store); // Add accesses for D3 -> P2 -> D2 -> P1 -> D1 <-> P3 @@ -602,7 +602,7 @@ public void testWorkflowLineage() throws ConflictException { new Relation(dataset1, program3, AccessType.UNKNOWN, twillRunId(run3)) ), oneLevelLineage.getRelations()); - + // Assert that in a different namespace both lineage and metadata should be empty NamespaceId customNamespace = new NamespaceId("custom_namespace"); DatasetId customDataset1 = customNamespace.dataset(dataset1.getEntityName()); @@ -671,7 +671,7 @@ public void testLocalDatasetsInWorkflow() throws Exception { ApplicationMeta meta = new ApplicationMeta(appSpec.getName(), appSpec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(testApp, meta); + store.addLatestApplication(testApp, meta); LineageAdmin lineageAdmin = new LineageAdmin(lineageReader, store); // Add accesses for D1 -| diff --git a/cdap-app-fabric/src/test/java/io/cdap/cdap/metadata/MetadataSubscriberServiceTest.java b/cdap-app-fabric/src/test/java/io/cdap/cdap/metadata/MetadataSubscriberServiceTest.java index 63e80d37d372..a61d28a4f9f8 100644 --- a/cdap-app-fabric/src/test/java/io/cdap/cdap/metadata/MetadataSubscriberServiceTest.java +++ b/cdap-app-fabric/src/test/java/io/cdap/cdap/metadata/MetadataSubscriberServiceTest.java @@ -447,7 +447,7 @@ public void testProfileMetadata() throws Exception { ApplicationMeta meta = new ApplicationMeta(appSpec.getName(), appSpec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(appId, meta); + store.addLatestApplication(appId, meta); // set default namespace to use the profile, since now MetadataSubscriberService is not started, // it should not affect the mds @@ -591,7 +591,7 @@ public void testProfileMetadataWithNoProfilePreferences() throws Exception { ApplicationMeta meta = new ApplicationMeta(appSpec.getName(), appSpec, new ChangeDetail(null, null, null, System.currentTimeMillis())); - store.addApplication(appId, meta); + store.addLatestApplication(appId, meta); // set default namespace to use the profile, since now MetadataSubscriberService is not started, // it should not affect the mds diff --git a/cdap-authenticator-ext-gcp/src/main/java/io/cdap/cdap/authenticator/gcp/GCPRemoteAuthenticator.java b/cdap-authenticator-ext-gcp/src/main/java/io/cdap/cdap/authenticator/gcp/GCPRemoteAuthenticator.java index ed04d4bc0aaa..d9c45aab14c7 100644 --- a/cdap-authenticator-ext-gcp/src/main/java/io/cdap/cdap/authenticator/gcp/GCPRemoteAuthenticator.java +++ b/cdap-authenticator-ext-gcp/src/main/java/io/cdap/cdap/authenticator/gcp/GCPRemoteAuthenticator.java @@ -64,6 +64,7 @@ public Credential getCredentials() throws IOException { if (accessToken == null || accessToken.getExpirationTime().before(Date.from(clock.instant()))) { accessToken = googleCredentials.refreshAccessToken(); } - return new Credential(accessToken.getTokenValue(), Credential.CredentialType.EXTERNAL_BEARER); + return new Credential(accessToken.getTokenValue(), Credential.CredentialType.EXTERNAL_BEARER, + accessToken.getExpirationTime().getTime() / 1000L); } } diff --git a/cdap-common/src/main/java/io/cdap/cdap/common/conf/Constants.java b/cdap-common/src/main/java/io/cdap/cdap/common/conf/Constants.java index f3a5dc32a179..9341e9c9d043 100644 --- a/cdap-common/src/main/java/io/cdap/cdap/common/conf/Constants.java +++ b/cdap-common/src/main/java/io/cdap/cdap/common/conf/Constants.java @@ -594,6 +594,8 @@ public static final class ArtifactLocalizer { public static final String WORKER_THREADS = "artifact.localizer.worker.threads"; public static final String PRELOAD_LIST = "artifact.localizer.preload.list"; public static final String PRELOAD_VERSION_LIMIT = "artifact.localizer.preload.version.limit"; + public static final String REMOTE_AUTHENTICATOR_NAME = + "artifact.localizer.remote.authenticator.name"; } /** @@ -2432,4 +2434,31 @@ public static final class InternalRouter { public static final String CLIENT_ENABLED = "internal.router.client.enabled"; public static final String SERVER_ENABLED = "internal.router.server.enabled"; } + + /** + * Constants for operations. + */ + public static final class Operation { + + /** + * Topic prefix for publishing status transitioning events of operation runs to the messaging + * system. + */ + public static final String STATUS_EVENT_TOPIC = "operation.status.event.topic"; + /** + * Number of topics to use for operation status events. All events related to same run should + * always go to same topic. If this value is 1, {@link #STATUS_EVENT_TOPIC} is a topic name. If + * it's more than 1, {@link #STATUS_EVENT_TOPIC} is a prefix, but bare name should still be + * subscribed to ensure any pending messages / active run events are processed properly. + */ + public static final String STATUS_EVENT_NUM_PARTITIONS = "operation.status.event.topic.num.partitions"; + public static final String STATUS_RETRY_STRATEGY_PREFIX = "operation.status."; + + // Notification keys + public static final String RUN_ID_NOTIFICATION_KEY = "operation.notification.run.id"; + public static final String STATUS_NOTIFICATION_KEY = "operation.notification.status"; + public static final String RESOURCES_NOTIFICATION_KEY = "operation.notification.resources"; + public static final String ENDTIME_NOTIFICATION_KEY = "operation.notification.endtime"; + public static final String ERROR_NOTIFICATION_KEY = "operation.notification.error"; + } } diff --git a/cdap-credential-ext-gcp-wi/src/main/java/io/cdap/cdap/security/spi/credential/GcpWorkloadIdentityCredentialProvider.java b/cdap-credential-ext-gcp-wi/src/main/java/io/cdap/cdap/security/spi/credential/GcpWorkloadIdentityCredentialProvider.java index aec8224f8db9..ab380073b261 100644 --- a/cdap-credential-ext-gcp-wi/src/main/java/io/cdap/cdap/security/spi/credential/GcpWorkloadIdentityCredentialProvider.java +++ b/cdap-credential-ext-gcp-wi/src/main/java/io/cdap/cdap/security/spi/credential/GcpWorkloadIdentityCredentialProvider.java @@ -63,9 +63,8 @@ import org.slf4j.LoggerFactory; /** - * {@link CredentialProvider} Credential Provider which returns application default credentials. - * For more details, see - * medium.com/google-cloud/gcp-workload-identity-federation-with-federated-tokens-d03b8bad0228 + * {@link CredentialProvider} Credential Provider which returns application default credentials. For + * more details, see medium.com/google-cloud/gcp-workload-identity-federation-with-federated-tokens-d03b8bad0228 */ public class GcpWorkloadIdentityCredentialProvider implements CredentialProvider { @@ -189,9 +188,11 @@ public ProvisionedCredential provision(NamespaceMeta namespaceMeta, private ProvisionedCredential getProvisionedCredential(NamespaceMeta namespaceMeta, CredentialIdentity identity, @Nullable String scopes) throws IOException, ApiException { - // get k8s namespace from namespace metadata if namespace creation hook is enabled. + // Get k8s namespace from namespace metadata if using a non-default namespace and namespace + // creation hook is enabled. String k8sNamespace = NamespaceId.DEFAULT.getNamespace(); - if (credentialProviderContext.isNamespaceCreationHookEnabled()) { + if (!namespaceMeta.getName().equals(NamespaceId.DEFAULT.getNamespace()) + && credentialProviderContext.isNamespaceCreationHookEnabled()) { k8sNamespace = namespaceMeta.getConfig().getConfigs().get("k8s.namespace"); } @@ -227,7 +228,7 @@ private ProvisionedCredential getProvisionedCredential(NamespaceMeta namespaceMe // get GSA token using Federating Token as credential IamCredentialGenerateAccessTokenResponse iamCredentialGenerateAccessTokenResponse = GSON.fromJson(fetchIamServiceAccountToken(securityTokenServiceResponse.getAccessToken(), - scopes, identity.getSecureValue()), + scopes, identity.getSecureValue()), IamCredentialGenerateAccessTokenResponse.class); LOG.trace("Successfully generated GSA token using Federating Token as credential."); @@ -235,7 +236,8 @@ private ProvisionedCredential getProvisionedCredential(NamespaceMeta namespaceMe Instant.parse(iamCredentialGenerateAccessTokenResponse.getExpireTime())); } catch (ApiException e) { - if ((e.getCode() / 100) != 4) { + // Returned code can be 0 if parameter validation fails before an API call is made. + if (e.getCode() != 0 && (e.getCode() / 100) != 4) { // if there was an API exception that was not a 4xx, we can just retry throw new RetryableException(e); } diff --git a/cdap-features/src/main/java/io/cdap/cdap/features/Feature.java b/cdap-features/src/main/java/io/cdap/cdap/features/Feature.java index 4f15fab8a6a7..a68c37634b23 100644 --- a/cdap-features/src/main/java/io/cdap/cdap/features/Feature.java +++ b/cdap-features/src/main/java/io/cdap/cdap/features/Feature.java @@ -38,6 +38,7 @@ public enum Feature { STREAMING_PIPELINE_NATIVE_STATE_TRACKING("6.8.0", false), PUSHDOWN_TRANSFORMATION_WINDOWAGGREGATION("6.9.1"), SOURCE_CONTROL_MANAGEMENT_GIT("6.9.0"), + SOURCE_CONTROL_MANAGEMENT_MULTI_APP("6.10.0"), WRANGLER_PRECONDITION_SQL("6.9.1"), WRANGLER_EXECUTION_SQL("6.10.0"), WRANGLER_SCHEMA_MANAGEMENT("6.10.0"), diff --git a/cdap-kubernetes/src/main/java/io/cdap/cdap/k8s/runtime/KubeTwillPreparer.java b/cdap-kubernetes/src/main/java/io/cdap/cdap/k8s/runtime/KubeTwillPreparer.java index 5e85f18baee1..46797f0c77b4 100644 --- a/cdap-kubernetes/src/main/java/io/cdap/cdap/k8s/runtime/KubeTwillPreparer.java +++ b/cdap-kubernetes/src/main/java/io/cdap/cdap/k8s/runtime/KubeTwillPreparer.java @@ -189,6 +189,7 @@ class KubeTwillPreparer implements DependentTwillPreparer, StatefulTwillPreparer private final String resourcePrefix; private final Map extraLabels; private final Map secretDiskRunnables; + private final Set withNamespaceWorkloadIdentityRunnables; private final Map containerSecurityContexts; private final Map> readonlyDisks; private final Map> runnableConfigs; @@ -240,6 +241,7 @@ class KubeTwillPreparer implements DependentTwillPreparer, StatefulTwillPreparer this.dependentRunnableNames = new HashSet<>(); this.serviceAccountName = null; this.secretDiskRunnables = new HashMap<>(); + this.withNamespaceWorkloadIdentityRunnables = new HashSet<>(); this.containerSecurityContexts = new HashMap<>(); this.readonlyDisks = new HashMap<>(); this.runnableConfigs = runnables.stream() @@ -368,6 +370,12 @@ public SecureTwillPreparer withSecretDisk(String runnableName, SecretDisk... sec return this; } + @Override + public SecureTwillPreparer withNamespacedWorkloadIdentity(String runnableName) { + withNamespaceWorkloadIdentityRunnables.add(runnableName); + return this; + } + @Override public SecureTwillPreparer withSecurityContext(String runnableName, SecurityContext securityContext) { @@ -1285,9 +1293,8 @@ private List createContainers(Map run environs.put(JAVA_OPTS_KEY, jvmOpts); // Add workload identity environment variable if applicable. if (workloadIdentityEnabled && WorkloadIdentityUtil.shouldMountWorkloadIdentity( - cdapInstallNamespace, - programRuntimeNamespace, - workloadIdentityServiceAccount)) { + cdapInstallNamespace, programRuntimeNamespace, workloadIdentityServiceAccount) + && !withNamespaceWorkloadIdentityRunnables.contains(runnableName)) { V1EnvVar workloadIdentityEnvVar = WorkloadIdentityUtil.generateWorkloadIdentityEnvVar(); environs.put(workloadIdentityEnvVar.getName(), workloadIdentityEnvVar.getValue()); } @@ -1314,6 +1321,13 @@ private List createContainers(Map run .filter(entry -> !entry.getKey().equals(GCE_METADATA_HOST_ENV_VAR)) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + // Add workload identity environment variable in the dependent runnable if applicable. + if (workloadIdentityEnabled && WorkloadIdentityUtil.shouldMountWorkloadIdentity( + cdapInstallNamespace, programRuntimeNamespace, workloadIdentityServiceAccount) + && !withNamespaceWorkloadIdentityRunnables.contains(name)) { + V1EnvVar workloadIdentityEnvVar = WorkloadIdentityUtil.generateWorkloadIdentityEnvVar(); + envs.put(workloadIdentityEnvVar.getName(), workloadIdentityEnvVar.getValue()); + } mounts = addSecreteVolMountIfNeeded(spec, volumeMounts); containers.add( createContainer(name, podInfo.getContainerImage(), podInfo.getImagePullPolicy(), workDir, diff --git a/cdap-master-spi/src/main/java/io/cdap/cdap/master/spi/twill/SecureTwillPreparer.java b/cdap-master-spi/src/main/java/io/cdap/cdap/master/spi/twill/SecureTwillPreparer.java index 8355750eb74c..d49f5d455108 100644 --- a/cdap-master-spi/src/main/java/io/cdap/cdap/master/spi/twill/SecureTwillPreparer.java +++ b/cdap-master-spi/src/main/java/io/cdap/cdap/master/spi/twill/SecureTwillPreparer.java @@ -44,4 +44,13 @@ public interface SecureTwillPreparer extends TwillPreparer { SecureTwillPreparer withSecurityContext(String runnableName, SecurityContext securityContext); + /** + * Runs the given runnable with namespace workload identity, + * this feature removes the GOOGLE_APPLICATION_CREDENTIALS environment variable + * to enable namespaced service accounts. + * + * @param runnableName name of the {@link TwillRunnable} + * @return this {@link TwillPreparer} + */ + SecureTwillPreparer withNamespacedWorkloadIdentity(String runnableName); } diff --git a/cdap-proto/src/main/java/io/cdap/cdap/proto/Notification.java b/cdap-proto/src/main/java/io/cdap/cdap/proto/Notification.java index c9a5e803d439..9f3fb676e6c3 100644 --- a/cdap-proto/src/main/java/io/cdap/cdap/proto/Notification.java +++ b/cdap-proto/src/main/java/io/cdap/cdap/proto/Notification.java @@ -45,7 +45,8 @@ public enum Type { TIME, PARTITION, PROGRAM_STATUS, - PROGRAM_HEART_BEAT + PROGRAM_HEART_BEAT, + OPERATION_STATUS } private final Type notificationType; diff --git a/cdap-proto/src/main/java/io/cdap/cdap/proto/operation/OperationMeta.java b/cdap-proto/src/main/java/io/cdap/cdap/proto/operation/OperationMeta.java index f71ec8ade734..2aff0a046fe9 100644 --- a/cdap-proto/src/main/java/io/cdap/cdap/proto/operation/OperationMeta.java +++ b/cdap-proto/src/main/java/io/cdap/cdap/proto/operation/OperationMeta.java @@ -17,17 +17,18 @@ package io.cdap.cdap.proto.operation; import java.time.Instant; +import java.util.Collections; +import java.util.HashSet; import java.util.Objects; import java.util.Set; import javax.annotation.Nullable; /** - * Metadata for an operation includes - * 1. The resources on which operation is executed - * 2. Timestamp of operation create - * 3. Timestamp of operation endtime + * Metadata for an operation includes 1. The resources on which operation is executed 2. Timestamp + * of operation create 3. Timestamp of operation endtime */ public class OperationMeta { + private final Set resources; private final Instant createTime; @@ -41,7 +42,8 @@ public class OperationMeta { * @param createTime timestamp when the operation was created * @param endTime timestamp when the operation reached an end state */ - public OperationMeta(Set resources, Instant createTime, @Nullable Instant endTime) { + private OperationMeta(Set resources, Instant createTime, + @Nullable Instant endTime) { this.resources = resources; this.createTime = createTime; this.endTime = endTime; @@ -69,7 +71,7 @@ public int hashCode() { } public Set getResources() { - return resources; + return Collections.unmodifiableSet(resources); } public Instant getCreateTime() { @@ -80,4 +82,67 @@ public Instant getCreateTime() { public Instant getEndTime() { return endTime; } + + /** + * Creates a Builder. + */ + public static Builder builder() { + return new Builder(); + } + + /** + * Create a Builder from existing run. + * + * @param meta existing meta to copy fields from + */ + public static Builder builder(OperationMeta meta) { + return new Builder() + .setCreateTime(meta.getCreateTime()) + .setEndTime(meta.endTime) + .setResources(meta.resources); + } + + /** + * Builder to create OperationMeta. + */ + @SuppressWarnings("unchecked") + public static class Builder { + + private final Set resources; + private Instant createTime; + private Instant endTime; + + protected Builder() { + this.resources = new HashSet<>(); + } + + /** + * Clear current resources and add given resources. + */ + public Builder setResources(Set resources) { + this.resources.clear(); + this.resources.addAll(resources); + return this; + } + + public Builder setCreateTime(Instant createTime) { + this.createTime = createTime; + return this; + } + + public Builder setEndTime(Instant endTime) { + this.endTime = endTime; + return this; + } + + /** + * Builds the OperationMeta. + */ + public OperationMeta build() { + if (createTime == null) { + throw new IllegalArgumentException("create time must be specified"); + } + return new OperationMeta(resources, createTime, endTime); + } + } } diff --git a/cdap-proto/src/main/java/io/cdap/cdap/proto/operation/OperationRunStatus.java b/cdap-proto/src/main/java/io/cdap/cdap/proto/operation/OperationRunStatus.java index 406364db79a4..38ceabb75eb2 100644 --- a/cdap-proto/src/main/java/io/cdap/cdap/proto/operation/OperationRunStatus.java +++ b/cdap-proto/src/main/java/io/cdap/cdap/proto/operation/OperationRunStatus.java @@ -48,6 +48,8 @@ public boolean canTransitionTo(OperationRunStatus status) { return true; } switch (this) { + case PENDING: + return status == STARTING || status == STOPPING; case STARTING: // RUNNING is the happy path // STOPPING happens if the run was manually stopped gracefully(may include a timeout) diff --git a/cdap-proto/src/main/java/io/cdap/cdap/proto/security/Credential.java b/cdap-proto/src/main/java/io/cdap/cdap/proto/security/Credential.java index 0f8887611d98..b22eb9be2cd6 100644 --- a/cdap-proto/src/main/java/io/cdap/cdap/proto/security/Credential.java +++ b/cdap-proto/src/main/java/io/cdap/cdap/proto/security/Credential.java @@ -84,10 +84,31 @@ public static CredentialType fromQualifiedName(String qualifiedName) { private final String value; private final CredentialType type; + private final Long expirationTimeSecs; + /** + * Constructs the Credential. + * + * @param value credential value + * @param type credential type + */ public Credential(String value, CredentialType type) { this.value = value; this.type = type; + this.expirationTimeSecs = null; + } + + /** + * Constructs the Credential. + * + * @param value credential value + * @param type credential type + * @param expirationTimeSecs the time in seconds after which credential will expire + */ + public Credential(String value, CredentialType type, Long expirationTimeSecs) { + this.value = value; + this.type = type; + this.expirationTimeSecs = expirationTimeSecs; } public String getValue() { @@ -98,10 +119,15 @@ public CredentialType getType() { return type; } + public Long getExpirationTimeSecs() { + return expirationTimeSecs; + } + @Override public String toString() { return "Credential{" + "type=" + type + + ", expires_in=" + expirationTimeSecs + ", length=" + value.length() + "}"; } diff --git a/cdap-source-control/src/main/java/io/cdap/cdap/sourcecontrol/RepositoryManager.java b/cdap-source-control/src/main/java/io/cdap/cdap/sourcecontrol/RepositoryManager.java index 4d4bfe7cdb71..06fd4f55f835 100644 --- a/cdap-source-control/src/main/java/io/cdap/cdap/sourcecontrol/RepositoryManager.java +++ b/cdap-source-control/src/main/java/io/cdap/cdap/sourcecontrol/RepositoryManager.java @@ -39,12 +39,15 @@ import java.nio.file.LinkOption; import java.nio.file.Path; import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.Date; import java.util.List; import java.util.Map; import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.function.BiFunction; import java.util.function.Supplier; import java.util.stream.Stream; import javax.annotation.Nullable; @@ -58,6 +61,8 @@ import org.eclipse.jgit.api.TransportCommand; import org.eclipse.jgit.api.errors.GitAPIException; import org.eclipse.jgit.api.errors.TransportException; +import org.eclipse.jgit.dircache.DirCache; +import org.eclipse.jgit.dircache.DirCacheEntry; import org.eclipse.jgit.errors.MissingObjectException; import org.eclipse.jgit.internal.storage.dfs.InMemoryRepository; import org.eclipse.jgit.lib.Constants; @@ -75,8 +80,8 @@ import org.slf4j.LoggerFactory; /** - * A git repository manager that is responsible for handling interfacing with - * git. It provides version control operations. This is not thread safe. + * A git repository manager that is responsible for handling interfacing with git. It provides + * version control operations. This is not thread safe. */ public class RepositoryManager implements AutoCloseable { @@ -99,8 +104,8 @@ public class RepositoryManager implements AutoCloseable { * @param cConf CDAP configurations. * @param namespace The CDAP namespace to whcih the Git repository is linked. * @param repoConfig The repository configuration for the CDAP namespace. - * @param metricsCollectionService A metrics service to emit metrics related - * to SCM operations. + * @param metricsCollectionService A metrics service to emit metrics related to SCM + * operations. */ public RepositoryManager(final SecureStore secureStore, final CConfiguration cConf, final NamespaceId namespace, @@ -123,9 +128,8 @@ public RepositoryManager(final SecureStore secureStore, } /** - * Returns the base path in the git repo to store CDAP applications. If an - * optional Path prefix is provided in the repository configuration, the - * returned path includes it. + * Returns the base path in the git repo to store CDAP applications. If an optional Path prefix is + * provided in the repository configuration, the returned path includes it. * * @return the path for the repository base directory. */ @@ -140,8 +144,7 @@ public Path getBasePath() { } /** - * Gets the relative path of a file in git repository based on the user - * configured path prefix. + * Gets the relative path of a file in git repository based on the user configured path prefix. * * @param fileName The filename * @return the relative {@link Path} @@ -158,11 +161,10 @@ public Path getFileRelativePath(final String fileName) { /** * Validates the provided configuration. * - * @param secureStore A secure store for fetching credentials if - * required. + * @param secureStore A secure store for fetching credentials if required. * @param sourceControlConfig Configuration for source control operations. - * @throws RepositoryConfigValidationException when provided repository - * configuration is invalid. + * @throws RepositoryConfigValidationException when provided repository configuration is + * invalid. */ public static void validateConfig(final SecureStore secureStore, final SourceControlConfig sourceControlConfig) @@ -211,24 +213,24 @@ public static void validateConfig(final SecureStore secureStore, } /** - * Commits and pushes the changes of a given file under the repository root - * path. + * Commits and pushes the changes of given files under the repository root path. + * Processes the hashes to a list of given type S * - * @param commitMeta Details for the commit including author, committer and - * commit message - * @param fileChanged The relative path to repository root where the file is - * updated - * @return the hash of the written file. - * @throws GitAPIException when the underlying git commands fail - * @throws NoChangesToPushException when there's no file changes for the - * commit - * @throws GitOperationException when failed to get filehash due to IOException - * or the {@link PushResult} status is not OK - * @throws SourceControlException when failed to get the fileHash before - * push + * @param Type of elements of the output collection + * @param Type of elements of the input collection, must implement {@link Supplier} + * @param commitMeta Details for the commit including author, committer and commit message + * @param filesChanged Supplies relative paths to repository root of the updated files + * @param hashConsumer a {@link BiFunction} that takes a value of the input collection element type (S), + * a filehash and returns a value of the output collection element type (T) + * @return a {@link Collection} of elements of type T + * @throws GitAPIException when the underlying git commands fail + * @throws NoChangesToPushException when there's no file changes for the commit + * @throws GitOperationException when failed to get filehash due to IOException or the + * {@link PushResult} status is not OK + * @throws SourceControlException when failed to get the fileHash before push */ - public String commitAndPush(final CommitMeta commitMeta, - final Path fileChanged) + public > Collection commitAndPush(CommitMeta commitMeta, Collection filesChanged, + BiFunction hashConsumer) throws NoChangesToPushException, GitAPIException { validateInitialized(); final Stopwatch stopwatch = new Stopwatch().start(); @@ -240,25 +242,21 @@ public String commitAndPush(final CommitMeta commitMeta, "No changes have been made for the applications to push."); } - git.add().addFilepattern(fileChanged.toString()).call(); - - RevCommit commit = getCommitCommand(commitMeta).call(); - - String fileHash; - try { - fileHash = getFileHash(fileChanged, commit); - } catch (IOException e) { - throw new GitOperationException( - String.format("Failed to get fileHash for %s", fileChanged), - e); + List output = new ArrayList<>(filesChanged.size()); + for (S fileChanged : filesChanged) { + DirCache cache = git.add().addFilepattern(fileChanged.get().toString()).call(); + DirCacheEntry entry = cache.getEntry(fileChanged.get().toString()); + // This should not happen usually as we have just added the entry + if (entry == null) { + throw new SourceControlException( + String.format( + "Failed to get fileHash for %s because some paths are not " + + "found in Git tree", fileChanged.get())); + } + output.add(hashConsumer.apply(fileChanged, entry.getObjectId().name())); } - if (fileHash == null) { - throw new SourceControlException( - String.format( - "Failed to get fileHash for %s, because the path is not " - + "found in Git tree", fileChanged)); - } + getCommitCommand(commitMeta).call(); PushCommand pushCommand = createCommand(git::push, sourceControlConfig, credentialsProvider); @@ -269,7 +267,7 @@ public String commitAndPush(final CommitMeta commitMeta, if (rru.getStatus() != RemoteRefUpdate.Status.OK && rru.getStatus() != RemoteRefUpdate.Status.UP_TO_DATE) { throw new GitOperationException( - String.format("Push failed for %s: %s", fileChanged, + String.format("Push failed for %s: %s", filesChanged, rru.getStatus())); } } @@ -278,7 +276,7 @@ public String commitAndPush(final CommitMeta commitMeta, metricsContext.event( SourceControlManagement.COMMIT_PUSH_LATENCY_MILLIS, stopwatch.stop().elapsedTime(TimeUnit.MILLISECONDS)); - return fileHash; + return output; } private CommitCommand getCommitCommand(final CommitMeta commitMeta) { @@ -294,16 +292,14 @@ private CommitCommand getCommitCommand(final CommitMeta commitMeta) { } /** - * Initializes the Git repository by cloning remote. If the repository is - * already cloned, it resets the git repository to the state in the remote. - * All local changes will be lost. + * Initializes the Git repository by cloning remote. If the repository is already cloned, it + * resets the git repository to the state in the remote. All local changes will be lost. * * @return the commit ID of the present HEAD. - * @throws GitAPIException when a Git operation fails. - * @throws IOException when file or network I/O fails. - * @throws AuthenticationConfigException when there is a failure while - * fetching authentication credentials - * for Git. + * @throws GitAPIException when a Git operation fails. + * @throws IOException when file or network I/O fails. + * @throws AuthenticationConfigException when there is a failure while fetching authentication + * credentials for Git. */ public String cloneRemote() throws IOException, AuthenticationConfigException, GitAPIException { @@ -371,20 +367,19 @@ public Path getRepositoryRoot() { } /** - * Returns the Git Hash - * of the requested file path in the provided commit. It is the caller's - * responsibility to handle paths that refer to directories or symlinks. + * Returns the Git Hash of the requested + * file path in the provided commit. It is the caller's responsibility to handle paths that refer + * to directories or symlinks. * - * @param relativePath The path relative to the repository base path (returned - * by {@link RepositoryManager#getRepositoryRoot()}) on - * the filesystem. - * @param commitHash The commit ID hash for which to get the file hash. + * @param relativePath The path relative to the repository base path (returned by + * {@link RepositoryManager#getRepositoryRoot()}) on the filesystem. + * @param commitHash The commit ID hash for which to get the file hash. * @return The git file hash of the requested file. - * @throws IOException when file or commit isn't found, there are - * circular symlinks or other file IO errors. - * @throws NotFoundException when the file isn't committed to Git. - * @throws IllegalStateException when {@link RepositoryManager#cloneRemote()} - * isn't called before this. + * @throws IOException when file or commit isn't found, there are circular symlinks or other + * file IO errors. + * @throws NotFoundException when the file isn't committed to Git. + * @throws IllegalStateException when {@link RepositoryManager#cloneRemote()} isn't called + * before this. */ public String getFileHash(final Path relativePath, final String commitHash) throws IOException, NotFoundException, @@ -524,17 +519,14 @@ ObjectId resolveHead() throws IOException { } /** - * Gets the {@link RevCommit} for a {@link ObjectId} from the local git - * repository. + * Gets the {@link RevCommit} for a {@link ObjectId} from the local git repository. * - * @param commitId the ID of the commit to be fetched. - * @param fetchCommitsIfRequired Whether to fetch commits from remote - * repository when commit isn't found locally. + * @param commitId the ID of the commit to be fetched. + * @param fetchCommitsIfRequired Whether to fetch commits from remote repository when commit + * isn't found locally. * @return the {@link RevCommit} - * @throws IOException when the commit for the provided commitId isn't - * found. - * @throws GitAPIException when there are failures while fetching commits from - * the remote. + * @throws IOException when the commit for the provided commitId isn't found. + * @throws GitAPIException when there are failures while fetching commits from the remote. */ private RevCommit getRevCommit(final ObjectId commitId, final boolean fetchCommitsIfRequired) @@ -582,8 +574,8 @@ private void fetchRemote(final RemoteConfig remote) throws GitAPIException { } /** - * Does a hard reset to the - * remote branch being tracked. + * Does a hard reset to the remote branch being + * tracked. * * @param remoteBranchName branch to reset to. * @return the commit ID of the new head. @@ -605,8 +597,8 @@ private String resetToCleanBranch(final String remoteBranchName) * * @param directoryPath the path of the root directory. * @return the size of the directory in bytes. - * @throws IOException when there are failures while reading the file or {@link UncheckedIOException} - * is thrown out and we throw to the cause. + * @throws IOException when there are failures while reading the file or + * {@link UncheckedIOException} is thrown out and we throw to the cause. */ @VisibleForTesting static long calculateDirectorySize(Path directoryPath) throws IOException { diff --git a/cdap-source-control/src/main/java/io/cdap/cdap/sourcecontrol/operationrunner/InMemorySourceControlOperationRunner.java b/cdap-source-control/src/main/java/io/cdap/cdap/sourcecontrol/operationrunner/InMemorySourceControlOperationRunner.java index c48ef85e0e22..8152795869bc 100644 --- a/cdap-source-control/src/main/java/io/cdap/cdap/sourcecontrol/operationrunner/InMemorySourceControlOperationRunner.java +++ b/cdap-source-control/src/main/java/io/cdap/cdap/sourcecontrol/operationrunner/InMemorySourceControlOperationRunner.java @@ -16,6 +16,7 @@ package io.cdap.cdap.sourcecontrol.operationrunner; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.AbstractIdleService; import com.google.gson.Gson; @@ -29,6 +30,8 @@ import io.cdap.cdap.common.utils.FileUtils; import io.cdap.cdap.proto.ApplicationDetail; import io.cdap.cdap.proto.artifact.AppRequest; +import io.cdap.cdap.proto.id.ApplicationReference; +import io.cdap.cdap.sourcecontrol.ApplicationManager; import io.cdap.cdap.sourcecontrol.AuthenticationConfigException; import io.cdap.cdap.sourcecontrol.CommitMeta; import io.cdap.cdap.sourcecontrol.ConfigFileWriteException; @@ -49,9 +52,13 @@ import java.nio.file.LinkOption; import java.nio.file.Path; import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.LinkedList; import java.util.List; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; +import java.util.function.Supplier; import org.eclipse.jgit.api.errors.GitAPIException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -83,21 +90,56 @@ public PushAppResponse push(PushAppOperationRequest pushRequest) throws NoChange pushRequest.getNamespaceId(), pushRequest.getRepositoryConfig()) ) { - try { - repositoryManager.cloneRemote(); - } catch (GitAPIException | IOException e) { - throw new GitOperationException(String.format("Failed to clone remote repository: %s", - e.getMessage()), e); - } + cloneAndCreateBaseDir(repositoryManager); + ApplicationDetail app = pushRequest.getApp(); + writeAppDetail(repositoryManager, app); - LOG.info("Pushing application configs for : {}", pushRequest.getApp().getName()); + List responses = new ArrayList<>(commitAndPush(repositoryManager, + pushRequest.getCommitDetails(), + ImmutableList.of(new PushFile( + app.getName(), + app.getAppVersion(), + repositoryManager.getFileRelativePath(generateConfigFileName(app.getName())) + ) + ) + )); - //TODO: CDAP-20371, Add retry logic here in case the head at remote moved while we are doing push - return writeAppDetailAndPush( - repositoryManager, - pushRequest.getApp(), - pushRequest.getCommitDetails() - ); + // it should never be empty as in case of any error we will get an exception + return responses.get(0); + } + } + + @Override + public Collection multiPush(MultiPushAppOperationRequest pushRequest, + ApplicationManager appManager) + throws NoChangesToPushException, AuthenticationConfigException { + try ( + RepositoryManager repositoryManager = repoManagerFactory.create( + pushRequest.getNamespaceId(), + pushRequest.getRepositoryConfig()) + ) { + cloneAndCreateBaseDir(repositoryManager); + + LOG.debug("Pushing application configs for {} apps.", pushRequest.getApps().size()); + + List filesToPush = new LinkedList<>(); + + for (String appToPush : pushRequest.getApps()) { + LOG.trace("Pushing application configs for : {}.", appToPush); + ApplicationReference appRef = new ApplicationReference(pushRequest.getNamespaceId(), + appToPush); + try { + ApplicationDetail detail = appManager.get(appRef); + writeAppDetail(repositoryManager, detail); + filesToPush.add(new PushFile(detail.getName(), detail.getAppVersion(), + repositoryManager.getFileRelativePath(generateConfigFileName(detail.getName())))); + } catch (IOException | NotFoundException e) { + throw new SourceControlException( + String.format("Failed to fetch details for app %s", appRef), e); + } + } + + return commitAndPush(repositoryManager, pushRequest.getCommitDetails(), filesToPush); } } @@ -105,7 +147,7 @@ public PushAppResponse push(PushAppOperationRequest pushRequest) throws NoChange public PullAppResponse pull(PullAppOperationRequest pullRequest) throws NotFoundException, AuthenticationConfigException { AtomicReference> response = new AtomicReference<>(); - pull( + multiPull( new MultiPullAppOperationRequest( pullRequest.getRepositoryConfig(), pullRequest.getApp().getNamespaceId(), @@ -118,7 +160,7 @@ public PullAppResponse pull(PullAppOperationRequest pullRequest) } @Override - public void pull(MultiPullAppOperationRequest pullRequest, Consumer> consumer) + public void multiPull(MultiPullAppOperationRequest pullRequest, Consumer> consumer) throws SourceControlAppConfigNotFoundException, AuthenticationConfigException { LOG.info("Cloning remote to pull applications {}", pullRequest.getApps()); @@ -137,7 +179,8 @@ public void pull(MultiPullAppOperationRequest pullRequest, Consumer pullSingle(RepositoryManager repositoryManager, String commitId, - String applicationName) throws SourceControlException, SourceControlAppConfigNotFoundException { + String applicationName) + throws SourceControlException, SourceControlAppConfigNotFoundException { String configFileName = generateConfigFileName(applicationName); Path appRelativePath = repositoryManager.getFileRelativePath(configFileName); Path filePathToRead = validateAppConfigRelativePath(repositoryManager, appRelativePath); @@ -164,20 +207,12 @@ private PullAppResponse pullSingle(RepositoryManager repositoryManager, Strin } } - /** - * Atomic operation of writing application and push, return the push response. - * - * @param repositoryManager {@link RepositoryManager} to conduct git operations - * @param appToPush {@link ApplicationDetail} to push - * @param commitDetails {@link CommitMeta} from user input - * @return {@link PushAppResponse} - * @throws NoChangesToPushException if there's no change between the application in namespace and git repository - * @throws SourceControlException for failures while writing config file or doing git operations - */ - private PushAppResponse writeAppDetailAndPush(RepositoryManager repositoryManager, - ApplicationDetail appToPush, - CommitMeta commitDetails) - throws NoChangesToPushException { + private void cloneAndCreateBaseDir(RepositoryManager repositoryManager) { + try { + repositoryManager.cloneRemote(); + } catch (GitAPIException | IOException e) { + throw new GitOperationException("Failed to clone remote repository.", e); + } try { // Creates the base directory if it does not exist. This method does not throw an exception if the directory // already exists. This is for the case that the repo is new and user configured prefix path. @@ -185,7 +220,17 @@ private PushAppResponse writeAppDetailAndPush(RepositoryManager repositoryManage } catch (IOException e) { throw new SourceControlException("Failed to create repository base directory", e); } + } + /** + * Write {@link ApplicationDetail} to config file in git repository. + * + * @param repositoryManager {@link RepositoryManager} to conduct git operations + * @param appToPush application details to write + * @throws SourceControlException for failures while writing config file or doing git + * operations + */ + private void writeAppDetail(RepositoryManager repositoryManager, ApplicationDetail appToPush) { String configFileName = generateConfigFileName(appToPush.getName()); Path appRelativePath = repositoryManager.getFileRelativePath(configFileName); @@ -207,21 +252,43 @@ private PushAppResponse writeAppDetailAndPush(RepositoryManager repositoryManage ); } - LOG.debug("Wrote application configs for {} in file {}", appToPush.getName(), appRelativePath); + LOG.debug("Wrote application configs for {} in file {}", appToPush.getName(), + appRelativePath); + } + /** + * Atomic operation of writing commit and push, return push responses. + * + * @param repositoryManager {@link RepositoryManager} to conduct git operations + * @param commitMeta {@link CommitMeta} from user input + * @param filesToPush an {@link Iterator} of objects of the generic type T. + * @throws NoChangesToPushException if there's no effective change between applications in + * namespace and git repository + * @throws SourceControlException for failures while writing config file or doing git + * operations + */ + //TODO: CDAP-20371, Add retry logic here in case the head at remote moved while we are doing push + private Collection commitAndPush(RepositoryManager repositoryManager, + CommitMeta commitMeta, List filesToPush) throws NoChangesToPushException { + + // TODO: CDAP-20383, handle NoChangesToPushException + // Define the case that the application to push does not have any changes try { - // TODO: CDAP-20383, handle NoChangesToPushException - // Define the case that the application to push does not have any changes - String gitFileHash = repositoryManager.commitAndPush(commitDetails, appRelativePath); - return new PushAppResponse(appToPush.getName(), appToPush.getAppVersion(), gitFileHash); + return repositoryManager.commitAndPush(commitMeta, filesToPush, + (PushFile pushFile, String hash) -> new PushAppResponse( + pushFile.getAppName(), + pushFile.getAppVersion(), + hash + )); } catch (GitAPIException e) { - throw new GitOperationException(String.format("Failed to push config to git: %s", e.getMessage()), e); + throw new GitOperationException( + String.format("Failed to push configs to git: %s", e.getMessage()), e); } } /** - * Generate config file name from app name. - * Currently, it only adds `.json` as extension with app name being the filename. + * Generate config file name from app name. Currently, it only adds `.json` as extension with app + * name being the filename. * * @param appName Name of the application * @return The file name we want to store application config in @@ -308,5 +375,40 @@ protected void startUp() throws Exception { protected void shutDown() throws Exception { // No-op. } + + /** + * Container class to encapsulate files to push information. + */ + public static class PushFile implements Supplier { + + private final String appName; + private final String appVersion; + private final Path path; + + /** + * Constructs an object encapsulating information for a file to push. + * @param appName name of the application to push + * @param appVersion version of the application to push + * @param path filepath of the application json to push + */ + public PushFile(String appName, String appVersion, Path path) { + this.appName = appName; + this.appVersion = appVersion; + this.path = path; + } + + public String getAppName() { + return appName; + } + + public String getAppVersion() { + return appVersion; + } + + @Override + public Path get() { + return path; + } + } } diff --git a/cdap-source-control/src/main/java/io/cdap/cdap/sourcecontrol/operationrunner/MultiPushAppOperationRequest.java b/cdap-source-control/src/main/java/io/cdap/cdap/sourcecontrol/operationrunner/MultiPushAppOperationRequest.java new file mode 100644 index 000000000000..79b4875a618b --- /dev/null +++ b/cdap-source-control/src/main/java/io/cdap/cdap/sourcecontrol/operationrunner/MultiPushAppOperationRequest.java @@ -0,0 +1,62 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.cdap.sourcecontrol.operationrunner; + +import io.cdap.cdap.proto.id.NamespaceId; +import io.cdap.cdap.proto.sourcecontrol.RepositoryConfig; +import io.cdap.cdap.sourcecontrol.CommitMeta; +import java.util.Set; + +/** + * Information required by {@link SourceControlOperationRunner} to execute the task of + * pushing multiple applications to linked repository. + */ +public class MultiPushAppOperationRequest { + private final NamespaceId namespace; + private final RepositoryConfig repoConfig; + private final Set apps; + private final CommitMeta commitDetails; + + /** + * Default constructor. + */ + public MultiPushAppOperationRequest(NamespaceId namespace, + RepositoryConfig repoConfig, + Set apps, + CommitMeta commitDetails) { + this.namespace = namespace; + this.repoConfig = repoConfig; + this.apps = apps; + this.commitDetails = commitDetails; + } + + public NamespaceId getNamespaceId() { + return namespace; + } + + public RepositoryConfig getRepositoryConfig() { + return repoConfig; + } + + public Set getApps() { + return apps; + } + + public CommitMeta getCommitDetails() { + return commitDetails; + } +} diff --git a/cdap-source-control/src/main/java/io/cdap/cdap/sourcecontrol/operationrunner/RemoteSourceControlOperationRunner.java b/cdap-source-control/src/main/java/io/cdap/cdap/sourcecontrol/operationrunner/RemoteSourceControlOperationRunner.java index 274872560a81..97878b81d9a3 100644 --- a/cdap-source-control/src/main/java/io/cdap/cdap/sourcecontrol/operationrunner/RemoteSourceControlOperationRunner.java +++ b/cdap-source-control/src/main/java/io/cdap/cdap/sourcecontrol/operationrunner/RemoteSourceControlOperationRunner.java @@ -29,6 +29,7 @@ import io.cdap.cdap.common.conf.Constants; import io.cdap.cdap.common.internal.remote.RemoteClientFactory; import io.cdap.cdap.common.internal.remote.RemoteTaskExecutor; +import io.cdap.cdap.sourcecontrol.ApplicationManager; import io.cdap.cdap.sourcecontrol.AuthenticationConfigException; import io.cdap.cdap.sourcecontrol.NoChangesToPushException; import io.cdap.cdap.sourcecontrol.SourceControlAppConfigNotFoundException; @@ -38,6 +39,7 @@ import io.cdap.cdap.sourcecontrol.worker.PushAppTask; import io.cdap.common.http.HttpRequestConfig; import java.nio.charset.StandardCharsets; +import java.util.Collection; import java.util.function.Consumer; import javax.inject.Inject; import org.slf4j.Logger; @@ -97,6 +99,13 @@ public PushAppResponse push(PushAppOperationRequest pushRequest) throws NoChange } } + @Override + public Collection multiPush(MultiPushAppOperationRequest pushRequest, + ApplicationManager appManager) + throws NoChangesToPushException, AuthenticationConfigException { + throw new UnsupportedOperationException("multi push not supported for RemoteSourceControlOperationRunner"); + } + @Override public PullAppResponse pull(PullAppOperationRequest pullRequest) throws NotFoundException, AuthenticationConfigException { @@ -117,7 +126,7 @@ public PullAppResponse pull(PullAppOperationRequest pullRequest) throws NotFo } @Override - public void pull(MultiPullAppOperationRequest pullRequest, Consumer> consumer) + public void multiPull(MultiPullAppOperationRequest pullRequest, Consumer> consumer) throws NotFoundException, AuthenticationConfigException { throw new UnsupportedOperationException("multi pull not supported for RemoteSourceControlOperationRunner"); } diff --git a/cdap-source-control/src/main/java/io/cdap/cdap/sourcecontrol/operationrunner/SourceControlOperationRunner.java b/cdap-source-control/src/main/java/io/cdap/cdap/sourcecontrol/operationrunner/SourceControlOperationRunner.java index 2daeb50c9150..8f332ab1599e 100644 --- a/cdap-source-control/src/main/java/io/cdap/cdap/sourcecontrol/operationrunner/SourceControlOperationRunner.java +++ b/cdap-source-control/src/main/java/io/cdap/cdap/sourcecontrol/operationrunner/SourceControlOperationRunner.java @@ -18,9 +18,11 @@ import com.google.common.util.concurrent.Service; import io.cdap.cdap.common.NotFoundException; +import io.cdap.cdap.sourcecontrol.ApplicationManager; import io.cdap.cdap.sourcecontrol.AuthenticationConfigException; import io.cdap.cdap.sourcecontrol.NoChangesToPushException; import io.cdap.cdap.sourcecontrol.SourceControlException; +import java.util.Collection; import java.util.function.Consumer; /** @@ -40,6 +42,20 @@ public interface SourceControlOperationRunner extends Service { PushAppResponse push(PushAppOperationRequest pushRequest) throws NoChangesToPushException, AuthenticationConfigException; + /** + * Push application configs to remote git repository. + * + * @param pushRequest {@link MultiPushAppOperationRequest} of the applications to be pushed + * @param appManager {@link ApplicationManager} to fetch the app details + * @return file-paths and file-hashes for the updated configs. + * @throws AuthenticationConfigException when there is an error while creating the authentication credentials to + * call remote Git. + * @throws SourceControlException when the push operation fails for any other reason. + */ + Collection multiPush(MultiPushAppOperationRequest pushRequest, + ApplicationManager appManager) + throws NoChangesToPushException, AuthenticationConfigException; + /** * Gets an application spec from a Git repository. * @@ -66,7 +82,7 @@ PullAppResponse pull(PullAppOperationRequest pullRequest) throws NotFoundExce * @throws IllegalArgumentException when the fetched application json or file path is invalid. * @throws SourceControlException when the operation fails for any other reason. */ - void pull(MultiPullAppOperationRequest pullRequest, Consumer> consumer) + void multiPull(MultiPullAppOperationRequest pullRequest, Consumer> consumer) throws NotFoundException, AuthenticationConfigException; /** diff --git a/cdap-source-control/src/test/java/io/cdap/cdap/sourcecontrol/RepositoryManagerTest.java b/cdap-source-control/src/test/java/io/cdap/cdap/sourcecontrol/RepositoryManagerTest.java index 644cee29c745..af12b0a92c79 100644 --- a/cdap-source-control/src/test/java/io/cdap/cdap/sourcecontrol/RepositoryManagerTest.java +++ b/cdap-source-control/src/test/java/io/cdap/cdap/sourcecontrol/RepositoryManagerTest.java @@ -16,6 +16,7 @@ package io.cdap.cdap.sourcecontrol; +import com.google.common.collect.ImmutableSet; import io.cdap.cdap.api.metrics.MetricsCollectionService; import io.cdap.cdap.api.metrics.MetricsContext; import io.cdap.cdap.api.security.store.SecureStore; @@ -39,14 +40,18 @@ import java.nio.file.Path; import java.nio.file.Paths; import java.nio.file.attribute.PosixFilePermissions; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.StreamSupport; import org.eclipse.jgit.api.Git; import org.eclipse.jgit.api.errors.GitAPIException; import org.eclipse.jgit.revwalk.RevCommit; +import org.eclipse.jgit.treewalk.TreeWalk; import org.eclipse.jgit.util.SystemReader; import org.junit.Assert; import org.junit.Before; @@ -359,12 +364,24 @@ public void testCommitAndPushSuccess() throws Exception { manager.cloneRemote(); CommitMeta commitMeta = new CommitMeta("author", "committer", 100, "message"); - Path filePath = manager.getBasePath().resolve("file1"); + Path filePath1 = manager.getBasePath().resolve("file1"); + Path filePath2 = manager.getBasePath().resolve("file2"); String fileContent = "content"; - Files.write(filePath, fileContent.getBytes(StandardCharsets.UTF_8)); - manager.commitAndPush(commitMeta, - manager.getBasePath().relativize(filePath)); + Files.write(filePath1, fileContent.getBytes(StandardCharsets.UTF_8)); + Files.write(filePath2, fileContent.getBytes(StandardCharsets.UTF_8)); + + Set filePaths = ImmutableSet.of( + new PathSupplier(manager.getBasePath().relativize(filePath1)), + new PathSupplier(manager.getBasePath().relativize(filePath2)) + ); + + Collection suppliers = manager.commitAndPush(commitMeta, filePaths, + (path, hash) -> { + path.setHash(hash); + return path; + }); + verifyCommit(suppliers, fileContent, commitMeta); // Verify metrics. Mockito.verify(mockMetricsContext).event( Mockito.eq(SourceControlManagement.CLONE_REPOSITORY_SIZE_BYTES), @@ -375,7 +392,6 @@ public void testCommitAndPushSuccess() throws Exception { Mockito.verify(mockMetricsContext).event( Mockito.eq(SourceControlManagement.COMMIT_PUSH_LATENCY_MILLIS), Mockito.anyLong()); - verifyCommit(filePath, fileContent, commitMeta); } } @@ -398,11 +414,16 @@ public void testCommitAndPushSourceControlFailure() throws Exception { String fileContent = "content"; Files.write(filePath, fileContent.getBytes(StandardCharsets.UTF_8)); - manager.commitAndPush(commitMeta, Paths.get("fileThatDoesNotExist.json")); + manager.commitAndPush( + commitMeta, + ImmutableSet.of(new PathSupplier(Paths.get("file1")), + new PathSupplier(Paths.get("fileThatDoesNotExist.json"))), + (path, hash) -> "" + ); } } - @Test(expected = NoChangesToPushException.class) + @Test public void testCommitAndPushNoChangeSuccess() throws Exception { String pathPrefix = "prefix"; String serverUrl = gitServer.getServerUrl(); @@ -410,7 +431,6 @@ public void testCommitAndPushNoChangeSuccess() throws Exception { Provider.GITHUB) .setLink(serverUrl + "ignored") .setDefaultBranch("develop") - .setPathPrefix(pathPrefix) .setAuth(AUTH_CONFIG) .build(); @@ -419,9 +439,34 @@ public void testCommitAndPushNoChangeSuccess() throws Exception { manager.cloneRemote(); CommitMeta commitMeta = new CommitMeta("author", "committer", 100, "message"); - manager.commitAndPush(commitMeta, Paths.get(pathPrefix)); - verifyNoCommit(); + Path filePath1 = manager.getBasePath().resolve("file1"); + Path filePath2 = manager.getBasePath().resolve("file2"); + String fileContent = "content"; + Files.write(filePath1, fileContent.getBytes(StandardCharsets.UTF_8)); + Files.write(filePath2, fileContent.getBytes(StandardCharsets.UTF_8)); + + Path file2 = Paths.get("file2"); + Path file1 = Paths.get("file1"); + manager.commitAndPush(commitMeta, + ImmutableSet.of(new PathSupplier(file1), new PathSupplier(file2)), + (path, hash) -> "" + ); + + Files.write(filePath1, fileContent.getBytes(StandardCharsets.UTF_8)); + Files.write(filePath2, fileContent.getBytes(StandardCharsets.UTF_8)); + + try { + manager.commitAndPush(commitMeta, + ImmutableSet.of(new PathSupplier(file1), new PathSupplier(file2)), + (path, hash) -> "" + ); + Assert.fail(); + } catch (NoChangesToPushException e) { + // expected + } + + verifyCommitCount(2); } } @@ -445,8 +490,8 @@ public void testCommitAndPushFails() throws Exception { Files.write(filePath, fileContent.getBytes(StandardCharsets.UTF_8)); gitServer.after(); - manager.commitAndPush(commitMeta, - manager.getBasePath().relativize(filePath)); + manager.commitAndPush(commitMeta, ImmutableSet.of(new PathSupplier(manager.getBasePath().relativize(filePath))), + (path, hash) -> ""); } } @@ -466,29 +511,24 @@ public void testDirectorySizeMb() throws IOException { RepositoryManager.calculateDirectorySize(root)); } - private void verifyNoCommit() throws GitAPIException, IOException { + private void verifyCommitCount(int count) throws GitAPIException, IOException { Path tempDirPath = baseTempFolder.newFolder("temp-local-git-verify") .toPath(); Git localGit = getClonedGit(tempDirPath, gitServer); List commits = StreamSupport.stream( localGit.log().all().call().spliterator(), false) .collect(Collectors.toList()); - Assert.assertEquals(1, commits.size()); + Assert.assertEquals(count, commits.size()); } - private void verifyCommit(Path pathFromRepoRoot, String expectedContent, - CommitMeta commitMeta) + private void verifyCommit(Collection pathSuppliers, String expectedContent, CommitMeta commitMeta) throws GitAPIException, IOException { Path tempDirPath = baseTempFolder.newFolder("temp-local-git-verify") .toPath(); Git localGit = getClonedGit(tempDirPath, gitServer); - Path filePath = tempDirPath.resolve(pathFromRepoRoot); - - String actualContent = new String(Files.readAllBytes(filePath), - StandardCharsets.UTF_8); - Assert.assertEquals(expectedContent, actualContent); + // verify commit List commits = StreamSupport.stream(localGit.log().all().call().spliterator(), false) .collect(Collectors.toList()); @@ -501,6 +541,19 @@ private void verifyCommit(Path pathFromRepoRoot, String expectedContent, Assert.assertEquals(commitMeta.getCommitter(), latestCommit.getCommitterIdent().getName()); + for (PathSupplier pathSupplier : pathSuppliers) { + // verify content + Path filePath = tempDirPath.resolve(pathSupplier.get()); + String actualContent = new String(Files.readAllBytes(filePath), + StandardCharsets.UTF_8); + Assert.assertEquals(expectedContent, actualContent); + + // verify hash + String gotHash = TreeWalk.forPath(localGit.getRepository(), pathSupplier.get().toString(), + latestCommit.getTree()).getObjectId(0).getName(); + Assert.assertEquals(pathSupplier.getHash(), gotHash); + } + localGit.close(); DirUtils.deleteDirectoryContents(tempDirPath.toFile()); } @@ -539,7 +592,8 @@ private void commitFileToLocalRepository(Path filePath, String contents, throws IOException, NoChangesToPushException, GitAPIException { Files.write(manager.getRepositoryRoot().resolve(filePath), contents.getBytes(StandardCharsets.UTF_8)); - manager.commitAndPush(commitMeta, filePath); + manager.commitAndPush(commitMeta, ImmutableSet.of(new PathSupplier(filePath)), + (path, hash) -> ""); } private void installHook(RepositoryManager manager) throws IOException { @@ -552,4 +606,27 @@ private void installHook(RepositoryManager manager) throws IOException { Files.write(gitHookPath, "touch hook_executed\nexit 0".getBytes( StandardCharsets.UTF_8)); } + + private static class PathSupplier implements Supplier { + + final Path path; + String hash; + + private PathSupplier(Path path) { + this.path = path; + } + + public void setHash(String hash) { + this.hash = hash; + } + + private String getHash() { + return hash; + } + + @Override + public Path get() { + return path; + } + } } diff --git a/cdap-source-control/src/test/java/io/cdap/cdap/sourcecontrol/operationrunner/InMemorySourceControlOperationRunnerTest.java b/cdap-source-control/src/test/java/io/cdap/cdap/sourcecontrol/operationrunner/InMemorySourceControlOperationRunnerTest.java index 32df67ae538f..2cef65a68b67 100644 --- a/cdap-source-control/src/test/java/io/cdap/cdap/sourcecontrol/operationrunner/InMemorySourceControlOperationRunnerTest.java +++ b/cdap-source-control/src/test/java/io/cdap/cdap/sourcecontrol/operationrunner/InMemorySourceControlOperationRunnerTest.java @@ -16,6 +16,7 @@ package io.cdap.cdap.sourcecontrol.operationrunner; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; import com.google.gson.Gson; import com.google.gson.GsonBuilder; @@ -28,6 +29,7 @@ import io.cdap.cdap.proto.sourcecontrol.PatConfig; import io.cdap.cdap.proto.sourcecontrol.Provider; import io.cdap.cdap.proto.sourcecontrol.RepositoryConfig; +import io.cdap.cdap.sourcecontrol.ApplicationManager; import io.cdap.cdap.sourcecontrol.AuthenticationConfigException; import io.cdap.cdap.sourcecontrol.CommitMeta; import io.cdap.cdap.sourcecontrol.GitOperationException; @@ -43,8 +45,12 @@ import java.nio.file.Path; import java.nio.file.Paths; import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.Comparator; +import java.util.HashSet; import java.util.List; +import java.util.Set; import java.util.stream.Collectors; import org.junit.Assert; import org.junit.Before; @@ -52,10 +58,11 @@ import org.mockito.Mockito; public class InMemorySourceControlOperationRunnerTest extends SourceControlTestBase { + private static final String FAKE_COMMIT_HASH = "5905258bb958ceda80b6a37938050ad876920f10"; private static final ApplicationDetail testAppDetails = new ApplicationDetail( - TEST_APP_NAME, "v1", "description1", null, null, "conf1", new ArrayList<>(), - new ArrayList<>(), new ArrayList<>(), null, null); + TEST_APP_NAME, "v1", "description1", null, null, "conf1", new ArrayList<>(), + new ArrayList<>(), new ArrayList<>(), null, null); private static final ApplicationDetail testApp2Details = new ApplicationDetail( TEST_APP2_NAME, "v1", "description1", null, null, "conf1", new ArrayList<>(), new ArrayList<>(), new ArrayList<>(), null, null); @@ -67,31 +74,36 @@ public class InMemorySourceControlOperationRunnerTest extends SourceControlTestB .setAuth(new AuthConfig(AuthType.PAT, new PatConfig("GITHUB_TOKEN_NAME", null))) .build(); private static final CommitMeta testCommit = - new CommitMeta("author1", "committer1", 123, "message1"); + new CommitMeta("author1", "committer1", 123, "message1"); private static final Gson GSON = new GsonBuilder().setPrettyPrinting().create(); private static final NamespaceId NAMESPACE = NamespaceId.DEFAULT; private static final PushAppOperationRequest pushContext = - new PushAppOperationRequest(NAMESPACE, testRepoConfig, testAppDetails, testCommit); - private static final NamespaceRepository nameSpaceRepository = new NamespaceRepository(NAMESPACE, testRepoConfig); + new PushAppOperationRequest(NAMESPACE, testRepoConfig, testAppDetails, testCommit); + private static final NamespaceRepository nameSpaceRepository = new NamespaceRepository(NAMESPACE, + testRepoConfig); private InMemorySourceControlOperationRunner operationRunner; private RepositoryManager mockRepositoryManager; @Before public void setUp() throws Exception { - RepositoryManagerFactory mockRepositoryManagerFactory = Mockito.mock(RepositoryManagerFactory.class); + RepositoryManagerFactory mockRepositoryManagerFactory = Mockito.mock( + RepositoryManagerFactory.class); this.mockRepositoryManager = Mockito.mock(RepositoryManager.class); - Mockito.doReturn(mockRepositoryManager).when(mockRepositoryManagerFactory).create(Mockito.any(), Mockito.any()); + Mockito.doReturn(mockRepositoryManager).when(mockRepositoryManagerFactory) + .create(Mockito.any(), Mockito.any()); Mockito.doReturn(FAKE_COMMIT_HASH).when(mockRepositoryManager).cloneRemote(); this.operationRunner = new InMemorySourceControlOperationRunner(mockRepositoryManagerFactory); Path appRelativePath = Paths.get(PATH_PREFIX, testAppDetails.getName() + ".json"); - Mockito.doReturn(appRelativePath).when(mockRepositoryManager).getFileRelativePath(Mockito.any()); + Mockito.doReturn(appRelativePath).when(mockRepositoryManager) + .getFileRelativePath(Mockito.any()); } - private boolean verifyConfigFileContent(Path repoDirPath) throws IOException { + private boolean verifyConfigFileContent(Path repoDirPath, ApplicationDetail detail) + throws IOException { String fileData = new String(Files.readAllBytes( - repoDirPath.resolve(String.format("%s.json", testAppDetails.getName()))), StandardCharsets.UTF_8); - return fileData.equals(GSON.toJson(testAppDetails)); + repoDirPath.resolve(String.format("%s.json", detail.getName()))), StandardCharsets.UTF_8); + return fileData.equals(GSON.toJson(detail)); } @Test @@ -102,12 +114,53 @@ public void testPushSuccess() throws Exception { Mockito.doReturn(tmpRepoDirPath).when(mockRepositoryManager).getRepositoryRoot(); Mockito.doReturn(baseRepoDirPath).when(mockRepositoryManager).getBasePath(); - Mockito.doReturn("file Hash").when(mockRepositoryManager).commitAndPush(Mockito.anyObject(), - Mockito.any()); + Mockito.doReturn( + ImmutableSet.of( + new PushAppResponse(pushContext.getApp().getName(), + pushContext.getApp().getAppVersion(), "file-hash") + )) + .when(mockRepositoryManager).commitAndPush(Mockito.anyObject(), + Mockito.any(), Mockito.any()); operationRunner.push(pushContext); - Assert.assertTrue(verifyConfigFileContent(baseRepoDirPath)); + Assert.assertTrue(verifyConfigFileContent(baseRepoDirPath, testAppDetails)); + } + + @Test + public void testMultiPushSuccess() throws Exception { + setupPushTest(); + Path tmpRepoDirPath = baseTempFolder.newFolder().toPath(); + Path baseRepoDirPath = tmpRepoDirPath.resolve(PATH_PREFIX); + + ApplicationManager mockManager = Mockito.mock(ApplicationManager.class); + Mockito.doReturn(testAppDetails).when(mockManager) + .get(new ApplicationReference(NAMESPACE, TEST_APP_NAME)); + Mockito.doReturn(testApp2Details).when(mockManager) + .get(new ApplicationReference(NAMESPACE, TEST_APP2_NAME)); + + Mockito.doReturn(tmpRepoDirPath).when(mockRepositoryManager).getRepositoryRoot(); + Mockito.doReturn(baseRepoDirPath).when(mockRepositoryManager).getBasePath(); + Set expectedResponses = ImmutableSet.of( + new PushAppResponse(testAppDetails.getName(), testAppDetails.getAppVersion(), "file-hash1"), + new PushAppResponse(testApp2Details.getName(), testApp2Details.getAppVersion(), + "file-hash2") + ); + + Mockito.doReturn(expectedResponses) + .when(mockRepositoryManager) + .commitAndPush(Mockito.anyObject(), Mockito.any(), Mockito.any()); + + MultiPushAppOperationRequest request = + new MultiPushAppOperationRequest(NAMESPACE, testRepoConfig, + ImmutableSet.of(TEST_APP_NAME, TEST_APP2_NAME), testCommit); + + Collection gotResponses = operationRunner.multiPush(request, mockManager); + + Assert.assertEquals(expectedResponses, new HashSet<>(gotResponses)); + + Assert.assertTrue(verifyConfigFileContent(baseRepoDirPath, testAppDetails)); + Assert.assertTrue(verifyConfigFileContent(baseRepoDirPath, testApp2Details)); } @Test(expected = SourceControlException.class) @@ -145,8 +198,9 @@ public void testPushFailedInvalidSymlinkPath() throws Exception { Mockito.doReturn(tmpRepoDirPath).when(mockRepositoryManager).getRepositoryRoot(); Mockito.doReturn(baseRepoDirPath).when(mockRepositoryManager).getBasePath(); - Mockito.doReturn("file Hash").when(mockRepositoryManager).commitAndPush(Mockito.anyObject(), - Mockito.any()); + Mockito.doReturn(Collections.emptyList()) + .when(mockRepositoryManager).commitAndPush(Mockito.anyObject(), + Mockito.any(), Mockito.any()); Path target = tmpRepoDirPath.resolve("target"); Files.createDirectories(baseRepoDirPath); @@ -157,7 +211,8 @@ public void testPushFailedInvalidSymlinkPath() throws Exception { @Test(expected = AuthenticationConfigException.class) public void testPushFailedToClone() throws Exception { - Mockito.doThrow(new AuthenticationConfigException("config not exists")).when(mockRepositoryManager).cloneRemote(); + Mockito.doThrow(new AuthenticationConfigException("config not exists")) + .when(mockRepositoryManager).cloneRemote(); operationRunner.push(pushContext); } @@ -170,7 +225,7 @@ public void testPushNoChanges() throws Exception { Mockito.doReturn(tmpRepoDirPath).when(mockRepositoryManager).getRepositoryRoot(); Mockito.doReturn(baseRepoDirPath).when(mockRepositoryManager).getBasePath(); Mockito.doThrow(new NoChangesToPushException("no changes to push")) - .when(mockRepositoryManager).commitAndPush(Mockito.any(), Mockito.any()); + .when(mockRepositoryManager).commitAndPush(Mockito.any(), Mockito.any(), Mockito.any()); operationRunner.push(pushContext); } @@ -196,18 +251,18 @@ public void testListSuccess() throws Exception { Mockito.doReturn(FAKE_COMMIT_HASH).when(mockRepositoryManager).cloneRemote(); Mockito.doReturn(tmpRepoDirPath).when(mockRepositoryManager).getBasePath(); Mockito.doReturn(file1.getFileName()).when(mockRepositoryManager) - .getFileRelativePath(file1.getFileName().toString()); + .getFileRelativePath(file1.getFileName().toString()); Mockito.doReturn(app1.getFileHash()).when(mockRepositoryManager) - .getFileHash(file1.getFileName(), FAKE_COMMIT_HASH); + .getFileHash(file1.getFileName(), FAKE_COMMIT_HASH); Mockito.doReturn(file2.getFileName()).when(mockRepositoryManager) - .getFileRelativePath(file2.getFileName().toString()); + .getFileRelativePath(file2.getFileName().toString()); Mockito.doReturn(app2.getFileHash()).when(mockRepositoryManager) - .getFileHash(file2.getFileName(), FAKE_COMMIT_HASH); + .getFileHash(file2.getFileName(), FAKE_COMMIT_HASH); Mockito.doNothing().when(mockRepositoryManager).close(); List sortedListedApps = - operationRunner.list(nameSpaceRepository).getApps().stream() - .sorted(Comparator.comparing(RepositoryApp::getName)).collect(Collectors.toList()); + operationRunner.list(nameSpaceRepository).getApps().stream() + .sorted(Comparator.comparing(RepositoryApp::getName)).collect(Collectors.toList()); Assert.assertEquals(2, sortedListedApps.size()); Assert.assertEquals(sortedListedApps.get(0), app1); @@ -241,8 +296,10 @@ public void testListFailedToGetHash() throws Exception { Mockito.doReturn(tmpRepoDirPath).when(mockRepositoryManager).getBasePath(); Path file1 = tmpRepoDirPath.resolve("file1.some.json"); Files.write(file1, new byte[]{}); - Mockito.doReturn(file1.getFileName()).when(mockRepositoryManager).getFileRelativePath(Mockito.any()); - Mockito.doThrow(new IOException()).when(mockRepositoryManager).getFileHash(file1.getFileName(), FAKE_COMMIT_HASH); + Mockito.doReturn(file1.getFileName()).when(mockRepositoryManager) + .getFileRelativePath(Mockito.any()); + Mockito.doThrow(new IOException()).when(mockRepositoryManager) + .getFileHash(file1.getFileName(), FAKE_COMMIT_HASH); operationRunner.list(nameSpaceRepository); } @@ -251,7 +308,8 @@ public void testListFailedToGetHash() throws Exception { public void testPullSuccess() throws Exception { setupPullTest(); ApplicationReference appRef = new ApplicationReference(NAMESPACE, TEST_APP_NAME); - PullAppResponse response = operationRunner.pull(new PullAppOperationRequest(appRef, testRepoConfig)); + PullAppResponse response = operationRunner.pull( + new PullAppOperationRequest(appRef, testRepoConfig)); validatePullResponse(response, TEST_APP_NAME); Mockito.verify(mockRepositoryManager, Mockito.times(1)).cloneRemote(); Mockito.verify(mockRepositoryManager, Mockito.times(1)).close(); @@ -261,8 +319,9 @@ public void testPullSuccess() throws Exception { public void testMultiPullSuccess() throws Exception { setupPullTest(); List> responses = new ArrayList<>(); - operationRunner.pull( - new MultiPullAppOperationRequest(testRepoConfig, NAMESPACE, Sets.newHashSet(TEST_APP_NAME, TEST_APP2_NAME)), + operationRunner.multiPull( + new MultiPullAppOperationRequest(testRepoConfig, NAMESPACE, + Sets.newHashSet(TEST_APP_NAME, TEST_APP2_NAME)), responses::add ); Assert.assertEquals(responses.size(), 2); @@ -281,8 +340,8 @@ public void testPullFailedToReadHash() throws Exception { setupPullTest(); ApplicationReference appRef = new ApplicationReference(NAMESPACE, TEST_APP_NAME); Mockito.doThrow(new NotFoundException("object not found")) - .when(mockRepositoryManager) - .getFileHash(Mockito.any(Path.class), Mockito.any(String.class)); + .when(mockRepositoryManager) + .getFileHash(Mockito.any(Path.class), Mockito.any(String.class)); try { operationRunner.pull(new PullAppOperationRequest(appRef, testRepoConfig)); } finally { @@ -296,7 +355,7 @@ public void testPullFileNotFound() throws Exception { setupPullTest(); ApplicationReference appRef = new ApplicationReference(NAMESPACE, TEST_APP_NAME); Mockito.doReturn(Paths.get(PATH_PREFIX, "notpresent.json")) - .when(mockRepositoryManager).getFileRelativePath(Mockito.any()); + .when(mockRepositoryManager).getFileRelativePath(Mockito.any()); try { operationRunner.pull(new PullAppOperationRequest(appRef, testRepoConfig)); } finally { @@ -309,7 +368,8 @@ public void testPullFileNotFound() throws Exception { public void testPullCloneFailure() throws Exception { setupPullTest(); ApplicationReference appRef = new ApplicationReference(NAMESPACE, TEST_APP_NAME); - Mockito.doThrow(new IOException("secure store failure")).when(mockRepositoryManager).cloneRemote(); + Mockito.doThrow(new IOException("secure store failure")).when(mockRepositoryManager) + .cloneRemote(); try { operationRunner.pull(new PullAppOperationRequest(appRef, testRepoConfig)); } finally { @@ -319,8 +379,12 @@ public void testPullCloneFailure() throws Exception { } private void setupPushTest() { - Path appRelativePath = Paths.get(PATH_PREFIX, testAppDetails.getName() + ".json"); - Mockito.doReturn(appRelativePath).when(mockRepositoryManager).getFileRelativePath(Mockito.any()); + String fileName1 = testAppDetails.getName() + ".json"; + String fileName2 = testApp2Details.getName() + ".json"; + Mockito.doReturn(Paths.get(PATH_PREFIX, fileName1)).when(mockRepositoryManager) + .getFileRelativePath(fileName1); + Mockito.doReturn(Paths.get(PATH_PREFIX, fileName2)).when(mockRepositoryManager) + .getFileRelativePath(fileName2); } private void setupPullTest() throws Exception { @@ -330,14 +394,18 @@ private void setupPullTest() throws Exception { Mockito.doReturn(baseRepoDirPath).when(mockRepositoryManager).getBasePath(); Mockito.doReturn(FAKE_COMMIT_HASH).when(mockRepositoryManager).cloneRemote(); Mockito.doReturn(TEST_FILE_HASH) - .when(mockRepositoryManager) - .getFileHash(Mockito.eq(Paths.get(PATH_PREFIX, TEST_APP_NAME + ".json")), Mockito.eq(FAKE_COMMIT_HASH)); + .when(mockRepositoryManager) + .getFileHash(Mockito.eq(Paths.get(PATH_PREFIX, TEST_APP_NAME + ".json")), + Mockito.eq(FAKE_COMMIT_HASH)); Mockito.doReturn(TEST_FILE_HASH) .when(mockRepositoryManager) - .getFileHash(Mockito.eq(Paths.get(PATH_PREFIX, TEST_APP2_NAME + ".json")), Mockito.eq(FAKE_COMMIT_HASH)); + .getFileHash(Mockito.eq(Paths.get(PATH_PREFIX, TEST_APP2_NAME + ".json")), + Mockito.eq(FAKE_COMMIT_HASH)); Files.createDirectories(baseRepoDirPath); - Files.write(baseRepoDirPath.resolve(TEST_APP_NAME + ".json"), TEST_APP_SPEC.getBytes(StandardCharsets.UTF_8)); - Files.write(baseRepoDirPath.resolve(TEST_APP2_NAME + ".json"), TEST_APP_SPEC.getBytes(StandardCharsets.UTF_8)); + Files.write(baseRepoDirPath.resolve(TEST_APP_NAME + ".json"), + TEST_APP_SPEC.getBytes(StandardCharsets.UTF_8)); + Files.write(baseRepoDirPath.resolve(TEST_APP2_NAME + ".json"), + TEST_APP_SPEC.getBytes(StandardCharsets.UTF_8)); Mockito.doNothing().when(mockRepositoryManager).close(); Path appRelativePath = Paths.get(PATH_PREFIX, testAppDetails.getName() + ".json"); Path app2RelativePath = Paths.get(PATH_PREFIX, testApp2Details.getName() + ".json"); diff --git a/cdap-unit-test/src/main/java/io/cdap/cdap/test/TestBase.java b/cdap-unit-test/src/main/java/io/cdap/cdap/test/TestBase.java index de95b0386e0f..8b74f96b2722 100644 --- a/cdap-unit-test/src/main/java/io/cdap/cdap/test/TestBase.java +++ b/cdap-unit-test/src/main/java/io/cdap/cdap/test/TestBase.java @@ -94,10 +94,10 @@ import io.cdap.cdap.internal.provision.MockProvisionerModule; import io.cdap.cdap.logging.guice.LocalLogAppenderModule; import io.cdap.cdap.logging.guice.LogReaderRuntimeModules; -import io.cdap.cdap.messaging.spi.MessagingService; import io.cdap.cdap.messaging.context.BasicMessagingAdmin; import io.cdap.cdap.messaging.context.MultiThreadMessagingContext; import io.cdap.cdap.messaging.guice.MessagingServerRuntimeModule; +import io.cdap.cdap.messaging.spi.MessagingService; import io.cdap.cdap.metadata.FieldLineageAdmin; import io.cdap.cdap.metadata.LineageAdmin; import io.cdap.cdap.metadata.MetadataAdmin;