diff --git a/.github/workflows/pr_check.yml b/.github/workflows/pr_check.yml
index a0063b37..40139e1e 100644
--- a/.github/workflows/pr_check.yml
+++ b/.github/workflows/pr_check.yml
@@ -2,7 +2,7 @@ name: PR check
on:
pull_request:
- branches: [ master ]
+ branches: [master, develop]
jobs:
engine-lint:
diff --git a/README.md b/README.md
index da01162d..9affdb31 100644
--- a/README.md
+++ b/README.md
@@ -44,6 +44,7 @@ bamboo-engine 是一个通用的流程引擎,他可以解析,执行,调度
- [重置某个异常节点的输出](./docs/user_guide/update_node_output.md)
- [设置](./docs/user_guide/settings.md)
- [增强包 - 节点超时功能](./docs/user_guide/node_timeout_introduction.md)
+ - [流程从指定位置开始](./docs/user_guide/start_the_pipeline_at_the_specified_location.md)
## 整体设计
diff --git a/bamboo_engine/__version__.py b/bamboo_engine/__version__.py
index c47db039..aeff28ea 100644
--- a/bamboo_engine/__version__.py
+++ b/bamboo_engine/__version__.py
@@ -11,4 +11,4 @@
specific language governing permissions and limitations under the License.
"""
-__version__ = "2.9.0"
+__version__ = "2.10.0rc4"
diff --git a/bamboo_engine/builder/builder.py b/bamboo_engine/builder/builder.py
index 92a5eb53..30d44041 100644
--- a/bamboo_engine/builder/builder.py
+++ b/bamboo_engine/builder/builder.py
@@ -16,10 +16,10 @@
from bamboo_engine.utils.string import unique_id
+from ..validator.connection import validate_graph_without_circle
from .flow.data import Data, Params
from .flow.event import ExecutableEndEvent
-
__all__ = ["build_tree"]
__skeleton = {
@@ -94,6 +94,185 @@ def build_tree(start_elem, id=None, data=None):
return tree
+def _get_next_node(node, pipeline_tree):
+ """
+ 获取当前节点的下一个节点
+ """
+
+ out_goings = node["outgoing"]
+
+ # 说明曾经去除过环,此时没有out_goings
+ if out_goings == "":
+ return []
+
+ # 当只有一个输出时,
+ if not isinstance(out_goings, list):
+ out_goings = [out_goings]
+
+ next_nodes = []
+ for out_going in out_goings:
+ target_id = pipeline_tree["flows"][out_going]["target"]
+ if target_id in pipeline_tree["activities"]:
+ next_nodes.append(pipeline_tree["activities"][target_id])
+ elif target_id in pipeline_tree["gateways"]:
+ next_nodes.append(pipeline_tree["gateways"][target_id])
+ elif target_id == pipeline_tree["end_event"]["id"]:
+ next_nodes.append(pipeline_tree["end_event"])
+
+ return next_nodes
+
+
+def _get_all_nodes(pipeline_tree: dict, with_subprocess: bool = False) -> dict:
+ """
+ 获取 pipeline_tree 中所有 activity 的信息
+
+ :param pipeline_tree: pipeline web tree
+ :param with_subprocess: 是否是子流程的 tree
+ :return: 包含 pipeline_tree 中所有 activity 的字典(包括子流程的 acitivity)
+ """
+ all_nodes = {}
+ all_nodes.update(pipeline_tree["activities"])
+ all_nodes.update(pipeline_tree["gateways"])
+ all_nodes.update(
+ {
+ pipeline_tree["start_event"]["id"]: pipeline_tree["start_event"],
+ pipeline_tree["end_event"]["id"]: pipeline_tree["end_event"],
+ }
+ )
+ if with_subprocess:
+ for act in pipeline_tree["activities"].values():
+ if act["type"] == "SubProcess":
+ all_nodes.update(_get_all_nodes(act["pipeline"], with_subprocess=True))
+ return all_nodes
+
+
+def _delete_flow_id_from_node_io(node, flow_id, io_type):
+ """
+ 删除节点的某条连线,io_type(incoming or outgoing)
+ """
+ if node[io_type] == flow_id:
+ node[io_type] = ""
+ elif isinstance(node[io_type], list):
+ if len(node[io_type]) == 1 and node[io_type][0] == flow_id:
+ node[io_type] = (
+ "" if node["type"] not in ["ExclusiveGateway", "ParallelGateway", "ConditionalParallelGateway"] else []
+ )
+ else:
+ node[io_type].pop(node[io_type].index(flow_id))
+
+ # recover to original format
+ if (
+ len(node[io_type]) == 1
+ and io_type == "outgoing"
+ and node["type"] in ["EmptyStartEvent", "ServiceActivity", "ConvergeGateway"]
+ ):
+ node[io_type] = node[io_type][0]
+
+
+def _acyclic(pipeline):
+ """
+ @summary: 逆转反向边
+ @return:
+ """
+
+ pipeline["all_nodes"] = _get_all_nodes(pipeline, with_subprocess=True)
+
+ deformed_flows = {
+ "{}.{}".format(flow["source"], flow["target"]): flow_id for flow_id, flow in pipeline["flows"].items()
+ }
+ while True:
+ no_circle = validate_graph_without_circle(pipeline)
+ if no_circle["result"]:
+ break
+ source = no_circle["error_data"][-2]
+ target = no_circle["error_data"][-1]
+ circle_flow_key = "{}.{}".format(source, target)
+ flow_id = deformed_flows[circle_flow_key]
+ pipeline["flows"][flow_id].update({"source": target, "target": source})
+
+ source_node = pipeline["all_nodes"][source]
+ _delete_flow_id_from_node_io(source_node, flow_id, "outgoing")
+
+ target_node = pipeline["all_nodes"][target]
+ _delete_flow_id_from_node_io(target_node, flow_id, "incoming")
+
+
+def generate_pipeline_token(pipeline_tree):
+ tree = copy.deepcopy(pipeline_tree)
+ # 去环
+ _acyclic(tree)
+
+ start_node = tree["start_event"]
+ token = unique_id("t")
+ node_token_map = {start_node["id"]: token}
+ inject_pipeline_token(start_node, tree, node_token_map, token)
+ return node_token_map
+
+
+# 需要处理子流程的问题
+def inject_pipeline_token(node, pipeline_tree, node_token_map, token):
+ # 如果是网关
+ if node["type"] in ["ParallelGateway", "ExclusiveGateway", "ConditionalParallelGateway"]:
+ next_nodes = _get_next_node(node, pipeline_tree)
+ target_nodes = {}
+ for next_node in next_nodes:
+ # 分支网关各个分支token相同
+ node_token = token
+ node_token_map[next_node["id"]] = node_token
+ # 并行网关token不同
+ if node["type"] in ["ParallelGateway", "ConditionalParallelGateway"]:
+ node_token = unique_id("t")
+ node_token_map[next_node["id"]] = node_token
+
+ # 如果是并行网关,沿着路径向内搜索,最终遇到对应的汇聚网关会返回
+ target_node = inject_pipeline_token(next_node, pipeline_tree, node_token_map, node_token)
+ if target_node:
+ target_nodes[target_node["id"]] = target_node
+
+ for target_node in target_nodes.values():
+ # 汇聚网关可以直连结束节点,所以可能会存在找不到对应的汇聚网关的情况
+ if target_node["type"] in ["EmptyEndEvent", "ExecutableEndEvent"]:
+ node_token_map[target_node["id"]] = token
+ continue
+ # 汇聚网关的token等于对应的网关的token
+ node_token_map[target_node["id"]] = token
+ # 到汇聚网关之后,此时继续向下遍历
+ next_node = _get_next_node(target_node, pipeline_tree)[0]
+ # 汇聚网关只会有一个出度
+ node_token_map[next_node["id"]] = token
+ inject_pipeline_token(next_node, pipeline_tree, node_token_map, token)
+
+ # 如果是汇聚网关,并且id等于converge_id,说明此时遍历在某个单元
+ if node["type"] == "ConvergeGateway":
+ return node
+
+ # 如果是普通的节点,说明只有一个出度,此时直接向下遍历就好
+ if node["type"] in ["ServiceActivity", "EmptyStartEvent"]:
+ next_node_list = _get_next_node(node, pipeline_tree)
+ # 此时有可能遇到一个去环的节点,该节点没有
+ if not next_node_list:
+ return
+ next_node = next_node_list[0]
+ node_token_map[next_node["id"]] = token
+ return inject_pipeline_token(next_node, pipeline_tree, node_token_map, token)
+
+ # 如果遇到结束节点,直接返回
+ if node["type"] in ["EmptyEndEvent", "ExecutableEndEvent"]:
+ return node
+
+ if node["type"] == "SubProcess":
+ subprocess_pipeline_tree = node["pipeline"]
+ subprocess_start_node = subprocess_pipeline_tree["start_event"]
+ subprocess_start_node_token = unique_id("t")
+ node_token_map[subprocess_start_node["id"]] = subprocess_start_node_token
+ inject_pipeline_token(
+ subprocess_start_node, subprocess_pipeline_tree, node_token_map, subprocess_start_node_token
+ )
+ next_node = _get_next_node(node, pipeline_tree)[0]
+ node_token_map[next_node["id"]] = token
+ return inject_pipeline_token(next_node, pipeline_tree, node_token_map, token)
+
+
def __update(tree, elem):
node_type = __node_type[elem.type()]
node = tree[node_type] if node_type == "end_event" else tree[node_type][elem.id]
diff --git a/bamboo_engine/config.py b/bamboo_engine/config.py
index bebf2d21..f4a02cc5 100644
--- a/bamboo_engine/config.py
+++ b/bamboo_engine/config.py
@@ -70,3 +70,5 @@ class Settings:
PIPELINE_EXCLUSIVE_GATEWAY_EXPR_FUNC = default_expr_func
PIPELINE_EXCLUSIVE_GATEWAY_STRATEGY = ExclusiveGatewayStrategy.ONLY.value
+
+ PIPELINE_ENABLE_ROLLBACK = False
diff --git a/bamboo_engine/engine.py b/bamboo_engine/engine.py
index 6f929705..b8fdd66b 100644
--- a/bamboo_engine/engine.py
+++ b/bamboo_engine/engine.py
@@ -57,6 +57,7 @@
setup_gauge,
setup_histogram,
)
+from .utils.constants import RuntimeSettings
from .utils.host import get_hostname
from .utils.string import get_lower_case_name
@@ -115,6 +116,10 @@ def run_pipeline(
cycle_tolerate = options.get("cycle_tolerate", False)
validator.validate_and_process_pipeline(pipeline, cycle_tolerate)
+ start_node_id = options.get("start_node_id", pipeline["start_event"]["id"])
+ # 如果起始位置不是开始节点,则需要进行额外校验
+ validator.validate_pipeline_start_node(pipeline, start_node_id)
+
self.runtime.pre_prepare_run_pipeline(
pipeline, root_pipeline_data, root_pipeline_context, subprocess_context, **options
)
@@ -122,10 +127,11 @@ def run_pipeline(
process_id = self.runtime.prepare_run_pipeline(
pipeline, root_pipeline_data, root_pipeline_context, subprocess_context, **options
)
+
# execute from start event
self.runtime.execute(
process_id=process_id,
- node_id=pipeline["start_event"]["id"],
+ node_id=start_node_id,
root_pipeline_id=pipeline["id"],
parent_pipeline_id=pipeline["id"],
)
@@ -912,7 +918,8 @@ def execute(
# 设置状态前检测
if node_state.name not in states.INVERTED_TRANSITION[states.RUNNING]:
logger.info(
- "[pipeline-trace](root_pipeline: %s) can not transit state from %s to RUNNING for exist state", # noqa
+ "[pipeline-trace](root_pipeline: %s) can not transit state from %s to RUNNING "
+ "for exist state",
process_info.root_pipeline_id,
node_state.name,
)
@@ -1010,6 +1017,15 @@ def execute(
hook=HookType.NODE_FINISH,
node=node,
)
+ if node.type == NodeType.ServiceActivity and self.runtime.get_config(
+ RuntimeSettings.PIPELINE_ENABLE_ROLLBACK.value
+ ):
+ self._set_snapshot(root_pipeline_id, node)
+ # 判断是否已经预约了回滚,如果已经预约,则kill掉当前的process,直接return
+ if node.reserve_rollback:
+ self.runtime.die(process_id)
+ self.runtime.start_rollback(root_pipeline_id, node_id)
+ return
# 进程是否要进入睡眠
if execute_result.should_sleep:
@@ -1177,7 +1193,9 @@ def schedule(
# only retry at multiple calback type
if schedule.type is not ScheduleType.MULTIPLE_CALLBACK:
logger.info(
- "root pipeline[%s] schedule(%s) %s with version %s is not multiple callback type, will not retry to get lock", # noqa
+ "root pipeline[%s] schedule(%s) %s with version %s is not multiple callback type, "
+ "will not retry to get lock",
+ # noqa
root_pipeline_id,
schedule_id,
node_id,
@@ -1290,6 +1308,15 @@ def schedule(
node=node,
callback_data=callback_data,
)
+ if node.type == NodeType.ServiceActivity and self.runtime.get_config(
+ RuntimeSettings.PIPELINE_ENABLE_ROLLBACK.value
+ ):
+ self._set_snapshot(root_pipeline_id, node)
+ # 判断是否已经预约了回滚,如果已经预约,启动回滚流程
+ if node.reserve_rollback:
+ self.runtime.start_rollback(root_pipeline_id, node_id)
+ return
+
self.runtime.execute(
process_id=process_id,
node_id=schedule_result.next_node_id,
@@ -1302,6 +1329,20 @@ def schedule(
time.time() - engine_post_schedule_start_at
)
+ def _set_snapshot(self, root_pipeline_id, node):
+ inputs = self.runtime.get_execution_data_inputs(node.id)
+ outputs = self.runtime.get_execution_data_outputs(node.id)
+ root_pipeline_input = {key: di.value for key, di in self.runtime.get_data_inputs(root_pipeline_id).items()}
+ self.runtime.set_node_snapshot(
+ root_pipeline_id=root_pipeline_id,
+ node_id=node.id,
+ code=node.code,
+ version=node.version,
+ context_values=root_pipeline_input,
+ inputs=inputs,
+ outputs=outputs,
+ )
+
def _add_history(
self,
node_id: str,
diff --git a/bamboo_engine/eri/interfaces.py b/bamboo_engine/eri/interfaces.py
index 99c722db..8ac6fb23 100644
--- a/bamboo_engine/eri/interfaces.py
+++ b/bamboo_engine/eri/interfaces.py
@@ -1577,6 +1577,35 @@ def get_config(self, name):
"""
+class RollbackMixin:
+ @abstractmethod
+ def set_pipeline_token(self, pipeline_tree: dict):
+ """
+ 设置pipeline token
+ """
+
+ @abstractmethod
+ def set_node_snapshot(
+ self,
+ root_pipeline_id: str,
+ node_id: str,
+ code: str,
+ version: str,
+ context_values: dict,
+ inputs: dict,
+ outputs: dict,
+ ):
+ """
+ 创建一份节点快照
+ """
+
+ @abstractmethod
+ def start_rollback(self, root_pipeline_id: str, node_id: str):
+ """
+ 开始回滚
+ """
+
+
class EngineRuntimeInterface(
PluginManagerMixin,
EngineAPIHooksMixin,
@@ -1591,6 +1620,7 @@ class EngineRuntimeInterface(
ExecutionHistoryMixin,
InterruptMixin,
ConfigMixin,
+ RollbackMixin,
metaclass=ABCMeta,
):
@abstractmethod
diff --git a/bamboo_engine/eri/models/node.py b/bamboo_engine/eri/models/node.py
index 7ac8e87a..ce17b509 100644
--- a/bamboo_engine/eri/models/node.py
+++ b/bamboo_engine/eri/models/node.py
@@ -12,7 +12,7 @@
"""
from enum import Enum
-from typing import List, Dict
+from typing import Dict, List
from bamboo_engine.utils.object import Representable
@@ -50,6 +50,7 @@ def __init__(
can_skip: bool = True,
can_retry: bool = True,
name: str = None,
+ reserve_rollback: bool = False,
):
"""
@@ -82,6 +83,7 @@ def __init__(
self.can_skip = can_skip
self.can_retry = can_retry
self.name = name
+ self.reserve_rollback = reserve_rollback
class EmptyStartEvent(Node):
diff --git a/bamboo_engine/exceptions.py b/bamboo_engine/exceptions.py
index 6a198162..39d4fe7d 100644
--- a/bamboo_engine/exceptions.py
+++ b/bamboo_engine/exceptions.py
@@ -11,6 +11,7 @@
specific language governing permissions and limitations under the License.
"""
+
# 异常定义模块
@@ -38,6 +39,10 @@ class TreeInvalidException(EngineException):
pass
+class StartPositionInvalidException(EngineException):
+ pass
+
+
class ConnectionValidateError(TreeInvalidException):
def __init__(self, failed_nodes, detail, *args):
self.failed_nodes = failed_nodes
diff --git a/bamboo_engine/states.py b/bamboo_engine/states.py
index 6a4a460d..319960b1 100644
--- a/bamboo_engine/states.py
+++ b/bamboo_engine/states.py
@@ -29,6 +29,9 @@ class StateType(Enum):
FINISHED = "FINISHED"
FAILED = "FAILED"
REVOKED = "REVOKED"
+ ROLLING_BACK = "ROLLING_BACK"
+ ROLL_BACK_SUCCESS = "ROLL_BACK_SUCCESS"
+ ROLL_BACK_FAILED = "ROLL_BACK_FAILED"
CREATED = StateType.CREATED.value
@@ -39,8 +42,11 @@ class StateType(Enum):
FINISHED = StateType.FINISHED.value
FAILED = StateType.FAILED.value
REVOKED = StateType.REVOKED.value
+ROLLING_BACK = StateType.ROLLING_BACK.value
+ROLL_BACK_SUCCESS = StateType.ROLL_BACK_SUCCESS.value
+ROLL_BACK_FAILED = StateType.ROLL_BACK_FAILED.value
-ALL_STATES = frozenset([READY, RUNNING, SUSPENDED, BLOCKED, FINISHED, FAILED, REVOKED])
+ALL_STATES = frozenset([READY, RUNNING, SUSPENDED, BLOCKED, FINISHED, FAILED, REVOKED, ROLLING_BACK])
ARCHIVED_STATES = frozenset([FINISHED, FAILED, REVOKED])
SLEEP_STATES = frozenset([SUSPENDED, REVOKED])
@@ -51,18 +57,20 @@ class StateType(Enum):
TRANSITION = ConstantDict(
{
READY: frozenset([RUNNING, SUSPENDED]),
- RUNNING: frozenset([FINISHED, FAILED, REVOKED, SUSPENDED]),
- SUSPENDED: frozenset([READY, REVOKED, RUNNING]),
+ RUNNING: frozenset([FINISHED, FAILED, REVOKED, SUSPENDED, ROLLING_BACK]),
+ SUSPENDED: frozenset([READY, REVOKED, RUNNING, ROLLING_BACK]),
BLOCKED: frozenset([]),
- FINISHED: frozenset([RUNNING, FAILED]),
+ FINISHED: frozenset([RUNNING, FAILED, ROLLING_BACK]),
FAILED: frozenset([READY, FINISHED]),
REVOKED: frozenset([]),
+ ROLLING_BACK: frozenset([ROLL_BACK_SUCCESS, ROLL_BACK_FAILED]),
+ ROLL_BACK_SUCCESS: frozenset([READY, FINISHED]),
+ ROLL_BACK_FAILED: frozenset([READY, FINISHED, ROLLING_BACK]),
}
)
def can_transit(from_state, to_state):
-
if from_state in TRANSITION:
if to_state in TRANSITION[from_state]:
return True
diff --git a/bamboo_engine/utils/constants.py b/bamboo_engine/utils/constants.py
index 1eb0f7df..5f458433 100644
--- a/bamboo_engine/utils/constants.py
+++ b/bamboo_engine/utils/constants.py
@@ -35,9 +35,11 @@ class ExclusiveGatewayStrategy(Enum):
class RuntimeSettings(Enum):
PIPELINE_EXCLUSIVE_GATEWAY_EXPR_FUNC = "PIPELINE_EXCLUSIVE_GATEWAY_EXPR_FUNC"
PIPELINE_EXCLUSIVE_GATEWAY_STRATEGY = "PIPELINE_EXCLUSIVE_GATEWAY_STRATEGY"
+ PIPELINE_ENABLE_ROLLBACK = "PIPELINE_ENABLE_ROLLBACK"
RUNTIME_ALLOWED_CONFIG = [
RuntimeSettings.PIPELINE_EXCLUSIVE_GATEWAY_EXPR_FUNC.value,
RuntimeSettings.PIPELINE_EXCLUSIVE_GATEWAY_STRATEGY.value,
+ RuntimeSettings.PIPELINE_ENABLE_ROLLBACK.value,
]
diff --git a/bamboo_engine/utils/graph.py b/bamboo_engine/utils/graph.py
index c3232c25..df162a6f 100644
--- a/bamboo_engine/utils/graph.py
+++ b/bamboo_engine/utils/graph.py
@@ -60,6 +60,55 @@ def get_cycle(self):
return []
+class RollbackGraph(Graph):
+ def __init__(self, nodes=None, flows=None):
+ self.nodes = nodes or []
+ self.flows = flows or []
+ super().__init__(self.nodes, self.flows)
+ self.edges = self.build_edges()
+ self.path = []
+ self.last_visited_node = ""
+ self.graph = {node: [] for node in self.nodes}
+ for flow in self.flows:
+ self.graph[flow[0]].append(flow[1])
+
+ def build_edges(self):
+ edges = {}
+ for flow in self.flows:
+ edges.setdefault(flow[0], set()).add(flow[1])
+ return edges
+
+ def add_node(self, node):
+ if node not in self.nodes:
+ self.nodes.append(node)
+
+ def add_edge(self, source, target):
+ self.flows.append([source, target])
+ self.edges.setdefault(source, set()).add(target)
+
+ def next(self, node):
+ return self.edges.get(node, {})
+
+ def reverse(self):
+ graph = RollbackGraph()
+ graph.nodes = self.nodes
+ for flow in self.flows:
+ graph.add_edge(flow[1], flow[0])
+
+ return graph
+
+ def in_degrees(self):
+ ingress = {node: 0 for node in self.nodes}
+ for node, targets in self.edges.items():
+ for target in targets:
+ ingress[target] += 1
+
+ return ingress
+
+ def as_dict(self):
+ return {"nodes": self.nodes, "flows": self.flows}
+
+
if __name__ == "__main__":
graph1 = Graph([1, 2, 3, 4], [[1, 2], [2, 3], [3, 4]])
assert not graph1.has_cycle()
diff --git a/bamboo_engine/validator/__init__.py b/bamboo_engine/validator/__init__.py
index 4d01c48f..8474434a 100644
--- a/bamboo_engine/validator/__init__.py
+++ b/bamboo_engine/validator/__init__.py
@@ -11,4 +11,8 @@
specific language governing permissions and limitations under the License.
"""
-from .api import validate_and_process_pipeline # noqa
+from .api import ( # noqa
+ get_allowed_start_node_ids,
+ validate_and_process_pipeline,
+ validate_pipeline_start_node,
+)
diff --git a/bamboo_engine/validator/api.py b/bamboo_engine/validator/api.py
index 170b62dc..e4edcf60 100644
--- a/bamboo_engine/validator/api.py
+++ b/bamboo_engine/validator/api.py
@@ -10,17 +10,71 @@
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.
"""
+import copy
-from bamboo_engine.eri import NodeType
from bamboo_engine import exceptions
+from bamboo_engine.eri import NodeType
from . import rules
-from .connection import (
- validate_graph_connection,
- validate_graph_without_circle,
-)
+from .connection import validate_graph_connection, validate_graph_without_circle
from .gateway import validate_gateways, validate_stream
-from .utils import format_pipeline_tree_io_to_list
+from .utils import (
+ compute_pipeline_main_nodes,
+ compute_pipeline_skip_executed_map,
+ format_pipeline_tree_io_to_list,
+ get_nodes_dict,
+)
+
+
+def validate_pipeline_start_node(pipeline: dict, node_id: str):
+ # 当开始位置位于开始节点时,则直接返回
+ if node_id == pipeline["start_event"]["id"]:
+ return
+
+ allowed_start_node_ids = get_allowed_start_node_ids(pipeline)
+ if node_id not in allowed_start_node_ids:
+ raise exceptions.StartPositionInvalidException("this node_id is not allowed as a starting node")
+
+
+def get_skipped_execute_node_ids(pipeline_tree, start_node_id, validate=True):
+ if validate and start_node_id not in get_allowed_start_node_ids(pipeline_tree):
+ raise Exception("the start_node_id is not legal, please check")
+ start_event_id = pipeline_tree["start_event"]["id"]
+
+ # 如果开始节点 = start_node_id, 说明要从开始节点开始执行,此时没有任何节点被跳过
+ if start_node_id == start_event_id:
+ return []
+
+ node_dict = get_nodes_dict(pipeline_tree)
+ # 流程的开始位置只允许出现在主干,子流程/并行网关内的节点不允许作为起始位置
+ will_skipped_nodes = compute_pipeline_skip_executed_map(start_event_id, node_dict, start_node_id)
+ return list(will_skipped_nodes)
+
+
+def get_allowed_start_node_ids(pipeline_tree):
+ # 检查该流程是否已经经过汇聚网关填充
+ def check_converge_gateway():
+ gateways = pipeline_tree["gateways"]
+ if not gateways:
+ return True
+ # 经过填充的网关会有converge_gateway_id 字段
+ for gateway in gateways.values():
+ if (
+ gateway["type"] in ["ParallelGateway", "ConditionalParallelGateway"]
+ and "converge_gateway_id" not in gateway
+ ):
+ return False
+
+ return True
+
+ if check_converge_gateway():
+ pipeline_tree = copy.deepcopy(pipeline_tree)
+ validate_gateways(pipeline_tree)
+ start_event_id = pipeline_tree["start_event"]["id"]
+ node_dict = get_nodes_dict(pipeline_tree)
+ # 流程的开始位置只允许出现在主干,子流程/并行网关内的节点不允许作为起始位置
+ allowed_start_node_ids = compute_pipeline_main_nodes(start_event_id, node_dict)
+ return allowed_start_node_ids
def validate_and_process_pipeline(pipeline: dict, cycle_tolerate=False):
diff --git a/bamboo_engine/validator/utils.py b/bamboo_engine/validator/utils.py
index c7f20cfe..10f97b6c 100644
--- a/bamboo_engine/validator/utils.py
+++ b/bamboo_engine/validator/utils.py
@@ -88,3 +88,50 @@ def get_nodes_dict(data):
node["target"] = [data["flows"][outgoing]["target"] for outgoing in node["outgoing"]]
return nodes
+
+
+def compute_pipeline_main_nodes(node_id, node_dict):
+ """
+ 计算流程中的主线节点,遇到并行网关/分支并行网关/子流程,则会跳过
+ 最后计算出来主干分支所允许开始的节点范围
+ """
+ nodes = []
+ node_detail = node_dict[node_id]
+ node_type = node_detail["type"]
+ if node_type in [
+ "EmptyStartEvent",
+ "ServiceActivity",
+ "ExclusiveGateway",
+ "ParallelGateway",
+ "ConditionalParallelGateway",
+ ]:
+ nodes.append(node_id)
+
+ if node_type in ["EmptyStartEvent", "ServiceActivity", "ExclusiveGateway", "ConvergeGateway", "SubProcess"]:
+ next_nodes = node_detail.get("target", [])
+ for next_node_id in next_nodes:
+ nodes += compute_pipeline_main_nodes(next_node_id, node_dict)
+ elif node_type in ["ParallelGateway", "ConditionalParallelGateway"]:
+ next_node_id = node_detail["converge_gateway_id"]
+ nodes += compute_pipeline_main_nodes(next_node_id, node_dict)
+
+ return nodes
+
+
+def compute_pipeline_skip_executed_map(node_id, node_dict, start_node_id):
+ nodes = [node_id]
+ if node_id == start_node_id:
+ return nodes
+ node_detail = node_dict[node_id]
+ next_nodes = node_detail.get("target", [])
+ if node_detail["type"] in ["ExclusiveGateway"]:
+ for next_node_id in next_nodes:
+ node_ids = compute_pipeline_skip_executed_map(next_node_id, node_dict, start_node_id)
+ # 如果开始的位置在分支网关内,只处理该分支
+ if start_node_id in node_ids:
+ nodes += node_ids
+ else:
+ for next_node_id in next_nodes:
+ nodes += compute_pipeline_skip_executed_map(next_node_id, node_dict, start_node_id)
+
+ return set(nodes) - {start_node_id}
diff --git a/docs/assets/img/rollback/rollback.png b/docs/assets/img/rollback/rollback.png
index 2029187e..537ef21b 100644
Binary files a/docs/assets/img/rollback/rollback.png and b/docs/assets/img/rollback/rollback.png differ
diff --git a/docs/assets/img/start_the_pipeline_at_the_specified_location/run_pipeline.png b/docs/assets/img/start_the_pipeline_at_the_specified_location/run_pipeline.png
new file mode 100644
index 00000000..e85e7576
Binary files /dev/null and b/docs/assets/img/start_the_pipeline_at_the_specified_location/run_pipeline.png differ
diff --git a/docs/user_guide/node_timer_event_introduction.md b/docs/user_guide/node_timer_event_introduction.md
new file mode 100644
index 00000000..d2ef878c
--- /dev/null
+++ b/docs/user_guide/node_timer_event_introduction.md
@@ -0,0 +1,125 @@
+# 增强包 - 节点计时器边界事件
+
+## 特性
+
+- 支持节点边界事件扫描与处理
+- 支持自定义节点计时器边界事件处理方式
+- 支持自定义节点计时器边界事件相关消息队列
+
+## 启动配置
+
+0. 该功能依赖 `mq`、`Redis` 和 `DB` 三个服务,需要在启动时保证这三个服务已经启动。
+1. 在 Django 项目配置文件的 INSTALLED_APPS 中添加 `pipeline.contrib.node_timer_event` 应用。
+2. 执行 `python manage.py migrate` 命令,创建数据库表。
+3. 启动计时器到期扫描进程,执行 `python manage.py start_node_timer_event_process` 命令。
+4. 启动对应的 worker 进程,执行 `python manage.py celery worker -l info -c 4` 命令。
+
+## 接口
+
+目前,pipeline.contrib.node_timer_event 模块提供了以下接口或扩展:
+
+1. Action
+
+ SDK 内置了两个 Action 提供「节点超时」处理能力
+ - `bamboo_engine_forced_fail`: 节点超时强制失败
+ - `bamboo_engine_forced_fail_and_skip`: 节点超时强制失败并跳过
+
+ SDK 也提供了比较友好的自定义 Action 扩展和接入能力,用于定义业务层计时器边界事件的处理动作,例如定义一个名为 `example`
+ 的 Action
+
+ ```python
+ import logging
+
+ from pipeline.core.data.base import DataObject
+ from pipeline.contrib.node_timer_event.handlers import BaseAction
+
+ logger = logging.getLogger(__name__)
+
+ class ExampleAction(BaseAction):
+ def do(self, data: DataObject, parent_data: DataObject, *args, **kwargs) -> bool:
+ logger.info("[Action] example do: data -> %s, parent_data -> %s", data, parent_data)
+ return True
+
+ class Meta:
+ action_name = "example"
+
+ ```
+
+2. apply_node_timer_event_configs
+ 该接口用于在 pipeline_tree 中应用节点计时器边界事件,接口定义如下:
+ ```python
+ def apply_node_timer_event_configs(pipeline_tree: dict, configs: dict):
+ """
+ 在 pipeline_tree 中应用节点时间事件配置
+ :param pipeline_tree: pipeline_tree
+ :param configs: 节点时间时间配置
+ """
+ ```
+ 例如,创建一个节点运行 10 min 后启动的计时器边界事件,事件处理动作为步骤 1. 定义的 `example` 的计时器边界事件配置,可以这样写:
+ ```python
+ pipeline_tree = {} # 此处省略 pipeline_tree 的创建过程
+ configs = {"node_id": [{"enable": True, "action": "example", "timer_type": "time_duration", "defined": "PT10M"}]}
+ new_pipeline_tree = apply_node_timer_event_configs(pipeline_tree, configs)
+ ```
+
+ 节点计时器边界事件配置中
+ - enable 代表是否启用该节点的计时器事件配置
+ - action 表示计时器触发时执行的动作
+ - defined 代表计时器定义
+ - timer_type 表示计时器类型
+ - defined & timer_type 更多配置方式,请参考文末「附录」
+
+3. batch_create_node_timeout_config
+ 该接口用于批量创建节点计时器边界事件,接口定义如下:
+ ```python
+ def batch_create_node_timer_event_config(root_pipeline_id: str, pipeline_tree: dict):
+ """
+ 批量创建节点计时器边界事件配置
+ :param root_pipeline_id: pipeline root ID
+ :param pipeline_tree: pipeline_tree
+ :return: 节点计时器边界事件配置数据插入结果
+ """
+ ```
+ 插入结果示例:
+ ``` python
+ {
+ "result": True, # 是否操作成功, True 时关注 data,False 时关注 message
+ "data": [...], # NodeTimerEventConfig Model objects
+ "message": ""
+ }
+ ```
+
+## 自定义
+
+节点计时器边界事件模块的自定义配置 Django Settings 来实现,配置项和默认值如下:
+
+``` python
+from pipeline.contrib.node_timer_event.handlers import node_timeout_handler
+
+PIPELINE_NODE_TIMER_EVENT_KEY_PREFIX = "bamboo:v1:node_timer_event" # Redis key 前缀,用于记录正在执行的节点,命名示例: {app_code}:{app_env}:{module}:node_timer_event
+PIPELINE_NODE_TIMER_EVENT_HANDLE_QUEUE = None # 节点计时器边界事件处理队列名称, 用于处理计时器边界事件, 需要 worker 接收该队列消息,默认为 None,即使用 celery 默认队列
+PIPELINE_NODE_TIMER_EVENT_DISPATCH_QUEUE = None # 节点计时器边界事件分发队列名称, 用于记录计时器边界事件, 需要 worker 接收该队列消息,默认为 None,即使用 celery 默认队列
+PIPELINE_NODE_TIMER_EVENT_EXECUTING_POOL = "bamboo:v1:node_timer_event:executing_node_pool" # 执行节点池名称,用于记录正在执行的节点,需要保证 Redis key 唯一,命名示例: {app_code}:{app_env}:{module}:executing_node_pool
+PIPELINE_NODE_TIMER_EVENT_POOL_SCAN_INTERVAL = 1 # 节点池扫描间隔,间隔越小,边界事件触发时间越精准,相应的事件处理的 workload 负载也会提升,默认为 1 s
+PIPELINE_NODE_TIMER_EVENT_MAX_EXPIRE_TIME = 60 * 60 * 24 * 15 # 最长过期时间,兜底删除 Redis 冗余数据,默认为 15 Days,请根据业务场景调整
+PIPELINE_NODE_TIMER_EVENT_ADAPTER_CLASS = "pipeline.contrib.node_timer_event.adapter.NodeTimerEventAdapter" # 边界事件处理适配器,默认为 `pipeline.contrib.node_timer_event.adapter.NodeTimerEventAdapter`
+```
+
+## 使用样例
+
+假设当前开发者已经准备好了对应的 pipeline_tree 和对应的节点计时器边界事件配置,那么在进行项目配置并启动对应的进程后,可以按照以下步骤进行处理:
+
+1. 调用 apply_node_timer_event_configs 接口,将节点计时器边界事件配置应用到 pipeline_tree 中
+2. 调用 batch_create_node_timeout_config 接口,将节点计时器边界事件配置插入到数据库中
+3. 启动 pipeline 运行,等待动计时器到期扫描进程处理到期边界事件,验证时请确认节点执行完成时间大于设置的计时器到期时间
+4. 查看节点计时器边界事件处理结果是否符合预期
+
+## 附录
+
+### 支持的计时器定义
+
+| 计时器类型 | 配置值 | 描述 | `defined` 样例 | 备注 |
+|---------------------|-----------------|----------------------------------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------|
+| 时间日期(Time date) | `time_date` | ISO 8601 组合日期和时间格式 | `2019-10-01T12:00:00Z` - UTC 时间
`2019-10-02T08:09:40+02:00` - UTC 加上两小时时区偏移
`2019-10-02T08:09:40+02:00[Europe/Berlin]` - UTC 加上柏林两小时时区偏移 | |
+| 持续时间(Time duration) | `time_duration` | ISO 8601 持续时间格式,模式:`P(n)Y(n)M(n)DT(n)H(n)M(n)S` | `PT15S` - 15 秒
`PT1H30M` - 1 小时 30 分钟
`P14D` - 14 天
`P14DT1H30M` - 14 天 1 小时 30 分钟 | `P` - 持续事件标识
`Y` - 年
`M` - 月
`D` - 天
`T` - 时间分量开始标识
`H` - 小时
`M` - 分钟
`S` - 秒 |
+| 时间周期(Time cycle) | `time_cycle` | ISO 8601 重复间隔格式,包含重复次数模式:`R(n)` 及持续时间模式:`P(n)Y(n)M(n)DT(n)H(n)M(n)S` | `R5/PT10S` - 每10秒一次,最多五次
`R1/P1D` - 每天一次,最多一次 | |
\ No newline at end of file
diff --git a/docs/user_guide/rollback.md b/docs/user_guide/rollback.md
index a4e066e5..2c08268b 100644
--- a/docs/user_guide/rollback.md
+++ b/docs/user_guide/rollback.md
@@ -1,39 +1,116 @@
### 功能介绍:
-节点回退允许流程回退到某个特定的节点重新开始。**该回退会删除目标节点之后所有已执行过的节点信息和数据,让流程表现的是第一次执行的样子。**
-需要注意的流程的回退并不是无限制的,需要满足以下条件的节点才允许回退。
-- 只能回退运行中的流程
-- 子流程暂时不支持回退
-- 目标节点的状态必须为已完成状态
-- 并行网关内的节点不允许回退。并行网关包含条件并行网关和并行网关
-- 网关节点不允许回退
-- 条件网关只允许条件网关内已经运行的节点允许回退,未执行到的分支不允许回退。
+## 回滚配置
-节点在回退前会强制失败掉当前运行的节点,只有流程中没有正在运行的节点时才会开始执行回退逻辑。
-节点回退的过程无法中断,因为中断导致的回退失败可能会导致无法通过再次回退重试成功
+### 开启配置项
-针对如下图的流程,蓝色框所标注的节点是允许回退的节点。
+在开始体验回滚之前,我们需要在django_settings 中开启以下的配置:
-![rollback.png](..%2Fassets%2Fimg%2Frollback%2Frollback.png)
+```python
+PIPELINE_ENABLE_ROLLBACK = True
+ROLLBACK_QUEUE = "default_test"
+PIPELINE_ENABLE_AUTO_EXECUTE_WHEN_ROLL_BACKED = False
+```
-### 使用事例:
+其中:
+- PIPELINE_ENABLE_ROLLBACK 表示开启回滚
+- ROLLBACK_QUEUE: 表示回滚所使用的队列
+- PIPELINE_ENABLE_AUTO_EXECUTE_WHEN_ROLL_BACKED: 是否开启回滚后自动开始,开启时,回滚到目标节点将会自动开始,未开启时流程回到目标节点将会暂停。
-查询可以回退的节点列表:
+### Install App
+之后需要在 `INSTALLED_APPS` 增加配置:
```python
-from pipeline.contrib.rollback import api
+INSTALLED_APPS += (
+ "pipeline.contrib.rollback",
+)
+```
-# 获取该pipeline允许回滚的节点列表
-result = api.get_allowed_rollback_node_id_list(pipeline_id)
-node_ids = result.data
+### 执行 migrate 操作
+```bash
+python manage.py migrate rollback
```
-节点的回退使用非常简单,只需要指定pipeline_id和node_id即可,如下所示:
-```python
+之后回滚的一切便已经就绪了。
+
+## 回滚的边界条件
+
+### Token 模式:
+
+
+现阶段的回滚的行为受到严格限制,在TOKEN 模式下,回滚将不能随意的指向某个节点。流程回滚时将沿着原路径依次回滚(如果存在子流程,则先回滚子流程,再继续主流程的回滚)。
+在流程回滚时, 节点的状态机如下:
+
+![rollback.png](..%2Fassets%2Fimg%2Frollback%2Frollback.png)
+
+以下是回滚的各项边界条件:
+
+#### 任务状态
+
+只有处于 `RUNNING` 和 `ROLL_BACK_FAILED` 状态的任务才允许回滚。当任务处于结束,暂停时,将不允许回滚。
+
+#### 任务节点
+
+在 token 模式下,**回滚的起始和目标节点的token须保持一致**。同时不允许同 token下 **存在正在运行的节点**。
+
+##### 回滚的起点
+
+- 回滚开始的节点的状态只支持`FINISHED` 或 `FAILED`, 正在运行的节点将不允许回滚。
+- 回滚开始的节点必须是流程当前正在运行的节点之一,也就是流程的末端节点。
+
+##### 回滚的目标节点
+
+- 回滚的目标节点只支持`FINISHED` 状态,回滚的目标节点只支持任务类型的节点,网关节点不支持回滚。
+
+#### 回滚预约
+- 回滚预约只能预约为RUNNING状态的节点,当该节点结束时,将自动开始回滚。
+- **一个流程同时只能有一个预约回滚的任务**
+
+
+### ANY 模式:
+
+当为ANY 模式的回滚时,流程可以从任何地方开始,回滚到之前的任意节点上去,此时流程**将不会按照路径调用回滚(不会调用节点的rollback方法)**,而是直接回到目标节点,并删除回滚路径上已经执行过的节点信息,从目标位置开始。
+
+#### 任务状态
+
+只有处于 `RUNNING` 的任务才允许回滚。当任务处于结束,暂停时,将不允许回滚。
+
+#### 任务节点
+
+在 any 模式下,回滚的边界条件将少得多,由于 any 状态下的回滚将直接回到目标节点并开始,类似于任务的任意节点跳转。
+
+在 any 模式下,回滚开始前**不允许当前流程存在处于 running 状态的节点。**
+
+##### 回滚的起点
+
+- 回滚开始的节点必须是流程当前正在运行的节点,也就是流程的末端节点。
+- -回滚开始的节点的状态只支持`FINISHED` 或 `FAILED`, 正在运行的节点将不再允许回滚。
+
+##### 回滚的目标节点
+
+- 回滚的目标节点只支持`FINISHED` 状态,回滚的目标节点只支持任务节点类型。
+
+#### 回滚预约
+- 回滚预约只能预约为running状态的节点,当该节点结束时,将自动开始回滚。
+- **一个流程同时只能有一个预约回滚的任务**
+
+#### 回滚的使用:
+
+``` python
from pipeline.contrib.rollback import api
-result = api.rollback(pipeline_id, node_id)
-if result.result:
- pass
-```
\ No newline at end of file
+# 节点回滚,其中mode 有 TOKEN 和 ANY 两种模式可选
+api.rollback(root_pipeline_id, start_node_id, target_node_id, mode="TOKEN")
+
+# 回滚预约
+api.reserve_rollback(root_pipeline_id, start_node_id, target_node_id, mode="TOKEN")
+
+# 取消回滚预约
+api.cancel_reserved_rollback(root_pipeline_id, start_node_id, target_node_id, mode="TOKEN")
+
+
+# 获取本次回滚支持的范围
+api.get_allowed_rollback_node_id_list(root_pipeline_id, start_node_id, mode="TOKEN")
+
+```
diff --git a/docs/user_guide/start_the_pipeline_at_the_specified_location.md b/docs/user_guide/start_the_pipeline_at_the_specified_location.md
new file mode 100644
index 00000000..34245ebf
--- /dev/null
+++ b/docs/user_guide/start_the_pipeline_at_the_specified_location.md
@@ -0,0 +1,130 @@
+# 指定流程从某个位置开始执行
+
+默认的 run_pipeline_api 只允许流程从开始节点开始,再某些特殊的场景下,用户可能会新起一个任务,并期望从指定的位置开始。
+因此run_pipeline 支持了该特性,不过需要注意的是,该功能是受限的,这意味着你不能选择流程内的任意一个位置开始流程。
+
+使用方式:
+
+```python
+from pipeline.eri.runtime import BambooDjangoRuntime
+from bamboo_engine import api
+
+pipeline = {}
+# 可以使用root_pipeline_context的方式补充缺失的上下文信息
+api.run_pipeline(runtime=BambooDjangoRuntime(),
+ pipeline=pipeline,
+ start_node_id="xxxxx",
+ root_pipeline_context={})
+```
+
+使用范围:
+
+start_node_id 的指定需要遵循如下规则:
+
+- 只允许开始节点和位于流程中的主干节点和分支网关内的节点进行回滚,不允许并行网关内的节点作为开始的起始位置,当分支网关处于并行网关内时,该分支网关内的节点也无法作为开始的起始位置。
+- 位于主流程上的并行网关/条件并行网关/条件网关 允许作为起始节点,汇聚网关不允许作为流程的开始节点。
+- 子流程节点不允许作为流程的开始节点
+- 结束节点不允许作为流程的开始节点
+
+下图红框内的节点表示允许作为起始位置的节点。
+
+![run_pipeline.png](..%2Fassets%2Fimg%2Fstart_the_pipeline_at_the_specified_location%2Frun_pipeline.png)
+
+其他工具方法:
+
+1. 获取某个流程所允许的回滚范围
+
+```python
+
+from bamboo_engine.builder import (
+ ConditionalParallelGateway,
+ ConvergeGateway,
+ EmptyEndEvent,
+ EmptyStartEvent,
+ ServiceActivity,
+ build_tree,
+)
+
+from bamboo_engine.validator.api import get_allowed_start_node_ids
+
+start = EmptyStartEvent()
+act_1 = ServiceActivity(component_code="pipe_example_component", name="act_1")
+cpg = ConditionalParallelGateway(
+ conditions={0: "${act_1_output} < 0", 1: "${act_1_output} >= 0", 2: "${act_1_output} >= 0"},
+ name="[act_2] or [act_3 and act_4]",
+)
+act_2 = ServiceActivity(component_code="pipe_example_component", name="act_2")
+act_3 = ServiceActivity(component_code="pipe_example_component", name="act_3")
+act_4 = ServiceActivity(component_code="pipe_example_component", name="act_4")
+cg = ConvergeGateway()
+end = EmptyEndEvent()
+start.extend(act_1).extend(cpg).connect(act_2, act_3, act_4).to(cpg).converge(cg).extend(end)
+
+pipeline = build_tree(start)
+allowed_start_node_ids = get_allowed_start_node_ids(pipeline)
+```
+
+2. 检查某个节点是否可作为开始节点:
+
+```python
+from bamboo_engine.builder import (
+ ConditionalParallelGateway,
+ ConvergeGateway,
+ EmptyEndEvent,
+ EmptyStartEvent,
+ ServiceActivity,
+ build_tree,
+)
+
+from bamboo_engine.validator.api import validate_pipeline_start_node
+
+start = EmptyStartEvent()
+act_1 = ServiceActivity(component_code="pipe_example_component", name="act_1")
+cpg = ConditionalParallelGateway(
+ conditions={0: "${act_1_output} < 0", 1: "${act_1_output} >= 0", 2: "${act_1_output} >= 0"},
+ name="[act_2] or [act_3 and act_4]",
+)
+act_2 = ServiceActivity(component_code="pipe_example_component", name="act_2")
+act_3 = ServiceActivity(component_code="pipe_example_component", name="act_3")
+act_4 = ServiceActivity(component_code="pipe_example_component", name="act_4")
+cg = ConvergeGateway()
+end = EmptyEndEvent()
+start.extend(act_1).extend(cpg).connect(act_2, act_3, act_4).to(cpg).converge(cg).extend(end)
+
+pipeline = build_tree(start)
+validate_pipeline_start_node(pipeline, act_2.id)
+```
+
+2.当开始节点为某个节点时,流程被跳过执行的节点列表:
+
+```python
+from bamboo_engine.builder import (
+ ConditionalParallelGateway,
+ ConvergeGateway,
+ EmptyEndEvent,
+ EmptyStartEvent,
+ ServiceActivity,
+ build_tree,
+)
+
+from bamboo_engine.validator.api import get_skipped_execute_node_ids
+
+start = EmptyStartEvent()
+act_1 = ServiceActivity(component_code="pipe_example_component", name="act_1")
+cpg = ConditionalParallelGateway(
+ conditions={0: "${act_1_output} < 0", 1: "${act_1_output} >= 0", 2: "${act_1_output} >= 0"},
+ name="[act_2] or [act_3 and act_4]",
+)
+act_2 = ServiceActivity(component_code="pipe_example_component", name="act_2")
+act_3 = ServiceActivity(component_code="pipe_example_component", name="act_3")
+act_4 = ServiceActivity(component_code="pipe_example_component", name="act_4")
+cg = ConvergeGateway()
+end = EmptyEndEvent()
+start.extend(act_1).extend(cpg).connect(act_2, act_3, act_4).to(cpg).converge(cg).extend(end)
+
+pipeline = build_tree(start)
+
+# validate = True 将会校验节点合法性
+skipped_execute_node_ids = get_skipped_execute_node_ids(pipeline, act_2.id, validate=True)
+
+```
\ No newline at end of file
diff --git a/pyproject.toml b/pyproject.toml
index abd6a44f..64a610ad 100644
--- a/pyproject.toml
+++ b/pyproject.toml
@@ -1,6 +1,6 @@
[tool.poetry]
name = "bamboo-engine"
-version = "2.9.0"
+version = "2.10.0rc4"
description = "Bamboo-engine is a general-purpose workflow engine"
authors = ["homholueng "]
license = "MIT"
diff --git a/runtime/bamboo-pipeline/pipeline/__init__.py b/runtime/bamboo-pipeline/pipeline/__init__.py
index e59e754c..a3f486e8 100644
--- a/runtime/bamboo-pipeline/pipeline/__init__.py
+++ b/runtime/bamboo-pipeline/pipeline/__init__.py
@@ -13,4 +13,4 @@
default_app_config = "pipeline.apps.PipelineConfig"
-__version__ = "3.28.0"
+__version__ = "3.29.0rc4"
diff --git a/runtime/bamboo-pipeline/pipeline/conf/default_settings.py b/runtime/bamboo-pipeline/pipeline/conf/default_settings.py
index 437cfe9d..eda27c28 100644
--- a/runtime/bamboo-pipeline/pipeline/conf/default_settings.py
+++ b/runtime/bamboo-pipeline/pipeline/conf/default_settings.py
@@ -104,3 +104,6 @@
# 是否开启PIPELINE HOOKS 事件通知
ENABLE_PIPELINE_EVENT_SIGNALS = getattr(settings, "ENABLE_PIPELINE_EVENT_SIGNALS", False)
+
+ROLLBACK_QUEUE = getattr(settings, "ROLLBACK_QUEUE", "rollback")
+PLUGIN_EXECUTE_QUEUE = getattr(settings, "PLUGIN_EXECUTE_QUEUE", "plugin_execute")
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/exceptions.py b/runtime/bamboo-pipeline/pipeline/contrib/exceptions.py
index 186ea958..a769bc4d 100644
--- a/runtime/bamboo-pipeline/pipeline/contrib/exceptions.py
+++ b/runtime/bamboo-pipeline/pipeline/contrib/exceptions.py
@@ -20,3 +20,7 @@ class RollBackException(PipelineException):
class UpdatePipelineContextException(PipelineException):
pass
+
+
+class PluginExecuteException(PipelineException):
+ pass
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/fields.py b/runtime/bamboo-pipeline/pipeline/contrib/fields.py
new file mode 100644
index 00000000..8f6462c8
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/fields.py
@@ -0,0 +1,31 @@
+# -*- coding: utf-8 -*-
+import codecs
+import json
+import pickle
+
+from django.db.models import TextField
+
+
+class SerializerField(TextField):
+ """
+ 特定的序列化类,用于兼容json和pickle两种序列化数据
+ """
+
+ def to_python(self, value):
+ try:
+ return json.loads(value)
+ except Exception:
+ return pickle.loads(codecs.decode(value.encode(), "base64"))
+
+ def from_db_value(self, value, expression, connection, context=None):
+ try:
+ return json.loads(value)
+ except Exception:
+ return pickle.loads(codecs.decode(value.encode(), "base64"))
+
+ def get_prep_value(self, value):
+ try:
+ return json.dumps(value)
+ except TypeError:
+ return codecs.encode(pickle.dumps(value), "base64").decode()
+ pass
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/node_timeout/management/commands/start_node_timeout_process.py b/runtime/bamboo-pipeline/pipeline/contrib/node_timeout/management/commands/start_node_timeout_process.py
index 6ab1fdbb..3a5513a4 100644
--- a/runtime/bamboo-pipeline/pipeline/contrib/node_timeout/management/commands/start_node_timeout_process.py
+++ b/runtime/bamboo-pipeline/pipeline/contrib/node_timeout/management/commands/start_node_timeout_process.py
@@ -12,17 +12,15 @@
"""
import datetime
import json
+import logging
import signal
import time
-import logging
-from django.conf import settings
from django.core.management import BaseCommand
from django.db import connections
-
+from pipeline.contrib.node_timeout.models import TimeoutNodesRecord
from pipeline.contrib.node_timeout.settings import node_timeout_settings
from pipeline.contrib.node_timeout.tasks import dispatch_timeout_nodes
-from pipeline.contrib.node_timeout.models import TimeoutNodesRecord
logger = logging.getLogger("root")
@@ -33,8 +31,8 @@ class Command(BaseCommand):
def handle(self, *args, **options):
signal.signal(signal.SIGTERM, self._graceful_exit)
- redis_inst = settings.redis_inst
- nodes_pool = settings.EXECUTING_NODE_POOL
+ redis_inst = node_timeout_settings.redis_inst
+ nodes_pool = node_timeout_settings.executing_pool
while not self.has_killed:
try:
start = time.time()
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/__init__.py b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/__init__.py
new file mode 100644
index 00000000..26a6d1c2
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/__init__.py
@@ -0,0 +1,12 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/adapter.py b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/adapter.py
new file mode 100644
index 00000000..ca0d5683
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/adapter.py
@@ -0,0 +1,138 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+
+import abc
+import datetime
+import json
+import logging
+import re
+from typing import Dict, List, Optional, Union
+
+from pipeline.contrib.node_timer_event.models import NodeTimerEventConfig
+from pipeline.contrib.node_timer_event.settings import node_timer_event_settings
+from pipeline.contrib.node_timer_event.types import TimerEvent, TimerEvents
+from pipeline.contrib.node_timer_event.utils import parse_timer_defined
+
+logger = logging.getLogger(__name__)
+
+EVENT_KEY_PATTERN = re.compile(r".*node:(?P.+):version:(?P.+):index:(?P\d+)")
+
+
+class NodeTimerEventBaseAdapter(abc.ABC):
+
+ node_id: str = None
+ version: str = None
+ root_pipeline_id: Optional[str] = None
+ events: Optional[TimerEvents] = None
+ index__event_map: Optional[Dict[int, TimerEvent]] = None
+
+ def __init__(self, node_id: str, version: str):
+ self.node_id: str = node_id
+ self.version: str = version
+
+ def is_ready(self) -> bool:
+ """适配器是否就绪"""
+ if not self.events:
+ return False
+ return True
+
+ def fetch_keys_to_be_rem(self) -> List[str]:
+ """
+ 获取需要被移除的事件 Key
+ :return:
+ """
+ return [self.get_event_key(event) for event in self.events]
+
+ def get_event_key(self, event: TimerEvent) -> str:
+ """
+ 获取事件 Key
+ :param event:
+ :return:
+ """
+
+ # zset 没有按字符串匹配模式批量删除 key 的支持,使用 key 的命名采用已检索的信息进行拼接
+ # 之前想把 loop 也维护进去,发觉移除操作非常麻烦,故采用 incr 的方式,单独维护每个事件事件的触发次数
+ key_prefix: str = f"{node_timer_event_settings.key_prefix}:node:{self.node_id}:version:{self.version}"
+ return f"{key_prefix}:index:{event['index']}"
+
+ @classmethod
+ def get_next_expired_time(cls, event: TimerEvent, start: Optional[datetime.datetime] = None) -> float:
+ """
+ 获取时间事件下一次过期时间
+ :param event: 事件详情
+ :param start: 开始时间,默认取 datetime.now()
+ :return:
+ """
+ return parse_timer_defined(event["timer_type"], event["defined"], start=start or datetime.datetime.now())[
+ "timestamp"
+ ]
+
+ def add_to_pool(self, redis_inst, event: TimerEvent):
+
+ key: str = self.get_event_key(event)
+
+ expired_time: float = self.get_next_expired_time(event)
+
+ # TODO 考虑 incr & zadd 合并,使用 lua 封装成原子操作
+ loop: int = int(redis_inst.incr(key, 1))
+ redis_inst.expire(key, node_timer_event_settings.max_expire_time)
+ if loop > event["repetitions"]:
+ logger.info(
+ "[add_to_pool] No need to add: node -> %s, version -> %s, loop -> %s, event -> %s",
+ self.node_id,
+ self.version,
+ loop,
+ event,
+ )
+ return
+
+ redis_inst.zadd(node_timer_event_settings.executing_pool, mapping={key: expired_time}, nx=True)
+
+ logger.info(
+ "[add_to_pool] add event to redis: "
+ "node_id -> %s, version -> %s, event -> %s, key -> %s, expired_time -> %s",
+ self.node_id,
+ self.version,
+ event,
+ key,
+ expired_time,
+ )
+
+ @classmethod
+ def parse_event_key(cls, key: str) -> Dict[str, Union[str, int]]:
+ match = EVENT_KEY_PATTERN.match(key)
+ if match:
+ key_info: Dict[str, Union[str, int]] = match.groupdict()
+ # to int
+ key_info["index"] = int(key_info["index"])
+
+ return key_info
+
+ else:
+ raise ValueError(f"invalid key -> {key}")
+
+
+class NodeTimerEventAdapter(NodeTimerEventBaseAdapter):
+ def __init__(self, node_id: str, version: str):
+ super().__init__(node_id, version)
+
+ node_timer_event_config: NodeTimerEventConfig = NodeTimerEventConfig.objects.filter(
+ node_id=self.node_id
+ ).first()
+
+ if not node_timer_event_config:
+ return
+
+ self.root_pipeline_id: str = node_timer_event_config.root_pipeline_id
+ self.events: TimerEvents = json.loads(node_timer_event_config.events)
+ self.index__event_map: Dict[int, TimerEvent] = {event["index"]: event for event in self.events}
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/admin.py b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/admin.py
new file mode 100644
index 00000000..9fa6711b
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/admin.py
@@ -0,0 +1,26 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+
+from django.contrib import admin
+from pipeline.contrib.node_timer_event import models
+
+
+@admin.register(models.NodeTimerEventConfig)
+class NodeTimerEventConfigAdmin(admin.ModelAdmin):
+ list_display = ["root_pipeline_id", "node_id", "events"]
+ search_fields = ["root_pipeline_id__exact", "node_id__exact"]
+
+
+@admin.register(models.ExpiredNodesRecord)
+class ExpiredNodesRecordAdmin(admin.ModelAdmin):
+ list_display = ["id", "nodes"]
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/api.py b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/api.py
new file mode 100644
index 00000000..26ce1e01
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/api.py
@@ -0,0 +1,54 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+import copy
+from typing import Any, Dict, List
+
+from pipeline.contrib.node_timer_event.models import NodeTimerEventConfig
+from pipeline.contrib.utils import ensure_return_pipeline_contrib_api_result
+from pipeline.core.constants import PE
+
+
+@ensure_return_pipeline_contrib_api_result
+def apply_node_timer_event_configs(pipeline_tree: Dict[str, Any], configs: Dict[str, List[Dict[str, Any]]]):
+ """
+ 在 pipeline_tree 中应用节点计时器边界事件配置
+ :param pipeline_tree: pipeline_tree
+ :param configs: 节点计时器边界事件配置
+ e.g. {"node_id": [{"enable": True, "action": "forced_fail", "timer_type": "time_duration", "defined": "PT10M"}]}
+ """
+ new_pipeline_tree = copy.deepcopy(pipeline_tree)
+ for act_id, act in pipeline_tree[PE.activities].items():
+ if act["type"] == PE.SubProcess:
+ apply_node_timer_event_configs(act[PE.pipeline], configs)
+ elif act["type"] == PE.ServiceActivity and act_id in configs:
+ act.setdefault("events", {})["timer_events"] = [
+ {
+ "enable": config["enable"],
+ "timer_type": config["timer_type"],
+ "action": config["action"],
+ "defined": config["defined"],
+ }
+ for config in configs[act_id]
+ ]
+ return new_pipeline_tree
+
+
+@ensure_return_pipeline_contrib_api_result
+def batch_create_node_timer_event_config(root_pipeline_id: str, pipeline_tree: Dict[str, Any]):
+ """
+ 批量创建节点时间事件配置
+ :param root_pipeline_id: pipeline root ID
+ :param pipeline_tree: pipeline_tree
+ :return: 节点计时器边界事件配置数据插入结果,e.g. {"result": True, "data": objs, "message": ""}
+ """
+ return NodeTimerEventConfig.objects.batch_create_node_timer_event_config(root_pipeline_id, pipeline_tree)
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/apps.py b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/apps.py
new file mode 100644
index 00000000..2f7e43ee
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/apps.py
@@ -0,0 +1,31 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+from django.apps import AppConfig
+from django.conf import settings
+
+
+class NodeTimerEventConfig(AppConfig):
+ name = "pipeline.contrib.node_timer_event"
+ verbose_name = "PipelineNodeTimerEvent"
+
+ def ready(self):
+ from pipeline.contrib.node_timer_event.signals.handlers import ( # noqa
+ bamboo_engine_eri_node_state_handler,
+ )
+ from pipeline.contrib.node_timer_event.tasks import ( # noqa
+ dispatch_expired_nodes,
+ execute_node_timer_event_action,
+ )
+
+ if not hasattr(settings, "redis_inst"):
+ raise Exception("Django Settings should have redis_inst attribute")
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/constants.py b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/constants.py
new file mode 100644
index 00000000..97afb78f
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/constants.py
@@ -0,0 +1,35 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+
+from enum import Enum
+
+
+class TimerType(Enum):
+
+ # 时间日期(Time date)ISO 8601 组合日期和时间格式
+ # 2019-10-01T12:00:00Z - UTC 时间
+ # 2019-10-02T08:09:40+02:00 - UTC 加上两小时时区偏移
+ # 2019-10-02T08:09:40+02:00[Europe/Berlin] - UTC 加上柏林两小时时区偏移
+ TIME_DATE = "time_date"
+
+ # 时间周期(Time cycle)ISO 8601 重复间隔格式,包含重复次数模式:R(n) 及持续时间模式:P(n)Y(n)M(n)DT(n)H(n)M(n)S
+ # R5/PT10S - 每10秒一次,最多五次
+ # R/P1D - 每天,无限
+ TIME_CYCLE = "time_cycle"
+
+ # 持续时间(Time duration)ISO 8601 持续时间格式,模式:P(n)Y(n)M(n)DT(n)H(n)M(n)S
+ # PT15S - 15 秒
+ # PT1H30M - 1 小时 30 分钟
+ # P14D - 14 天
+ # P14DT1H30M - 14 天 1 小时 30 分钟
+ TIME_DURATION = "time_duration"
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/handlers.py b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/handlers.py
new file mode 100644
index 00000000..1725b686
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/handlers.py
@@ -0,0 +1,166 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+import logging
+from abc import abstractmethod
+from typing import Any, Dict, Tuple
+
+from pipeline.core.data.base import DataObject
+from pipeline.eri.runtime import BambooDjangoRuntime
+
+from bamboo_engine import api as bamboo_engine_api
+from bamboo_engine.eri import ExecutionData
+
+logger = logging.getLogger(__name__)
+
+
+class ActionManager:
+ __hub = {}
+
+ @classmethod
+ def register_invocation_cls(cls, invocation_cls):
+ action_name = invocation_cls.Meta.action_name
+ existed_invocation_cls = cls.__hub.get(action_name)
+ if existed_invocation_cls:
+ raise RuntimeError(
+ "func register error, {}'s action_name {} conflict with {}".format(
+ existed_invocation_cls, action_name, invocation_cls
+ )
+ )
+
+ cls.__hub[action_name] = invocation_cls
+
+ @classmethod
+ def clear(cls):
+ cls.__hub = {}
+
+ @classmethod
+ def get_action(cls, root_pipeline_id: str, node_id: str, version: str, action_name: str) -> "BaseAction":
+ """
+ 获取 Action 实例
+ :param root_pipeline_id: 根节点 ID
+ :param node_id: 节点 ID
+ :param version: 节点版本
+ :param action_name: Action 名称
+ :return:
+ """
+ if action_name not in cls.__hub:
+ raise ValueError("{} not found".format(action_name))
+ return cls.__hub[action_name](root_pipeline_id, node_id, version)
+
+
+class ActionMeta(type):
+ """
+ Metaclass for FEEL function invocation
+ """
+
+ def __new__(cls, name, bases, dct):
+ # ensure initialization is only performed for subclasses of Plugin
+ parents = [b for b in bases if isinstance(b, ActionMeta)]
+ if not parents:
+ return super().__new__(cls, name, bases, dct)
+
+ new_cls = super().__new__(cls, name, bases, dct)
+
+ # meta validation
+ meta_obj = getattr(new_cls, "Meta", None)
+ if not meta_obj:
+ raise AttributeError("Meta class is required")
+
+ action_name = getattr(meta_obj, "action_name", None)
+ if not action_name:
+ raise AttributeError("action_name is required in Meta")
+
+ # register func
+ ActionManager.register_invocation_cls(new_cls)
+
+ return new_cls
+
+
+class BaseAction(metaclass=ActionMeta):
+ def __init__(self, root_pipeline_id: str, node_id: str, version: str):
+ self.root_pipeline_id = root_pipeline_id
+ self.node_id = node_id
+ self.version = version
+
+ @classmethod
+ def get_execution_data(cls, root_pipeline_id: str, node_id: str) -> Tuple[DataObject, DataObject]:
+ runtime: BambooDjangoRuntime = BambooDjangoRuntime()
+ data: ExecutionData = runtime.get_execution_data(node_id)
+ root_pipeline_inputs: Dict[str, Any] = {
+ key: di.value for key, di in runtime.get_data_inputs(root_pipeline_id).items()
+ }
+ root_pipeline_data: ExecutionData = ExecutionData(inputs=root_pipeline_inputs, outputs={})
+
+ data_obj: DataObject = DataObject(inputs=data.inputs, outputs=data.outputs)
+ parent_data_obj: DataObject = DataObject(inputs=root_pipeline_data.inputs, outputs=root_pipeline_data.outputs)
+ return data_obj, parent_data_obj
+
+ def notify(self, *args, **kwargs) -> bool:
+ data, parent_data = self.get_execution_data(self.root_pipeline_id, self.node_id)
+ return self.do(data, parent_data, *args, **kwargs)
+
+ @abstractmethod
+ def do(self, data: DataObject, parent_data: DataObject, *args, **kwargs) -> bool:
+ raise NotImplementedError
+
+
+class ExampleAction(BaseAction):
+ def do(self, data: DataObject, parent_data: DataObject, *args, **kwargs) -> bool:
+ logger.info("[Action] example do: data -> %s, parent_data -> %s", data, parent_data)
+ return True
+
+ class Meta:
+ action_name = "example"
+
+
+class ForcedFailAction(BaseAction):
+
+ TIMEOUT_NODE_OPERATOR = "bamboo_engine"
+
+ def do(self, data: DataObject, parent_data: DataObject, *args, **kwargs) -> bool:
+ logger.info("[Action(bamboo_engine_forced_fail)] do: data -> %s, parent_data -> %s", data, parent_data)
+ result = bamboo_engine_api.forced_fail_activity(
+ runtime=BambooDjangoRuntime(),
+ node_id=self.node_id,
+ ex_data="forced fail by {}".format(self.TIMEOUT_NODE_OPERATOR),
+ send_post_set_state_signal=kwargs.get("send_post_set_state_signal", True),
+ )
+ return result.result
+
+ class Meta:
+ action_name = "bamboo_engine_forced_fail"
+
+
+class ForcedFailAndSkipAction(BaseAction):
+
+ TIMEOUT_NODE_OPERATOR = "bamboo_engine"
+
+ def do(self, data: DataObject, parent_data: DataObject, *args, **kwargs) -> bool:
+ logger.info("[Action(bamboo_engine_forced_fail_and_skip)] do: data -> %s, parent_data -> %s", data, parent_data)
+ result = bamboo_engine_api.forced_fail_activity(
+ runtime=BambooDjangoRuntime(),
+ node_id=self.node_id,
+ ex_data="forced fail by {}".format(self.TIMEOUT_NODE_OPERATOR),
+ send_post_set_state_signal=kwargs.get("send_post_set_state_signal", True),
+ )
+ if result.result:
+ result = bamboo_engine_api.skip_node(
+ runtime=BambooDjangoRuntime(),
+ node_id=self.node_id,
+ ex_data="forced skip by {}".format(self.TIMEOUT_NODE_OPERATOR),
+ send_post_set_state_signal=kwargs.get("send_post_set_state_signal", True),
+ )
+ return result.result
+
+ class Meta:
+ action_name = "bamboo_engine_forced_fail_and_skip"
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/management/__init__.py b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/management/__init__.py
new file mode 100644
index 00000000..40097292
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/management/__init__.py
@@ -0,0 +1,12 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017-2021 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/management/commands/__init__.py b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/management/commands/__init__.py
new file mode 100644
index 00000000..40097292
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/management/commands/__init__.py
@@ -0,0 +1,12 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017-2021 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/management/commands/start_node_timer_event_process.py b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/management/commands/start_node_timer_event_process.py
new file mode 100644
index 00000000..f02e3092
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/management/commands/start_node_timer_event_process.py
@@ -0,0 +1,74 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017-2021 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+import datetime
+import json
+import logging
+import signal
+import time
+
+from django.core.management import BaseCommand
+from django.db import connections
+from pipeline.contrib.node_timer_event.models import ExpiredNodesRecord
+from pipeline.contrib.node_timer_event.settings import node_timer_event_settings
+from pipeline.contrib.node_timer_event.tasks import dispatch_expired_nodes
+
+logger = logging.getLogger("root")
+
+
+class Command(BaseCommand):
+ help = "scanning expired nodes and dispatch them to celery task"
+ has_killed = False
+
+ def handle(self, *args, **options):
+ signal.signal(signal.SIGTERM, self._graceful_exit)
+ redis_inst = node_timer_event_settings.redis_inst
+ nodes_pool = node_timer_event_settings.executing_pool
+ while not self.has_killed:
+ try:
+ start = time.time()
+ self._pop_expired_nodes(redis_inst, nodes_pool)
+ end = time.time()
+ logger.info(f"[node_timeout_process] time consuming: {end-start}")
+ except Exception as e:
+ logger.exception(e)
+ time.sleep(node_timer_event_settings.pool_scan_interval)
+
+ def _graceful_exit(self, *args):
+ self.has_killed = True
+
+ def _pop_expired_nodes(self, redis_inst, nodes_pool) -> list:
+ now = datetime.datetime.now().timestamp()
+ expired_nodes = [
+ node.decode("utf-8") if isinstance(node, bytes) else node
+ for node in redis_inst.zrangebyscore(nodes_pool, "-inf", now)
+ ]
+ if expired_nodes:
+ self.record_expired_nodes(expired_nodes)
+ redis_inst.zrem(nodes_pool, *expired_nodes)
+ return expired_nodes
+
+ @staticmethod
+ def record_expired_nodes(timeout_nodes: list):
+ # 处理因为过长时间没有访问导致的链接失效问题
+ for conn in connections.all():
+ conn.close_if_unusable_or_obsolete()
+
+ record = ExpiredNodesRecord.objects.create(nodes=json.dumps(timeout_nodes))
+ if node_timer_event_settings.dispatch_queue is None:
+ dispatch_expired_nodes.apply_async(kwargs={"record_id": record.id})
+ else:
+ dispatch_expired_nodes.apply_async(
+ kwargs={"record_id": record.id},
+ queue=node_timer_event_settings.dispatch_queue,
+ routing_key=node_timer_event_settings.dispatch_queue,
+ )
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/migrations/0001_initial.py b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/migrations/0001_initial.py
new file mode 100644
index 00000000..381aea45
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/migrations/0001_initial.py
@@ -0,0 +1,40 @@
+# Generated by Django 3.2.18 on 2023-10-25 02:06
+
+from django.db import migrations, models
+
+
+class Migration(migrations.Migration):
+
+ initial = True
+
+ dependencies = []
+
+ operations = [
+ migrations.CreateModel(
+ name="ExpiredNodesRecord",
+ fields=[
+ ("id", models.BigAutoField(primary_key=True, serialize=False, verbose_name="ID")),
+ ("nodes", models.TextField(verbose_name="到期节点信息")),
+ ],
+ options={
+ "verbose_name": "到期节点数据记录 ExpiredNodesRecord",
+ "verbose_name_plural": "到期节点数据记录 ExpiredNodesRecord",
+ },
+ ),
+ migrations.CreateModel(
+ name="NodeTimerEventConfig",
+ fields=[
+ ("root_pipeline_id", models.CharField(max_length=64, verbose_name="root pipeline id")),
+ (
+ "node_id",
+ models.CharField(max_length=64, primary_key=True, serialize=False, verbose_name="task node id"),
+ ),
+ ("events", models.TextField(default="[]", verbose_name="timer events")),
+ ],
+ options={
+ "verbose_name": "节点时间事件配置 NodeTimerEventConfig",
+ "verbose_name_plural": "节点时间事件配置 NodeTimerEventConfig",
+ "index_together": {("root_pipeline_id", "node_id")},
+ },
+ ),
+ ]
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/migrations/__init__.py b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/migrations/__init__.py
new file mode 100644
index 00000000..40097292
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/migrations/__init__.py
@@ -0,0 +1,12 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017-2021 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/models.py b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/models.py
new file mode 100644
index 00000000..0075ee9f
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/models.py
@@ -0,0 +1,128 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+import json
+import logging
+import re
+import typing
+from typing import Any, Dict, List, Optional
+
+from django.db import models
+from django.utils.translation import ugettext_lazy as _
+from pipeline.contrib.node_timer_event.types import TimeDefined
+from pipeline.contrib.node_timer_event.utils import parse_timer_defined
+from pipeline.core.constants import PE
+
+logger = logging.getLogger(__name__)
+
+
+EVENT_KEY_PATTERN = re.compile(r".*node:(?P.+):version:(?P.+):index:(?P\d+)")
+
+
+class NodeTimerEventConfigManager(models.Manager):
+ def parse_node_timer_event_configs(self, pipeline_tree: Dict[str, Any]) -> Dict[str, Any]:
+ """解析事件时间"""
+ configs: List[Dict[str, Any]] = []
+ for act_id, act in pipeline_tree[PE.activities].items():
+ if act["type"] == PE.SubProcess:
+ result = self.parse_node_timer_event_configs(act[PE.pipeline])
+ if not result["result"]:
+ return result
+ configs.extend(result["data"])
+ elif act["type"] == PE.ServiceActivity:
+ index: int = 1
+ treated_timer_events: List[Dict[str, Any]] = []
+ timer_events: List[Dict[str, Any]] = (act.get("events") or {}).get("timer_events") or []
+ for timer_event in timer_events:
+ enable: bool = timer_event.get("enable") or False
+ if not enable:
+ continue
+
+ timer_type: Optional[str] = timer_event.get("timer_type")
+ defined: Optional[str] = timer_event.get("defined")
+
+ try:
+ timer_defined: TimeDefined = parse_timer_defined(timer_type, defined)
+ except Exception:
+ # 对于不符合格式要求的情况,记录日志并跳过
+ logger.exception(
+ "[parse_node_timer_event_configs] parse timer_defined failed: "
+ "node_id -> %s, timer_type -> %s, defined -> %s",
+ act_id,
+ timer_type,
+ defined,
+ )
+ continue
+
+ treated_timer_events.append(
+ {
+ "index": index,
+ "action": timer_event.get("action"),
+ "timer_type": timer_type,
+ "repetitions": timer_defined["repetitions"],
+ "defined": defined,
+ }
+ )
+
+ index += 1
+
+ if treated_timer_events:
+ configs.append({"node_id": act_id, "events": treated_timer_events})
+
+ return {"result": True, "data": configs, "message": ""}
+
+ def batch_create_node_timer_event_config(self, root_pipeline_id: str, pipeline_tree: dict):
+ """批量创建节点超时配置"""
+
+ config_parse_result: Dict[str, Any] = self.parse_node_timer_event_configs(pipeline_tree)
+ # 这里忽略解析失败的情况,保证即使解析失败也能正常创建任务
+ if not config_parse_result["result"]:
+ logger.error(
+ f"[batch_create_node_timer_event_config] parse node timer event config "
+ f'failed: {config_parse_result["result"]}'
+ )
+ return config_parse_result
+
+ configs: List[Dict[str, Any]] = config_parse_result["data"]
+ config_objs: typing.List[NodeTimerEventConfig] = [
+ NodeTimerEventConfig(
+ root_pipeline_id=root_pipeline_id, node_id=config["node_id"], events=json.dumps(config["events"])
+ )
+ for config in configs
+ ]
+ objs = self.bulk_create(config_objs, batch_size=1000)
+ return {"result": True, "data": objs, "message": ""}
+
+
+class NodeTimerEventConfig(models.Model):
+ root_pipeline_id = models.CharField(verbose_name="root pipeline id", max_length=64)
+ node_id = models.CharField(verbose_name="task node id", max_length=64, primary_key=True)
+ events = models.TextField(verbose_name="timer events", default="[]")
+
+ objects = NodeTimerEventConfigManager()
+
+ class Meta:
+ verbose_name = _("节点时间事件配置 NodeTimerEventConfig")
+ verbose_name_plural = _("节点时间事件配置 NodeTimerEventConfig")
+ index_together = [("root_pipeline_id", "node_id")]
+
+ def get_events(self) -> List[Dict[str, Any]]:
+ return json.loads(self.events)
+
+
+class ExpiredNodesRecord(models.Model):
+ id = models.BigAutoField(verbose_name="ID", primary_key=True)
+ nodes = models.TextField(verbose_name="到期节点信息")
+
+ class Meta:
+ verbose_name = _("到期节点数据记录 ExpiredNodesRecord")
+ verbose_name_plural = _("到期节点数据记录 ExpiredNodesRecord")
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/settings.py b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/settings.py
new file mode 100644
index 00000000..4eac23ff
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/settings.py
@@ -0,0 +1,71 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017-2021 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+
+from importlib import import_module
+
+from django.conf import settings
+from pipeline.contrib.node_timer_event import types
+
+
+def get_import_path(cls: types.T) -> str:
+ return f"{cls.__module__}.{cls.__name__}"
+
+
+def import_string(dotted_path: str):
+ """
+ Import a dotted module path and return the attribute/class designated by the
+ last name in the path. Raise ImportError if the import failed.
+ """
+ try:
+ module_path, class_name = dotted_path.rsplit(".", 1)
+ except ValueError as err:
+ raise ImportError(f"{dotted_path} doesn't look like a module path") from err
+
+ module = import_module(module_path)
+
+ try:
+ return getattr(module, class_name)
+ except AttributeError as err:
+ raise ImportError(f'Module "{module_path}" does not define a "{class_name}" attribute/class') from err
+
+
+class NodeTimerEventSettngs:
+ PREFIX = "PIPELINE_NODE_TIMER_EVENT"
+ DEFAULT_SETTINGS = {
+ # # Redis key 前缀,用于记录正在执行的节点,命名示例: {app_code}:{app_env}:{module}:node_timer_event
+ # v1 表示 node_timer_event 的版本,预留以隔离
+ "key_prefix": "bamboo:v1:node_timer_event",
+ # 节点计时器边界事件处理队列名称, 用于处理计时器边界事件, 需要 worker 接收该队列消息,默认为 None,即使用 celery 默认队列
+ "dispatch_queue": None,
+ # 节点计时器边界事件分发队列名称, 用于记录计时器边界事件, 需要 worker 接收该队列消息,默认为 None,即使用 celery 默认队列
+ "handle_queue": None,
+ # 执行节点池名称,用于记录正在执行的节点,需要保证 Redis key 唯一,命名示例: {app_code}:{app_env}:{module}:executing_node_pool
+ "executing_pool": "bamboo:v1:node_timer_event:executing_node_pool",
+ # 节点池扫描间隔,间隔越小,边界事件触发时间越精准,相应的事件处理的 workload 负载也会提升
+ "pool_scan_interval": 1,
+ # 最长过期时间,兜底删除 Redis 冗余数据,默认为 15 Days,请根据业务场景调整
+ "max_expire_time": 60 * 60 * 24 * 15,
+ # 边界事件处理适配器,默认为 `pipeline.contrib.node_timer_event.adapter.NodeTimerEventAdapter`
+ "adapter_class": "pipeline.contrib.node_timer_event.adapter.NodeTimerEventAdapter",
+ }
+
+ def __getattr__(self, item: str):
+ if item == "redis_inst":
+ return settings.redis_inst
+ if item == "adapter_class":
+ return import_string(getattr(settings, f"{self.PREFIX}_{item.upper()}", self.DEFAULT_SETTINGS.get(item)))
+
+ return getattr(settings, f"{self.PREFIX}_{item.upper()}", self.DEFAULT_SETTINGS.get(item))
+
+
+node_timer_event_settings = NodeTimerEventSettngs()
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/signals/__init__.py b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/signals/__init__.py
new file mode 100644
index 00000000..26a6d1c2
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/signals/__init__.py
@@ -0,0 +1,12 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/signals/handlers.py b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/signals/handlers.py
new file mode 100644
index 00000000..4b3c4d56
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/signals/handlers.py
@@ -0,0 +1,80 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+import logging
+from typing import List, Optional, Type
+
+from django.dispatch import receiver
+from pipeline.contrib.node_timer_event.adapter import NodeTimerEventBaseAdapter
+from pipeline.contrib.node_timer_event.settings import node_timer_event_settings
+from pipeline.eri.signals import post_set_state
+
+from bamboo_engine import states as bamboo_engine_states
+
+logger = logging.getLogger(__name__)
+
+
+def _node_timer_event_info_update(redis_inst, to_state: str, node_id: str, version: str):
+
+ adapter: Optional[NodeTimerEventBaseAdapter] = None
+
+ if to_state in [
+ bamboo_engine_states.RUNNING,
+ bamboo_engine_states.FAILED,
+ bamboo_engine_states.FINISHED,
+ bamboo_engine_states.SUSPENDED,
+ ]:
+
+ adapter_class: Type[NodeTimerEventBaseAdapter] = node_timer_event_settings.adapter_class
+ adapter: NodeTimerEventBaseAdapter = adapter_class(node_id=node_id, version=version)
+
+ if not adapter.is_ready():
+ logger.info(
+ "[node_timer_event_info_update] node_timer_event_config not exist and skipped: "
+ "node_id -> %s, version -> %s",
+ node_id,
+ version,
+ )
+ return
+
+ logger.info(
+ "[node_timer_event_info_update] load node_timer_event_config: node_id -> %s, version -> %s, events -> %s",
+ node_id,
+ version,
+ adapter.events,
+ )
+
+ if to_state == bamboo_engine_states.RUNNING:
+ # 遍历节点时间事件,丢进待调度节点池
+ for event in adapter.events:
+ adapter.add_to_pool(redis_inst, event)
+
+ elif to_state in [bamboo_engine_states.FAILED, bamboo_engine_states.FINISHED, bamboo_engine_states.SUSPENDED]:
+ keys: List[str] = adapter.fetch_keys_to_be_rem()
+ redis_inst.zrem(node_timer_event_settings.executing_pool, *keys)
+ redis_inst.delete(*keys)
+ logger.info(
+ "[node_timer_event_info_update] removed events from redis: "
+ "node_id -> %s, version -> %s, events -> %s, keys -> %s",
+ node_id,
+ version,
+ adapter.events,
+ keys,
+ )
+
+
+@receiver(post_set_state)
+def bamboo_engine_eri_node_state_handler(sender, node_id, to_state, version, root_id, parent_id, loop, **kwargs):
+ try:
+ _node_timer_event_info_update(node_timer_event_settings.redis_inst, to_state, node_id, version)
+ except Exception:
+ logger.exception("node_timeout_info_update error")
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/tasks.py b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/tasks.py
new file mode 100644
index 00000000..44970a5a
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/tasks.py
@@ -0,0 +1,110 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017-2021 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+import json
+import logging
+from typing import Any, Dict, List, Type, Union
+
+from celery import task
+from pipeline.contrib.node_timer_event.adapter import NodeTimerEventBaseAdapter
+from pipeline.contrib.node_timer_event.handlers import ActionManager
+from pipeline.contrib.node_timer_event.models import ExpiredNodesRecord
+from pipeline.contrib.node_timer_event.settings import node_timer_event_settings
+from pipeline.eri.models import Process, State
+
+logger = logging.getLogger("celery")
+
+
+@task(acks_late=True)
+def dispatch_expired_nodes(record_id: int):
+ record: ExpiredNodesRecord = ExpiredNodesRecord.objects.get(id=record_id)
+ node_keys: List[str] = json.loads(record.nodes)
+ logger.info("[dispatch_expired_nodes] record -> %s, nodes -> %s", record_id, node_keys)
+
+ adapter_class: Type[NodeTimerEventBaseAdapter] = node_timer_event_settings.adapter_class
+
+ for node_key in node_keys:
+ try:
+ key_info: Dict[str, Union[str, int]] = adapter_class.parse_event_key(node_key)
+ except ValueError:
+ logger.warning(
+ "[dispatch_expired_nodes] failed to parse key, skipped: record -> %s, node_key -> %s",
+ record_id,
+ node_key,
+ )
+ continue
+
+ index: int = key_info["index"]
+ node_id: str = key_info["node_id"]
+ version: str = key_info["version"]
+
+ if node_timer_event_settings.handle_queue is None:
+ execute_node_timer_event_action.apply_async(kwargs={"node_id": node_id, "version": version, "index": index})
+ else:
+ execute_node_timer_event_action.apply_async(
+ kwargs={"node_id": node_id, "version": version, "index": index},
+ queue=node_timer_event_settings.handle_queue,
+ routing_key=node_timer_event_settings.handle_queue,
+ )
+
+ logger.info("[dispatch_expired_nodes] dispatch finished: record -> %s, nodes -> %s", record_id, node_keys)
+ # 删除临时记录
+ record.delete()
+ logger.info("[dispatch_expired_nodes] record deleted: record -> %s", record_id)
+
+
+@task(ignore_result=True)
+def execute_node_timer_event_action(node_id: str, version: str, index: int):
+
+ adapter_class: Type[NodeTimerEventBaseAdapter] = node_timer_event_settings.adapter_class
+ adapter: NodeTimerEventBaseAdapter = adapter_class(node_id=node_id, version=version)
+ if not adapter.is_ready() or (adapter.index__event_map and index not in adapter.index__event_map):
+ message: str = (
+ f"[execute_node_timer_event_action] no timer config: "
+ f"node_id -> {node_id}, version -> {version}, index -> {index}"
+ )
+ logger.exception(message)
+ return {"result": False, "message": message, "data": None}
+
+ event: Dict[str, Any] = adapter.index__event_map[index]
+
+ # 判断当前节点是否符合策略执行要求
+ is_process_current_node: bool = Process.objects.filter(
+ root_pipeline_id=adapter.root_pipeline_id, current_node_id=node_id
+ ).exists()
+ is_node_match = State.objects.filter(node_id=node_id, version=version).exists()
+ if not (is_node_match and is_process_current_node):
+ message = (
+ f"[execute_node_timer_event_action] node {node_id} with version {version} "
+ f"in pipeline {adapter.root_pipeline_id} has been passed."
+ )
+ logger.error(message)
+ return {"result": False, "message": message, "data": None}
+
+ # 计算事件下一次触发事件并丢进待调度节点池
+ adapter.add_to_pool(node_timer_event_settings.redis_inst, event)
+
+ try:
+ is_success: bool = ActionManager.get_action(
+ adapter.root_pipeline_id, node_id, version, event["action"]
+ ).notify()
+ logger.info(
+ f"[execute_node_timer_event_action] node {node_id} with version {version} in pipeline "
+ f"{adapter.root_pipeline_id} action result is: {is_success}."
+ )
+ return {"result": is_success, "data": None}
+ except Exception as e:
+ logger.exception(
+ f"[execute_node_timer_event_action] node {node_id} with version {version} in pipeline "
+ f"{adapter.root_pipeline_id} error is: {e}."
+ )
+ return {"result": False, "data": None, "message": str(e)}
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/types.py b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/types.py
new file mode 100644
index 00000000..092524ff
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/types.py
@@ -0,0 +1,25 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+
+
+import typing
+
+T = typing.TypeVar("T")
+
+TimeDefined = typing.Dict[str, typing.Any]
+
+
+TimerEvent = typing.Dict[str, typing.Any]
+
+
+TimerEvents = typing.List[TimerEvent]
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/utils.py b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/utils.py
new file mode 100644
index 00000000..31c4c3e2
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/node_timer_event/utils.py
@@ -0,0 +1,64 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+
+
+import datetime
+import re
+from typing import Any, Callable, Dict, Optional
+
+import isodate
+from pipeline.contrib.node_timer_event.constants import TimerType
+from pipeline.contrib.node_timer_event.types import TimeDefined
+
+TIME_CYCLE_DEFINED_PATTERN = re.compile(r"(^R\d+)/")
+
+
+def handle_result(timestamp: float, repetitions: int = 1, *args, **kwargs) -> TimeDefined:
+ return {"timestamp": timestamp, "repetitions": repetitions}
+
+
+def handle_timedate(defined: str, start: Optional[datetime.datetime] = None, *args, **kwargs) -> TimeDefined:
+ return handle_result(timestamp=isodate.parse_datetime(defined).timestamp())
+
+
+def handle_time_duration(defined: str, start: Optional[datetime.datetime] = None, *args, **kwargs) -> TimeDefined:
+ start = start or datetime.datetime.now()
+ return handle_result(timestamp=(start + isodate.parse_duration(defined)).timestamp())
+
+
+def handle_time_cycle(defined: str, start: Optional[datetime.datetime] = None, *args, **kwargs) -> TimeDefined:
+ repeat_match = TIME_CYCLE_DEFINED_PATTERN.match(defined)
+ if repeat_match:
+ repetitions: int = int(repeat_match.group(1)[1:])
+ duration_string = TIME_CYCLE_DEFINED_PATTERN.sub("", defined)
+ else:
+ repetitions: int = 1
+ duration_string = defined
+
+ return handle_result(timestamp=handle_time_duration(duration_string, start)["timestamp"], repetitions=repetitions)
+
+
+TIMER_TYPE_ROUTES: Dict[str, Callable[[str, Optional[datetime.datetime], Any, Any], TimeDefined]] = {
+ TimerType.TIME_DURATION.value: handle_time_duration,
+ TimerType.TIME_CYCLE.value: handle_time_cycle,
+ TimerType.TIME_DATE.value: handle_timedate,
+}
+
+
+def parse_timer_defined(
+ timer_type: str, defined: str, start: Optional[datetime.datetime] = None, *args, **kwargs
+) -> TimeDefined:
+ if timer_type not in TIMER_TYPE_ROUTES:
+ raise ValueError(f"Unsupported timer_type -> {timer_type}")
+
+ return TIMER_TYPE_ROUTES[timer_type](defined, start, *args, **kwargs)
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/__init__.py b/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/__init__.py
new file mode 100644
index 00000000..26a6d1c2
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/__init__.py
@@ -0,0 +1,12 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/api.py b/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/api.py
new file mode 100644
index 00000000..1a03fc63
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/api.py
@@ -0,0 +1,41 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+
+from pipeline.contrib.plugin_execute.handler import PluginExecuteHandler
+from pipeline.contrib.utils import ensure_return_pipeline_contrib_api_result
+
+
+@ensure_return_pipeline_contrib_api_result
+def run(component_code: str, version: str, inputs: dict, contexts: dict, runtime_attr: dict = None):
+ task_id = PluginExecuteHandler.run(component_code, version, inputs, contexts, runtime_attr)
+ return task_id
+
+
+@ensure_return_pipeline_contrib_api_result
+def get_state(task_id: int):
+ return PluginExecuteHandler.get_state(task_id)
+
+
+@ensure_return_pipeline_contrib_api_result
+def callback(task_id: int, callback_data: dict = None):
+ PluginExecuteHandler.callback(task_id, callback_data)
+
+
+@ensure_return_pipeline_contrib_api_result
+def forced_fail(task_id):
+ PluginExecuteHandler.forced_fail(task_id)
+
+
+@ensure_return_pipeline_contrib_api_result
+def retry(task_id: int, inputs: dict = None, context: dict = None, runtime_attr: dict = None):
+ PluginExecuteHandler.retry_node(task_id, inputs, context, runtime_attr)
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/apps.py b/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/apps.py
new file mode 100644
index 00000000..8bbefc32
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/apps.py
@@ -0,0 +1,21 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+from django.apps import AppConfig
+
+
+class RollbackConfig(AppConfig):
+ name = "pipeline.contrib.plugin_execute"
+ verbose_name = "PipelinePluginExecute"
+
+ def ready(self):
+ from pipeline.contrib.plugin_execute import tasks # noqa
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/contants.py b/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/contants.py
new file mode 100644
index 00000000..008c9397
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/contants.py
@@ -0,0 +1,19 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+
+
+class State:
+ READY = "READY"
+ RUNNING = "RUNNING"
+ FINISHED = "FINISHED"
+ FAILED = "FAILED"
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/handler.py b/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/handler.py
new file mode 100644
index 00000000..7eaae4d9
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/handler.py
@@ -0,0 +1,169 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+
+import logging
+
+from pipeline.conf.default_settings import PLUGIN_EXECUTE_QUEUE
+from pipeline.contrib.exceptions import PluginExecuteException
+from pipeline.contrib.plugin_execute.contants import State
+from pipeline.contrib.plugin_execute.models import PluginExecuteTask, get_schedule_lock
+from pipeline.contrib.plugin_execute.tasks import execute, schedule
+
+logger = logging.getLogger("celery")
+
+
+def _retry_once(action: callable):
+ try:
+ action()
+ except Exception:
+ try:
+ action()
+ except Exception as e:
+ raise e
+
+
+class PluginExecuteHandler:
+ @classmethod
+ def run(cls, component_code: str, version: str, inputs: dict, contexts: dict, runtime_attrs: dict = None):
+ if runtime_attrs is None:
+ runtime_attrs = {}
+
+ if not (isinstance(inputs, dict) and isinstance(contexts, dict) and isinstance(runtime_attrs, dict)):
+ raise PluginExecuteException("[plugin_execute_run] error, the inputs, contexts, runtime_attrs must be dict")
+ plugin_execute_task = PluginExecuteTask.objects.create(
+ state=State.READY,
+ inputs=inputs,
+ version=version,
+ component_code=component_code,
+ contexts=contexts,
+ runtime_attrs=runtime_attrs,
+ )
+
+ def action():
+ # 发送执行任务
+ execute.apply_async(
+ kwargs={"task_id": plugin_execute_task.id}, queue=PLUGIN_EXECUTE_QUEUE, ignore_result=True
+ )
+ logger.info(
+ "[plugin_execute_run] send execute task, plugin_execute_task_id = {}".format(plugin_execute_task.id)
+ )
+
+ try:
+ _retry_once(action=action)
+ except Exception as e:
+ # 如果任务启动出现异常,则删除任务
+ plugin_execute_task.delete()
+ raise e
+
+ return plugin_execute_task.id
+
+ @classmethod
+ def get_state(cls, task_id):
+ """
+ 获取任务状态
+ @param task_id:
+ @return:
+ """
+ # 直接抛出异常让上层去捕获
+ plugin_execute_task = PluginExecuteTask.objects.get(id=task_id)
+ return {
+ "task_id": plugin_execute_task.id,
+ "state": plugin_execute_task.state,
+ "component_code": plugin_execute_task.component_code,
+ "version": plugin_execute_task.version,
+ "invoke_count": plugin_execute_task.invoke_count,
+ "inputs": plugin_execute_task.inputs,
+ "outputs": plugin_execute_task.outputs,
+ "contexts": plugin_execute_task.contexts,
+ "runtime_attrs": plugin_execute_task.runtime_attrs,
+ "create_at": plugin_execute_task.created_at,
+ "finish_at": plugin_execute_task.finish_at,
+ }
+
+ @classmethod
+ def forced_fail(cls, task_id):
+ plugin_execute_task = PluginExecuteTask.objects.get(id=task_id)
+ if plugin_execute_task.state != State.RUNNING:
+ raise PluginExecuteException(
+ "[forced_fail] error, the plugin_execute_task.state is not RUNNING, state={}".format(
+ plugin_execute_task.state
+ )
+ )
+ # 插件状态改成 FAILED, 在schdule会自动停止
+ plugin_execute_task.state = State.FAILED
+ plugin_execute_task.save()
+
+ @classmethod
+ def callback(cls, task_id: int, callback_data: dict = None):
+
+ if callback_data is None:
+ callback_data = {}
+
+ if not isinstance(callback_data, dict):
+ raise PluginExecuteException("[plugin_execute_callback] error, the callback must be dict")
+
+ plugin_execute_task = PluginExecuteTask.objects.get(id=task_id)
+ if plugin_execute_task.state != State.RUNNING:
+ raise PluginExecuteException(
+ "[callback] error, the plugin_execute_task.state is not RUNNING, state={}".format(
+ plugin_execute_task.state
+ )
+ )
+
+ def action():
+ # 需要加锁,防止流程处在回调的过程中
+ with get_schedule_lock(task_id) as locked:
+ if not locked:
+ raise PluginExecuteException("[plugin_execute_callback] error, it`s have callback task, please try")
+ plugin_execute_task.callback_data = callback_data
+ plugin_execute_task.save()
+ schedule.apply_async(
+ kwargs={"task_id": plugin_execute_task.id}, queue=PLUGIN_EXECUTE_QUEUE, ignore_result=True
+ )
+ logger.info("[plugin_execute_callback] send callback task, plugin_execute_task_id = {}".format(task_id))
+
+ _retry_once(action=action)
+
+ @classmethod
+ def retry_node(cls, task_id: int, inputs: dict = None, contexts: dict = None, runtime_attrs: dict = None):
+
+ plugin_execute_task = PluginExecuteTask.objects.get(id=task_id)
+ if plugin_execute_task.state != State.FAILED:
+ raise PluginExecuteException(
+ "[retry_node] error, the plugin_execute_task.state is not FAILED, state={}".format(
+ plugin_execute_task.state
+ )
+ )
+
+ if contexts and isinstance(contexts, dict):
+ plugin_execute_task.contexts = contexts
+ if inputs and isinstance(inputs, dict):
+ plugin_execute_task.inputs = inputs
+ if runtime_attrs and isinstance(runtime_attrs, dict):
+ plugin_execute_task.runtime_attrs = runtime_attrs
+
+ plugin_execute_task.state = State.READY
+ plugin_execute_task.inputs = inputs
+ plugin_execute_task.invoke_count += 1
+ # 清空输出和callback_data
+ plugin_execute_task.outputs = {}
+ plugin_execute_task.callback_data = {}
+ plugin_execute_task.save()
+
+ def action():
+ execute.apply_async(kwargs={"task_id": plugin_execute_task.id}, queue=PLUGIN_EXECUTE_QUEUE)
+ logger.info(
+ "[plugin_execute_retry_node] send retry_node task, plugin_execute_task_id = {}".format(task_id)
+ )
+
+ _retry_once(action=action)
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/migrations/0001_initial.py b/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/migrations/0001_initial.py
new file mode 100644
index 00000000..969e3ae8
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/migrations/0001_initial.py
@@ -0,0 +1,38 @@
+# Generated by Django 3.2.18 on 2023-11-28 11:49
+
+import pipeline.contrib.fields
+from django.db import migrations, models
+
+
+class Migration(migrations.Migration):
+ initial = True
+
+ dependencies = []
+
+ operations = [
+ migrations.CreateModel(
+ name="PluginExecuteTask",
+ fields=[
+ ("id", models.AutoField(auto_created=True, primary_key=True, serialize=False, verbose_name="ID")),
+ ("state", models.CharField(max_length=64, verbose_name="状态名")),
+ ("invoke_count", models.IntegerField(default=1, verbose_name="invoke count")),
+ ("component_code", models.CharField(db_index=True, max_length=255, verbose_name="组件编码")),
+ ("version", models.CharField(default="legacy", max_length=255, verbose_name="插件版本")),
+ ("inputs", pipeline.contrib.fields.SerializerField(default={}, verbose_name="node inputs")),
+ ("outputs", pipeline.contrib.fields.SerializerField(default={}, verbose_name="node outputs")),
+ ("callback_data", pipeline.contrib.fields.SerializerField(default={}, verbose_name="callback data")),
+ (
+ "contexts",
+ pipeline.contrib.fields.SerializerField(default={}, verbose_name="pipeline context values"),
+ ),
+ ("runtime_attrs", pipeline.contrib.fields.SerializerField(default={}, verbose_name="runtime attr")),
+ ("scheduling", models.BooleanField(db_index=True, default=False, verbose_name="是否正在调度")),
+ ("created_at", models.DateTimeField(auto_now_add=True, verbose_name="create time")),
+ ("finish_at", models.DateTimeField(null=True, verbose_name="finish time")),
+ ],
+ ),
+ migrations.AddIndex(
+ model_name="pluginexecutetask",
+ index=models.Index(fields=["id", "scheduling"], name="plugin_exec_id_e7c7b2_idx"),
+ ),
+ ]
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/migrations/__init__.py b/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/migrations/__init__.py
new file mode 100644
index 00000000..26a6d1c2
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/migrations/__init__.py
@@ -0,0 +1,12 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/models.py b/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/models.py
new file mode 100644
index 00000000..d7bd1353
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/models.py
@@ -0,0 +1,81 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+
+from django.db import models
+from django.utils.translation import ugettext_lazy as _
+from pipeline.contrib.fields import SerializerField
+
+
+class ScheduleManger(models.Manager):
+ def apply_schedule_lock(self, task_id: int) -> bool:
+ """
+ 获取 Schedule 对象的调度锁,返回是否成功获取锁
+
+ :return: True or False
+ """
+ return self.filter(id=task_id, scheduling=False).update(scheduling=True) == 1
+
+ def release_schedule_lock(self, task_id: int) -> None:
+ """
+ 释放指定 Schedule 的调度锁
+ :return:
+ """
+ self.filter(id=task_id, scheduling=True).update(scheduling=False)
+
+
+class ScheduleLock(object):
+ def __init__(self, task_id: int):
+ self.task_id = task_id
+ self.locked = False
+
+ def __enter__(self):
+ self.locked = PluginExecuteTask.objects.apply_schedule_lock(self.task_id)
+
+ return self
+
+ def __exit__(self, exc_type, exc_val, exc_tb):
+ if self.locked:
+ PluginExecuteTask.objects.release_schedule_lock(self.task_id)
+
+
+def get_schedule_lock(task_id: int) -> ScheduleLock:
+ """
+ 获取 schedule lock 的 context 对象
+ :param task_id:
+ :return:
+ """
+ return ScheduleLock(task_id)
+
+
+class PluginExecuteTask(models.Model):
+ """
+ 单节点执行任务
+ """
+
+ state = models.CharField(_("状态名"), null=False, max_length=64)
+ invoke_count = models.IntegerField("invoke count", default=1)
+ component_code = models.CharField(_("组件编码"), max_length=255, db_index=True)
+ version = models.CharField(_("插件版本"), max_length=255, default="legacy")
+ inputs = SerializerField(verbose_name=_("node inputs"), default={})
+ outputs = SerializerField(verbose_name=_("node outputs"), default={})
+ callback_data = SerializerField(verbose_name=_("callback data"), default={})
+ contexts = SerializerField(verbose_name=_("pipeline context values"), default={})
+ runtime_attrs = SerializerField(verbose_name=_("runtime attr"), default={})
+ scheduling = models.BooleanField("是否正在调度", default=False, db_index=True)
+ created_at = models.DateTimeField("create time", auto_now_add=True)
+ finish_at = models.DateTimeField("finish time", null=True)
+
+ objects = ScheduleManger()
+
+ class Meta:
+ indexes = [models.Index(fields=["id", "scheduling"])]
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/tasks.py b/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/tasks.py
new file mode 100644
index 00000000..b1898431
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/plugin_execute/tasks.py
@@ -0,0 +1,151 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+
+import logging
+import traceback
+
+from celery import task
+from django.utils import timezone
+from pipeline.component_framework.library import ComponentLibrary
+from pipeline.conf.default_settings import PLUGIN_EXECUTE_QUEUE
+from pipeline.contrib.plugin_execute.contants import State
+from pipeline.contrib.plugin_execute.models import PluginExecuteTask
+from pipeline.core.data.base import DataObject
+
+logger = logging.getLogger("celery")
+
+
+@task
+def execute(task_id):
+ try:
+ plugin_execute_task = PluginExecuteTask.objects.get(id=task_id)
+ except PluginExecuteTask.DoesNotExist:
+ logger.exception("[plugin_execute] execute error, task not exist, task_id={}".format(task_id))
+ return
+
+ # 更新插件的状态
+ plugin_execute_task.state = State.RUNNING
+ plugin_execute_task.save(update_fields=["state"])
+
+ # 封装data
+ data = DataObject(inputs=plugin_execute_task.inputs, outputs={})
+ parent_data = DataObject(inputs=plugin_execute_task.contexts, outputs={})
+
+ try:
+ # 获取 component
+ comp_cls = ComponentLibrary.get_component_class(plugin_execute_task.component_code, plugin_execute_task.version)
+ # 获取service
+ service = comp_cls.bound_service(name=plugin_execute_task.runtime_attrs.get("name", None))
+
+ # 封装运行时
+ service.setup_runtime_attrs(**plugin_execute_task.runtime_attrs, logger=logger)
+ execute_success = service.execute(data, parent_data)
+ # 在 pipeline 中,如果插件返回为None,则表示成功
+ if execute_success is None:
+ execute_success = True
+ plugin_execute_task.outputs = data.outputs
+ plugin_execute_task.save()
+ except Exception as e:
+ # 处理异常情况
+ ex_data = traceback.format_exc()
+ data.outputs.ex_data = ex_data
+ logger.exception("[plugin_execute] plugin execute failed, err={}".format(e))
+ plugin_execute_task.outputs = data.outputs
+ plugin_execute_task.state = State.FAILED
+ plugin_execute_task.save()
+ return
+
+ # 单纯的执行失败, 更新状态和输出信息
+ if not execute_success:
+ plugin_execute_task.state = State.FAILED
+ plugin_execute_task.save()
+ return
+
+ # 执行成功, 需要判断是否需要调度
+ need_schedule = service.need_schedule()
+ if not need_schedule:
+ plugin_execute_task.state = State.FINISHED
+ plugin_execute_task.finish_at = timezone.now()
+ plugin_execute_task.save()
+ return
+
+ # 需要调度,则调度自身
+ if service.interval:
+ schedule.apply_async(
+ kwargs={"task_id": task_id},
+ queue=PLUGIN_EXECUTE_QUEUE,
+ countdown=service.interval.next(),
+ ignore_result=True,
+ )
+
+
+@task
+def schedule(task_id):
+ try:
+ plugin_execute_task = PluginExecuteTask.objects.get(id=task_id)
+ except PluginExecuteTask.DoesNotExist:
+ logger.exception("[plugin_execute] schedule error, task not exist, task_id={}".format(task_id))
+ return
+
+ # 只有处于运行状态的节点才允许被调度
+ if plugin_execute_task.state != State.RUNNING:
+ logger.exception("[plugin_execute] schedule error, task not exist, task_id={}".format(task_id))
+ return
+
+ data = DataObject(inputs=plugin_execute_task.inputs, outputs=plugin_execute_task.outputs)
+ parent_data = DataObject(inputs=plugin_execute_task.contexts, outputs={})
+
+ try:
+ comp_cls = ComponentLibrary.get_component_class(plugin_execute_task.component_code, plugin_execute_task.version)
+ # 获取service
+ service = comp_cls.bound_service(name=plugin_execute_task.runtime_attrs.get("name", None))
+ # 封装运行时
+ service.setup_runtime_attrs(**plugin_execute_task.runtime_attrs, logger=logger)
+ schedule_success = service.schedule(
+ data=data, parent_data=parent_data, callback_data=plugin_execute_task.callback_data
+ )
+ # 在 pipeline 中,如果插件返回为None,则表示成功
+ if schedule_success is None:
+ schedule_success = True
+ plugin_execute_task.outputs = data.outputs
+ plugin_execute_task.save()
+ except Exception as e:
+ # 处理异常情况
+ ex_data = traceback.format_exc()
+ data.outputs.ex_data = ex_data
+ logger.exception("[plugin_execute] plugin execute failed, err={}".format(e))
+ plugin_execute_task.outputs = data.outputs
+ plugin_execute_task.state = State.FAILED
+ plugin_execute_task.save()
+ return
+
+ if not schedule_success:
+ plugin_execute_task.state = State.FAILED
+ plugin_execute_task.save()
+ return
+
+ if service.is_schedule_finished():
+ plugin_execute_task.state = State.FINISHED
+ plugin_execute_task.finish_at = timezone.now()
+ plugin_execute_task.save()
+ return
+
+ # 还需要下一次的调度
+ # 需要调度,则调度自身
+ if service.interval:
+ schedule.apply_async(
+ kwargs={"task_id": task_id},
+ queue=PLUGIN_EXECUTE_QUEUE,
+ countdown=service.interval.next(),
+ ignore_result=True,
+ )
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/rollback/api.py b/runtime/bamboo-pipeline/pipeline/contrib/rollback/api.py
index d123c1a7..f8deac76 100644
--- a/runtime/bamboo-pipeline/pipeline/contrib/rollback/api.py
+++ b/runtime/bamboo-pipeline/pipeline/contrib/rollback/api.py
@@ -10,23 +10,40 @@
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.
"""
-from pipeline.contrib.rollback.handler import RollBackHandler
+from pipeline.contrib.rollback.constants import TOKEN
+from pipeline.contrib.rollback.handler import RollbackDispatcher
from pipeline.contrib.utils import ensure_return_pipeline_contrib_api_result
@ensure_return_pipeline_contrib_api_result
-def rollback(root_pipeline_id: str, node_id: str):
+def rollback(
+ root_pipeline_id: str, start_node_id: str, target_node_id: str, skip_rollback_nodes: list = None, mode: str = TOKEN
+):
"""
:param root_pipeline_id: pipeline id
- :param node_id: 节点 id
+ :param start_node_id: 开始的 id
+ :param target_node_id: 开始的 id
+ :param mode 回滚模式
:return: True or False
-
- 回退的思路是,先搜索计算出来当前允许跳过的节点,在计算的过程中网关节点会合并成一个节点
- 只允许回退到已经执行过的节点
"""
- RollBackHandler(root_pipeline_id, node_id).rollback()
+ RollbackDispatcher(root_pipeline_id, mode).rollback(start_node_id, target_node_id)
+
+
+@ensure_return_pipeline_contrib_api_result
+def reserve_rollback(root_pipeline_id: str, start_node_id: str, target_node_id: str, mode: str = TOKEN):
+ RollbackDispatcher(root_pipeline_id, mode).reserve_rollback(start_node_id, target_node_id)
+
+
+@ensure_return_pipeline_contrib_api_result
+def cancel_reserved_rollback(root_pipeline_id: str, start_node_id: str, target_node_id: str, mode: str = TOKEN):
+ RollbackDispatcher(root_pipeline_id, mode).cancel_reserved_rollback(start_node_id, target_node_id)
+
+
+@ensure_return_pipeline_contrib_api_result
+def retry_rollback_failed_node(root_pipeline_id: str, node_id: str, retry_data: dict = None, mode: str = TOKEN):
+ RollbackDispatcher(root_pipeline_id, mode).retry_rollback_failed_node(node_id, retry_data)
@ensure_return_pipeline_contrib_api_result
-def get_allowed_rollback_node_id_list(root_pipeline_id: str):
- return RollBackHandler(root_pipeline_id, None).get_allowed_rollback_node_id_list()
+def get_allowed_rollback_node_id_list(root_pipeline_id: str, start_node_id: str, mode: str = TOKEN):
+ return RollbackDispatcher(root_pipeline_id, mode).get_allowed_rollback_node_id_list(start_node_id)
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/rollback/apps.py b/runtime/bamboo-pipeline/pipeline/contrib/rollback/apps.py
new file mode 100644
index 00000000..5072026b
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/rollback/apps.py
@@ -0,0 +1,21 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+from django.apps import AppConfig
+
+
+class RollbackConfig(AppConfig):
+ name = "pipeline.contrib.rollback"
+ verbose_name = "PipelineRollback"
+
+ def ready(self):
+ pass
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/rollback/constants.py b/runtime/bamboo-pipeline/pipeline/contrib/rollback/constants.py
new file mode 100644
index 00000000..3798671b
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/rollback/constants.py
@@ -0,0 +1,9 @@
+# -*- coding: utf-8 -*-
+# 回滚开始的标识位
+START_FLAG = "START"
+
+# 回滚结束的标志位
+END_FLAG = "END"
+
+ANY = "ANY" # 任意跳转模式,此时将不再检查token,可以任意回退到指定节点
+TOKEN = "TOKEN" # TOKEN 跳转模式,只允许跳转到指定的范围的节点
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/rollback/graph.py b/runtime/bamboo-pipeline/pipeline/contrib/rollback/graph.py
new file mode 100644
index 00000000..8b2863cd
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/rollback/graph.py
@@ -0,0 +1,132 @@
+# -*- coding: utf-8 -*-
+import copy
+
+from pipeline.contrib.rollback import constants
+from pipeline.core.constants import PE
+
+from bamboo_engine.utils.graph import RollbackGraph
+
+
+class CycleHandler:
+ """
+ 环处理器,作用是去除拓扑中的环
+ """
+
+ def __init__(self, node_map):
+ self.node_map = copy.deepcopy(node_map)
+
+ def get_nodes_and_edges(self):
+ """
+ 从node_map 中解析出环和边
+ """
+ nodes = []
+ edges = []
+ for node, value in self.node_map.items():
+ nodes.append(node)
+ targets = value["targets"]
+ for target in targets.values():
+ # 过滤掉那些没有执行的分支
+ if target not in self.node_map:
+ continue
+ edges.append([node, target])
+ return nodes, edges
+
+ def has_cycle(self, nodes, edges) -> list:
+ """
+ 判断是否有环,存在环是,将返回一个有效的list
+ """
+ graph = RollbackGraph(nodes, edges)
+ return graph.get_cycle()
+
+ def delete_edge(self, source, target):
+ """
+ 删除环边
+ """
+ targets = self.node_map[source]["targets"]
+
+ keys_to_remove = []
+ for key, val in targets.items():
+ if val == target:
+ keys_to_remove.append(key)
+ for key in keys_to_remove:
+ del targets[key]
+
+ def remove_cycle(self):
+ while True:
+ nodes, edges = self.get_nodes_and_edges()
+ cycles = self.has_cycle(nodes, edges)
+ if not cycles:
+ break
+ source = cycles[-2]
+ target = cycles[-1]
+ self.delete_edge(source, target)
+ return self.node_map
+
+
+class RollbackGraphHandler:
+ def __init__(self, node_map, start_id, target_id):
+ self.graph = RollbackGraph()
+ # 回滚开始的节点
+ self.start_id = start_id
+ # 回滚结束的节点
+ self.target_id = target_id
+ self.graph.add_node(start_id)
+ self.graph.add_node(target_id)
+ # 去除自环边
+ self.node_map = CycleHandler(node_map).remove_cycle()
+ # 其他不参与回滚,但是需要被清理的节点,主要是网关节点和子流程节点
+ self.others_nodes = []
+
+ def build(self, node_id, source_id=None):
+ """
+ 使用递归构建用于回滚的图谱,最终会生成一条连线 source_id -> node_id
+ @param node_id 本次遍历到的节点id
+ @param source_id 上一个遍历到的节点id
+ """
+ node_detail = self.node_map.get(node_id)
+ if node_detail is None:
+ return
+ node_type = node_detail["type"]
+
+ if node_type not in [PE.ServiceActivity]:
+ self.others_nodes.append(node_id)
+
+ if node_type == PE.ServiceActivity:
+ next_node_id = node_detail.get("id")
+ self.graph.add_node(next_node_id)
+ if source_id and source_id != next_node_id:
+ self.graph.add_edge(source_id, next_node_id)
+
+ # 如果遍历到目标节点,则返回
+ if node_id == self.start_id:
+ return
+ source_id = next_node_id
+ targets = node_detail.get("targets", {}).values()
+ elif node_type == PE.SubProcess:
+ # 处理子流程
+ source_id = self.build(node_detail["start_event_id"], source_id)
+ targets = node_detail.get("targets", {}).values()
+ elif node_type == PE.ExclusiveGateway:
+ targets = [target for target in node_detail.get("targets", {}).values() if target in self.node_map.keys()]
+ else:
+ targets = node_detail.get("targets", {}).values()
+
+ # 为了避免循环的过程中source_id值被覆盖,需要额外临时存储source_id
+ temporary_source_id = source_id
+ for target in targets:
+ source_id = self.build(target, temporary_source_id)
+
+ return source_id
+
+ def build_rollback_graph(self):
+ """
+ 这里将会从结束的节点往开始的节点进行遍历,之后再反转图
+ """
+ self.graph.add_node(constants.END_FLAG)
+ # 未整个流程加上结束节点
+ self.graph.add_edge(constants.END_FLAG, self.target_id)
+ self.graph.add_node(constants.START_FLAG)
+ self.graph.add_edge(self.start_id, constants.START_FLAG)
+ self.build(self.target_id, self.target_id)
+
+ return self.graph.reverse(), self.others_nodes
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/rollback/handler.py b/runtime/bamboo-pipeline/pipeline/contrib/rollback/handler.py
index b11fff64..de5240e4 100644
--- a/runtime/bamboo-pipeline/pipeline/contrib/rollback/handler.py
+++ b/runtime/bamboo-pipeline/pipeline/contrib/rollback/handler.py
@@ -14,197 +14,403 @@
import json
from django.db import transaction
+from django.db.models import Q
+from pipeline.conf.default_settings import ROLLBACK_QUEUE
from pipeline.contrib.exceptions import RollBackException
-from pipeline.core.constants import PE
-from pipeline.eri.models import (
- CallbackData,
- ExecutionData,
- ExecutionHistory,
- LogEntry,
- Node,
- Process,
- Schedule,
- State,
+from pipeline.contrib.rollback import constants
+from pipeline.contrib.rollback.constants import ANY, TOKEN
+from pipeline.contrib.rollback.graph import RollbackGraphHandler
+from pipeline.contrib.rollback.models import (
+ RollbackPlan,
+ RollbackSnapshot,
+ RollbackToken,
)
+from pipeline.contrib.rollback.tasks import any_rollback, token_rollback
+from pipeline.core.constants import PE
+from pipeline.eri.models import Node, Process, State
from pipeline.eri.runtime import BambooDjangoRuntime
-from bamboo_engine import api, states
+from bamboo_engine import states
-class RollBackHandler:
- def __init__(self, root_pipeline_id, node_id):
- self.root_pipeline_id = root_pipeline_id
- self.node_id = node_id
- self.runtime = BambooDjangoRuntime()
+class RollbackValidator:
+ @staticmethod
+ def validate_pipeline(root_pipeline_id):
+ pipeline_state = State.objects.filter(node_id=root_pipeline_id).first()
+ if not pipeline_state:
+ raise RollBackException(
+ "rollback failed: pipeline state not exist, pipeline_id={}".format(root_pipeline_id)
+ )
+
+ if pipeline_state.name not in [states.RUNNING, states.ROLL_BACK_FAILED]:
+ raise RollBackException(
+ "rollback failed: the task of non-running state is not allowed to roll back, "
+ "pipeline_id={}, state={}".format(root_pipeline_id, pipeline_state.name)
+ )
+
+ @staticmethod
+ def validate_node(node_id, allow_failed=False):
+ node = Node.objects.filter(node_id=node_id).first()
+ if node is None:
+ raise RollBackException("rollback failed: node not exist, node={}".format(node_id))
+
+ node_detail = json.loads(node.detail)
+ if node_detail["type"] not in [PE.ServiceActivity, PE.EmptyStartEvent]:
+ raise RollBackException("rollback failed: only allows rollback to ServiceActivity type nodes")
+
+ target_node_state = State.objects.filter(node_id=node_id).first()
+
+ if target_node_state is None:
+ raise RollBackException("rollback failed: node state not exist, node={}".format(node_id))
+
+ allow_states = [states.FINISHED]
+ if allow_failed:
+ allow_states = [states.FINISHED, states.FAILED]
+ if target_node_state.name not in allow_states:
+ raise RollBackException(
+ "rollback failed: only allows rollback to finished node, allowed states {}".format(allow_states)
+ )
+
+ @staticmethod
+ def validate_token(root_pipeline_id, start_node_id, target_node_id):
+ try:
+ rollback_token = RollbackToken.objects.get(root_pipeline_id=root_pipeline_id)
+ except RollbackToken.DoesNotExist:
+ raise RollBackException(
+ "rollback failed: pipeline token not exist, pipeline_id={}".format(root_pipeline_id)
+ )
+
+ tokens = json.loads(rollback_token.token)
- def _compute_validate_nodes(self, node_id, node_map, nodes=None):
+ start_node_token = tokens.get(start_node_id)
+ target_node_token = tokens.get(target_node_id)
+
+ if start_node_token is None or target_node_token is None:
+ raise RollBackException("rollback failed: token not found, pipeline_id={}".format(root_pipeline_id))
+
+ if start_node_token != target_node_token:
+ raise RollBackException(
+ "rollback failed: start node token must equal target node, pipeline_id={}".format(root_pipeline_id)
+ )
+
+ @staticmethod
+ def validate_node_state(root_pipeline_id, start_node_id):
"""
- 计算并得到一个允许回调的节点列表。
- 该方法的实现思路如下,从开始节点开始遍历,通过每个节点的 targets 获取到该节点的下一个节点
- - 对于并行网关和条件并行网关将直接跳过
- - 对于分支网关,则会裁剪只保留执行的那条分支
- - node_map 记录了所有已经执行过的节点的信息,当遍历到node_map中不存在的节点时,意味着已经遍历到了当前未执行的节点
- 此时会停止计算
+ 使用token模式下的回滚,相同token的节点不允许有正在运行的节点
"""
+ try:
+ rollback_token = RollbackToken.objects.get(root_pipeline_id=root_pipeline_id)
+ except RollbackToken.DoesNotExist:
+ raise RollBackException(
+ "rollback failed: pipeline token not exist, pipeline_id={}".format(root_pipeline_id)
+ )
- if nodes is None:
- nodes = []
- node_detail = node_map.get(node_id)
- # 当搜索不到时,说明已经扫描了所有已经执行过的节点了,此时直接结束
- if node_detail is None:
- return nodes
-
- if node_detail["type"] == PE.ServiceActivity:
- nodes.append(node_id)
-
- # 对于并行网关,无法跳转到任何路径
- if node_detail["type"] in [PE.ParallelGateway, PE.ConditionalParallelGateway]:
- targets = [node_detail.get(PE.converge_gateway_id)]
- # 对于分支网关内的, 只允许跳转到已经执行过的路径
- elif node_detail["type"] == PE.ExclusiveGateway:
- targets = [target for target in node_detail.get("targets", {}).values() if target in node_map.keys()]
- else:
- targets = node_detail.get("targets", {}).values()
+ tokens = json.loads(rollback_token.token)
+ start_token = tokens.get(start_node_id)
+ if start_token is None:
+ raise RollBackException("rollback failed: can't find the not token, node_id={}".format(start_node_id))
- for target in targets:
- # 如果目标节点已经出现在了node中,说明出现了环,跳过该分支
- if target in nodes:
- continue
- self._compute_validate_nodes(target, node_map, nodes)
+ node_id_list = []
+ for node_id, token in node_id_list:
+ if token == start_token:
+ node_id_list.append(node_id)
- return nodes
+ if State.objects.filter(node_id__in=node_id_list, name=states.RUNNING).exists():
+ raise RollBackException(
+ "rollback failed: there is currently the same node that the same token is running, node_id={}".format(
+ start_node_id
+ )
+ )
- def _clean_engine_data(self, target_state):
+ @staticmethod
+ def validate_start_node_id(root_pipeline_id, start_node_id):
"""
- 执行清理工作
+ 回滚的开始节点必须是流程的末尾节点
"""
- # 获取当前正在运行的节点
- state_list = State.objects.filter(root_id=self.root_pipeline_id, name=states.RUNNING).exclude(
- node_id=self.root_pipeline_id
- )
- for state in state_list:
- # 强制失败这些节点
- result = api.forced_fail_activity(self.runtime, node_id=state.node_id, ex_data="")
- if not result.result:
- raise RollBackException(
- "rollback failed: forced_fail_activity failed, node_id={}, message={}".format(
- target_state.node_id, result.message
- )
- )
+ if not Process.objects.filter(root_pipeline_id=root_pipeline_id, current_node_id=start_node_id).exists():
+ raise RollBackException("rollback failed: The node to be rolled back must be the current node!")
- # 之后清理多余的进程信息,只保留主process即可。
- Process.objects.filter(root_pipeline_id=self.root_pipeline_id).exclude(parent_id=-1).delete()
- # 查询到所有在该节点之后创建的状态信息
- need_clean_node_id_list = list(
- State.objects.filter(root_id=self.root_pipeline_id, created_time__gt=target_state.created_time).values_list(
- "node_id", flat=True
- )
- )
- # 同时清理掉目标节点的信息
- need_clean_node_id_list.append(target_state.node_id)
-
- # 清理状态信息
- State.objects.filter(root_id=self.root_pipeline_id, node_id__in=need_clean_node_id_list).delete()
- # 清理Schedule 信息
- Schedule.objects.filter(node_id__in=need_clean_node_id_list).delete()
- # 清理日志信息
- LogEntry.objects.filter(node_id__in=need_clean_node_id_list).delete()
- ExecutionHistory.objects.filter(node_id__in=need_clean_node_id_list).delete()
- ExecutionData.objects.filter(node_id__in=need_clean_node_id_list).delete()
- CallbackData.objects.filter(node_id__in=need_clean_node_id_list).delete()
-
- def get_allowed_rollback_node_id_list(self):
+class BaseRollbackHandler:
+ mode = None
+
+ def __init__(self, root_pipeline_id):
+ self.root_pipeline_id = root_pipeline_id
+ self.runtime = BambooDjangoRuntime()
+ # 检查pipeline 回滚的合法性
+ RollbackValidator.validate_pipeline(root_pipeline_id)
+
+ def get_allowed_rollback_node_id_list(self, start_node_id):
"""
- 获取允许回退的节点id列表
+ 获取允许回滚的节点范围
+ 规则:token 一致的节点允许回滚
"""
- # 不需要遍历整颗树,获取到现在已经执行成功的所有列表
+ try:
+ rollback_token = RollbackToken.objects.get(root_pipeline_id=self.root_pipeline_id)
+ except RollbackToken.DoesNotExist:
+ raise RollBackException(
+ "rollback failed: pipeline token not exist, pipeline_id={}".format(self.root_pipeline_id)
+ )
+ node_map = self._get_allowed_rollback_node_map()
+ service_activity_node_list = [
+ node_id for node_id, node_detail in node_map.items() if node_detail["type"] == PE.ServiceActivity
+ ]
+
+ tokens = json.loads(rollback_token.token)
+ start_token = tokens.get(start_node_id)
+ if not start_token:
+ return []
+
+ nodes = []
+ for node_id, token in tokens.items():
+ if start_token == token and node_id != start_node_id and node_id in service_activity_node_list:
+ nodes.append(node_id)
+
+ return nodes
+
+ def _get_allowed_rollback_node_map(self):
+ # 不需要遍历整颗树,获取到现在已经执行成功和失败节点的所有列表
finished_node_id_list = (
- State.objects.filter(root_id=self.root_pipeline_id, name=states.FINISHED)
+ State.objects.filter(root_id=self.root_pipeline_id, name__in=[states.FINISHED, states.FAILED])
.exclude(node_id=self.root_pipeline_id)
.values_list("node_id", flat=True)
)
+ node_detail_list = Node.objects.filter(node_id__in=finished_node_id_list)
+ # 获取node_id 到 node_detail的映射
+ return {n.node_id: json.loads(n.detail) for n in node_detail_list}
- # 获取到除pipeline节点之外第一个被创建的节点,此时是开始节点
- start_node_state = (
- State.objects.filter(root_id=self.root_pipeline_id)
- .exclude(node_id=self.root_pipeline_id)
- .order_by("created_time")
- .first()
+ def _reserve(self, start_node_id, target_node_id, reserve_rollback=True):
+ # 节点预约 需要在 Node 里面 插入 reserve_rollback = True, 为 True的节点执行完将暂停
+ RollbackValidator.validate_start_node_id(self.root_pipeline_id, start_node_id)
+ RollbackValidator.validate_node(target_node_id)
+ node = Node.objects.filter(node_id=start_node_id).first()
+ if node is None:
+ raise RollBackException("reserve rollback failed, the node is not exists, node_id={}".format(start_node_id))
+
+ state = State.objects.filter(node_id=start_node_id).first()
+ if state is None:
+ raise RollBackException(
+ "reserve rollback failed, the node state is not exists, node_id={}".format(start_node_id)
+ )
+
+ # 不在执行中的节点不允许预约
+ if state.name != states.RUNNING:
+ raise RollBackException(
+ "reserve rollback failed, the node state is not Running, current state={}, node_id={}".format(
+ state.name, start_node_id
+ )
+ )
+
+ with transaction.atomic():
+ if reserve_rollback:
+ # 一个流程只能同时拥有一个预约任务
+ if RollbackPlan.objects.filter(root_pipeline_id=self.root_pipeline_id, is_expired=False).exists():
+ raise RollBackException(
+ "reserve rollback failed, the rollbackPlan, current state={}, node_id={}".format(
+ state.name, start_node_id
+ )
+ )
+ RollbackPlan.objects.create(
+ root_pipeline_id=self.root_pipeline_id,
+ start_node_id=start_node_id,
+ target_node_id=target_node_id,
+ mode=self.mode,
+ )
+ else:
+ # 取消回滚,删除所有的任务
+ RollbackPlan.objects.filter(root_pipeline_id=self.root_pipeline_id, start_node_id=start_node_id).update(
+ is_expired=True
+ )
+
+ node_detail = json.loads(node.detail)
+ node_detail["reserve_rollback"] = reserve_rollback
+ node.detail = json.dumps(node_detail)
+ node.save()
+
+ def reserve_rollback(self, start_node_id, target_node_id):
+ """
+ 预约回滚
+ """
+ RollbackValidator.validate_token(self.root_pipeline_id, start_node_id, target_node_id)
+ self._reserve(start_node_id, target_node_id)
+
+ def cancel_reserved_rollback(self, start_node_id, target_node_id):
+ """
+ 取消预约回滚
+ """
+ RollbackValidator.validate_token(self.root_pipeline_id, start_node_id, target_node_id)
+ self._reserve(start_node_id, target_node_id, reserve_rollback=False)
+
+
+class AnyRollbackHandler(BaseRollbackHandler):
+ mode = ANY
+
+ def retry_rollback_failed_node(self, node_id, retry_data):
+ """ """
+ raise RollBackException("rollback failed: when mode is any, not support retry")
+
+ def rollback(self, start_node_id, target_node_id, skip_rollback_nodes=None):
+ RollbackValidator.validate_start_node_id(self.root_pipeline_id, start_node_id)
+ RollbackValidator.validate_node(start_node_id, allow_failed=True)
+ RollbackValidator.validate_node(target_node_id)
+ RollbackValidator.validate_token(self.root_pipeline_id, start_node_id, target_node_id)
+ # 相同token回滚时,不允许同一token路径上有正在运行的节点
+ RollbackValidator.validate_node_state(self.root_pipeline_id, start_node_id)
+
+ node_map = self._get_allowed_rollback_node_map()
+ rollback_graph = RollbackGraphHandler(node_map=node_map, start_id=start_node_id, target_id=target_node_id)
+
+ graph, other_nodes = rollback_graph.build_rollback_graph()
+ node_access_record = {node: 0 for node in graph.nodes}
+
+ rollback_snapshot = RollbackSnapshot.objects.create(
+ root_pipeline_id=self.root_pipeline_id,
+ graph=json.dumps(graph.as_dict()),
+ node_access_record=json.dumps(node_access_record),
+ start_node_id=start_node_id,
+ target_node_id=target_node_id,
+ other_nodes=json.dumps(other_nodes),
+ skip_rollback_nodes=json.dumps([]),
)
- # 获取到所有当前已经运行完节点的详情
- node_detail_list = Node.objects.filter(node_id__in=finished_node_id_list)
- # 获取node_id 到 node_detail的映射
- node_map = {n.node_id: json.loads(n.detail) for n in node_detail_list}
+ any_rollback.apply_async(
+ kwargs={"snapshot_id": rollback_snapshot.id},
+ queue=ROLLBACK_QUEUE,
+ )
- # 计算当前允许跳过的合法的节点
- validate_nodes_list = self._compute_validate_nodes(start_node_state.node_id, node_map)
- return validate_nodes_list
+class TokenRollbackHandler(BaseRollbackHandler):
+ mode = TOKEN
- def rollback(self):
+ def retry_rollback_failed_node(self, node_id, retry_data):
+ """
+ 重试回滚失败的节点
+ """
pipeline_state = State.objects.filter(node_id=self.root_pipeline_id).first()
- if not pipeline_state:
+ if pipeline_state.name != states.ROLL_BACK_FAILED:
raise RollBackException(
- "rollback failed: pipeline state not exist, pipeline_id={}".format(self.root_pipeline_id)
+ "rollback failed: only retry the failed pipeline, current_status={}".format(pipeline_state.name)
)
-
- if pipeline_state.name != states.RUNNING:
+ node_state = State.objects.filter(node_id=node_id).first()
+ if node_state.name != states.ROLL_BACK_FAILED:
raise RollBackException(
- "rollback failed: the task of non-running state is not allowed to roll back, pipeline_id={}".format(
- self.root_pipeline_id
- )
+ "rollback failed: only retry the failed node, current_status={}".format(node_state.name)
)
- node = Node.objects.filter(node_id=self.node_id).first()
- if node is None:
- raise RollBackException("rollback failed: node not exist, node={}".format(self.node_id))
+ # 获取镜像
+ try:
+ rollback_snapshot = RollbackSnapshot.objects.get(root_pipeline_id=self.root_pipeline_id, is_expired=False)
+ except RollbackSnapshot.DoesNotExist:
+ raise RollBackException("rollback failed: the rollback snapshot is not exists, please check")
+ except RollbackSnapshot.MultipleObjectsReturned:
+ raise RollBackException("rollback failed: found multi not expired rollback snapshot, please check")
+
+ # 重置pipeline的状态为回滚中
+ self.runtime.set_state(
+ node_id=self.root_pipeline_id,
+ to_state=states.ROLLING_BACK,
+ )
- node_detail = json.loads(node.detail)
- if node_detail["type"] not in [PE.ServiceActivity, PE.EmptyStartEvent]:
- raise RollBackException("rollback failed: only allows rollback to ServiceActivity type nodes")
+ # 驱动这个任务
+ token_rollback.apply_async(
+ kwargs={
+ "snapshot_id": rollback_snapshot.id,
+ "node_id": node_id,
+ "retry": True,
+ "retry_data": retry_data,
+ },
+ queue=ROLLBACK_QUEUE,
+ )
- target_node_state = State.objects.filter(node_id=self.node_id).first()
+ def _node_state_is_failed(self, node_id):
+ """
+ 判断该节点是不是失败的状态
+ """
+ node_state = State.objects.filter(node_id=node_id).first()
+ if node_state.name == states.FAILED:
+ return True
+ return False
+
+ def _get_failed_skip_node_id_list(self, node_id_list):
+ failed_skip_node_id_list = State.objects.filter(
+ Q(Q(skip=True) | Q(error_ignored=True)) & Q(node_id__in=node_id_list)
+ ).values_list("node_id", flat=True)
+ return failed_skip_node_id_list
+
+ def rollback(self, start_node_id, target_node_id, skip_rollback_nodes=None):
+
+ if skip_rollback_nodes is None:
+ skip_rollback_nodes = []
+
+ # 回滚的开始节点运行失败的情况
+ RollbackValidator.validate_node(start_node_id, allow_failed=True)
+ RollbackValidator.validate_node(target_node_id)
+ # 相同token回滚时,不允许同一token路径上有正在运行的节点
+ RollbackValidator.validate_node_state(self.root_pipeline_id, start_node_id)
+ RollbackValidator.validate_token(self.root_pipeline_id, start_node_id, target_node_id)
+
+ # 如果开始节点是失败的情况,则跳过该节点的回滚操作
+ if self._node_state_is_failed(start_node_id):
+ skip_rollback_nodes.append(start_node_id)
+
+ node_map = self._get_allowed_rollback_node_map()
+ rollback_graph = RollbackGraphHandler(node_map=node_map, start_id=start_node_id, target_id=target_node_id)
+
+ runtime = BambooDjangoRuntime()
+
+ graph, other_nodes = rollback_graph.build_rollback_graph()
+ node_access_record = {node: 0 for node in graph.nodes}
+
+ # 所有失败并跳过的节点不再参与回滚
+ failed_skip_node_id_list = self._get_failed_skip_node_id_list(node_map.keys())
+ skip_rollback_nodes.extend(list(failed_skip_node_id_list))
+
+ rollback_snapshot = RollbackSnapshot.objects.create(
+ root_pipeline_id=self.root_pipeline_id,
+ graph=json.dumps(graph.as_dict()),
+ node_access_record=json.dumps(node_access_record),
+ start_node_id=start_node_id,
+ target_node_id=target_node_id,
+ other_nodes=json.dumps(other_nodes),
+ skip_rollback_nodes=json.dumps(skip_rollback_nodes),
+ )
- if target_node_state is None:
- raise RollBackException("rollback failed: node state not exist, node={}".format(self.node_id))
+ runtime.set_state(
+ node_id=self.root_pipeline_id,
+ to_state=states.ROLLING_BACK,
+ )
+ # 驱动这个任务
+ token_rollback.apply_async(
+ kwargs={
+ "snapshot_id": rollback_snapshot.id,
+ "node_id": constants.START_FLAG,
+ "retry": False,
+ "retry_data": None,
+ },
+ queue=ROLLBACK_QUEUE,
+ )
- if target_node_state.name != states.FINISHED:
- raise RollBackException("rollback failed: only allows rollback to finished node")
- validate_nodes_list = self.get_allowed_rollback_node_id_list()
+class RollbackDispatcher:
+ def __init__(self, root_pipeline_id, mode):
+ if mode == ANY:
+ self.handler = AnyRollbackHandler(root_pipeline_id)
+ elif mode == TOKEN:
+ self.handler = TokenRollbackHandler(root_pipeline_id)
+ else:
+ raise RollBackException("rollback failed: not support this mode, please check")
- if self.node_id not in validate_nodes_list:
- raise RollBackException("rollback failed: node is not allow to rollback, node={}".format(self.node_id))
+ def rollback(self, start_node_id: str, target_node_id: str, skip_rollback_nodes: list = None):
+ self.handler.rollback(start_node_id, target_node_id, skip_rollback_nodes)
- with transaction.atomic():
- try:
- self._clean_engine_data(target_node_state)
- except Exception as e:
- raise RollBackException("rollback failed: clean engine data error, error={}".format(str(e)))
-
- try:
- # 将当前住进程的正在运行的节点指向目标ID
- main_process = Process.objects.get(root_pipeline_id=self.root_pipeline_id, parent_id=-1)
- main_process.current_node_id = self.node_id
- main_process.save()
-
- # 重置该节点的状态信息
- self.runtime.set_state(
- node_id=self.node_id,
- to_state=states.READY,
- is_retry=True,
- refresh_version=True,
- clear_started_time=True,
- clear_archived_time=True,
- )
- process_info = self.runtime.get_process_info(main_process.id)
- self.runtime.execute(
- process_id=process_info.process_id,
- node_id=self.node_id,
- root_pipeline_id=process_info.root_pipeline_id,
- parent_pipeline_id=process_info.top_pipeline_id,
- )
- except Exception as e:
- raise RollBackException("rollback failed: rollback to node error, error={}".format(str(e)))
+ def reserve_rollback(self, start_node_id: str, target_node_id: str):
+ self.handler.reserve_rollback(start_node_id, target_node_id)
+
+ def retry_rollback_failed_node(self, node_id: str, retry_data: dict = None):
+ self.handler.retry_rollback_failed_node(node_id, retry_data)
+
+ def cancel_reserved_rollback(self, start_node_id: str, target_node_id: str):
+ self.handler.cancel_reserved_rollback(start_node_id, target_node_id)
+
+ def get_allowed_rollback_node_id_list(self, start_node_id: str):
+ return self.handler.get_allowed_rollback_node_id_list(start_node_id)
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/rollback/migrations/0001_initial.py b/runtime/bamboo-pipeline/pipeline/contrib/rollback/migrations/0001_initial.py
new file mode 100644
index 00000000..fd0a18d2
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/rollback/migrations/0001_initial.py
@@ -0,0 +1,65 @@
+# Generated by Django 3.2.18 on 2023-10-16 07:12
+
+from django.db import migrations, models
+
+
+class Migration(migrations.Migration):
+ initial = True
+
+ dependencies = []
+
+ operations = [
+ migrations.CreateModel(
+ name="RollbackNodeSnapshot",
+ fields=[
+ ("id", models.AutoField(auto_created=True, primary_key=True, serialize=False, verbose_name="ID")),
+ ("root_pipeline_id", models.CharField(db_index=True, max_length=64, verbose_name="root pipeline id")),
+ ("node_id", models.CharField(db_index=True, max_length=64, verbose_name="node_id")),
+ ("code", models.CharField(max_length=64, verbose_name="node_code")),
+ ("version", models.CharField(max_length=33, verbose_name="version")),
+ ("inputs", models.TextField(verbose_name="node inputs")),
+ ("outputs", models.TextField(verbose_name="node outputs")),
+ ("context_values", models.TextField(verbose_name="pipeline context values")),
+ ("rolled_back", models.BooleanField(default=False, verbose_name="whether the node rolls back")),
+ ],
+ ),
+ migrations.CreateModel(
+ name="RollbackPlan",
+ fields=[
+ ("id", models.AutoField(auto_created=True, primary_key=True, serialize=False, verbose_name="ID")),
+ ("root_pipeline_id", models.CharField(db_index=True, max_length=64, verbose_name="root pipeline id")),
+ ("start_node_id", models.CharField(db_index=True, max_length=64, verbose_name="start node id")),
+ ("target_node_id", models.CharField(db_index=True, max_length=64, verbose_name="target_node_id")),
+ ("mode", models.CharField(default="TOKEN", max_length=32, verbose_name="rollback mode")),
+ ("is_expired", models.BooleanField(default=False, verbose_name="is expired")),
+ ],
+ ),
+ migrations.CreateModel(
+ name="RollbackSnapshot",
+ fields=[
+ ("id", models.AutoField(auto_created=True, primary_key=True, serialize=False, verbose_name="ID")),
+ ("root_pipeline_id", models.CharField(db_index=True, max_length=64, verbose_name="root pipeline id")),
+ ("graph", models.TextField(verbose_name="rollback graph")),
+ ("node_access_record", models.TextField(verbose_name="node access record")),
+ ("skip_rollback_nodes", models.TextField(verbose_name="skip rollback nodes")),
+ ("other_nodes", models.TextField(verbose_name="other nodes")),
+ ("start_node_id", models.CharField(db_index=True, max_length=64, verbose_name="start node id")),
+ ("target_node_id", models.CharField(db_index=True, max_length=64, verbose_name="target_node_id")),
+ ("is_expired", models.BooleanField(db_index=True, default=False, verbose_name="is expired")),
+ ],
+ ),
+ migrations.CreateModel(
+ name="RollbackToken",
+ fields=[
+ ("id", models.AutoField(auto_created=True, primary_key=True, serialize=False, verbose_name="ID")),
+ ("root_pipeline_id", models.CharField(db_index=True, max_length=64, verbose_name="root pipeline id")),
+ ("token", models.TextField(verbose_name="token map")),
+ (
+ "is_deleted",
+ models.BooleanField(
+ db_index=True, default=False, help_text="is deleted", verbose_name="is deleted"
+ ),
+ ),
+ ],
+ ),
+ ]
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/rollback/migrations/0002_auto_20231020_1234.py b/runtime/bamboo-pipeline/pipeline/contrib/rollback/migrations/0002_auto_20231020_1234.py
new file mode 100644
index 00000000..26b00ac0
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/rollback/migrations/0002_auto_20231020_1234.py
@@ -0,0 +1,29 @@
+# Generated by Django 3.2.18 on 2023-10-20 12:34
+
+import pipeline.contrib.fields
+from django.db import migrations
+
+
+class Migration(migrations.Migration):
+
+ dependencies = [
+ ("rollback", "0001_initial"),
+ ]
+
+ operations = [
+ migrations.AlterField(
+ model_name="rollbacknodesnapshot",
+ name="context_values",
+ field=pipeline.contrib.fields.SerializerField(verbose_name="pipeline context values"),
+ ),
+ migrations.AlterField(
+ model_name="rollbacknodesnapshot",
+ name="inputs",
+ field=pipeline.contrib.fields.SerializerField(verbose_name="node inputs"),
+ ),
+ migrations.AlterField(
+ model_name="rollbacknodesnapshot",
+ name="outputs",
+ field=pipeline.contrib.fields.SerializerField(verbose_name="node outputs"),
+ ),
+ ]
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/rollback/migrations/__init__.py b/runtime/bamboo-pipeline/pipeline/contrib/rollback/migrations/__init__.py
new file mode 100644
index 00000000..e69de29b
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/rollback/models.py b/runtime/bamboo-pipeline/pipeline/contrib/rollback/models.py
new file mode 100644
index 00000000..cefac0b9
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/rollback/models.py
@@ -0,0 +1,53 @@
+# -*- coding: utf-8 -*-
+from django.db import models
+from django.utils.translation import ugettext_lazy as _
+from pipeline.contrib.fields import SerializerField
+from pipeline.contrib.rollback.constants import TOKEN
+
+
+class RollbackToken(models.Model):
+ """
+ 回滚配置token信息
+ """
+
+ root_pipeline_id = models.CharField(verbose_name="root pipeline id", max_length=64, db_index=True)
+ token = models.TextField(_("token map"), null=False)
+ is_deleted = models.BooleanField(_("is deleted"), default=False, help_text=_("is deleted"), db_index=True)
+
+
+class RollbackSnapshot(models.Model):
+ """
+ 节点执行的快照信息
+ """
+
+ root_pipeline_id = models.CharField(verbose_name="root pipeline id", max_length=64, db_index=True)
+ graph = models.TextField(verbose_name="rollback graph", null=False)
+ node_access_record = models.TextField(verbose_name="node access record")
+ skip_rollback_nodes = models.TextField(verbose_name="skip rollback nodes")
+ other_nodes = models.TextField(verbose_name="other nodes")
+ start_node_id = models.CharField(verbose_name="start node id", max_length=64, db_index=True)
+ target_node_id = models.CharField(verbose_name="target_node_id", max_length=64, db_index=True)
+ is_expired = models.BooleanField(verbose_name="is expired", default=False, db_index=True)
+
+
+class RollbackNodeSnapshot(models.Model):
+ """
+ 节点快照
+ """
+
+ root_pipeline_id = models.CharField(verbose_name="root pipeline id", max_length=64, db_index=True)
+ node_id = models.CharField(verbose_name="node_id", max_length=64, db_index=True)
+ code = models.CharField(verbose_name="node_code", max_length=64)
+ version = models.CharField(verbose_name=_("version"), null=False, max_length=33)
+ inputs = SerializerField(verbose_name=_("node inputs"))
+ outputs = SerializerField(verbose_name=_("node outputs"))
+ context_values = SerializerField(verbose_name=_("pipeline context values"))
+ rolled_back = models.BooleanField(_("whether the node rolls back"), default=False)
+
+
+class RollbackPlan(models.Model):
+ root_pipeline_id = models.CharField(verbose_name="root pipeline id", max_length=64, db_index=True)
+ start_node_id = models.CharField(verbose_name="start node id", max_length=64, db_index=True)
+ target_node_id = models.CharField(verbose_name="target_node_id", max_length=64, db_index=True)
+ mode = models.CharField(verbose_name="rollback mode", max_length=32, default=TOKEN)
+ is_expired = models.BooleanField(verbose_name="is expired", default=False)
diff --git a/runtime/bamboo-pipeline/pipeline/contrib/rollback/tasks.py b/runtime/bamboo-pipeline/pipeline/contrib/rollback/tasks.py
new file mode 100644
index 00000000..259df0cb
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/contrib/rollback/tasks.py
@@ -0,0 +1,301 @@
+# -*- coding: utf-8 -*-
+import json
+import logging
+
+from celery import task
+from django.conf import settings
+from django.db import transaction
+from pipeline.conf.default_settings import ROLLBACK_QUEUE
+from pipeline.contrib.rollback import constants
+from pipeline.contrib.rollback.models import (
+ RollbackNodeSnapshot,
+ RollbackPlan,
+ RollbackSnapshot,
+)
+from pipeline.eri.models import CallbackData
+from pipeline.eri.models import ExecutionData as DBExecutionData
+from pipeline.eri.models import (
+ ExecutionHistory,
+ LogEntry,
+ Node,
+ Process,
+ Schedule,
+ State,
+)
+from pipeline.eri.runtime import BambooDjangoRuntime
+
+from bamboo_engine import states
+from bamboo_engine.eri import ExecutionData
+from bamboo_engine.utils.graph import RollbackGraph
+
+logger = logging.getLogger("celery")
+
+
+class RollbackCleaner:
+ def __init__(self, snapshot):
+ self.snapshot = snapshot
+
+ def _clear_node_reserve_flag(self, node_id):
+ node = Node.objects.get(node_id=node_id)
+ node_detail = json.loads(node.detail)
+ node_detail["reserve_rollback"] = False
+ node.detail = json.dumps(node_detail)
+ node.save()
+
+ def clear_data(self):
+ # 节点快照需要全部删除,可能会有下一次的回滚
+ RollbackNodeSnapshot.objects.filter(root_pipeline_id=self.snapshot.root_pipeline_id).delete()
+ # 回滚快照需要置为已过期
+ RollbackSnapshot.objects.filter(root_pipeline_id=self.snapshot.root_pipeline_id).update(is_expired=True)
+ # 预约计划需要修改为已过期
+ RollbackPlan.objects.filter(
+ root_pipeline_id=self.snapshot.root_pipeline_id, start_node_id=self.snapshot.start_node_id
+ ).update(is_expired=True)
+ # 节点的预约信息需要清理掉
+ self._clear_node_reserve_flag(self.snapshot.start_node_id)
+ # 需要删除该节点的进程信息/非主进程的,防止网关再分支处回滚时,仍然有正在运行的process得不到清理
+ Process.objects.filter(
+ root_pipeline_id=self.snapshot.root_pipeline_id, current_node_id=self.snapshot.start_node_id
+ ).exclude(parent_id=-1).delete()
+
+ graph = json.loads(self.snapshot.graph)
+ need_clean_node_id_list = graph["nodes"] + json.loads(self.snapshot.other_nodes)
+ # 清理状态信息
+ State.objects.filter(root_id=self.snapshot.root_pipeline_id, node_id__in=need_clean_node_id_list).delete()
+ # 清理Schedule 信息
+ Schedule.objects.filter(node_id__in=need_clean_node_id_list).delete()
+ # 清理日志信息
+ LogEntry.objects.filter(node_id__in=need_clean_node_id_list).delete()
+ ExecutionHistory.objects.filter(node_id__in=need_clean_node_id_list).delete()
+ DBExecutionData.objects.filter(node_id__in=need_clean_node_id_list).delete()
+ CallbackData.objects.filter(node_id__in=need_clean_node_id_list).delete()
+
+
+class TokenRollbackTaskHandler:
+ def __init__(self, snapshot_id, node_id, retry, retry_data):
+ self.snapshot_id = snapshot_id
+ self.node_id = node_id
+ self.retry_data = retry_data
+ self.retry = retry
+ self.runtime = BambooDjangoRuntime()
+
+ def set_state(self, node_id, state):
+ logger.info("[TokenRollbackTaskHandler][set_state] set node_id state to {}".format(state))
+ # 开始和结束节点直接跳过回滚
+ if node_id in [constants.END_FLAG, constants.START_FLAG]:
+ return
+ self.runtime.set_state(
+ node_id=node_id,
+ to_state=state,
+ )
+
+ def execute_rollback(self):
+ """
+ 执行回滚的操作
+ """
+ if self.node_id in [constants.END_FLAG, constants.START_FLAG]:
+ return True
+
+ # 获取节点快照,可能会有多多份快照,需要多次回滚
+ node_snapshots = RollbackNodeSnapshot.objects.filter(node_id=self.node_id, rolled_back=False).order_by("-id")
+ for node_snapshot in node_snapshots:
+ service = self.runtime.get_service(code=node_snapshot.code, version=node_snapshot.version)
+ data = ExecutionData(inputs=node_snapshot.inputs, outputs=node_snapshot.outputs)
+ parent_data = ExecutionData(inputs=node_snapshot.context_values, outputs={})
+ result = service.service.rollback(data, parent_data, self.retry_data)
+ node_snapshot.rolled_back = True
+ node_snapshot.save()
+ if not result:
+ return False
+
+ return True
+
+ def start_pipeline(self, root_pipeline_id, target_node_id):
+ """
+ 启动pipeline
+ """
+ # 将当前住进程的正在运行的节点指向目标ID
+ main_process = Process.objects.get(root_pipeline_id=root_pipeline_id, parent_id=-1)
+ main_process.current_node_id = target_node_id
+ main_process.save()
+
+ # 重置该节点的状态信息
+ self.runtime.set_state(
+ node_id=target_node_id,
+ to_state=states.READY,
+ is_retry=True,
+ refresh_version=True,
+ clear_started_time=True,
+ clear_archived_time=True,
+ )
+
+ process_info = self.runtime.get_process_info(main_process.id)
+ # 设置pipeline的状态
+ self.runtime.set_state(
+ node_id=root_pipeline_id,
+ to_state=states.READY,
+ )
+
+ # 如果开启了流程自动回滚,则会开启到目标节点之后自动开始
+ if getattr(settings, "PIPELINE_ENABLE_AUTO_EXECUTE_WHEN_ROLL_BACKED", True):
+ self.runtime.set_state(
+ node_id=root_pipeline_id,
+ to_state=states.RUNNING,
+ )
+ else:
+ # 流程设置为暂停状态,需要用户点击才可以继续开始
+ self.runtime.set_state(
+ node_id=root_pipeline_id,
+ to_state=states.SUSPENDED,
+ )
+
+ self.runtime.execute(
+ process_id=process_info.process_id,
+ node_id=target_node_id,
+ root_pipeline_id=process_info.root_pipeline_id,
+ parent_pipeline_id=process_info.top_pipeline_id,
+ )
+
+ def rollback(self):
+ with transaction.atomic():
+ rollback_snapshot = RollbackSnapshot.objects.select_for_update().get(id=self.snapshot_id, is_expired=False)
+ node_access_record = json.loads(rollback_snapshot.node_access_record)
+ # 只有非重试状态下才需要记录访问
+ if not self.retry:
+ node_access_record[self.node_id] += 1
+ rollback_snapshot.node_access_record = json.dumps(node_access_record)
+ rollback_snapshot.save()
+
+ graph = json.loads(rollback_snapshot.graph)
+ target_node_id = rollback_snapshot.target_node_id
+ rollback_graph = RollbackGraph(nodes=graph["nodes"], flows=graph["flows"])
+ skip_rollback_nodes = json.loads(rollback_snapshot.skip_rollback_nodes)
+ in_degrees = rollback_graph.in_degrees()
+
+ clearner = RollbackCleaner(rollback_snapshot)
+
+ if node_access_record[self.node_id] >= in_degrees[self.node_id]:
+ # 对于不需要跳过的节点才会执行具体的回滚行为
+ if self.node_id not in skip_rollback_nodes:
+ try:
+ # 设置节点状态为回滚中
+ self.set_state(self.node_id, states.ROLLING_BACK)
+ # 执行同步回滚的操作
+ result = self.execute_rollback()
+ except Exception as e:
+ logger.error(
+ "[TokenRollbackTaskHandler][rollback] execute rollback error,"
+ "snapshot_id={}, node_id={}, err={}".format(self.snapshot_id, self.node_id, e)
+ )
+ # 节点和流程重置为回滚失败的状态
+ self.set_state(rollback_snapshot.root_pipeline_id, states.ROLL_BACK_FAILED)
+ # 回滚失败的节点将不再向下执行
+ self.set_state(self.node_id, states.ROLL_BACK_FAILED)
+ return
+
+ # 节点回滚成功
+ if result:
+ self.set_state(self.node_id, states.ROLL_BACK_SUCCESS)
+ else:
+ logger.info(
+ "[TokenRollbackTaskHandler][rollback], execute rollback failed, "
+ "result=False, snapshot_id={}, node_id={}".format(self.snapshot_id, self.node_id)
+ )
+ self.set_state(self.node_id, states.ROLL_BACK_FAILED)
+ # 回滚失败的节点将不再向下执行
+ self.set_state(rollback_snapshot.root_pipeline_id, states.ROLL_BACK_FAILED)
+ return
+
+ next_node = rollback_graph.next(self.node_id)
+ if list(next_node)[0] == constants.END_FLAG:
+ self.set_state(rollback_snapshot.root_pipeline_id, states.ROLL_BACK_SUCCESS)
+ try:
+ clearner.clear_data()
+ self.start_pipeline(
+ root_pipeline_id=rollback_snapshot.root_pipeline_id, target_node_id=target_node_id
+ )
+ except Exception as e:
+ logger.error("[TokenRollbackTaskHandler][rollback] start_pipeline failed, err={}".format(e))
+ return
+ return
+
+ for node in next_node:
+ token_rollback.apply_async(
+ kwargs={
+ "snapshot_id": self.snapshot_id,
+ "node_id": node,
+ },
+ queue=ROLLBACK_QUEUE,
+ )
+
+
+class AnyRollbackHandler:
+ def __init__(self, snapshot_id):
+ self.snapshot_id = snapshot_id
+ self.runtime = BambooDjangoRuntime()
+
+ def start_pipeline(self, root_pipeline_id, target_node_id):
+ """
+ 启动pipeline
+ """
+
+ main_process = Process.objects.get(root_pipeline_id=root_pipeline_id, parent_id=-1)
+ main_process.current_node_id = target_node_id
+ main_process.save()
+
+ # 重置该节点的状态信息
+ self.runtime.set_state(
+ node_id=target_node_id,
+ to_state=states.READY,
+ is_retry=True,
+ refresh_version=True,
+ clear_started_time=True,
+ clear_archived_time=True,
+ )
+
+ process_info = self.runtime.get_process_info(main_process.id)
+
+ # 如果PIPELINE_ENABLE_AUTO_EXECUTE_WHEN_ROLL_BACKED为False, 那么则会重制流程为暂停状态
+ if not getattr(settings, "PIPELINE_ENABLE_AUTO_EXECUTE_WHEN_ROLL_BACKED", True):
+ self.runtime.set_state(
+ node_id=root_pipeline_id,
+ to_state=states.SUSPENDED,
+ )
+
+ self.runtime.execute(
+ process_id=process_info.process_id,
+ node_id=target_node_id,
+ root_pipeline_id=process_info.root_pipeline_id,
+ parent_pipeline_id=process_info.top_pipeline_id,
+ )
+
+ def rollback(self):
+ with transaction.atomic():
+ rollback_snapshot = RollbackSnapshot.objects.get(id=self.snapshot_id, is_expired=False)
+ clearner = RollbackCleaner(rollback_snapshot)
+ try:
+ clearner.clear_data()
+ self.start_pipeline(
+ root_pipeline_id=rollback_snapshot.root_pipeline_id, target_node_id=rollback_snapshot.target_node_id
+ )
+ except Exception as e:
+ logger.error(
+ "rollback failed: start pipeline, pipeline_id={}, target_node_id={}, error={}".format(
+ rollback_snapshot.root_pipeline_id, rollback_snapshot.target_node_id, str(e)
+ )
+ )
+ raise e
+
+
+@task
+def token_rollback(snapshot_id, node_id, retry=False, retry_data=None):
+ """
+ snapshot_id 本次回滚的快照id
+ node_id 当前要回滚的节点id
+ """
+ TokenRollbackTaskHandler(snapshot_id=snapshot_id, node_id=node_id, retry=retry, retry_data=retry_data).rollback()
+
+
+@task
+def any_rollback(snapshot_id):
+ AnyRollbackHandler(snapshot_id=snapshot_id).rollback()
diff --git a/runtime/bamboo-pipeline/pipeline/core/flow/activity/service_activity.py b/runtime/bamboo-pipeline/pipeline/core/flow/activity/service_activity.py
index 6cb32fb3..d0076b4b 100644
--- a/runtime/bamboo-pipeline/pipeline/core/flow/activity/service_activity.py
+++ b/runtime/bamboo-pipeline/pipeline/core/flow/activity/service_activity.py
@@ -100,6 +100,9 @@ def need_run_hook(self):
def schedule(self, data, parent_data, callback_data=None):
return True
+ def rollback(self, data, parent_data, rollback_data=None):
+ return True
+
def finish_schedule(self):
setattr(self, self.schedule_result_attr, True)
diff --git a/runtime/bamboo-pipeline/pipeline/engine/tasks.py b/runtime/bamboo-pipeline/pipeline/engine/tasks.py
index 956350f8..3a6470b1 100644
--- a/runtime/bamboo-pipeline/pipeline/engine/tasks.py
+++ b/runtime/bamboo-pipeline/pipeline/engine/tasks.py
@@ -11,27 +11,28 @@
specific language governing permissions and limitations under the License.
"""
-import logging
import datetime
-from dateutil.relativedelta import relativedelta
+import logging
+
from celery import task
from celery.schedules import crontab
from celery.task import periodic_task
-from django.db import transaction, connection
-
+from dateutil.relativedelta import relativedelta
+from django.apps import apps
+from django.db import connection, transaction
from pipeline.conf import default_settings
from pipeline.core.pipeline import Pipeline
from pipeline.engine import api, signals, states
from pipeline.engine.core import runtime, schedule
from pipeline.engine.health import zombie
from pipeline.engine.models import (
+ History,
NodeCeleryTask,
NodeRelationship,
PipelineProcess,
ProcessCeleryTask,
- Status,
ScheduleService,
- History,
+ Status,
)
from pipeline.models import PipelineInstance
@@ -257,6 +258,15 @@ def _clean_pipeline_instance_data(instance_id, timestamp):
delete_pipeline_process = (
"DELETE FROM `engine_pipelineprocess` " "WHERE `engine_pipelineprocess`.`root_pipeline_id` = %s"
)
+
+ delete_rollback_plan = "DELETE FROM `rollback_rollbackplan` WHERE `rollback_rollbackplan`.`root_pipeline_id` = %s"
+ delete_rollback_snapshot = (
+ "DELETE FROM `rollback_rollbacksnapshot` WHERE `rollback_rollbacksnapshot`.`root_pipeline_id` = %s"
+ )
+ delete_rollback_token = (
+ "DELETE FROM `rollback_rollbacktoken` WHERE `rollback_rollbacktoken`.`root_pipeline_id` = %s"
+ )
+
with transaction.atomic():
with connection.cursor() as cursor:
if pipeline_process_ids:
@@ -289,6 +299,15 @@ def _clean_pipeline_instance_data(instance_id, timestamp):
_raw_sql_execute(cursor, delete_pipeline_process, [instance_id], timestamp)
PipelineInstance.objects.filter(instance_id=instance_id).update(is_expired=True)
+ try:
+ apps.get_model("rollback", "RollbackToken")
+ except Exception:
+ logger.error("[_clean_pipeline_rollback_data] delete error, the rollback app not installed")
+ return
+ _raw_sql_execute(cursor, delete_rollback_plan, [instance_id], timestamp)
+ _raw_sql_execute(cursor, delete_rollback_snapshot, [instance_id], timestamp)
+ _raw_sql_execute(cursor, delete_rollback_token, [instance_id], timestamp)
+
def _sql_log(sql, params, timestamp):
if isinstance(params, list):
diff --git a/runtime/bamboo-pipeline/pipeline/eri/imp/hooks.py b/runtime/bamboo-pipeline/pipeline/eri/imp/hooks.py
index a0be2298..23904fbd 100644
--- a/runtime/bamboo-pipeline/pipeline/eri/imp/hooks.py
+++ b/runtime/bamboo-pipeline/pipeline/eri/imp/hooks.py
@@ -17,6 +17,8 @@
from pipeline.eri.models import LogEntry
from pipeline.eri.signals import pipeline_event
+from bamboo_engine.utils.constants import RuntimeSettings
+
class PipelineEvent:
def __init__(self, event_type, data):
@@ -64,6 +66,9 @@ def pre_prepare_run_pipeline(
)
)
+ if self.get_config(RuntimeSettings.PIPELINE_ENABLE_ROLLBACK.value):
+ self.set_pipeline_token(pipeline)
+
def post_prepare_run_pipeline(
self, pipeline: dict, root_pipeline_data: dict, root_pipeline_context: dict, subprocess_context: dict, **options
):
diff --git a/runtime/bamboo-pipeline/pipeline/eri/imp/node.py b/runtime/bamboo-pipeline/pipeline/eri/imp/node.py
index 1e693888..add1a058 100644
--- a/runtime/bamboo-pipeline/pipeline/eri/imp/node.py
+++ b/runtime/bamboo-pipeline/pipeline/eri/imp/node.py
@@ -13,25 +13,25 @@
import json
+from pipeline.eri.models import Node as DBNode
+
from bamboo_engine import metrics
from bamboo_engine.eri import (
- Node,
- NodeType,
- ServiceActivity,
- SubProcess,
- ExclusiveGateway,
- ParallelGateway,
+ Condition,
ConditionalParallelGateway,
ConvergeGateway,
- EmptyStartEvent,
+ DefaultCondition,
EmptyEndEvent,
+ EmptyStartEvent,
+ ExclusiveGateway,
ExecutableEndEvent,
- Condition,
- DefaultCondition,
+ Node,
+ NodeType,
+ ParallelGateway,
+ ServiceActivity,
+ SubProcess,
)
-from pipeline.eri.models import Node as DBNode
-
class NodeMixin:
def _get_node(self, node: DBNode):
@@ -48,6 +48,7 @@ def _get_node(self, node: DBNode):
can_skip=node_detail["can_skip"],
name=node_detail.get("name"),
can_retry=node_detail["can_retry"],
+ reserve_rollback=node_detail.get("reserve_rollback", False),
)
if node_type == NodeType.ServiceActivity.value:
diff --git a/runtime/bamboo-pipeline/pipeline/eri/imp/process.py b/runtime/bamboo-pipeline/pipeline/eri/imp/process.py
index 05aef601..4e703dd6 100644
--- a/runtime/bamboo-pipeline/pipeline/eri/imp/process.py
+++ b/runtime/bamboo-pipeline/pipeline/eri/imp/process.py
@@ -12,17 +12,17 @@
"""
import json
-from typing import List, Optional, Dict
+from typing import Dict, List, Optional
-from django.utils import timezone
-from django.db.models import F
+from django.conf import settings
from django.db import transaction
-
-from bamboo_engine import metrics
-from bamboo_engine.eri import ProcessInfo, SuspendedProcessInfo, DispatchProcess
-
+from django.db.models import F
+from django.utils import timezone
from pipeline.eri.models import Process
+from bamboo_engine import metrics, states
+from bamboo_engine.eri import DispatchProcess, ProcessInfo, SuspendedProcessInfo
+
class ProcessMixin:
def beat(self, process_id: int):
@@ -175,7 +175,7 @@ def get_sleep_process_info_with_current_node_id(self, node_id: str) -> Optional[
parent_id=qs[0].parent_id,
)
- def get_process_id_with_current_node_id(self, node_id: str) -> Optional[str]:
+ def get_process_id_with_current_node_id(self, node_id: str) -> Optional[int]:
"""
获取当前节点 ID 为 node_id 且存活的进程 ID
@@ -273,6 +273,23 @@ def fork(
if not qs:
raise Process.DoesNotExist("Process with id({}) does not exist".format(parent_id))
+ if getattr(settings, "PIPELINE_ENABLE_ROLLBACK", False):
+ # 如果开启了回滚,则会自动删除相关的process信息,防止异常
+ state = self.get_state(root_pipeline_id)
+ # 如果流程处在回滚中,才会删除
+ if state.name == states.ROLLING_BACK:
+ for current_node, destination in from_to.items():
+ Process.objects.filter(
+ parent_id=parent_id,
+ asleep=True,
+ destination_id=destination,
+ current_node_id=current_node,
+ root_pipeline_id=root_pipeline_id,
+ pipeline_stack=stack_json,
+ priority=qs[0].priority,
+ queue=qs[0].queue,
+ ).delete()
+
children = [
Process(
parent_id=parent_id,
diff --git a/runtime/bamboo-pipeline/pipeline/eri/imp/rollback.py b/runtime/bamboo-pipeline/pipeline/eri/imp/rollback.py
new file mode 100644
index 00000000..ea1a15a9
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/eri/imp/rollback.py
@@ -0,0 +1,82 @@
+# -*- coding: utf-8 -*-
+import json
+import logging
+
+from django.apps import apps
+
+from bamboo_engine.builder.builder import generate_pipeline_token
+
+logger = logging.getLogger("bamboo_engine")
+
+
+class RollbackMixin:
+ def set_pipeline_token(self, pipeline_tree: dict):
+ """
+ 设置pipeline token
+ """
+ try:
+ # 引用成功说明pipeline rollback 这个 app 是安装过的
+ RollbackToken = apps.get_model("rollback", "RollbackToken")
+ except Exception as e:
+ logger.error(
+ "[RollbackMixin][set_pipeline_token] import RollbackToken error, "
+ "Please check whether the rollback app is installed correctly, err={}".format(e)
+ )
+ return
+
+ root_pipeline_id = pipeline_tree["id"]
+ node_map = generate_pipeline_token(pipeline_tree)
+
+ RollbackToken.objects.create(root_pipeline_id=root_pipeline_id, token=json.dumps(node_map))
+
+ def set_node_snapshot(self, root_pipeline_id, node_id, code, version, context_values, inputs, outputs):
+ """
+ 创建一分节点快照
+ """
+ try:
+ RollbackNodeSnapshot = apps.get_model("rollback", "RollbackNodeSnapshot")
+ # 引用成功说明pipeline rollback 这个 app 是安装过的
+ except Exception as e:
+ logger.error(
+ "[RollbackMixin][set_node_snapshot] import RollbackNodeSnapshot error, "
+ "Please check whether the rollback app is installed correctly, err={}".format(e)
+ )
+ return
+
+ RollbackNodeSnapshot.objects.create(
+ root_pipeline_id=root_pipeline_id,
+ node_id=node_id,
+ code=code,
+ version=version,
+ context_values=context_values,
+ inputs=inputs,
+ outputs=outputs,
+ )
+
+ def start_rollback(self, root_pipeline_id, node_id):
+ """
+ 新建一个回滚任务
+ """
+ try:
+ # 引用成功说明pipeline rollback 这个 app 是安装过的
+ from pipeline.contrib.rollback.handler import RollbackDispatcher
+
+ RollbackPlan = apps.get_model("rollback", "RollbackPlan")
+ except Exception as e:
+ logger.error(
+ "[RollbackMixin][set_pipeline_token] import RollbackDispatcher or RollbackPlan error, "
+ "Please check whether the rollback app is installed correctly, err={}".format(e)
+ )
+ return
+
+ try:
+ rollback_plan = RollbackPlan.objects.get(
+ root_pipeline_id=root_pipeline_id, start_node_id=node_id, is_expired=False
+ )
+ handler = RollbackDispatcher(root_pipeline_id=root_pipeline_id, mode=rollback_plan.mode)
+ handler.rollback(rollback_plan.start_node_id, rollback_plan.target_node_id)
+ rollback_plan.is_expired = True
+ rollback_plan.save(update_fields=["is_expired"])
+ except Exception as e:
+ logger.error("[RollbackMixin][start_rollback] start a rollback task error, err={}".format(e))
+ return
diff --git a/runtime/bamboo-pipeline/pipeline/eri/runtime.py b/runtime/bamboo-pipeline/pipeline/eri/runtime.py
index 440374a3..62aebf00 100644
--- a/runtime/bamboo-pipeline/pipeline/eri/runtime.py
+++ b/runtime/bamboo-pipeline/pipeline/eri/runtime.py
@@ -29,6 +29,7 @@
from pipeline.eri.imp.node import NodeMixin
from pipeline.eri.imp.plugin_manager import PipelinePluginManagerMixin
from pipeline.eri.imp.process import ProcessMixin
+from pipeline.eri.imp.rollback import RollbackMixin
from pipeline.eri.imp.schedule import ScheduleMixin
from pipeline.eri.imp.state import StateMixin
from pipeline.eri.imp.task import TaskMixin
@@ -68,10 +69,10 @@ class BambooDjangoRuntime(
InterruptMixin,
EventMixin,
ConfigMixin,
+ RollbackMixin,
EngineRuntimeInterface,
):
-
- ERI_SUPPORT_VERSION = 7
+ ERI_SUPPORT_VERSION = 8
def __init__(self):
try:
diff --git a/runtime/bamboo-pipeline/pipeline/parser/utils.py b/runtime/bamboo-pipeline/pipeline/parser/utils.py
index 9b1f7203..669dbe01 100644
--- a/runtime/bamboo-pipeline/pipeline/parser/utils.py
+++ b/runtime/bamboo-pipeline/pipeline/parser/utils.py
@@ -13,23 +13,35 @@
import logging
-from pipeline.utils.uniqid import node_uniqid, line_uniqid
from pipeline.core.constants import PE
from pipeline.exceptions import NodeNotExistException
+from pipeline.utils.uniqid import line_uniqid, node_uniqid
logger = logging.getLogger("root")
BRANCH_SELECT_GATEWAYS = {PE.ExclusiveGateway, PE.ConditionalParallelGateway}
-def recursive_replace_id(pipeline_data):
+def _recursive_replace_id_with_node_map(pipeline_data, subprocess_id=None):
+ """
+ 替换pipeline_id 并返回 对应的 node_map 映射
+ """
pipeline_data[PE.id] = node_uniqid()
- replace_all_id(pipeline_data)
+ node_map = {}
+ replace_result_map = replace_all_id(pipeline_data)
+ pipeline_id = subprocess_id or pipeline_data[PE.id]
+ node_map[pipeline_id] = replace_result_map
activities = pipeline_data[PE.activities]
for act_id, act in list(activities.items()):
if act[PE.type] == PE.SubProcess:
- recursive_replace_id(act[PE.pipeline])
+ replace_result_map = _recursive_replace_id_with_node_map(act[PE.pipeline], act_id)
act[PE.pipeline][PE.id] = act_id
+ node_map[pipeline_id].setdefault("subprocess", {}).update(replace_result_map)
+ return node_map
+
+
+def recursive_replace_id(pipeline_data):
+ return _recursive_replace_id_with_node_map(pipeline_data)
def replace_all_id(pipeline_data):
diff --git a/runtime/bamboo-pipeline/pipeline/tests/contrib/test_graph.py b/runtime/bamboo-pipeline/pipeline/tests/contrib/test_graph.py
new file mode 100644
index 00000000..f439b1dc
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/tests/contrib/test_graph.py
@@ -0,0 +1,56 @@
+# -*- coding: utf-8 -*-
+from unittest import TestCase
+
+from pipeline.contrib.rollback.graph import RollbackGraphHandler
+
+
+class TestGraph(TestCase):
+ def test_build_rollback_graph_with_cycle(self):
+ node_map = {
+ "start": {
+ "id": "start",
+ "type": "EmptyStartEvent",
+ "targets": {"lf67ec0280323668ba383bc61c92bdce": "node_1"},
+ },
+ "node_1": {
+ "id": "node_1",
+ "type": "ServiceActivity",
+ "targets": {"l5c6729c70b83c81bd98eebefe0c46e3": "node_2"},
+ },
+ "node_2": {
+ "id": "node_2",
+ "type": "ServiceActivity",
+ "targets": {"ld09dcdaaae53cd9868b652fd4b7b074": "node_3"},
+ },
+ "node_3": {
+ "id": "node_3",
+ "type": "ExclusiveGateway",
+ "targets": {"ld9beef12dd33812bb9b697afd5f2728": "node_4", "lffeab3bdb0139b69ac6978a415e3f54": "node_1"},
+ },
+ "node_4": {
+ "id": "node_4",
+ "type": "ServiceActivity",
+ "targets": {"l995fa16e367312e99a1f8b54458ed6a": "node_5"},
+ },
+ "node_5": {
+ "id": "node_5",
+ "type": "ServiceActivity",
+ "targets": {"l802b3f8e60e39518915f85d4c943a18": "node_6"},
+ },
+ "node_6": {
+ "id": "node_6",
+ "type": "ExclusiveGateway",
+ "targets": {"l8ff0721ec8c3745b6f2183a7006d2c6": "node_7", "l5df5ee5497f3616aec4347c0e5913b8": "node_5"},
+ },
+ "node_7": {"id": "node_7", "type": "EmptyEndEvent", "targets": {}},
+ }
+
+ rollback_graph = RollbackGraphHandler(node_map=node_map, start_id="node_5", target_id="node_1")
+ graph, other_nodes = rollback_graph.build_rollback_graph()
+ self.assertListEqual(other_nodes, ["node_3"])
+ self.assertListEqual(graph.as_dict()["nodes"], ["node_5", "node_1", "END", "START", "node_2", "node_4"])
+ self.assertListEqual(
+ graph.as_dict()["flows"],
+ [["node_1", "END"], ["START", "node_5"], ["node_2", "node_1"], ["node_4", "node_2"], ["node_5", "node_4"]],
+ )
+ self.assertListEqual(list(graph.next("START")), ["node_5"])
diff --git a/runtime/bamboo-pipeline/pipeline/tests/contrib/test_node_timer_event.py b/runtime/bamboo-pipeline/pipeline/tests/contrib/test_node_timer_event.py
new file mode 100644
index 00000000..93cc0f4f
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/tests/contrib/test_node_timer_event.py
@@ -0,0 +1,288 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017-2021 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
+import copy
+import datetime
+import json
+from typing import Any, Dict, List
+
+from django.test import TestCase
+from mock import MagicMock, call, patch
+from pipeline.contrib.node_timer_event import constants, types, utils
+from pipeline.contrib.node_timer_event.models import (
+ ExpiredNodesRecord,
+ NodeTimerEventConfig,
+)
+from pipeline.contrib.node_timer_event.tasks import (
+ dispatch_expired_nodes,
+ execute_node_timer_event_action,
+)
+from pipeline.eri.models import Process, State
+
+from bamboo_engine.eri import DataInput, ExecutionData
+
+
+class ParseTimerDefinedTestCase(TestCase):
+
+ TIME_FORMAT: str = "%Y-%m-%d %H:%M:%S"
+
+ def test_time_cycle(self):
+
+ start: datetime.datetime = datetime.datetime.strptime("2022-01-01 00:00:00", self.TIME_FORMAT)
+ cases: List[Dict[str, Any]] = [
+ {
+ "defined": "R5/PT10S",
+ "repetitions": 5,
+ "timestamp": (start + datetime.timedelta(seconds=10)).timestamp(),
+ },
+ {"defined": "R1/P1D", "repetitions": 1, "timestamp": (start + datetime.timedelta(days=1)).timestamp()},
+ ]
+
+ for case in cases:
+ time_defined: types.TimeDefined = utils.parse_timer_defined(
+ timer_type=constants.TimerType.TIME_CYCLE.value, defined=case["defined"], start=start
+ )
+ self.assertEqual(time_defined["repetitions"], case["repetitions"])
+ self.assertEqual(time_defined["timestamp"], case["timestamp"])
+
+ def test_time_duration(self):
+ start: datetime.datetime = datetime.datetime.strptime("2022-01-01 00:00:00", self.TIME_FORMAT)
+ cases: List[Dict[str, Any]] = [
+ {
+ "defined": "P14DT1H30M",
+ "timestamp": (start + datetime.timedelta(days=14, hours=1, minutes=30)).timestamp(),
+ },
+ {"defined": "P14D", "timestamp": (start + datetime.timedelta(days=14)).timestamp()},
+ {
+ "defined": "P14DT1H30M",
+ "timestamp": (start + datetime.timedelta(days=14, hours=1, minutes=30)).timestamp(),
+ },
+ {"defined": "PT15S", "timestamp": (start + datetime.timedelta(seconds=15)).timestamp()},
+ ]
+
+ for case in cases:
+ time_defined: types.TimeDefined = utils.parse_timer_defined(
+ timer_type=constants.TimerType.TIME_DURATION.value, defined=case["defined"], start=start
+ )
+ self.assertEqual(time_defined["repetitions"], 1)
+ self.assertEqual(time_defined["timestamp"], case["timestamp"])
+
+ def test_time_date(self):
+ start: datetime.datetime = datetime.datetime.strptime("2022-01-01 00:00:00", self.TIME_FORMAT)
+ cases: List[Dict[str, Any]] = [
+ {"defined": "2019-10-01T12:00:00Z", "timestamp": 1569931200.0},
+ {"defined": "2019-10-02T08:09:40+02:00", "timestamp": 1569996580.0},
+ {"defined": "2019-10-02T08:09:40+02:00[Europe/Berlin]", "timestamp": 1569996580.0},
+ ]
+
+ for case in cases:
+ time_defined: types.TimeDefined = utils.parse_timer_defined(
+ timer_type=constants.TimerType.TIME_DATE.value, defined=case["defined"], start=start
+ )
+ self.assertEqual(time_defined["repetitions"], 1)
+ self.assertEqual(time_defined["timestamp"], case["timestamp"])
+
+
+class NodeTimerEventTestCase(TestCase):
+ def setUp(self):
+ self.node_id = "node_id"
+ self.version = "version"
+ self.action = "example"
+ self.root_pipeline_id = "root_pipeline_id"
+ self.pipeline_tree = {}
+ self.timer_events = [
+ {
+ "index": 1,
+ "action": self.action,
+ "timer_type": constants.TimerType.TIME_CYCLE.value,
+ "repetitions": 5,
+ "defined": "R5/PT10S",
+ },
+ {
+ "index": 2,
+ "action": self.action,
+ "timer_type": constants.TimerType.TIME_DATE.value,
+ "repetitions": 1,
+ "defined": "2019-10-01T12:00:00Z",
+ },
+ ]
+ self.timer_events_in_tree = [
+ {
+ "enable": True,
+ "action": self.action,
+ "timer_type": constants.TimerType.TIME_CYCLE.value,
+ "defined": "R5/PT10S",
+ },
+ {
+ "enable": True,
+ "action": self.action,
+ "timer_type": constants.TimerType.TIME_DATE.value,
+ "defined": "2019-10-01T12:00:00Z",
+ },
+ ]
+ runtime = MagicMock()
+ runtime.get_execution_data = MagicMock(
+ return_value=ExecutionData({"key": "value", "from": "node"}, {"key": "value"})
+ )
+ runtime.get_data_inputs = MagicMock(return_value={"key": DataInput(need_render=False, value=1)})
+ self.runtime = runtime
+ self.mock_runtime = MagicMock(return_value=runtime)
+
+ def test_dispatch_expired_nodes(self):
+ mock_execute_node_timer_event_strategy = MagicMock()
+ mock_execute_node_timer_event_strategy.apply_async = MagicMock()
+ with patch(
+ "pipeline.contrib.node_timer_event.tasks.execute_node_timer_event_action",
+ mock_execute_node_timer_event_strategy,
+ ):
+ ExpiredNodesRecord.objects.create(
+ id=1,
+ nodes=json.dumps(
+ [
+ "bamboo:v1:node_timer_event:node:node1:version:version1:index:1",
+ "bamboo:v1:node_timer_event:node:node2:version:version2:index:1",
+ ]
+ ),
+ )
+
+ dispatch_expired_nodes(record_id=1)
+ mock_execute_node_timer_event_strategy.apply_async.assert_has_calls(
+ [
+ call(kwargs={"node_id": "node1", "version": "version1", "index": 1}),
+ call(kwargs={"node_id": "node2", "version": "version2", "index": 1}),
+ ]
+ )
+
+ self.assertFalse(ExpiredNodesRecord.objects.filter(id=1).exists())
+
+ def execute_node_timeout_action_success_test_helper(self, index: int):
+ NodeTimerEventConfig.objects.create(
+ root_pipeline_id=self.root_pipeline_id, node_id=self.node_id, events=json.dumps(self.timer_events)
+ )
+ Process.objects.create(root_pipeline_id=self.root_pipeline_id, current_node_id=self.node_id, priority=1)
+ State.objects.create(node_id=self.node_id, name="name", version=self.version)
+
+ redis_inst = MagicMock()
+ redis_inst.incr = MagicMock(return_value=b"2")
+ redis_inst.zadd = MagicMock(return_value=b"1")
+
+ key: str = f"bamboo:v1:node_timer_event:node:{self.node_id}:version:{self.version}:index:{index}"
+ with patch("pipeline.contrib.node_timer_event.handlers.BambooDjangoRuntime", self.mock_runtime):
+ with patch("pipeline.contrib.node_timer_event.adapter.node_timer_event_settings.redis_inst", redis_inst):
+ result = execute_node_timer_event_action(self.node_id, self.version, index=index)
+ self.assertEqual(result["result"], True)
+ self.runtime.get_execution_data.assert_called_once_with(self.node_id)
+ self.runtime.get_data_inputs.assert_called_once_with(self.root_pipeline_id)
+ redis_inst.incr.assert_called_once_with(key, 1)
+ if index == 1:
+ redis_inst.zadd.assert_called_once()
+ else:
+ redis_inst.zadd.assert_not_called()
+
+ def test_execute_node_timer_event_action_success__time_cycle(self):
+ """测试时间循环:成功调度时,投递下一个节点"""
+ self.execute_node_timeout_action_success_test_helper(index=1)
+
+ def test_execute_node_timer_event_action_success__time_date(self):
+ """测试具体时间日期:无需进行下次调度"""
+ self.execute_node_timeout_action_success_test_helper(index=2)
+
+ def test_execute_node_timer_event_action_not_current_node(self):
+ NodeTimerEventConfig.objects.create(
+ root_pipeline_id=self.root_pipeline_id, node_id=self.node_id, events=json.dumps(self.timer_events)
+ )
+ Process.objects.create(root_pipeline_id=self.root_pipeline_id, current_node_id="next_node", priority=1)
+ State.objects.create(node_id=self.node_id, name="name", version=self.version)
+ with patch("pipeline.contrib.node_timer_event.handlers.BambooDjangoRuntime", self.mock_runtime):
+ result = execute_node_timer_event_action(self.node_id, self.version, index=1)
+ self.assertEqual(result["result"], False)
+ self.runtime.get_data_inputs.assert_not_called()
+ self.runtime.get_execution_data.assert_not_called()
+
+ def test_execute_node_timer_event_action_not_current_version(self):
+ NodeTimerEventConfig.objects.create(
+ root_pipeline_id=self.root_pipeline_id, node_id=self.node_id, events=json.dumps(self.timer_events)
+ )
+ Process.objects.create(root_pipeline_id=self.root_pipeline_id, current_node_id=self.node_id, priority=1)
+ State.objects.create(node_id=self.node_id, name="name", version="ano_version")
+
+ with patch("pipeline.contrib.node_timer_event.handlers.BambooDjangoRuntime", self.mock_runtime):
+ result = execute_node_timer_event_action(self.node_id, self.version, index=1)
+ self.assertEqual(result["result"], False)
+ self.runtime.get_data_inputs.assert_not_called()
+ self.runtime.get_execution_data.assert_not_called()
+
+ def test_parse_node_timer_event_configs_success(self):
+ pipeline_tree = {
+ "activities": {
+ "act_1": {"type": "ServiceActivity", "events": {"timer_events": self.timer_events_in_tree}},
+ "act_2": {"type": "ServiceActivity", "events": {"timer_events": self.timer_events_in_tree}},
+ }
+ }
+
+ parse_result = NodeTimerEventConfig.objects.parse_node_timer_event_configs(pipeline_tree)
+ self.assertEqual(parse_result["result"], True)
+ self.assertEqual(
+ parse_result["data"],
+ [{"node_id": "act_1", "events": self.timer_events}, {"node_id": "act_2", "events": self.timer_events}],
+ )
+
+ def test_parse_node_timer_event_configs_fail_and_ignore(self):
+
+ timer_events_in_tree_act_1 = copy.deepcopy(self.timer_events_in_tree)
+ timer_events_in_tree_act_1[1]["defined"] = "invalid defined"
+
+ timer_events_in_tree_act_2 = copy.deepcopy(self.timer_events_in_tree)
+ timer_events_in_tree_act_2[1]["timer_type"] = "invalid timer_type"
+
+ pipeline_tree = {
+ "activities": {
+ "act_1": {"type": "ServiceActivity", "events": {"timer_events": timer_events_in_tree_act_1}},
+ "act_2": {"type": "ServiceActivity", "events": {"timer_events": timer_events_in_tree_act_2}},
+ }
+ }
+ parse_result = NodeTimerEventConfig.objects.parse_node_timer_event_configs(pipeline_tree)
+ self.assertEqual(parse_result["result"], True)
+ self.assertEqual(
+ parse_result["data"],
+ [
+ {"node_id": "act_1", "events": [self.timer_events[0]]},
+ {"node_id": "act_2", "events": [self.timer_events[0]]},
+ ],
+ )
+
+ def test_batch_create_node_timer_config_success(self):
+ config_parse_result = {
+ "result": True,
+ "data": [
+ {"node_id": "act_1", "events": self.timer_events},
+ {"node_id": "act_2", "events": self.timer_events},
+ ],
+ "message": "",
+ }
+ with patch(
+ "pipeline.contrib.node_timer_event.models.NodeTimerEventConfig.objects.parse_node_timer_event_configs",
+ MagicMock(return_value=config_parse_result),
+ ):
+ NodeTimerEventConfig.objects.batch_create_node_timer_event_config(self.root_pipeline_id, self.pipeline_tree)
+ config_count = len(NodeTimerEventConfig.objects.all())
+ self.assertEqual(config_count, 2)
+
+ def test_batch_create_node_timer_config_fail(self):
+ config_parse_result = {"result": False, "data": "", "message": "test fail"}
+ with patch(
+ "pipeline.contrib.node_timer_event.models.NodeTimerEventConfig.objects.parse_node_timer_event_configs",
+ MagicMock(return_value=config_parse_result),
+ ):
+ NodeTimerEventConfig.objects.batch_create_node_timer_event_config(self.root_pipeline_id, self.pipeline_tree)
+ config_count = NodeTimerEventConfig.objects.count()
+ self.assertEqual(config_count, 0)
diff --git a/runtime/bamboo-pipeline/pipeline/tests/contrib/test_plugin_execute.py b/runtime/bamboo-pipeline/pipeline/tests/contrib/test_plugin_execute.py
new file mode 100644
index 00000000..1bb3e0f0
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/tests/contrib/test_plugin_execute.py
@@ -0,0 +1,138 @@
+# # -*- coding: utf-8 -*-
+# """
+# Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+# Edition) available.
+# Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+# Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+# http://opensource.org/licenses/MIT
+# 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.
+# """
+
+from unittest import TestCase
+
+from mock.mock import MagicMock
+from pipeline.component_framework.library import ComponentLibrary
+from pipeline.contrib.plugin_execute import api
+from pipeline.contrib.plugin_execute.models import PluginExecuteTask
+from pipeline.contrib.plugin_execute.tasks import execute, schedule
+from pipeline.tests import mock
+from pipeline_test_use.components.collections.atom import (
+ DebugCallbackComponent,
+ InterruptDummyExecuteComponent,
+)
+
+mock_execute = MagicMock()
+mock_execute.apply_async = MagicMock(return_value=True)
+
+mock_schedule = MagicMock()
+mock_schedule.apply_async = MagicMock(return_value=True)
+
+
+class TestPluginExecuteBase(TestCase):
+ @mock.patch("pipeline.contrib.plugin_execute.handler.execute", MagicMock(return_value=mock_execute))
+ def test_run(self):
+ task_id = api.run("debug_callback_node", "legacy", {"hello": "world"}, {"hello": "world"}).data
+ task = PluginExecuteTask.objects.get(id=task_id)
+ self.assertEqual(task.state, "READY")
+ self.assertDictEqual(task.callback_data, {})
+ self.assertDictEqual(task.contexts, {"hello": "world"})
+ self.assertDictEqual(task.inputs, {"hello": "world"})
+
+ @mock.patch("pipeline.contrib.plugin_execute.handler.execute", MagicMock(return_value=mock_execute))
+ def test_get_state(self):
+ task_id = api.run("debug_callback_node", "legacy", {"hello": "world"}, {"hello": "world"}).data
+ state = api.get_state(task_id).data
+ self.assertEqual(state["state"], "READY")
+ self.assertDictEqual(state["inputs"], {"hello": "world"})
+ self.assertDictEqual(state["contexts"], {"hello": "world"})
+
+ @mock.patch("pipeline.contrib.plugin_execute.handler.execute", MagicMock(return_value=mock_execute))
+ def test_retry(self):
+ task_id = api.run("debug_callback_node", "legacy", {"hello": "world"}, {"hello": "world"}).data
+ task = PluginExecuteTask.objects.get(id=task_id)
+ result = api.retry(task_id, {})
+
+ self.assertFalse(result.result)
+
+ task.state = "FAILED"
+ task.save()
+
+ result = api.retry(task_id, {"hello": "tim"}, {"hello": "jav"})
+ self.assertEqual(result.result, True)
+
+ task.refresh_from_db()
+
+ self.assertEqual(task.state, "READY")
+ self.assertDictEqual(task.inputs, {"hello": "tim"})
+ self.assertDictEqual(task.contexts, {"hello": "jav"})
+
+ @mock.patch("pipeline.contrib.plugin_execute.handler.schedule", MagicMock(return_value=mock_schedule))
+ @mock.patch("pipeline.contrib.plugin_execute.handler.execute", MagicMock(return_value=mock_execute))
+ def test_callback(self):
+ task_id = api.run("debug_callback_node", "legacy", {"hello": "world"}, {"hello": "world"}).data
+ task = PluginExecuteTask.objects.get(id=task_id)
+ result = api.retry(task_id, {})
+
+ self.assertFalse(result.result)
+
+ task.state = "RUNNING"
+ task.save()
+
+ result = api.callback(task_id, {"hello": "sandri"})
+ self.assertEqual(result.result, True)
+
+ task.refresh_from_db()
+ self.assertDictEqual(task.callback_data, {"hello": "sandri"})
+
+ @mock.patch("pipeline.contrib.plugin_execute.handler.execute", MagicMock(return_value=mock_execute))
+ def test_force_fail(self):
+ task_id = api.run("debug_callback_node", "legacy", {"hello": "world"}, {"hello": "world"}).data
+ task = PluginExecuteTask.objects.get(id=task_id)
+ result = api.forced_fail(task_id)
+
+ self.assertFalse(result.result)
+
+ task.state = "RUNNING"
+ task.save()
+
+ result = api.forced_fail(task_id)
+ self.assertEqual(result.result, True)
+
+ task.refresh_from_db()
+ self.assertEqual(task.state, "FAILED")
+
+ def test_execute_task(self):
+ task = PluginExecuteTask.objects.create(
+ state="READY",
+ inputs={"time": 1},
+ version="legacy",
+ component_code="interrupt_dummy_exec_node",
+ contexts={},
+ runtime_attrs={},
+ )
+ ComponentLibrary.register_component("interrupt_dummy_exec_node", "legacy", InterruptDummyExecuteComponent)
+ execute(task.id)
+ task.refresh_from_db()
+ self.assertEqual(task.state, "FINISHED")
+ self.assertDictEqual(task.outputs, {"execute_count": 1})
+
+ def test_schedule_task(self):
+ task = PluginExecuteTask.objects.create(
+ state="READY",
+ inputs={},
+ version="legacy",
+ component_code="debug_callback_node",
+ contexts={},
+ runtime_attrs={},
+ )
+ ComponentLibrary.register_component("debug_callback_node", "legacy", DebugCallbackComponent)
+ task = PluginExecuteTask.objects.get(id=task.id)
+ task.callback_data = {"bit": 1}
+ task.save()
+ execute(task.id)
+ schedule(task.id)
+ task.refresh_from_db()
+ self.assertEqual(task.state, "FINISHED")
diff --git a/runtime/bamboo-pipeline/pipeline/tests/contrib/test_rollback.py b/runtime/bamboo-pipeline/pipeline/tests/contrib/test_rollback.py
index e52b2982..9fc47f6b 100644
--- a/runtime/bamboo-pipeline/pipeline/tests/contrib/test_rollback.py
+++ b/runtime/bamboo-pipeline/pipeline/tests/contrib/test_rollback.py
@@ -1,80 +1,87 @@
-# -*- coding: utf-8 -*-
-"""
-Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
-Edition) available.
-Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
-Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
-You may obtain a copy of the License at
-http://opensource.org/licenses/MIT
-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.
-"""
+# # -*- coding: utf-8 -*-
+# """
+# Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+# Edition) available.
+# Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+# Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+# http://opensource.org/licenses/MIT
+# 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.
+# """
import json
import mock
from django.test import TestCase
-from django.utils import timezone
from mock.mock import MagicMock
from pipeline.contrib.rollback import api
-from pipeline.contrib.rollback.handler import RollBackHandler
+from pipeline.contrib.rollback.models import (
+ RollbackPlan,
+ RollbackSnapshot,
+ RollbackToken,
+)
from pipeline.core.constants import PE
from pipeline.eri.models import Node, Process, State
from bamboo_engine import states
from bamboo_engine.utils.string import unique_id
-forced_fail_activity_mock = MagicMock()
-forced_fail_activity_mock.result = True
+token_rollback = MagicMock()
+token_rollback.apply_async = MagicMock(return_value=True)
class TestRollBackBase(TestCase):
- def setUp(self) -> None:
- self.started_time = timezone.now()
- self.archived_time = timezone.now()
-
- @mock.patch("bamboo_engine.api.forced_fail_activity", MagicMock(return_value=forced_fail_activity_mock))
- @mock.patch("pipeline.eri.runtime.BambooDjangoRuntime.execute", MagicMock())
+ @mock.patch("pipeline.contrib.rollback.handler.token_rollback", MagicMock(return_value=token_rollback))
def test_rollback(self):
pipeline_id = unique_id("n")
- State.objects.create(
- node_id=pipeline_id,
- root_id=pipeline_id,
- parent_id=pipeline_id,
- name=states.FINISHED,
- version=unique_id("v"),
- started_time=self.started_time,
- archived_time=self.archived_time,
+ pipeline_state = State.objects.create(
+ node_id=pipeline_id, root_id=pipeline_id, parent_id=pipeline_id, name=states.FAILED, version=unique_id("v")
)
- node_id_1 = unique_id("n")
- node_id_2 = unique_id("n")
- State.objects.create(
- node_id=node_id_1,
+ start_node_id = unique_id("n")
+ start_state = State.objects.create(
+ node_id=start_node_id,
root_id=pipeline_id,
parent_id=pipeline_id,
name=states.RUNNING,
version=unique_id("v"),
- started_time=self.started_time,
- archived_time=self.archived_time,
)
+ target_node_id = unique_id("n")
State.objects.create(
- node_id=node_id_2,
+ node_id=target_node_id,
root_id=pipeline_id,
parent_id=pipeline_id,
- name=states.RUNNING,
+ name=states.FINISHED,
version=unique_id("v"),
- started_time=self.started_time,
- archived_time=self.archived_time,
)
- node_id_1_detail = {
- "id": "n0be4eaa13413f9184863776255312f1",
- "type": PE.ParallelGateway,
- "targets": {"l7895e18cd7c33b198d56534ca332227": node_id_2},
- "root_pipeline_id": "n3369d7ce884357f987af1631bda69cb",
- "parent_pipeline_id": "n3369d7ce884357f987af1631bda69cb",
+ result = api.rollback(pipeline_id, start_node_id, target_node_id)
+ self.assertFalse(result.result)
+ message = (
+ "rollback failed: the task of non-running state is not allowed to roll back,"
+ " pipeline_id={}, state=FAILED".format(pipeline_id)
+ )
+ self.assertEqual(str(result.exc), message)
+ pipeline_state.name = states.RUNNING
+ pipeline_state.save()
+
+ token = RollbackToken.objects.create(
+ root_pipeline_id=pipeline_id, token=json.dumps({target_node_id: "xxx", start_node_id: "xsx"})
+ )
+
+ result = api.rollback(pipeline_id, start_node_id, target_node_id)
+ self.assertFalse(result.result)
+ message = "rollback failed: node not exist, node={}".format(start_node_id)
+ self.assertEqual(str(result.exc), message)
+
+ target_node_detail = {
+ "id": target_node_id,
+ "type": PE.ServiceActivity,
+ "targets": {target_node_id: start_node_id},
+ "root_pipeline_id": pipeline_id,
+ "parent_pipeline_id": pipeline_id,
"can_skip": True,
"code": "bk_display",
"version": "v1.0",
@@ -82,14 +89,12 @@ def test_rollback(self):
"can_retry": True,
}
- Node.objects.create(node_id=node_id_1, detail=json.dumps(node_id_1_detail))
-
- node_id_2_detail = {
- "id": "n0be4eaa13413f9184863776255312f1",
- "type": PE.ParallelGateway,
- "targets": {"l7895e18cd7c33b198d56534ca332227": unique_id("n")},
- "root_pipeline_id": "n3369d7ce884357f987af1631bda69cb",
- "parent_pipeline_id": "n3369d7ce884357f987af1631bda69cb",
+ start_node_detail = {
+ "id": start_node_id,
+ "type": PE.ServiceActivity,
+ "targets": {},
+ "root_pipeline_id": pipeline_id,
+ "parent_pipeline_id": pipeline_id,
"can_skip": True,
"code": "bk_display",
"version": "v1.0",
@@ -97,151 +102,163 @@ def test_rollback(self):
"can_retry": True,
}
- Node.objects.create(node_id=node_id_2, detail=json.dumps(node_id_2_detail))
+ Node.objects.create(node_id=target_node_id, detail=json.dumps(target_node_detail))
+ Node.objects.create(node_id=start_node_id, detail=json.dumps(start_node_detail))
- # pipeline_id 非running的情况下会异常
- message = "rollback failed: the task of non-running state is not allowed to roll back, pipeline_id={}".format(
- pipeline_id
- )
- result = api.rollback(pipeline_id, pipeline_id)
+ result = api.rollback(pipeline_id, start_node_id, target_node_id)
self.assertFalse(result.result)
+ message = "rollback failed: only allows rollback to finished node, allowed states ['FINISHED', 'FAILED']"
self.assertEqual(str(result.exc), message)
- State.objects.filter(node_id=pipeline_id).update(name=states.RUNNING)
- # pipeline_id 非running的情况下会异常
- message = "rollback failed: only allows rollback to ServiceActivity type nodes"
- result = api.rollback(pipeline_id, node_id_1)
+ start_state.name = states.FINISHED
+ start_state.save()
+
+ result = api.rollback(pipeline_id, start_node_id, target_node_id)
self.assertFalse(result.result)
+ message = "rollback failed: start node token must equal target node, pipeline_id={}".format(pipeline_id)
self.assertEqual(str(result.exc), message)
- node_id_1_detail["type"] = PE.ServiceActivity
- Node.objects.filter(node_id=node_id_1).update(detail=json.dumps(node_id_1_detail))
+ token.token = json.dumps({target_node_id: "xxx", start_node_id: "xxx"})
+ token.save()
+
+ result = api.rollback(pipeline_id, start_node_id, target_node_id)
+ self.assertTrue(result.result)
+ rollback_snapshot = RollbackSnapshot.objects.get(root_pipeline_id=pipeline_id)
+ self.assertEqual(json.loads(rollback_snapshot.skip_rollback_nodes), [])
+ self.assertEqual(len(json.loads(rollback_snapshot.graph)["nodes"]), 4)
+
+ pipeline_state.refresh_from_db()
+ self.assertEqual(pipeline_state.name, states.ROLLING_BACK)
- message = "rollback failed: only allows rollback to finished node"
- result = api.rollback(pipeline_id, node_id_1)
+ def test_reserve_rollback(self):
+ pipeline_id = unique_id("n")
+ State.objects.create(
+ node_id=pipeline_id,
+ root_id=pipeline_id,
+ parent_id=pipeline_id,
+ name=states.RUNNING,
+ version=unique_id("v")
+ # noqa
+ )
+
+ start_node_id = unique_id("n")
+ start_state = State.objects.create(
+ node_id=start_node_id,
+ root_id=pipeline_id,
+ parent_id=pipeline_id,
+ name=states.FINISHED,
+ version=unique_id("v"),
+ )
+
+ target_node_id = unique_id("n")
+ State.objects.create(
+ node_id=target_node_id,
+ root_id=pipeline_id,
+ parent_id=pipeline_id,
+ name=states.FINISHED,
+ version=unique_id("v"),
+ )
+
+ Process.objects.create(root_pipeline_id=pipeline_id, current_node_id=start_node_id, priority=1)
+ result = api.reserve_rollback(pipeline_id, start_node_id, target_node_id)
self.assertFalse(result.result)
+ message = "rollback failed: pipeline token not exist, pipeline_id={}".format(pipeline_id)
self.assertEqual(str(result.exc), message)
- State.objects.filter(node_id=node_id_1).update(name=states.FINISHED)
-
- p = Process.objects.create(
- root_pipeline_id=pipeline_id,
- parent_id=-1,
- current_node_id=node_id_2,
- pipeline_stack=json.dumps([pipeline_id]),
- priority=1,
+
+ RollbackToken.objects.create(
+ root_pipeline_id=pipeline_id, token=json.dumps({target_node_id: "xxx", start_node_id: "xxx"})
)
- result = api.rollback(pipeline_id, node_id_1)
- self.assertTrue(result.result)
+ result = api.reserve_rollback(pipeline_id, start_node_id, target_node_id)
+ self.assertFalse(result.result)
+ message = "rollback failed: node not exist, node={}".format(target_node_id)
+ self.assertEqual(str(result.exc), message)
- p.refresh_from_db()
- self.assertEqual(p.current_node_id, node_id_1)
- # 验证Node2 是不是被删除了
- self.assertFalse(State.objects.filter(node_id=node_id_2).exists())
-
- state = State.objects.get(node_id=node_id_1)
- self.assertEqual(state.name, states.READY)
-
- def test_compute_validate_nodes(self):
- node_map = {
- "node_1": {
- "id": "node_1",
- "type": "EmptyStartEvent",
- "targets": {"n": "node_2"},
- },
- "node_2": {
- "id": "node_2",
- "type": "ServiceActivity",
- "targets": {"n": "node_3"},
- },
- "node_3": {
- "id": "node_3",
- "type": "ServiceActivity",
- "targets": {"n": "node_4"},
- },
- "node_4": {
- "id": "node_4",
- "type": "ParallelGateway",
- "targets": {"n": "node_5", "n1": "node_6"},
- "converge_gateway_id": "node_7",
- },
- "node_5": {
- "id": "node_5",
- "type": "ServiceActivity",
- "targets": {"n": "node_7"},
- },
- "node_6": {
- "id": "node_6",
- "type": "ServiceActivity",
- "targets": {"n": "node_7"},
- },
- "node_7": {
- "id": "node_7",
- "type": "ConvergeGateway",
- "targets": {"n": "node_8"},
- },
- "node_8": {
- "id": "node_8",
- "type": "ExclusiveGateway",
- "targets": {"n1": "node_13", "n2": "node_9", "n3": "node_3"},
- },
- "node_9": {
- "id": "node_9",
- "type": "ServiceActivity",
- "targets": {"n": "node_10"},
- },
- "node_10": {
- "id": "node_10",
- "type": "ExclusiveGateway",
- "targets": {"n": "node_11", "n2": "node_12"},
- },
+ target_node_detail = {
+ "id": target_node_id,
+ "type": PE.ServiceActivity,
+ "targets": {target_node_id: start_node_id},
+ "root_pipeline_id": pipeline_id,
+ "parent_pipeline_id": pipeline_id,
+ "can_skip": True,
+ "code": "bk_display",
+ "version": "v1.0",
+ "error_ignorable": True,
+ "can_retry": True,
}
- node_id = "node_1"
- nodes = RollBackHandler("p", node_map)._compute_validate_nodes(node_id, node_map)
- self.assertListEqual(nodes, ["node_2", "node_3", "node_9"])
+ start_node_detail = {
+ "id": start_node_id,
+ "type": PE.ServiceActivity,
+ "targets": {},
+ "root_pipeline_id": pipeline_id,
+ "parent_pipeline_id": pipeline_id,
+ "can_skip": True,
+ "code": "bk_display",
+ "version": "v1.0",
+ "error_ignorable": True,
+ "can_retry": True,
+ }
- def test_get_allowed_rollback_node_id_list(self):
- pipeline_id = unique_id("n")
+ Node.objects.create(node_id=target_node_id, detail=json.dumps(target_node_detail))
+ Node.objects.create(node_id=start_node_id, detail=json.dumps(start_node_detail))
+ result = api.reserve_rollback(pipeline_id, start_node_id, target_node_id)
+ self.assertFalse(result.result)
+ message = "reserve rollback failed, the node state is not Running, current state=FINISHED, node_id={}".format(
+ # noqa
+ start_node_id
+ )
+ self.assertEqual(str(result.exc), message)
+
+ start_state.name = states.RUNNING
+ start_state.save()
+
+ result = api.reserve_rollback(pipeline_id, start_node_id, target_node_id)
+ self.assertTrue(result.result)
+
+ plan = RollbackPlan.objects.get(root_pipeline_id=pipeline_id)
+
+ self.assertEqual(plan.start_node_id, start_node_id)
+ self.assertEqual(plan.target_node_id, target_node_id)
+
+ result = api.cancel_reserved_rollback(pipeline_id, start_node_id, target_node_id)
+ self.assertTrue(result.result)
+
+ def test_allowed_rollback_node_id_list(self):
+ pipeline_id = unique_id("n")
State.objects.create(
node_id=pipeline_id,
root_id=pipeline_id,
parent_id=pipeline_id,
name=states.RUNNING,
- version=unique_id("v"),
- started_time=self.started_time,
- archived_time=self.archived_time,
+ version=unique_id("v")
+ # noqa
)
-
- node_id_1 = unique_id("n")
- node_id_2 = unique_id("n")
+ start_node_id = unique_id("n")
State.objects.create(
- node_id=node_id_1,
+ node_id=start_node_id,
root_id=pipeline_id,
parent_id=pipeline_id,
name=states.FINISHED,
version=unique_id("v"),
- started_time=self.started_time,
- archived_time=self.archived_time,
)
+ target_node_id = unique_id("n")
State.objects.create(
- node_id=node_id_2,
+ node_id=target_node_id,
root_id=pipeline_id,
parent_id=pipeline_id,
- name=states.RUNNING,
+ name=states.FINISHED,
version=unique_id("v"),
- started_time=self.started_time,
- archived_time=self.archived_time,
)
- node_id_1_detail = {
- "id": "n0be4eaa13413f9184863776255312f1",
+ target_node_detail = {
+ "id": target_node_id,
"type": PE.ServiceActivity,
- "targets": {"l7895e18cd7c33b198d56534ca332227": node_id_2},
- "root_pipeline_id": "n3369d7ce884357f987af1631bda69cb",
- "parent_pipeline_id": "n3369d7ce884357f987af1631bda69cb",
+ "targets": {target_node_id: start_node_id},
+ "root_pipeline_id": pipeline_id,
+ "parent_pipeline_id": pipeline_id,
"can_skip": True,
"code": "bk_display",
"version": "v1.0",
@@ -249,14 +266,12 @@ def test_get_allowed_rollback_node_id_list(self):
"can_retry": True,
}
- Node.objects.create(node_id=node_id_1, detail=json.dumps(node_id_1_detail))
-
- node_id_2_detail = {
- "id": "n0be4eaa13413f9184863776255312f1",
+ start_node_detail = {
+ "id": start_node_id,
"type": PE.ServiceActivity,
- "targets": {"l7895e18cd7c33b198d56534ca332227": unique_id("n")},
- "root_pipeline_id": "n3369d7ce884357f987af1631bda69cb",
- "parent_pipeline_id": "n3369d7ce884357f987af1631bda69cb",
+ "targets": {},
+ "root_pipeline_id": pipeline_id,
+ "parent_pipeline_id": pipeline_id,
"can_skip": True,
"code": "bk_display",
"version": "v1.0",
@@ -264,8 +279,62 @@ def test_get_allowed_rollback_node_id_list(self):
"can_retry": True,
}
- Node.objects.create(node_id=node_id_2, detail=json.dumps(node_id_2_detail))
+ Node.objects.create(node_id=target_node_id, detail=json.dumps(target_node_detail))
+ Node.objects.create(node_id=start_node_id, detail=json.dumps(start_node_detail))
+
+ RollbackToken.objects.create(
+ root_pipeline_id=pipeline_id, token=json.dumps({target_node_id: "xxx", start_node_id: "xxx"})
+ )
+
+ result = api.get_allowed_rollback_node_id_list(pipeline_id, start_node_id)
+ self.assertTrue(result.result)
+ self.assertEqual(len(result.data), 1)
+ self.assertEqual(result.data[0], target_node_id)
+
+ @mock.patch("pipeline.contrib.rollback.handler.token_rollback", MagicMock(return_value=token_rollback))
+ def test_retry_rollback_failed_node(self):
+ root_pipeline_id = unique_id("n")
+ pipeline_state = State.objects.create(
+ node_id=root_pipeline_id,
+ root_id=root_pipeline_id,
+ parent_id=root_pipeline_id,
+ name=states.RUNNING,
+ version=unique_id("v"),
+ )
+ node_id = unique_id("n")
+ node_state = State.objects.create(
+ node_id=node_id,
+ root_id=root_pipeline_id,
+ parent_id=root_pipeline_id,
+ name=states.FINISHED,
+ version=unique_id("v"),
+ )
+ result = api.retry_rollback_failed_node(root_pipeline_id, node_id)
+ self.assertFalse(result.result)
+ message = "rollback failed: only retry the failed pipeline, current_status=RUNNING"
+ self.assertEqual(str(result.exc), message)
+
+ pipeline_state.name = states.ROLL_BACK_FAILED
+ pipeline_state.save()
+
+ result = api.retry_rollback_failed_node(root_pipeline_id, node_id)
+ self.assertFalse(result.result)
+ message = "rollback failed: only retry the failed node, current_status=FINISHED"
+ self.assertEqual(str(result.exc), message)
+
+ node_state.name = states.ROLL_BACK_FAILED
+ node_state.save()
- result = api.get_allowed_rollback_node_id_list(pipeline_id)
- self.assertEqual(result.result, True)
- self.assertEqual(result.data, [node_id_1])
+ result = api.retry_rollback_failed_node(root_pipeline_id, node_id)
+ self.assertFalse(result.result)
+ message = "rollback failed: the rollback snapshot is not exists, please check"
+ self.assertEqual(str(result.exc), message)
+
+ RollbackSnapshot.objects.create(
+ root_pipeline_id=root_pipeline_id,
+ graph=json.dumps({}),
+ node_access_record=json.dumps({}),
+ )
+
+ result = api.retry_rollback_failed_node(root_pipeline_id, node_id)
+ self.assertTrue(result.result)
diff --git a/runtime/bamboo-pipeline/pipeline/tests/parser/__init__.py b/runtime/bamboo-pipeline/pipeline/tests/parser/__init__.py
new file mode 100644
index 00000000..40a96afc
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/tests/parser/__init__.py
@@ -0,0 +1 @@
+# -*- coding: utf-8 -*-
diff --git a/runtime/bamboo-pipeline/pipeline/tests/parser/test_replace.py b/runtime/bamboo-pipeline/pipeline/tests/parser/test_replace.py
new file mode 100644
index 00000000..b7830800
--- /dev/null
+++ b/runtime/bamboo-pipeline/pipeline/tests/parser/test_replace.py
@@ -0,0 +1,104 @@
+# -*- coding: utf-8 -*-
+
+from django.test import TestCase
+from pipeline.parser.utils import recursive_replace_id, replace_all_id
+
+from bamboo_engine.builder import (
+ ConvergeGateway,
+ Data,
+ EmptyEndEvent,
+ EmptyStartEvent,
+ ExclusiveGateway,
+ ParallelGateway,
+ ServiceActivity,
+ SubProcess,
+ Var,
+ build_tree,
+ builder,
+)
+
+
+class ReplaceTests(TestCase):
+ def test_replace_all_id(self):
+ start = EmptyStartEvent()
+ act = ServiceActivity(component_code="example_component")
+ end = EmptyEndEvent()
+ start.extend(act).extend(end)
+ pipeline = builder.build_tree(start)
+ node_map = replace_all_id(pipeline)
+ self.assertIsInstance(node_map, dict)
+ self.assertIn(pipeline["start_event"]["id"], node_map["start_event"][start.id])
+ self.assertIn(pipeline["end_event"]["id"], node_map["end_event"][end.id])
+ self.assertEqual(list(pipeline["activities"].keys())[0], node_map["activities"][act.id])
+
+ def test_replace_all_id_gateway(self):
+ start = EmptyStartEvent()
+ pg = ParallelGateway()
+ act_1 = ServiceActivity(component_code="pipe_example_component", name="act_1")
+ act_2 = ServiceActivity(component_code="pipe_example_component", name="act_2")
+ act_3 = ServiceActivity(component_code="pipe_example_component", name="act_3")
+ cg = ConvergeGateway()
+ end = EmptyEndEvent()
+
+ start.extend(pg).connect(act_1, act_2, act_3).to(pg).converge(cg).extend(end)
+ pipeline = build_tree(start)
+ node_map = replace_all_id(pipeline)
+
+ self.assertIn(pg.id, node_map["gateways"].keys())
+ self.assertIn(cg.id, node_map["gateways"].keys())
+
+ self.assertIn(node_map["gateways"][pg.id], pipeline["gateways"].keys())
+ self.assertIn(node_map["gateways"][cg.id], pipeline["gateways"].keys())
+
+ def test_recursive_replace_id(self):
+ start = EmptyStartEvent()
+ pg = ParallelGateway()
+ act_1 = ServiceActivity(component_code="pipe_example_component", name="act_1")
+ act_2 = ServiceActivity(component_code="pipe_example_component", name="act_2")
+ act_3 = ServiceActivity(component_code="pipe_example_component", name="act_3")
+ cg = ConvergeGateway()
+ end = EmptyEndEvent()
+ start.extend(pg).connect(act_1, act_2, act_3).to(pg).converge(cg).extend(end)
+ pipeline = build_tree(start)
+ node_map = recursive_replace_id(pipeline)
+ self.assertIn(pg.id, node_map[pipeline["id"]]["gateways"].keys())
+ self.assertIn(cg.id, node_map[pipeline["id"]]["gateways"].keys())
+ self.assertIn(node_map[pipeline["id"]]["gateways"][pg.id], pipeline["gateways"].keys())
+ self.assertIn(node_map[pipeline["id"]]["gateways"][cg.id], pipeline["gateways"].keys())
+ self.assertIn(act_1.id, node_map[pipeline["id"]]["activities"].keys())
+
+ def test_recursive_replace_id_with_subprocess(self):
+ def sub_process(data):
+ subproc_start = EmptyStartEvent()
+ subproc_act = ServiceActivity(component_code="pipe_example_component", name="sub_act")
+ subproc_end = EmptyEndEvent()
+
+ subproc_start.extend(subproc_act).extend(subproc_end)
+
+ subproc_act.component.inputs.sub_input = Var(type=Var.SPLICE, value="${sub_input}")
+
+ return SubProcess(start=subproc_start, data=data)
+
+ start = EmptyStartEvent()
+ act_1 = ServiceActivity(component_code="pipe_example_component", name="act_1")
+ eg = ExclusiveGateway(conditions={0: "${act_1_output} < 0", 1: "${act_1_output} >= 0"}, name="act_2 or act_3")
+
+ sub_pipeline_data_1 = Data(inputs={"${sub_input}": Var(type=Var.PLAIN, value=1)})
+ subproc_1 = sub_process(sub_pipeline_data_1)
+
+ sub_pipeline_data_2 = Data(inputs={"${sub_input}": Var(type=Var.PLAIN, value=2)})
+ subproc_2 = sub_process(sub_pipeline_data_2)
+ end = EmptyEndEvent()
+
+ start.extend(act_1).extend(eg).connect(subproc_1, subproc_2).converge(end)
+
+ pipeline = build_tree(start)
+ node_map = recursive_replace_id(pipeline)
+
+ self.assertEqual(len(node_map[pipeline["id"]]["subprocess"].keys()), 2)
+ self.assertIn(
+ node_map[pipeline["id"]]["activities"][subproc_1.id], node_map[pipeline["id"]]["subprocess"].keys()
+ )
+ self.assertIn(
+ node_map[pipeline["id"]]["activities"][subproc_2.id], node_map[pipeline["id"]]["subprocess"].keys()
+ )
diff --git a/runtime/bamboo-pipeline/poetry.lock b/runtime/bamboo-pipeline/poetry.lock
index f32e3435..1c575f02 100644
--- a/runtime/bamboo-pipeline/poetry.lock
+++ b/runtime/bamboo-pipeline/poetry.lock
@@ -57,7 +57,7 @@ tests_no_zope = ["hypothesis", "pympler", "pytest (>=4.3.0)", "pytest-xdist", "c
[[package]]
name = "bamboo-engine"
-version = "2.9.0"
+version = "2.10.0rc4"
description = "Bamboo-engine is a general-purpose workflow engine"
category = "main"
optional = false
@@ -343,6 +343,17 @@ category = "dev"
optional = false
python-versions = "*"
+[[package]]
+name = "isodate"
+version = "0.6.1"
+description = "An ISO 8601 date/time/duration parser and formatter"
+category = "main"
+optional = false
+python-versions = "*"
+
+[package.dependencies]
+six = "*"
+
[[package]]
name = "jmespath"
version = "0.10.0"
@@ -735,7 +746,7 @@ testing = ["pytest (>=4.6)", "pytest-checkdocs (>=2.4)", "pytest-flake8", "pytes
[metadata]
lock-version = "1.1"
python-versions = ">= 3.6, < 4"
-content-hash = "dd378de3f4e8e58678c31bc9c3898ee1b5d89765bfcda3c1de20c027f7058bd4"
+content-hash = "468798c3a0ae9c7014244e15919d01134dca7ed1be76c3d7d12f8acf985c6b4c"
[metadata.files]
amqp = []
@@ -763,6 +774,7 @@ faker = []
idna = []
importlib-metadata = []
iniconfig = []
+isodate = []
jmespath = []
jsonschema = []
kombu = []
diff --git a/runtime/bamboo-pipeline/pyproject.toml b/runtime/bamboo-pipeline/pyproject.toml
index 1ca6f1a2..05688181 100644
--- a/runtime/bamboo-pipeline/pyproject.toml
+++ b/runtime/bamboo-pipeline/pyproject.toml
@@ -1,6 +1,6 @@
[tool.poetry]
name = "bamboo-pipeline"
-version = "3.28.0"
+version = "3.29.0rc4"
description = "runtime for bamboo-engine base on Django and Celery"
authors = ["homholueng "]
license = "MIT"
@@ -16,7 +16,7 @@ requests = "^2.22.0"
django-celery-beat = "^2.1.0"
Mako = "^1.1.4"
pytz = "2019.3"
-bamboo-engine = "2.9.0"
+bamboo-engine = "2.10.0rc4"
jsonschema = "^2.5.1"
ujson = "4.1.*"
pyparsing = "^2.2.0"
@@ -26,6 +26,7 @@ django-timezone-field = "^4.0"
Werkzeug = "^1.0.0"
prometheus-client = "^0.9.0"
boto3 = "^1.9.130"
+isodate = "^0.6.1"
[tool.poetry.dev-dependencies]
pytest = "^6.2.2"
diff --git a/runtime/bamboo-pipeline/test/eri_imp_test_use/tests/control/test_rollback_node.py b/runtime/bamboo-pipeline/test/eri_imp_test_use/tests/control/test_rollback_node.py
deleted file mode 100644
index 992fdc99..00000000
--- a/runtime/bamboo-pipeline/test/eri_imp_test_use/tests/control/test_rollback_node.py
+++ /dev/null
@@ -1,149 +0,0 @@
-# -*- coding: utf-8 -*-
-"""
-Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
-Edition) available.
-Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
-Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
-You may obtain a copy of the License at
-http://opensource.org/licenses/MIT
-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.
-"""
-import time
-
-from bamboo_engine import Engine
-
-from bamboo_engine.builder import * # noqa
-from pipeline.engine import states
-from pipeline.eri.models import Process, State
-from pipeline.eri.runtime import BambooDjangoRuntime
-
-from pipeline.contrib.rollback import api
-
-
-def test_rollback_sample_pipeline():
- start = EmptyStartEvent()
- act_0 = ServiceActivity(component_code="callback_node")
- act_1 = ServiceActivity(component_code="callback_node")
- end = EmptyEndEvent()
- start.extend(act_0).extend(act_1).extend(end)
- pipeline = build_tree(start)
- runtime = BambooDjangoRuntime()
- engine = Engine(runtime)
- engine.run_pipeline(pipeline=pipeline, root_pipeline_data={})
- time.sleep(3)
-
- state = runtime.get_state(act_0.id)
- engine.callback(act_0.id, state.version, {"bit": 1})
- pipeline_id = pipeline["id"]
- time.sleep(3)
- assert State.objects.filter(node_id=act_0.id, name=states.FINISHED).exists()
- api.rollback(root_pipeline_id=pipeline_id, node_id=act_0.id)
- time.sleep(3)
- process = Process.objects.get(root_pipeline_id=pipeline_id, parent_id=-1)
- # 此时最新进程被指向了最新的node_id
- assert process.current_node_id == act_0.id
- # 此时第一个节点重回RUNNING状态
- assert State.objects.filter(node_id=act_0.id, name=states.RUNNING).exists()
-
-
-def test_rollback_pipeline_with_exclusive_gateway():
- """
- -> act_0
- 开始 -> 分支网关 -> 汇聚网关 -> act_2 -> 结束
- -> act_1
-
- 当执行到 act_2 时,此时回退到act_0 应当能够再次回到 act_2
- """
-
- runtime = BambooDjangoRuntime()
-
- start = EmptyStartEvent()
- eg = ExclusiveGateway(
- conditions={0: "True == True", 1: "True == False"}
- )
- act_0 = ServiceActivity(component_code="callback_node")
- act_1 = ServiceActivity(component_code="callback_node")
- act_2 = ServiceActivity(component_code="callback_node")
-
- cg = ConvergeGateway()
- end = EmptyEndEvent()
-
- start.extend(eg).connect(act_0, act_1).converge(cg).extend(act_2).extend(end)
-
- pipeline = build_tree(start)
- engine = Engine(BambooDjangoRuntime())
- engine.run_pipeline(pipeline=pipeline, root_pipeline_data={})
- time.sleep(3)
-
- state = runtime.get_state(act_0.id)
- engine.callback(act_0.id, state.version, {"bit": 1})
-
- time.sleep(3)
- pipeline_id = pipeline["id"]
-
- process = Process.objects.get(root_pipeline_id=pipeline_id, parent_id=-1)
-
- # 此时执行到了act_2
- assert process.current_node_id == act_2.id
-
- api.rollback(pipeline_id, act_0.id)
- time.sleep(3)
-
- process.refresh_from_db()
- # 此时最新进程被指向了最新的node_id
- assert process.current_node_id == act_0.id
- # 此时第一个节点重回RUNNING状态
- assert State.objects.filter(node_id=act_0.id, name=states.RUNNING).exists()
-
-
-def test_rollback_pipeline_with_conditional_parallel():
- """
- -> act_1
- 开始 -> act_0 并行网关 -> 汇聚网关 -> act_3 -> 结束
- -> act_2
-
- 当执行到 act_2 时,此时回退到act_0 应当能够再次回到 act_2
- """
-
- runtime = BambooDjangoRuntime()
-
- start = EmptyStartEvent()
- act_0 = ServiceActivity(component_code="debug_node")
- pg = ParallelGateway()
-
- act_1 = ServiceActivity(component_code="debug_node")
- act_2 = ServiceActivity(component_code="debug_node")
- cg = ConvergeGateway()
- act_3 = ServiceActivity(component_code="callback_node")
- end = EmptyEndEvent()
-
- start.extend(act_0).extend(pg).connect(act_1, act_2).converge(cg).extend(act_3).extend(end)
-
- pipeline = build_tree(start)
- engine = Engine(BambooDjangoRuntime())
- engine.run_pipeline(pipeline=pipeline, root_pipeline_data={})
-
- time.sleep(3)
- pipeline_id = pipeline["id"]
-
- process = Process.objects.get(root_pipeline_id=pipeline_id, parent_id=-1)
- # 此时执行到了act_2
- assert process.current_node_id == act_3.id
-
- # 此时回到开始节点
- api.rollback(pipeline_id, act_0.id)
- time.sleep(3)
-
- process.refresh_from_db()
- # 此时第二次执行到act_2
- assert process.current_node_id == act_3.id
- # 此时第一个节点重回RUNNING状态
- assert State.objects.filter(node_id=act_3.id, name=states.RUNNING).exists()
- # callback act_2 此时流程结束
- state = runtime.get_state(act_3.id)
- engine.callback(act_3.id, state.version, {"bit": 1})
- time.sleep(3)
-
- assert State.objects.filter(node_id=pipeline_id, name=states.FINISHED).exists()
diff --git a/runtime/bamboo-pipeline/test/eri_imp_test_use/tests/control/test_run_pipeline.py b/runtime/bamboo-pipeline/test/eri_imp_test_use/tests/control/test_run_pipeline.py
new file mode 100644
index 00000000..63e29be6
--- /dev/null
+++ b/runtime/bamboo-pipeline/test/eri_imp_test_use/tests/control/test_run_pipeline.py
@@ -0,0 +1,49 @@
+# -*- coding: utf-8 -*-
+import time
+
+import pytest
+from pipeline.eri.models import State
+from pipeline.eri.runtime import BambooDjangoRuntime
+
+from bamboo_engine.builder import (
+ EmptyEndEvent,
+ EmptyStartEvent,
+ ServiceActivity,
+ build_tree,
+)
+from bamboo_engine.engine import Engine
+
+
+def test_run_pipeline_with_start_node_id():
+ start = EmptyStartEvent()
+ act_1 = ServiceActivity(component_code="callback_node")
+ end = EmptyEndEvent()
+
+ start.extend(act_1).extend(end)
+
+ pipeline = build_tree(start)
+ runtime = BambooDjangoRuntime()
+ engine = Engine(runtime)
+ engine.run_pipeline(pipeline=pipeline, root_pipeline_data={}, start_node_id=act_1.id)
+
+ time.sleep(3)
+
+ with pytest.raises(State.DoesNotExist):
+ # 由于直接跳过了开始节点,此时应该抛异常
+ runtime.get_state(start.id)
+
+ state = runtime.get_state(act_1.id)
+
+ assert state.name == "RUNNING"
+
+ engine.callback(act_1.id, state.version, {})
+
+ time.sleep(2)
+
+ state = runtime.get_state(act_1.id)
+
+ assert state.name == "FINISHED"
+
+ pipeline_state = runtime.get_state(pipeline["id"])
+
+ assert pipeline_state.name == "FINISHED"
diff --git a/runtime/bamboo-pipeline/test/eri_imp_test_use/tests/plugin_execute/__init__.py b/runtime/bamboo-pipeline/test/eri_imp_test_use/tests/plugin_execute/__init__.py
new file mode 100644
index 00000000..26a6d1c2
--- /dev/null
+++ b/runtime/bamboo-pipeline/test/eri_imp_test_use/tests/plugin_execute/__init__.py
@@ -0,0 +1,12 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
diff --git a/runtime/bamboo-pipeline/test/eri_imp_test_use/tests/plugin_execute/test_plugin_execute.py b/runtime/bamboo-pipeline/test/eri_imp_test_use/tests/plugin_execute/test_plugin_execute.py
new file mode 100644
index 00000000..f45a4702
--- /dev/null
+++ b/runtime/bamboo-pipeline/test/eri_imp_test_use/tests/plugin_execute/test_plugin_execute.py
@@ -0,0 +1,83 @@
+# -*- coding: utf-8 -*-
+import time
+
+from pipeline.contrib.plugin_execute import api
+
+
+def test_run_plugin_no_schedule():
+ # 测试execute的情况
+ task_id = api.run("debug_no_schedule_node", "legacy", {}, {}).data
+ state = api.get_state(task_id).data
+ assert state["state"] == "READY"
+ time.sleep(2)
+ state = api.get_state(task_id).data
+ assert state["state"] == "FINISHED"
+
+
+def test_run_plugin_with_schedule():
+ # 测试schedule的情况
+ task_id = api.run("schedule_node", "legacy", {"count": 1}, {}).data
+ state = api.get_state(task_id).data
+ assert state["state"] == "READY"
+ time.sleep(30)
+ state = api.get_state(task_id).data
+ assert state["state"] == "FINISHED"
+ assert state["outputs"]["count"] == 5
+
+
+def test_run_plugin_with_callback():
+ # 测试callback的情况
+ task_id = api.run("hook_callback_node", "legacy", {}, {}).data
+ state = api.get_state(task_id).data
+ assert state["state"] == "READY"
+ time.sleep(5)
+ state = api.get_state(task_id).data
+ assert state["state"] == "RUNNING"
+
+ api.callback(task_id, {"bit": 0})
+ time.sleep(10)
+
+ state = api.get_state(task_id).data
+ assert state["state"] == "FAILED"
+
+ api.retry(task_id, inputs={})
+ time.sleep(5)
+ state = api.get_state(task_id).data
+ assert state["state"] == "RUNNING"
+
+ api.callback(task_id, {"bit": 1})
+ time.sleep(5)
+ state = api.get_state(task_id).data
+ assert state["state"] == "RUNNING"
+
+
+def test_run_plugin_with_callback_success():
+ task_id = api.run("debug_callback_node", "legacy", {}, {}).data
+ state = api.get_state(task_id).data
+
+ assert state["state"] == "READY"
+ time.sleep(5)
+ state = api.get_state(task_id).data
+ assert state["state"] == "RUNNING"
+
+ api.callback(task_id, {"bit": 1})
+ time.sleep(10)
+
+ state = api.get_state(task_id).data
+ assert state["state"] == "FINISHED"
+
+
+def test_run_plugin_with_force_fail():
+ task_id = api.run("debug_callback_node", "legacy", {}, {}).data
+ state = api.get_state(task_id).data
+
+ assert state["state"] == "READY"
+ time.sleep(5)
+ state = api.get_state(task_id).data
+ assert state["state"] == "RUNNING"
+
+ api.forced_fail(task_id)
+ time.sleep(3)
+
+ state = api.get_state(task_id).data
+ assert state["state"] == "FAILED"
diff --git a/runtime/bamboo-pipeline/test/pipeline_sdk_use/settings.py b/runtime/bamboo-pipeline/test/pipeline_sdk_use/settings.py
index ed03f568..3d7e8ed3 100755
--- a/runtime/bamboo-pipeline/test/pipeline_sdk_use/settings.py
+++ b/runtime/bamboo-pipeline/test/pipeline_sdk_use/settings.py
@@ -13,14 +13,13 @@
# Build paths inside the project like this: os.path.join(BASE_DIR, ...)
import os
+from celery import Celery
from pipeline.celery.queues import ScalableQueues # noqa
from pipeline.celery.settings import * # noqa
from pipeline.eri.celery import queues, step
-from celery import Celery
-
-CELERY_QUEUES.extend(queues.CELERY_QUEUES)
-CELERY_QUEUES.extend(queues.QueueResolver("api").queues())
+CELERY_QUEUES.extend(queues.CELERY_QUEUES) # noqa
+CELERY_QUEUES.extend(queues.QueueResolver("api").queues()) # noqa
step.PromServerStep.port = 8002
app = Celery("proj")
@@ -55,11 +54,14 @@
"pipeline",
"pipeline.log",
"pipeline.engine",
+ "pipeline.contrib.node_timer_event",
"pipeline.component_framework",
"pipeline.variable_framework",
"pipeline.django_signal_valve",
"pipeline.contrib.periodic_task",
"pipeline.contrib.node_timeout",
+ "pipeline.contrib.rollback",
+ "pipeline.contrib.plugin_execute",
"django_celery_beat",
"pipeline_test_use",
"variable_app",
@@ -154,14 +156,13 @@
STATIC_URL = "/static/"
-
ENABLE_EXAMPLE_COMPONENTS = True
BROKER_VHOST = "/"
BROKER_URL = "amqp://guest:guest@localhost:5672//"
-# BROKER_URL = 'redis://localhost:6379/0'
+# BROKER_URL = "redis://localhost:6379/0"
PIPELINE_DATA_BACKEND = "pipeline.engine.core.data.redis_backend.RedisDataBackend"
@@ -179,3 +180,6 @@
# }
# }
# ]
+
+
+PLUGIN_EXECUTE_QUEUE = "default"
diff --git a/runtime/bamboo-pipeline/test/pipeline_test_use/components/collections/atom.py b/runtime/bamboo-pipeline/test/pipeline_test_use/components/collections/atom.py
index e5fdba4e..b2ab9dc2 100755
--- a/runtime/bamboo-pipeline/test/pipeline_test_use/components/collections/atom.py
+++ b/runtime/bamboo-pipeline/test/pipeline_test_use/components/collections/atom.py
@@ -18,8 +18,9 @@ class HookMixin:
__need_run_hook__ = True
def recorder(self, hook: HookType, data, parent_data, callback_data=None):
- self.logger.info("hook_debug_node({}) id: {}".format(hook.value, self.id))
- self.logger.info("hook_debug_node({}) root_pipeline_id: {}".format(hook.value, self.root_pipeline_id))
+ if hasattr(hook.value, "id"):
+ self.logger.info("hook_debug_node({}) id: {}".format(hook.value, self.id))
+ self.logger.info("hook_debug_node({}) root_pipeline_id: {}".format(hook.value, self.root_pipeline_id))
logger.info("hook_debug_node hook(%s) data %s ", hook.value, pprint.pformat(data.inputs))
logger.info("hook_debug_node hook(%s) parent data %s ", hook.value, pprint.pformat(parent_data.inputs))
logger.info("hook_debug_node hook(%s) output data %s ", hook.value, pprint.pformat(data.outputs))
@@ -441,6 +442,32 @@ class CallbackComponent(Component):
form = "index.html"
+class DebugCallbackService(Service):
+ __need_schedule__ = True
+ interval = None
+
+ def execute(self, data, parent_data):
+ return True
+
+ def schedule(self, data, parent_data, callback_data=None):
+ if callback_data:
+ if int(callback_data.get("bit", 1)) == 1:
+ self.finish_schedule()
+ return True
+
+ return False
+
+ def outputs_format(self):
+ return []
+
+
+class DebugCallbackComponent(Component):
+ name = "callback component"
+ code = "debug_callback_node"
+ bound_service = DebugCallbackService
+ form = "index.html"
+
+
class HookCallbackService(HookMixin, CallbackService):
pass
@@ -469,7 +496,6 @@ def schedule(self, data, parent_data, callback_data=None):
logger.info("[{}]: callback_data={}".format(_scheduled_times, callback_data))
if callback_data:
if int(callback_data.get("bit", 0)) == 0:
- print("hahacai")
return False
_scheduled_times += 1
@@ -646,7 +672,7 @@ def outputs_format(self):
return []
-class InterruptScheduleComponent(Component):
+class InterruptRaiseScheduleComponent(Component):
name = "debug 组件"
code = "interrupt_raise_test"
bound_service = InterruptRaiseService
diff --git a/tests/builder/__init__.py b/tests/builder/__init__.py
new file mode 100644
index 00000000..26a6d1c2
--- /dev/null
+++ b/tests/builder/__init__.py
@@ -0,0 +1,12 @@
+# -*- coding: utf-8 -*-
+"""
+Tencent is pleased to support the open source community by making 蓝鲸智云PaaS平台社区版 (BlueKing PaaS Community
+Edition) available.
+Copyright (C) 2017 THL A29 Limited, a Tencent company. All rights reserved.
+Licensed under the MIT License (the "License"); you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+http://opensource.org/licenses/MIT
+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.
+"""
diff --git a/tests/builder/test_token.py b/tests/builder/test_token.py
new file mode 100644
index 00000000..acec08fd
--- /dev/null
+++ b/tests/builder/test_token.py
@@ -0,0 +1,236 @@
+# -*- coding: utf-8 -*-
+
+from bamboo_engine.builder import (
+ ConditionalParallelGateway,
+ ConvergeGateway,
+ EmptyEndEvent,
+ EmptyStartEvent,
+ ExclusiveGateway,
+ ParallelGateway,
+ ServiceActivity,
+ SubProcess,
+ build_tree,
+)
+from bamboo_engine.builder.builder import generate_pipeline_token
+
+
+def get_node_token(tree, name: str, node_map):
+ # 根据 name 获取对应的token
+ if name.startswith("act"):
+ for activity_id, value in tree["activities"].items():
+ if value["name"] == name:
+ return node_map[activity_id]
+
+ if (
+ name.startswith("ParallelGateway")
+ or name.startswith("ExclusiveGateway")
+ or name.startswith("ConvergeGateway")
+ or name.startswith("ConditionalParallelGateway")
+ ):
+ for gateway_id, value in tree["gateways"].items():
+ if value["name"] == name:
+ return node_map[gateway_id]
+
+ if name.startswith("start_event"):
+ return node_map[tree["start_event"]["id"]]
+
+ if name.startswith("end_event"):
+ return node_map[tree["end_event"]["id"]]
+
+
+def test_inject_pipeline_token_normal():
+ start = EmptyStartEvent()
+ act = ServiceActivity(name="act_1", component_code="example_component")
+ end = EmptyEndEvent()
+
+ start.extend(act).extend(end)
+ pipeline = build_tree(start)
+ node_token_map = generate_pipeline_token(pipeline)
+
+ assert get_node_token(pipeline, "act_1", node_token_map) == get_node_token(pipeline, "start_event", node_token_map)
+ assert get_node_token(pipeline, "start_event", node_token_map) == get_node_token(
+ pipeline, "end_event", node_token_map
+ )
+
+
+def test_inject_pipeline_token_with_complex_cycle():
+ start = EmptyStartEvent()
+ end = EmptyEndEvent()
+
+ write_document = ServiceActivity(name="act_1", component_code="example_component")
+ review_document = ServiceActivity(name="act_2", component_code="example_component")
+ rework_document = ServiceActivity(name="act_3", component_code="example_component")
+
+ release_document = ServiceActivity(name="act_4", component_code="example_component")
+ gateway = ExclusiveGateway(
+ conditions={0: "${act_1_output} < 0", 1: "${act_1_output} >= 0", 2: "${act_1_output} >= 0"},
+ name="ExclusiveGateway",
+ )
+
+ start.extend(write_document).extend(review_document).extend(gateway).connect(
+ release_document, end, rework_document
+ ).to(release_document).extend(end)
+ rework_document.extend(review_document)
+ pipeline = build_tree(start)
+ node_token_map = generate_pipeline_token(pipeline)
+
+ assert (
+ get_node_token(pipeline, "start_event", node_token_map)
+ == get_node_token(pipeline, "act_1", node_token_map)
+ == get_node_token(pipeline, "act_2", node_token_map)
+ == get_node_token(pipeline, "ExclusiveGateway", node_token_map)
+ == get_node_token(pipeline, "end_event", node_token_map)
+ == get_node_token(pipeline, "act_3", node_token_map)
+ )
+
+ assert get_node_token(pipeline, "act_4", node_token_map) == get_node_token(pipeline, "act_3", node_token_map)
+
+
+def test_inject_pipeline_token_parallel_gateway():
+ start = EmptyStartEvent()
+ pg = ParallelGateway(name="ParallelGateway")
+ act_1 = ServiceActivity(component_code="pipe_example_component", name="act_1")
+ act_2 = ServiceActivity(component_code="pipe_example_component", name="act_2")
+ act_3 = ServiceActivity(component_code="pipe_example_component", name="act_3")
+ cg = ConvergeGateway(name="ConvergeGateway")
+ end = EmptyEndEvent()
+
+ start.extend(pg).connect(act_1, act_2, act_3).to(pg).converge(cg).extend(end)
+
+ pipeline = build_tree(start)
+ node_token_map = generate_pipeline_token(pipeline)
+ assert (
+ get_node_token(pipeline, "start_event", node_token_map)
+ == get_node_token(pipeline, "ParallelGateway", node_token_map)
+ == get_node_token(pipeline, "ConvergeGateway", node_token_map)
+ == get_node_token(pipeline, "end_event", node_token_map)
+ != get_node_token(pipeline, "act_1", node_token_map)
+ )
+
+ assert (
+ get_node_token(pipeline, "act_1", node_token_map)
+ != get_node_token(pipeline, "act_2", node_token_map)
+ != get_node_token(pipeline, "act_3", node_token_map)
+ )
+
+
+def test_inject_pipeline_token_exclusive_gateway():
+ start = EmptyStartEvent()
+ act_1 = ServiceActivity(component_code="pipe_example_component", name="act_1")
+ eg = ExclusiveGateway(conditions={0: "${act_1_output} < 0", 1: "${act_1_output} >= 0"}, name="ExclusiveGateway")
+ act_2 = ServiceActivity(component_code="pipe_example_component", name="act_2")
+ act_3 = ServiceActivity(component_code="pipe_example_component", name="act_3")
+ end = EmptyEndEvent()
+ start.extend(act_1).extend(eg).connect(act_2, act_3).to(eg).converge(end)
+
+ pipeline = build_tree(start)
+ node_token_map = generate_pipeline_token(pipeline)
+
+ assert (
+ get_node_token(pipeline, "start_event", node_token_map)
+ == get_node_token(pipeline, "act_1", node_token_map)
+ == get_node_token(pipeline, "ExclusiveGateway", node_token_map)
+ == get_node_token(pipeline, "end_event", node_token_map)
+ == get_node_token(pipeline, "act_2", node_token_map)
+ )
+
+ assert get_node_token(pipeline, "act_2", node_token_map) == get_node_token(pipeline, "act_3", node_token_map)
+
+
+def test_inject_pipeline_token_conditional_exclusive_gateway():
+ start = EmptyStartEvent()
+ act_1 = ServiceActivity(component_code="pipe_example_component", name="act_1")
+ cpg = ConditionalParallelGateway(
+ conditions={0: "${act_1_output} < 0", 1: "${act_1_output} >= 0", 2: "${act_1_output} >= 0"},
+ name="ConditionalParallelGateway",
+ )
+ act_2 = ServiceActivity(component_code="pipe_example_component", name="act_2")
+ act_3 = ServiceActivity(component_code="pipe_example_component", name="act_3")
+ act_4 = ServiceActivity(component_code="pipe_example_component", name="act_4")
+ cg = ConvergeGateway(name="ConvergeGateway")
+ end = EmptyEndEvent()
+
+ start.extend(act_1).extend(cpg).connect(act_2, act_3, act_4).to(cpg).converge(cg).extend(end)
+
+ pipeline = build_tree(start)
+ node_token_map = generate_pipeline_token(pipeline)
+ assert (
+ get_node_token(pipeline, "start_event", node_token_map)
+ == get_node_token(pipeline, "act_1", node_token_map)
+ == get_node_token(pipeline, "ConditionalParallelGateway", node_token_map)
+ == get_node_token(pipeline, "ConvergeGateway", node_token_map)
+ == get_node_token(pipeline, "end_event", node_token_map)
+ != get_node_token(pipeline, "act_3", node_token_map)
+ )
+
+ assert (
+ get_node_token(pipeline, "act_2", node_token_map)
+ != get_node_token(pipeline, "act_3", node_token_map)
+ != get_node_token(pipeline, "act_4", node_token_map)
+ )
+
+
+def test_inject_pipeline_token_subprocess():
+ def sub_process(name):
+ subproc_start = EmptyStartEvent()
+ subproc_act = ServiceActivity(component_code="pipe_example_component", name="act_2")
+ subproc_end = EmptyEndEvent()
+ subproc_start.extend(subproc_act).extend(subproc_end)
+ return SubProcess(start=subproc_start, name=name)
+
+ start = EmptyStartEvent()
+ act_1 = ServiceActivity(component_code="pipe_example_component", name="act_1")
+ eg = ExclusiveGateway(conditions={0: "${act_1_output} < 0", 1: "${act_1_output} >= 0"}, name="ExclusiveGateway")
+
+ subproc_1 = sub_process(name="act_3")
+ subproc_2 = sub_process(name="act_4")
+ end = EmptyEndEvent()
+
+ start.extend(act_1).extend(eg).connect(subproc_1, subproc_2).converge(end)
+
+ pipeline = build_tree(start)
+ node_token_map = generate_pipeline_token(pipeline)
+
+ assert (
+ get_node_token(pipeline, "start_event", node_token_map)
+ == get_node_token(pipeline, "end_event", node_token_map)
+ == get_node_token(pipeline, "ExclusiveGateway", node_token_map)
+ == get_node_token(pipeline, "end_event", node_token_map)
+ == get_node_token(pipeline, "act_3", node_token_map)
+ )
+
+ assert get_node_token(pipeline, "act_3", node_token_map) == get_node_token(pipeline, "act_4", node_token_map)
+
+ subproc_pipeline = pipeline["activities"][subproc_1.id]["pipeline"]
+
+ assert (
+ get_node_token(subproc_pipeline, "start_event", node_token_map)
+ == get_node_token(subproc_pipeline, "end_event", node_token_map)
+ == get_node_token(subproc_pipeline, "act_2", node_token_map)
+ )
+
+
+def test_inject_pipeline_token_with_cycle():
+ start = EmptyStartEvent()
+ act_1 = ServiceActivity(component_code="pipe_example_component", name="act_1")
+ eg = ExclusiveGateway(
+ conditions={0: "${act_1_output} < 0", 1: "${act_1_output} >= 0", 2: "${act_1_output} >= 0"},
+ name="ExclusiveGateway",
+ )
+ act_2 = ServiceActivity(component_code="pipe_example_component", name="act_2")
+ act_3 = ServiceActivity(component_code="pipe_example_component", name="act_3")
+ end = EmptyEndEvent()
+ start.extend(act_1).extend(eg).connect(act_2, act_3, act_1).to(eg).converge(end)
+
+ pipeline = build_tree(start)
+ node_token_map = generate_pipeline_token(pipeline)
+
+ assert (
+ get_node_token(pipeline, "start_event", node_token_map)
+ == get_node_token(pipeline, "act_1", node_token_map)
+ == get_node_token(pipeline, "ExclusiveGateway", node_token_map)
+ == get_node_token(pipeline, "end_event", node_token_map)
+ == get_node_token(pipeline, "act_2", node_token_map)
+ )
+
+ assert get_node_token(pipeline, "act_2", node_token_map) == get_node_token(pipeline, "act_3", node_token_map)
diff --git a/tests/engine/test_engine_execute.py b/tests/engine/test_engine_execute.py
index 59cab573..362cfd2a 100644
--- a/tests/engine/test_engine_execute.py
+++ b/tests/engine/test_engine_execute.py
@@ -96,6 +96,7 @@ def node(node_id):
parent_pipeline_id="root",
code="",
version="",
+ reserve_rollback=True,
error_ignorable=False,
)
@@ -380,7 +381,6 @@ def test_execute__rerun_and_have_to_sleep(node_id, pi, interrupter, node, state)
def test_execute__have_to_sleep(node_id, pi, interrupter, node, state):
-
runtime = MagicMock()
runtime.get_process_info = MagicMock(return_value=pi)
runtime.batch_get_state_name = MagicMock(return_value={"root": states.RUNNING})
@@ -780,7 +780,6 @@ def test_execute__has_dispatch_processes(node_id, pi, interrupter, node, state):
def test_execute__have_to_die(node_id, pi, interrupter, node, state):
-
runtime = MagicMock()
runtime.get_process_info = MagicMock(return_value=pi)
runtime.batch_get_state_name = MagicMock(return_value={"root": states.RUNNING})
@@ -851,6 +850,81 @@ def test_execute__have_to_die(node_id, pi, interrupter, node, state):
assert interrupter.check_point.execute_result is not None
+def test_execute__has_reversed_rollback_plan(node_id, pi, interrupter, node, state):
+ runtime = MagicMock()
+ runtime.get_process_info = MagicMock(return_value=pi)
+ runtime.batch_get_state_name = MagicMock(return_value={"root": states.RUNNING})
+ runtime.get_node = MagicMock(return_value=node)
+ runtime.get_config = MagicMock(return_value=True)
+ runtime.start_rollback = MagicMock(return_value=True)
+ runtime.get_state_or_none = MagicMock(return_value=None)
+ runtime.get_state = MagicMock(return_value=state)
+ runtime.set_state = MagicMock(return_value=state.version)
+ handler = MagicMock()
+ handler.execute = MagicMock(
+ return_value=ExecuteResult(
+ should_sleep=False,
+ schedule_ready=False,
+ schedule_type=None,
+ schedule_after=-1,
+ dispatch_processes=[],
+ next_node_id=None,
+ should_die=True,
+ )
+ )
+
+ get_handler = MagicMock(return_value=handler)
+
+ engine = Engine(runtime=runtime)
+
+ with mock.patch(
+ "bamboo_engine.engine.HandlerFactory.get_handler",
+ get_handler,
+ ):
+ engine.execute(pi.process_id, node_id, pi.root_pipeline_id, pi.top_pipeline_id, interrupter, {})
+
+ runtime.beat.assert_called_once_with(pi.process_id)
+ runtime.get_node.assert_called_once_with(node_id)
+ runtime.get_state_or_none.assert_called_once_with(node_id)
+ runtime.node_rerun_limit.assert_not_called()
+ runtime.set_state.assert_called_once_with(
+ node_id=node.id,
+ to_state=states.RUNNING,
+ version=None,
+ loop=1,
+ inner_loop=1,
+ root_id=pi.root_pipeline_id,
+ parent_id=pi.top_pipeline_id,
+ set_started_time=True,
+ reset_skip=False,
+ reset_retry=False,
+ reset_error_ignored=False,
+ refresh_version=False,
+ ignore_boring_set=False,
+ )
+ runtime.start_rollback.assert_called_once_with(pi.root_pipeline_id, node_id)
+ runtime.sleep.assert_not_called()
+ runtime.set_schedule.assert_not_called()
+ runtime.schedule.assert_not_called()
+ runtime.execute.assert_not_called()
+ runtime.die.assert_called_once_with(pi.process_id)
+
+ get_handler.assert_called_once_with(node, runtime, interrupter)
+
+ handler.execute.assert_called_once_with(
+ process_info=pi,
+ loop=state.loop,
+ inner_loop=state.loop,
+ version=state.version,
+ recover_point=interrupter.recover_point,
+ )
+
+ assert interrupter.check_point.name == ExecuteKeyPoint.EXECUTE_NODE_DONE
+ assert interrupter.check_point.state_already_exist is False
+ assert interrupter.check_point.running_node_version == "v"
+ assert interrupter.check_point.execute_result is not None
+
+
def test_execute__recover_with_state_not_exsit(node_id, pi, interrupter, node, state, recover_point):
recover_point.state_already_exist = False
recover_point.running_node_version = "set_running_return_version"
diff --git a/tests/engine/test_engine_schedule.py b/tests/engine/test_engine_schedule.py
index f3cd6ddc..32ca385c 100644
--- a/tests/engine/test_engine_schedule.py
+++ b/tests/engine/test_engine_schedule.py
@@ -10,7 +10,7 @@
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.
"""
-
+import copy
import mock
import pytest
@@ -128,7 +128,6 @@ def recover_point():
def test_schedule__lock_get_failed(node_id, schedule_id, state, pi, schedule, interrupter):
-
runtime = MagicMock()
runtime.get_process_info = MagicMock(return_value=pi)
runtime.apply_schedule_lock = MagicMock(return_value=False)
@@ -470,7 +469,7 @@ def test_schedule__schedule_done(node_id, state, pi, schedule, node, interrupter
runtime.get_node.assert_called_once_with(node_id)
runtime.get_execution_data.assert_called_once_with(node_id)
runtime.set_execution_data.assert_called_once_with(node_id=node.id, data=execution_data)
- runtime.get_data_inputs.assert_called_once_with(pi.root_pipeline_id)
+ runtime.get_data_inputs.assert_has_calls([call("root"), call("root")])
runtime.get_callback_data.assert_not_called()
service.hook_dispatch.assert_called_once()
handler.schedule.assert_called_once_with(
@@ -496,6 +495,71 @@ def test_schedule__schedule_done(node_id, state, pi, schedule, node, interrupter
assert interrupter.check_point.lock_released is True
+def test_schedule__schedule_done_with_node_reserved_rollback(node_id, state, pi, schedule, node, interrupter):
+ node = copy.deepcopy(node)
+ node.reserve_rollback = True
+
+ schedule.type = ScheduleType.POLL
+ runtime = MagicMock()
+ runtime.get_process_info = MagicMock(return_value=pi)
+ runtime.apply_schedule_lock = MagicMock(return_value=True)
+ runtime.get_schedule = MagicMock(return_value=schedule)
+ runtime.start_rollback = MagicMock(return_value=True)
+ runtime.get_config = MagicMock(return_value=True)
+ runtime.get_state = MagicMock(return_value=state)
+ runtime.get_node = MagicMock(return_value=node)
+ runtime.node_finish = MagicMock()
+ handler = MagicMock()
+ handler.schedule = MagicMock(
+ return_value=ScheduleResult(
+ has_next_schedule=False,
+ schedule_after=-1,
+ schedule_done=True,
+ next_node_id="nid2",
+ )
+ )
+ get_handler = MagicMock(return_value=handler)
+
+ engine = Engine(runtime=runtime)
+
+ with mock.patch(
+ "bamboo_engine.engine.HandlerFactory.get_handler",
+ get_handler,
+ ):
+ engine.schedule(pi.process_id, node_id, schedule.id, interrupter, headers={})
+
+ runtime.beat.assert_called_once_with(pi.process_id)
+ runtime.get_process_info.assert_called_once_with(pi.process_id)
+ runtime.apply_schedule_lock.assert_called_once_with(schedule.id)
+ runtime.schedule.assert_not_called()
+
+ runtime.start_rollback.assert_called_once_with(pi.root_pipeline_id, node_id)
+
+ runtime.get_schedule.assert_called_once_with(schedule.id)
+ runtime.node_finish.assert_called_once_with(pi.root_pipeline_id, node.id)
+
+ runtime.get_state.assert_has_calls([call(node_id), call(node_id)])
+ runtime.get_node.assert_called_once_with(node_id)
+ runtime.get_callback_data.assert_not_called()
+ handler.schedule.assert_called_once_with(
+ process_info=pi,
+ loop=state.loop,
+ inner_loop=state.inner_loop,
+ schedule=schedule,
+ callback_data=None,
+ recover_point=interrupter.recover_point,
+ )
+ runtime.set_next_schedule.assert_not_called()
+ runtime.finish_schedule.assert_called_once_with(schedule.id)
+
+ assert interrupter.check_point.name == ScheduleKeyPoint.RELEASE_LOCK_DONE
+ assert interrupter.check_point.version_mismatch is False
+ assert interrupter.check_point.node_not_running is False
+ assert interrupter.check_point.lock_get is True
+ assert interrupter.check_point.schedule_result is not None
+ assert interrupter.check_point.lock_released is True
+
+
def test_schedule__recover_version_mismatch(node_id, pi, state, schedule, interrupter, recover_point):
recover_point.version_mismatch = True
interrupter.recover_point = recover_point
diff --git a/tests/utils/test_graph.py b/tests/utils/test_graph.py
new file mode 100644
index 00000000..599d362a
--- /dev/null
+++ b/tests/utils/test_graph.py
@@ -0,0 +1,206 @@
+# -*- coding: utf-8 -*-
+from bamboo_engine.utils.graph import RollbackGraph
+
+
+def test_graph():
+ graph1 = RollbackGraph([1, 2, 3, 4], [[1, 2], [2, 3], [3, 4]])
+ assert not graph1.has_cycle()
+ assert graph1.get_cycle() == []
+ graph2 = RollbackGraph([1, 2, 3, 4], [[1, 2], [2, 3], [3, 4], [4, 1]])
+ assert graph2.has_cycle()
+ assert graph2.get_cycle() == [1, 2, 3, 4, 1]
+ graph3 = RollbackGraph([1, 2, 3, 4], [[1, 2], [2, 3], [3, 4], [4, 2]])
+ assert graph3.has_cycle()
+ assert graph3.get_cycle() == [2, 3, 4, 2]
+ graph4 = RollbackGraph(
+ [
+ "n20c4a0601193f268bfa168f1192eacd",
+ "nef42d10350b3961b53df7af67e16d9b",
+ "n0ada7b4abe63771a43052eaf188dc4b",
+ "n0cd3b95c714388bacdf1a486ab432fc",
+ "n1430047af8537f88710c4bbf3cbfb0f",
+ "n383748fe27434d582f0ca17af9d968a",
+ "n51426abd4be3a4691c80a73c3f93b3c",
+ "n854753a77933562ae72ec87c365f23d",
+ "n89f083892a731d7b9d7edb0f372006d",
+ "n8d4568db0ad364692b0387e86a2f1e0",
+ "n8daedbb02273a0fbc94cc118c90649f",
+ "n90b7ef55fe839b181879e036b4f8ffe",
+ "n99817348b4a36a6931854c93eed8c5f",
+ "na02956eba6f3a36ab9b0af2f2350213",
+ "nc3d0d49adf530bbaffe53630c184c0a",
+ "nca50848d1aa340f8c2b4776ce81868d",
+ "ncab9a48e79d357195dcee68dad3a31f",
+ "ncb4e013a6a8348bab087cc8500a3876",
+ "ne1f86f902a23e7fa4a67192e8b38a05",
+ "ne26def77df1385caa206c64e7e3ea53",
+ "nf3ebee137c53da28091ad7d140ce00c",
+ "nfc1dcdd7476393b9a81a988c113e1cf",
+ "n0197f8f210b3a1b8a7fc2f90e94744e",
+ "n01fb40259ad3cf285bb11a8bbbe59f2",
+ "n03f39191e8a32629145ba6a677ed040",
+ "n03ffc3b9e12316d8be63261cb9dec71",
+ "n07982b8985139249bca3a046f3a4379",
+ "n0b9e36e6b633ddb906d2044f658f110",
+ "n136c4fedebe3eb0ba932495aff6a945",
+ "n17cdc62c5d43976a413bda8f35634eb",
+ "n1d48483d8023439ad98d61d156c85fb",
+ "n26725bdcc0931fab0bc73e7244545ca",
+ "n2890db24f6c3cd1bbcd6b7d8cf2c045",
+ "n2ad9caac5b737bd897d4c8844c85f12",
+ "n2c88d1c1d8b35aebf883cbf259fb6bc",
+ "n302d25dfc9c369ab13104d5208e7119",
+ "n31688b7ab44338e9e6cb8dcaf259eef",
+ "n374443fbdc1313d98ebbe19d535fec2",
+ "n38c3dd0344a3f86bc7511c454bcdf4c",
+ "n3934eef90463940a6a9cf4ba2e63b1c",
+ "n40d5f0ca4bc3dd99c0b264cb186f00f",
+ "n476ddcb6dd33e2abac43596b08c2bc1",
+ "n4790f8aa48e335aa712e2af757e180b",
+ "n48bbfdc912334fc89c4f48c05e8969e",
+ "n5bef4f4532a382eaf79a0af70b2396b",
+ "n5ced56bcc863060ac4977755f35a5f5",
+ "n66a0562670e37648a3e05c243335bff",
+ "n6dc118cd3f7341d9ef8c97c63e2e9d9",
+ "n6e9d52e1ea53958a93e5b34022e7037",
+ "n786694b5ed33295a885b5bcd8c7c1ce",
+ "n7dccd56c80233469a4609f684ebe457",
+ "n8492d92ab6a3da48c2b49d6fcb8a479",
+ "n86a8b1a56f9399f90c4c227594a9d03",
+ "n8a805c0cd02307bad9f7828880b53dc",
+ "n8c7e35b0457300d9d6a96a6b1d18329",
+ "n91fdaed36403d06a07f4afe85e2892c",
+ "n9335d0718a937f9a39ec5b36d5637fe",
+ "n9372fb07ad936cba31f3d4e440f395a",
+ "n9ab96f926d83a93a5d3ebe2888fd343",
+ "na2a8a54e68033d0a276eb88dbff91c3",
+ "na493a7b5d5b3cc29f4070a6c4589cb7",
+ "nadfa68cb2503a39aac6626d6c72484a",
+ "nae1218ddd2e3448b562bc79dc084401",
+ "nc012287be793377b975b0230b35d713",
+ "ncb2e01f0c5336fe82b0e0e496f2612b",
+ "ncb5843900903b4c8a0a8302474d8c51",
+ "ncbf4db2c48f3348b2c7081f9e3b363a",
+ "nd4ee6c3248935ce9239e4bb20a81ab8",
+ "ndb1cf7af0e2319c9868530d0df8fd93",
+ "ne36a6858a733430bffa4fec053dc1ab",
+ "ne7af4a7c3613b3d81fe9e6046425a36",
+ "ne8035dd8de732758c1cc623f80f2fc8",
+ "ned91fdb914c35f3a21f320f62d72ffd",
+ "nf5448b3c66430f4a299d08208d313a6",
+ "nfaa0756a06f300495fb2e2e45e05ed3",
+ ],
+ [
+ ["n8d4568db0ad364692b0387e86a2f1e0", "n5bef4f4532a382eaf79a0af70b2396b"],
+ ["n8daedbb02273a0fbc94cc118c90649f", "nf5448b3c66430f4a299d08208d313a6"],
+ ["n01fb40259ad3cf285bb11a8bbbe59f2", "ne1f86f902a23e7fa4a67192e8b38a05"],
+ ["ncab9a48e79d357195dcee68dad3a31f", "n0197f8f210b3a1b8a7fc2f90e94744e"],
+ ["na493a7b5d5b3cc29f4070a6c4589cb7", "ne1f86f902a23e7fa4a67192e8b38a05"],
+ ["n89f083892a731d7b9d7edb0f372006d", "n136c4fedebe3eb0ba932495aff6a945"],
+ ["n51426abd4be3a4691c80a73c3f93b3c", "n9ab96f926d83a93a5d3ebe2888fd343"],
+ ["n89f083892a731d7b9d7edb0f372006d", "n8492d92ab6a3da48c2b49d6fcb8a479"],
+ ["n17cdc62c5d43976a413bda8f35634eb", "n6e9d52e1ea53958a93e5b34022e7037"],
+ ["n476ddcb6dd33e2abac43596b08c2bc1", "ne1f86f902a23e7fa4a67192e8b38a05"],
+ ["n6dc118cd3f7341d9ef8c97c63e2e9d9", "nfc1dcdd7476393b9a81a988c113e1cf"],
+ ["n91fdaed36403d06a07f4afe85e2892c", "ncb4e013a6a8348bab087cc8500a3876"],
+ ["n8a805c0cd02307bad9f7828880b53dc", "n3934eef90463940a6a9cf4ba2e63b1c"],
+ ["n2890db24f6c3cd1bbcd6b7d8cf2c045", "n0ada7b4abe63771a43052eaf188dc4b"],
+ ["ned91fdb914c35f3a21f320f62d72ffd", "n383748fe27434d582f0ca17af9d968a"],
+ ["n89f083892a731d7b9d7edb0f372006d", "n0b9e36e6b633ddb906d2044f658f110"],
+ ["nc3d0d49adf530bbaffe53630c184c0a", "na493a7b5d5b3cc29f4070a6c4589cb7"],
+ ["ncb2e01f0c5336fe82b0e0e496f2612b", "nc012287be793377b975b0230b35d713"],
+ ["n86a8b1a56f9399f90c4c227594a9d03", "nf3ebee137c53da28091ad7d140ce00c"],
+ ["nc3d0d49adf530bbaffe53630c184c0a", "nadfa68cb2503a39aac6626d6c72484a"],
+ ["na02956eba6f3a36ab9b0af2f2350213", "na2a8a54e68033d0a276eb88dbff91c3"],
+ ["n8daedbb02273a0fbc94cc118c90649f", "n07982b8985139249bca3a046f3a4379"],
+ ["n136c4fedebe3eb0ba932495aff6a945", "nfc1dcdd7476393b9a81a988c113e1cf"],
+ ["n9372fb07ad936cba31f3d4e440f395a", "n1430047af8537f88710c4bbf3cbfb0f"],
+ ["n8d4568db0ad364692b0387e86a2f1e0", "n91fdaed36403d06a07f4afe85e2892c"],
+ ["n854753a77933562ae72ec87c365f23d", "n40d5f0ca4bc3dd99c0b264cb186f00f"],
+ ["n854753a77933562ae72ec87c365f23d", "n1d48483d8023439ad98d61d156c85fb"],
+ ["n9ab96f926d83a93a5d3ebe2888fd343", "n383748fe27434d582f0ca17af9d968a"],
+ ["ne36a6858a733430bffa4fec053dc1ab", "n0cd3b95c714388bacdf1a486ab432fc"],
+ ["n03ffc3b9e12316d8be63261cb9dec71", "nca50848d1aa340f8c2b4776ce81868d"],
+ ["ne8035dd8de732758c1cc623f80f2fc8", "n0ada7b4abe63771a43052eaf188dc4b"],
+ ["n51426abd4be3a4691c80a73c3f93b3c", "ned91fdb914c35f3a21f320f62d72ffd"],
+ ["nd4ee6c3248935ce9239e4bb20a81ab8", "nfaa0756a06f300495fb2e2e45e05ed3"],
+ ["n5bef4f4532a382eaf79a0af70b2396b", "ncb4e013a6a8348bab087cc8500a3876"],
+ ["ne26def77df1385caa206c64e7e3ea53", "n786694b5ed33295a885b5bcd8c7c1ce"],
+ ["n854753a77933562ae72ec87c365f23d", "ne8035dd8de732758c1cc623f80f2fc8"],
+ ["n374443fbdc1313d98ebbe19d535fec2", "ndb1cf7af0e2319c9868530d0df8fd93"],
+ ["nfaa0756a06f300495fb2e2e45e05ed3", "n8c7e35b0457300d9d6a96a6b1d18329"],
+ ["n90b7ef55fe839b181879e036b4f8ffe", "n26725bdcc0931fab0bc73e7244545ca"],
+ ["n8d4568db0ad364692b0387e86a2f1e0", "ncb2e01f0c5336fe82b0e0e496f2612b"],
+ ["ncb5843900903b4c8a0a8302474d8c51", "ncb4e013a6a8348bab087cc8500a3876"],
+ ["nf5448b3c66430f4a299d08208d313a6", "nf3ebee137c53da28091ad7d140ce00c"],
+ ["n20c4a0601193f268bfa168f1192eacd", "nd4ee6c3248935ce9239e4bb20a81ab8"],
+ ["nca50848d1aa340f8c2b4776ce81868d", "nc3d0d49adf530bbaffe53630c184c0a"],
+ ["na02956eba6f3a36ab9b0af2f2350213", "n03ffc3b9e12316d8be63261cb9dec71"],
+ ["n7dccd56c80233469a4609f684ebe457", "n8daedbb02273a0fbc94cc118c90649f"],
+ ["n0ada7b4abe63771a43052eaf188dc4b", "na02956eba6f3a36ab9b0af2f2350213"],
+ ["n9335d0718a937f9a39ec5b36d5637fe", "n99817348b4a36a6931854c93eed8c5f"],
+ ["n90b7ef55fe839b181879e036b4f8ffe", "n5ced56bcc863060ac4977755f35a5f5"],
+ ["ncb4e013a6a8348bab087cc8500a3876", "ne26def77df1385caa206c64e7e3ea53"],
+ ["na02956eba6f3a36ab9b0af2f2350213", "n4790f8aa48e335aa712e2af757e180b"],
+ ["nc012287be793377b975b0230b35d713", "ncb4e013a6a8348bab087cc8500a3876"],
+ ["n8d4568db0ad364692b0387e86a2f1e0", "ncb5843900903b4c8a0a8302474d8c51"],
+ ["n40d5f0ca4bc3dd99c0b264cb186f00f", "n0ada7b4abe63771a43052eaf188dc4b"],
+ ["n38c3dd0344a3f86bc7511c454bcdf4c", "n17cdc62c5d43976a413bda8f35634eb"],
+ ["n6e9d52e1ea53958a93e5b34022e7037", "n90b7ef55fe839b181879e036b4f8ffe"],
+ ["nf3ebee137c53da28091ad7d140ce00c", "n51426abd4be3a4691c80a73c3f93b3c"],
+ ["n99817348b4a36a6931854c93eed8c5f", "n89f083892a731d7b9d7edb0f372006d"],
+ ["n89f083892a731d7b9d7edb0f372006d", "n6dc118cd3f7341d9ef8c97c63e2e9d9"],
+ ["n8daedbb02273a0fbc94cc118c90649f", "n66a0562670e37648a3e05c243335bff"],
+ ["nadfa68cb2503a39aac6626d6c72484a", "ne1f86f902a23e7fa4a67192e8b38a05"],
+ ["n383748fe27434d582f0ca17af9d968a", "nef42d10350b3961b53df7af67e16d9b"],
+ ["na02956eba6f3a36ab9b0af2f2350213", "n03f39191e8a32629145ba6a677ed040"],
+ ["nae1218ddd2e3448b562bc79dc084401", "n383748fe27434d582f0ca17af9d968a"],
+ ["n26725bdcc0931fab0bc73e7244545ca", "n1430047af8537f88710c4bbf3cbfb0f"],
+ ["n48bbfdc912334fc89c4f48c05e8969e", "n8a805c0cd02307bad9f7828880b53dc"],
+ ["ne7af4a7c3613b3d81fe9e6046425a36", "ncb4e013a6a8348bab087cc8500a3876"],
+ ["nfc1dcdd7476393b9a81a988c113e1cf", "n8d4568db0ad364692b0387e86a2f1e0"],
+ ["n0197f8f210b3a1b8a7fc2f90e94744e", "n99817348b4a36a6931854c93eed8c5f"],
+ ["n90b7ef55fe839b181879e036b4f8ffe", "n302d25dfc9c369ab13104d5208e7119"],
+ ["n1d48483d8023439ad98d61d156c85fb", "n0ada7b4abe63771a43052eaf188dc4b"],
+ ["na2a8a54e68033d0a276eb88dbff91c3", "nca50848d1aa340f8c2b4776ce81868d"],
+ ["n90b7ef55fe839b181879e036b4f8ffe", "n9372fb07ad936cba31f3d4e440f395a"],
+ ["ndb1cf7af0e2319c9868530d0df8fd93", "n2ad9caac5b737bd897d4c8844c85f12"],
+ ["n8492d92ab6a3da48c2b49d6fcb8a479", "nfc1dcdd7476393b9a81a988c113e1cf"],
+ ["n8d4568db0ad364692b0387e86a2f1e0", "ne7af4a7c3613b3d81fe9e6046425a36"],
+ ["n302d25dfc9c369ab13104d5208e7119", "n1430047af8537f88710c4bbf3cbfb0f"],
+ ["n51426abd4be3a4691c80a73c3f93b3c", "n2c88d1c1d8b35aebf883cbf259fb6bc"],
+ ["n786694b5ed33295a885b5bcd8c7c1ce", "n0cd3b95c714388bacdf1a486ab432fc"],
+ ["n854753a77933562ae72ec87c365f23d", "n2890db24f6c3cd1bbcd6b7d8cf2c045"],
+ ["nc3d0d49adf530bbaffe53630c184c0a", "n476ddcb6dd33e2abac43596b08c2bc1"],
+ ["n2c88d1c1d8b35aebf883cbf259fb6bc", "n383748fe27434d582f0ca17af9d968a"],
+ ["n0cd3b95c714388bacdf1a486ab432fc", "n854753a77933562ae72ec87c365f23d"],
+ ["n51426abd4be3a4691c80a73c3f93b3c", "nae1218ddd2e3448b562bc79dc084401"],
+ ["nc3d0d49adf530bbaffe53630c184c0a", "n01fb40259ad3cf285bb11a8bbbe59f2"],
+ ["ne1f86f902a23e7fa4a67192e8b38a05", "n374443fbdc1313d98ebbe19d535fec2"],
+ ["n0b9e36e6b633ddb906d2044f658f110", "nfc1dcdd7476393b9a81a988c113e1cf"],
+ ["ncab9a48e79d357195dcee68dad3a31f", "ncbf4db2c48f3348b2c7081f9e3b363a"],
+ ["n8daedbb02273a0fbc94cc118c90649f", "n86a8b1a56f9399f90c4c227594a9d03"],
+ ["ncbf4db2c48f3348b2c7081f9e3b363a", "n99817348b4a36a6931854c93eed8c5f"],
+ ["n1430047af8537f88710c4bbf3cbfb0f", "ncab9a48e79d357195dcee68dad3a31f"],
+ ["n4790f8aa48e335aa712e2af757e180b", "nca50848d1aa340f8c2b4776ce81868d"],
+ ["ne26def77df1385caa206c64e7e3ea53", "ne36a6858a733430bffa4fec053dc1ab"],
+ ["ncab9a48e79d357195dcee68dad3a31f", "n31688b7ab44338e9e6cb8dcaf259eef"],
+ ["n07982b8985139249bca3a046f3a4379", "nf3ebee137c53da28091ad7d140ce00c"],
+ ["n66a0562670e37648a3e05c243335bff", "nf3ebee137c53da28091ad7d140ce00c"],
+ ["n03f39191e8a32629145ba6a677ed040", "nca50848d1aa340f8c2b4776ce81868d"],
+ ["n8c7e35b0457300d9d6a96a6b1d18329", "n38c3dd0344a3f86bc7511c454bcdf4c"],
+ ["n5ced56bcc863060ac4977755f35a5f5", "n1430047af8537f88710c4bbf3cbfb0f"],
+ ["n2ad9caac5b737bd897d4c8844c85f12", "n48bbfdc912334fc89c4f48c05e8969e"],
+ ["n31688b7ab44338e9e6cb8dcaf259eef", "n99817348b4a36a6931854c93eed8c5f"],
+ ["n3934eef90463940a6a9cf4ba2e63b1c", "n7dccd56c80233469a4609f684ebe457"],
+ ["ncab9a48e79d357195dcee68dad3a31f", "n9335d0718a937f9a39ec5b36d5637fe"],
+ ],
+ )
+ assert not graph4.has_cycle()
+ assert graph4.get_cycle() == []
+ graph5 = RollbackGraph([1, 2, 3, 4, 5], [[1, 2], [2, 3], [2, 4], [4, 5], [5, 2]])
+ assert graph5.has_cycle()
+ assert graph5.get_cycle() == [2, 4, 5, 2]
+
+ graph6 = graph5.reverse()
+ assert graph6.next(2), {1, 5}
diff --git a/tests/validator/test_validate_start_node.py b/tests/validator/test_validate_start_node.py
new file mode 100644
index 00000000..efb2743f
--- /dev/null
+++ b/tests/validator/test_validate_start_node.py
@@ -0,0 +1,133 @@
+# -*- coding: utf-8 -*-
+import pytest
+
+from bamboo_engine.builder import (
+ ConditionalParallelGateway,
+ ConvergeGateway,
+ EmptyEndEvent,
+ EmptyStartEvent,
+ ExclusiveGateway,
+ ParallelGateway,
+ ServiceActivity,
+ build_tree,
+)
+from bamboo_engine.exceptions import StartPositionInvalidException
+from bamboo_engine.validator.api import (
+ get_allowed_start_node_ids,
+ get_skipped_execute_node_ids,
+ validate_pipeline_start_node,
+)
+from bamboo_engine.validator.gateway import validate_gateways
+
+
+def test_get_allowed_start_node_ids_by_parallel_gateway():
+ """
+ 并行网关内的节点将会被忽略
+ """
+ start = EmptyStartEvent()
+ act_1 = ServiceActivity(component_code="pipe_example_component", name="act_1")
+ pg = ParallelGateway()
+ act_2 = ServiceActivity(component_code="pipe_example_component", name="act_2")
+ act_3 = ServiceActivity(component_code="pipe_example_component", name="act_3")
+ cg = ConvergeGateway()
+ end = EmptyEndEvent()
+ start.extend(act_1).extend(pg).connect(act_2, act_3).to(pg).converge(cg).extend(end)
+ pipeline = build_tree(start)
+ # 需要使用 validate_gateways 匹配网关对应的汇聚节点
+ validate_gateways(pipeline)
+ allowed_start_node_ids = get_allowed_start_node_ids(pipeline)
+
+ assert len(allowed_start_node_ids) == 3
+ assert allowed_start_node_ids == [start.id, act_1.id, pg.id]
+
+ skipped_execute_node_ids = get_skipped_execute_node_ids(pipeline, pg.id)
+ assert len(skipped_execute_node_ids) == 2
+ assert set(skipped_execute_node_ids) == {start.id, act_1.id}
+
+
+def test_get_allowed_start_node_ids_by_exclusive_gateway():
+ start = EmptyStartEvent()
+ act_1 = ServiceActivity(component_code="pipe_example_component", name="act_1")
+ eg = ExclusiveGateway(conditions={0: "${act_1_output} < 0", 1: "${act_1_output} >= 0"}, name="act_2 or act_3")
+ act_2 = ServiceActivity(component_code="pipe_example_component", name="act_2")
+ act_3 = ServiceActivity(component_code="pipe_example_component", name="act_3")
+ end = EmptyEndEvent()
+
+ start.extend(act_1).extend(eg).connect(act_2, act_3).to(eg).converge(end)
+ pipeline = build_tree(start)
+ validate_gateways(pipeline)
+ allowed_start_node_ids = get_allowed_start_node_ids(pipeline)
+
+ assert len(allowed_start_node_ids) == 5
+ assert allowed_start_node_ids == [start.id, act_1.id, eg.id, act_2.id, act_3.id]
+
+ skipped_execute_node_ids = get_skipped_execute_node_ids(pipeline, eg.id)
+ assert len(skipped_execute_node_ids) == 2
+ assert set(skipped_execute_node_ids) == {start.id, act_1.id}
+
+ skipped_execute_node_ids = get_skipped_execute_node_ids(pipeline, act_2.id)
+ assert len(skipped_execute_node_ids) == 3
+ assert set(skipped_execute_node_ids) == {start.id, act_1.id, eg.id}
+
+ skipped_execute_node_ids = get_skipped_execute_node_ids(pipeline, act_3.id)
+ assert len(skipped_execute_node_ids) == 3
+ assert set(skipped_execute_node_ids) == {start.id, act_1.id, eg.id}
+
+
+def test_get_allowed_start_node_ids_by_condition_parallel_gateway():
+ start = EmptyStartEvent()
+ act_1 = ServiceActivity(component_code="pipe_example_component", name="act_1")
+ cpg = ConditionalParallelGateway(
+ conditions={0: "${act_1_output} < 0", 1: "${act_1_output} >= 0", 2: "${act_1_output} >= 0"},
+ name="[act_2] or [act_3 and act_4]",
+ )
+ act_2 = ServiceActivity(component_code="pipe_example_component", name="act_2")
+ act_3 = ServiceActivity(component_code="pipe_example_component", name="act_3")
+ act_4 = ServiceActivity(component_code="pipe_example_component", name="act_4")
+ cg = ConvergeGateway()
+ end = EmptyEndEvent()
+ start.extend(act_1).extend(cpg).connect(act_2, act_3, act_4).to(cpg).converge(cg).extend(end)
+
+ pipeline = build_tree(start)
+ validate_gateways(pipeline)
+ allowed_start_node_ids = get_allowed_start_node_ids(pipeline)
+
+ assert len(allowed_start_node_ids) == 3
+ assert allowed_start_node_ids == [start.id, act_1.id, cpg.id]
+
+ skipped_execute_node_ids = get_skipped_execute_node_ids(pipeline, cpg.id)
+ assert len(skipped_execute_node_ids) == 2
+ assert set(skipped_execute_node_ids) == {start.id, act_1.id}
+
+
+def test_get_allowed_start_node_ids_by_normal():
+ start = EmptyStartEvent()
+ act_1 = ServiceActivity(component_code="pipe_example_component", name="act_1")
+ act_2 = ServiceActivity(component_code="pipe_example_component", name="act_2")
+ end = EmptyEndEvent()
+ start.extend(act_1).extend(act_2).extend(end)
+
+ pipeline = build_tree(start)
+ validate_gateways(pipeline)
+ allowed_start_node_ids = get_allowed_start_node_ids(pipeline)
+
+ assert len(allowed_start_node_ids) == 3
+ assert allowed_start_node_ids == [start.id, act_1.id, act_2.id]
+
+
+def test_validate_pipeline_start_node():
+ start = EmptyStartEvent()
+ act_1 = ServiceActivity(component_code="pipe_example_component", name="act_1")
+ eg = ExclusiveGateway(conditions={0: "${act_1_output} < 0", 1: "${act_1_output} >= 0"}, name="act_2 or act_3")
+ act_2 = ServiceActivity(component_code="pipe_example_component", name="act_2")
+ act_3 = ServiceActivity(component_code="pipe_example_component", name="act_3")
+ end = EmptyEndEvent()
+
+ start.extend(act_1).extend(eg).connect(act_2, act_3).to(eg).converge(end)
+ pipeline = build_tree(start)
+ validate_gateways(pipeline)
+
+ with pytest.raises(StartPositionInvalidException):
+ validate_pipeline_start_node(pipeline, end.id)
+
+ validate_pipeline_start_node(pipeline, act_1.id)