Skip to content

Commit

Permalink
Removed errors field
Browse files Browse the repository at this point in the history
  • Loading branch information
arkadius committed Feb 18, 2025
1 parent acfa370 commit 5a90c77
Show file tree
Hide file tree
Showing 18 changed files with 118 additions and 100 deletions.
22 changes: 0 additions & 22 deletions designer/client/src/components/Process/ProcessErrors.tsx

This file was deleted.

2 changes: 0 additions & 2 deletions designer/client/src/components/Process/ProcessStateIcon.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,6 @@ import { ProcessStateType, Scenario } from "./types";
import ProcessStateUtils from "./ProcessStateUtils";
import UrlIcon from "../UrlIcon";
import { Box, Divider, Popover, styled, Typography } from "@mui/material";
import { Errors } from "./ProcessErrors";

const StyledUrlIcon = styled(UrlIcon)(({ theme }) => ({
width: theme.spacing(2.5),
Expand Down Expand Up @@ -44,7 +43,6 @@ function ProcessStateIcon({ scenario, processState }: Props) {
<Typography variant="body2" style={{ whiteSpace: "pre-wrap" }}>
{tooltip}
</Typography>
<Errors state={processState} />
</Box>
</Popover>
</>
Expand Down
3 changes: 1 addition & 2 deletions designer/client/src/components/Process/types.ts
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/* eslint-disable i18next/no-literal-string */
import { UnknownRecord, Instant } from "../../types/common";
import { Instant } from "../../types/common";
import { ScenarioGraph, ValidationResult } from "../../types";
import { ProcessingMode } from "../../http/HttpService";

Expand Down Expand Up @@ -67,7 +67,6 @@ export type ProcessStateType = {
icon: string;
tooltip: string;
description: string;
errors?: Array<string>;
};

export type StatusType = {
Expand Down
1 change: 0 additions & 1 deletion designer/client/src/reducers/graph/utils.fixtures.ts
Original file line number Diff line number Diff line change
Expand Up @@ -171,7 +171,6 @@ export const state: GraphState = {
icon: "/assets/states/not-deployed.svg",
tooltip: "The scenario is not deployed.",
description: "The scenario is not deployed.",
errors: [],
},
validationResult: {
errors: {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,13 +5,12 @@ import pl.touk.nussknacker.engine.deployment.{DeploymentId, ExternalDeploymentId

case class StatusDetails(
status: StateStatus,
// FIXME abr: non optional
deploymentId: Option[DeploymentId],
// TODO: remove it after periodic mechanism will use UUID for DeploymentId
externalDeploymentId: Option[ExternalDeploymentId] = None,
version: Option[ProcessVersion] = None,
startTime: Option[Long] = None,
// FIXME abr: verify usage
errors: List[String] = List.empty
) {

def deploymentIdUnsafe: DeploymentId =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,15 +26,16 @@ object SimpleProcessStateDefinitionManager extends ProcessStateDefinitionManager
)

private[nussknacker] def statusDescription(status: StateStatus): String = status match {
case _ @ProblemStateStatus(message, _) => message
case _ => SimpleStateStatus.definitions(status.name).description
case _ @ProblemStateStatus(message, _, _) => message
case _ => SimpleStateStatus.definitions(status.name).description
}

override def statusTooltip(input: ScenarioStatusWithScenarioContext): String = statusTooltip(input.scenarioStatus)

private[nussknacker] def statusTooltip(status: StateStatus): String = status match {
case _ @ProblemStateStatus(message, _) => message
case _ => SimpleStateStatus.definitions(status.name).tooltip
case _ @ProblemStateStatus(message, _, Some(tooltip)) => tooltip
case _ @ProblemStateStatus(message, _, _) => message
case _ => SimpleStateStatus.definitions(status.name).tooltip
}

override def stateDefinitions: Map[StatusName, StateDefinitionDetails] =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ import pl.touk.nussknacker.engine.api.deployment.StateStatus.StatusName
import pl.touk.nussknacker.engine.api.deployment._
import pl.touk.nussknacker.engine.api.deployment.simple.SimpleStateStatus.ProblemStateStatus.defaultActions
import pl.touk.nussknacker.engine.api.process.VersionId
import pl.touk.nussknacker.engine.deployment.DeploymentId

import java.net.URI

Expand All @@ -19,8 +20,11 @@ object SimpleStateStatus {
}

// Represents general problem.
final case class ProblemStateStatus(description: String, allowedActions: Set[ScenarioActionName] = defaultActions)
extends StateStatus {
final case class ProblemStateStatus(
description: String,
allowedActions: Set[ScenarioActionName] = defaultActions,
tooltip: Option[String] = None
) extends StateStatus {
override def name: StatusName = ProblemStateStatus.name
}

Expand Down Expand Up @@ -66,8 +70,18 @@ object SimpleStateStatus {
def missingDeployedVersion(exceptedVersionId: VersionId, user: String): ProblemStateStatus =
ProblemStateStatus(s"Scenario deployed without version by $user, expected version $exceptedVersionId.")

val MultipleJobsRunning: ProblemStateStatus =
ProblemStateStatus("More than one deployment is running.", Set(ScenarioActionName.Cancel))
def multipleJobsRunning(nonFinalDeploymentIds: List[(DeploymentId, StateStatus)]): ProblemStateStatus =
ProblemStateStatus(
description = "More than one deployment is running.",
allowedActions = Set(ScenarioActionName.Cancel),
tooltip = Some(
nonFinalDeploymentIds
.map { case (deploymentId, deploymentStatus) =>
deploymentId + " - " + deploymentStatus
}
.mkString("Expected one job, instead: ", ", ", "")
)
)

}

Expand Down Expand Up @@ -105,7 +119,7 @@ object SimpleStateStatus {
Set(ScenarioActionName.Deploy, ScenarioActionName.Cancel)
// When Failed - process is in terminal state in Flink and it doesn't require any cleanup in Flink, but in NK it does
// - that's why Cancel action is available
case SimpleStateStatus.ProblemStateStatus(_, allowedActions) => allowedActions
case SimpleStateStatus.ProblemStateStatus(_, allowedActions, _) => allowedActions
}

val definitions: Map[StatusName, StateDefinitionDetails] = Map(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@ import java.net.URI
* - status itself and its evaluation moment: status, startTime
* - how to display in UI: icon, tooltip, description
* - which actions are allowed: allowedActions
* - additional properties: errors
*
* Statuses definition, allowed actions and current scenario presentation is defined by [[ProcessStateDefinitionManager]].
* @param description Short message displayed in top right panel of scenario diagram panel.
Expand All @@ -30,7 +29,6 @@ import java.net.URI
icon: URI,
tooltip: String,
description: String,
errors: List[String]
)

@JsonCodec case class LegacyScenarioStatusNameDto(name: String)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -173,7 +173,7 @@ class AppApiHttpService(
)
statusMap = processes.flatMap(process => process.state.map(process.name -> _)).toMap
withProblem = statusMap.collect {
case (name, processStatus @ ScenarioStatusDto(ProblemStateStatus.name, _, _, _, _, _, _, _, _)) =>
case (name, processStatus @ ScenarioStatusDto(ProblemStateStatus.name, _, _, _, _, _, _, _)) =>
(name, processStatus)
}
} yield withProblem
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@ class ScenarioStatusPresenter(dispatcher: DeploymentManagerDispatcher) {
icon = presentation.icon,
tooltip = presentation.tooltip,
description = presentation.description,
errors = statusDetails.errors,
)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,10 +46,11 @@ class InconsistentStateDetector extends LazyLogging {
case (_, firstNotFinished :: _ :: _) =>
Left(
firstNotFinished.copy(
status = ProblemStateStatus.MultipleJobsRunning,
errors = List(s"Expected one job, instead: ${notFinalStatuses
.map(details => details.deploymentId.map(_.value).getOrElse("missing") + " - " + details.status)
.mkString(", ")}")
status = ProblemStateStatus.multipleJobsRunning(
notFinalStatuses.map(deploymentStatus =>
deploymentStatus.deploymentId.getOrElse(DeploymentId("missing")) -> deploymentStatus.status
)
)
)
)
case (firstFinished :: _, Nil) => Right(Some(firstFinished))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ class InconsistentStateDetectorTest extends AnyFunSuiteLike with Matchers {

InconsistentStateDetector.extractAtMostOneStatus(List(firstDeploymentStatus, secondDeploymentStatus)) shouldBe Some(
StatusDetails(
ProblemStateStatus.MultipleJobsRunning,
ProblemStateStatus.multipleJobsRunning,
firstDeploymentStatus.deploymentId,
errors = List(
s"Expected one job, instead: ${firstDeploymentStatus.deploymentIdUnsafe} - RUNNING, ${secondDeploymentStatus.deploymentIdUnsafe} - RUNNING"
Expand All @@ -34,7 +34,7 @@ class InconsistentStateDetectorTest extends AnyFunSuiteLike with Matchers {

InconsistentStateDetector.extractAtMostOneStatus(List(firstDeploymentStatus, secondDeploymentStatus)) shouldBe Some(
StatusDetails(
ProblemStateStatus.MultipleJobsRunning,
ProblemStateStatus.multipleJobsRunning,
firstDeploymentStatus.deploymentId,
errors = List(
s"Expected one job, instead: ${firstDeploymentStatus.deploymentIdUnsafe} - RUNNING, ${secondDeploymentStatus.deploymentIdUnsafe} - RESTARTING"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -204,7 +204,7 @@ class FlinkDeploymentManager(
implicit val freshnessPolicy: DataFreshnessPolicy = DataFreshnessPolicy.Fresh
getScenarioDeploymentsStatuses(processName).flatMap { statuses =>
val runningDeploymentIds = statuses.value.filter(statusDetailsPredicate).collect {
case StatusDetails(SimpleStateStatus.Running, _, Some(deploymentId), _, _, _) => deploymentId
case StatusDetails(SimpleStateStatus.Running, _, Some(deploymentId), _, _) => deploymentId
}
runningDeploymentIds match {
case Nil =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@ class FlinkStatusDetailsDeterminer(
Some(ExternalDeploymentId(job.jid)),
version = Some(jobConfig.version),
startTime = Some(job.`start-time`),
errors = List.empty
)
} getOrElse {
logger.debug(
Expand All @@ -48,7 +47,6 @@ class FlinkStatusDetailsDeterminer(
Some(ExternalDeploymentId(job.jid)),
version = None,
startTime = Some(job.`start-time`),
errors = List.empty
)
}
name -> details
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -146,7 +146,7 @@ class StreamingEmbeddedDeploymentManagerTest
val FixtureParam(manager, _, _, _) = prepareFixture(inputTopic, outputTopic, List(deployedScenarioData))

manager.getScenarioDeploymentsStatuses(name).futureValue.value.map(_.status) should matchPattern {
case ProblemStateStatus("Scenario compilation errors", _) :: Nil =>
case ProblemStateStatus("Scenario compilation errors", _, _) :: Nil =>
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -318,14 +318,13 @@ class K8sDeploymentManager(
override def getScenarioDeploymentsStatuses(
scenarioName: ProcessName
)(implicit freshnessPolicy: DataFreshnessPolicy): Future[WithDataFreshnessStatus[List[StatusDetails]]] = {
val mapper = new K8sDeploymentStatusMapper(processStateDefinitionManager)
for {
deployments <- scenarioStateK8sClient
.listSelected[ListResource[Deployment]](requirementForName(scenarioName))
.map(_.items)
pods <- scenarioStateK8sClient.listSelected[ListResource[Pod]](requirementForName(scenarioName)).map(_.items)
} yield {
WithDataFreshnessStatus.fresh(deployments.map(mapper.status(_, pods)))
WithDataFreshnessStatus.fresh(deployments.map(K8sDeploymentStatusMapper.status(_, pods)))
}
}

Expand Down
Original file line number Diff line number Diff line change
@@ -1,16 +1,16 @@
package pl.touk.nussknacker.k8s.manager

import cats.data.NonEmptyList
import com.typesafe.scalalogging.LazyLogging
import io.circe.Json
import pl.touk.nussknacker.engine.api.deployment.simple.SimpleStateStatus
import pl.touk.nussknacker.engine.api.deployment.simple.SimpleStateStatus.ProblemStateStatus
import pl.touk.nussknacker.engine.api.deployment.{ProcessStateDefinitionManager, StateStatus, StatusDetails}
import pl.touk.nussknacker.engine.api.deployment.{ScenarioActionName, StateStatus, StatusDetails}
import pl.touk.nussknacker.k8s.manager.K8sDeploymentManager.parseVersionAnnotation
import pl.touk.nussknacker.k8s.manager.K8sDeploymentStatusMapper._
import skuber.apps.v1.Deployment
import skuber.{Container, Pod}

object K8sDeploymentStatusMapper {
//Based on https://kubernetes.io/docs/concepts/workloads/controllers/deployment/#deployment-status
object K8sDeploymentStatusMapper extends LazyLogging {

private val availableCondition = "Available"

Expand All @@ -23,10 +23,6 @@ object K8sDeploymentStatusMapper {
private val crashLoopBackOffReason = "CrashLoopBackOff"

private val newReplicaSetAvailable = "NewReplicaSetAvailable"
}

//Based on https://kubernetes.io/docs/concepts/workloads/controllers/deployment/#deployment-status
class K8sDeploymentStatusMapper(definitionManager: ProcessStateDefinitionManager) extends LazyLogging {

private[manager] def findStatusForDeploymentsAndPods(
deployments: List[Deployment],
Expand All @@ -38,17 +34,20 @@ class K8sDeploymentStatusMapper(definitionManager: ProcessStateDefinitionManager
case duplicates =>
Some(
StatusDetails(
ProblemStateStatus.MultipleJobsRunning,
ProblemStateStatus(
description = "More than one deployment is running.",
allowedActions = Set(ScenarioActionName.Cancel),
tooltip = Some(s"Expected one deployment, instead: ${duplicates.map(_.metadata.name).mkString(", ")}")
),
None,
errors = List(s"Expected one deployment, instead: ${duplicates.map(_.metadata.name).mkString(", ")}")
)
)
}
}

private[manager] def status(deployment: Deployment, pods: List[Pod]): StatusDetails = {
val (status, attrs, errors) = deployment.status match {
case None => (SimpleStateStatus.DuringDeploy, None, Nil)
val status = deployment.status match {
case None => SimpleStateStatus.DuringDeploy
case Some(status) => mapStatusWithPods(status, pods)
}
val startTime = deployment.metadata.creationTimestamp.map(_.toInstant.toEpochMilli)
Expand All @@ -59,32 +58,41 @@ class K8sDeploymentStatusMapper(definitionManager: ProcessStateDefinitionManager
None,
parseVersionAnnotation(deployment),
startTime,
errors
)
}

// TODO: should we add responses to status attributes?
private[manager] def mapStatusWithPods(
status: Deployment.Status,
pods: List[Pod]
): (StateStatus, Option[Json], List[String]) = {
): StateStatus = {
def condition(name: String): Option[Deployment.Condition] = status.conditions.find(cd => cd.`type` == name)
def anyContainerInState(state: Container.State) =
pods.flatMap(_.status.toList).flatMap(_.containerStatuses).exists(_.state.exists(_ == state))

(condition(availableCondition), condition(progressingCondition), condition(replicaFailureCondition)) match {
case (Some(available), None | ProgressingNewReplicaSetAvailable(), _) if isTrue(available) =>
(SimpleStateStatus.Running, None, Nil)
SimpleStateStatus.Running
case (_, Some(progressing), _)
if isTrue(progressing) && anyContainerInState(Container.Waiting(Some(crashLoopBackOffReason))) =>
logger.debug(
s"Some containers are in waiting state with CrashLoopBackOff reason - returning Restarting status. Pods: $pods"
)
(SimpleStateStatus.Restarting, None, Nil)
case (_, Some(progressing), _) if isTrue(progressing) => (SimpleStateStatus.DuringDeploy, None, Nil)
SimpleStateStatus.Restarting
case (_, Some(progressing), _) if isTrue(progressing) =>
SimpleStateStatus.DuringDeploy
case (_, _, Some(replicaFailure)) if isTrue(replicaFailure) =>
(ProblemStateStatus.Failed, None, replicaFailure.message.toList)
case (a, b, _) => (ProblemStateStatus.Failed, None, a.flatMap(_.message).toList ++ b.flatMap(_.message).toList)
ProblemStateStatus(
"There are some problems with scenario.",
tooltip = replicaFailure.message.map("Error: " + _)
)
case (a, b, _) =>
ProblemStateStatus(
"There are some problems with scenario.",
tooltip = NonEmptyList
.fromList(a.flatMap(_.message).toList ++ b.flatMap(_.message).toList)
.map(_.toList.mkString("Errors: ", ", ", ""))
)
}
}

Expand Down
Loading

0 comments on commit 5a90c77

Please sign in to comment.