You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by "bbovenzi (via GitHub)" <gi...@apache.org> on 2023/02/08 00:16:59 UTC

[GitHub] [airflow] bbovenzi opened a new pull request, #29413: Add a new graph view inside of the grid view

bbovenzi opened a new pull request, #29413:
URL: https://github.com/apache/airflow/pull/29413

   Embed the graph view inside of the grid view so a user can see history and dependencies at the same time.
   
   Closes: https://github.com/apache/airflow/issues/19939
   
   <img width="1058" alt="Screenshot 2023-02-07 at 7 05 31 PM" src="https://user-images.githubusercontent.com/4600967/217395086-1615924a-9431-4bbb-ab59-014b2d12b56b.png">
   
   Task Groups:
   <img width="1184" alt="Screenshot 2023-02-07 at 7 07 44 PM" src="https://user-images.githubusercontent.com/4600967/217395348-5d6e46a2-221c-439e-ad02-e671e936c367.png">
   
   A complex DAG:
   <img width="1705" alt="Screenshot 2023-02-07 at 7 04 41 PM" src="https://user-images.githubusercontent.com/4600967/217395201-18749573-3358-449b-b907-b22f72bda24e.png">
   
   Dynamic task groups:
   <img width="1005" alt="Screenshot 2023-02-07 at 7 08 51 PM" src="https://user-images.githubusercontent.com/4600967/217395497-5df7f62e-2777-481e-b13b-1bebe86da935.png">
   
   Edge labels:
   <img width="1027" alt="Screenshot 2023-02-07 at 7 10 35 PM" src="https://user-images.githubusercontent.com/4600967/217395704-0ea1eda8-55fa-4433-a22e-da4a09851bef.png">
   
   
   Allows users to quickly clear a task group. Closes: https://github.com/apache/airflow/issues/29380
   
   The current graph is built off of dagre-d3 which has been deprecated for quite some time now. So in one way or another we had to update the graph.
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code changes, an Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvement+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in a newsfragment file, named `{pr_number}.significant.rst` or `{issue_number}.significant.rst`, in [newsfragments](https://github.com/apache/airflow/tree/main/newsfragments).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] bbovenzi commented on pull request #29413: Add a new graph inside of the grid view

Posted by "bbovenzi (via GitHub)" <gi...@apache.org>.
bbovenzi commented on PR #29413:
URL: https://github.com/apache/airflow/pull/29413#issuecomment-1423165328

   Just added the ability to zoom to a selected node to help navigate large graphs:
   ![Feb-08-2023 14-50-45](https://user-images.githubusercontent.com/4600967/217637784-241b1efd-67d4-4076-814c-9a43eb36f8d8.gif)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] pierrejeambrun commented on a diff in pull request #29413: Add a new graph inside of the grid view

Posted by "pierrejeambrun (via GitHub)" <gi...@apache.org>.
pierrejeambrun commented on code in PR #29413:
URL: https://github.com/apache/airflow/pull/29413#discussion_r1103667019


##########
airflow/www/static/js/dag/details/index.tsx:
##########
@@ -17,23 +17,89 @@
  * under the License.
  */
 
-import React from 'react';
+import React, { useEffect, useState } from 'react';
 import {
   Flex,
-  Box,
   Divider,
+  Tabs,
+  TabList,
+  TabPanel,
+  TabPanels,
+  Tab,
+  Text,
 } from '@chakra-ui/react';
+import { useSearchParams } from 'react-router-dom';
 
 import useSelection from 'src/dag/useSelection';
+import URLSearchParamsWrapper from 'src/utils/URLSearchParamWrapper';
+import { getTask, getMetaValue } from 'src/utils';
+import { useGridData, useTaskInstance } from 'src/api';
 
 import Header from './Header';
 import TaskInstanceContent from './taskInstance';
 import DagRunContent from './dagRun';
 import DagContent from './Dag';
+import Graph from './graph';
+import MappedInstances from './taskInstance/MappedInstances';
+import Logs from './taskInstance/Logs';
+import BackToTaskSummary from './taskInstance/BackToTaskSummary';
 
-const Details = () => {
+const SHOW_GRAPH_PARAM = 'show_graph';
+const dagId = getMetaValue('dag_id')!;
+
+interface Props {
+  openGroupIds: string[];
+  onToggleGroups: (groupIds: string[]) => void;
+}
+
+const Details = ({ openGroupIds, onToggleGroups }: Props) => {
   const { selected: { runId, taskId, mapIndex }, onSelect } = useSelection();
 
+  const [searchParams, setSearchParams] = useSearchParams();
+  const showGraph = !!searchParams.get(SHOW_GRAPH_PARAM);
+  const [tabIndex, setTabIndex] = useState(showGraph ? 1 : 0);
+
+  // TODO: restore preferred tab index logic

Review Comment:
   Do we want to do this in this PR or as a follow up ?



##########
airflow/www/static/js/dag/details/graph/index.tsx:
##########
@@ -0,0 +1,260 @@
+/*!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import React, {
+  useRef, useMemo, useState, useEffect,
+} from 'react';
+import Color from 'color';
+import {
+  Box,
+  useTheme,
+  Select,
+  Text,
+} from '@chakra-ui/react';
+import ReactFlow, {
+  ReactFlowProvider,
+  Controls,
+  Background,
+  MiniMap,
+  Node as ReactFlowNode,
+  useReactFlow,
+  ControlButton,
+  Panel,
+} from 'reactflow';
+import { MdCenterFocusStrong } from 'react-icons/md';
+
+import { useGraphData, useGridData } from 'src/api';
+import useOffsetHeight from 'src/utils/useOffsetHeight';
+import useSelection from 'src/dag/useSelection';
+import { getTask } from 'src/utils';
+import type { TaskInstance } from 'src/types';
+import { useGraphLayout } from 'src/utils/graph';
+import type { NodeType } from 'src/datasets/Graph/Node';
+
+import Edge from './Edge';
+import Node, { CustomNodeProps } from './Node';
+
+interface Props {
+  openGroupIds: string[];
+  onToggleGroups: (groupIds: string[]) => void;
+}
+
+const Graph = ({ openGroupIds, onToggleGroups }: Props) => {
+  const graphRef = useRef(null);
+  const { data } = useGraphData();
+  const [arrange, setArrange] = useState(data?.arrange || 'LR');
+
+  useEffect(() => {
+    setArrange(data?.arrange || 'LR');
+  }, [data?.arrange]);
+
+  const { data: graphData } = useGraphLayout({
+    edges: data?.edges, nodes: data?.nodes, openGroupIds, arrange,
+  });
+  const { selected: { runId, taskId } } = useSelection();
+  const { data: { dagRuns, groups } } = useGridData();
+  const { colors } = useTheme();
+  const { setCenter } = useReactFlow();
+  const latestDagRunId = dagRuns[dagRuns.length - 1]?.runId;
+
+  const offsetHeight = useOffsetHeight(graphRef, undefined, 750);
+
+  const nodes: ReactFlowNode<CustomNodeProps>[] = [];
+  const edges = (graphData?.edges || []).map((edge) => ({
+    id: edge.id,
+    source: edge.sources[0],
+    target: edge.targets[0],
+    data: { rest: edge },
+    type: 'custom',
+  }));
+
+  const flattenNodes = (children: NodeType[], parent?: string) => {
+    const parentNode = parent ? { parentNode: parent } : undefined;
+    children.forEach((node) => {
+      let instance: TaskInstance | undefined;
+      const group = getTask({ taskId: node.id, task: groups });
+      if (!node.id.includes('join_id') && runId) {
+        instance = group?.instances.find((ti) => ti.runId === runId);
+      }
+      const isSelected = node.id === taskId && !!instance;
+
+      nodes.push({
+        id: node.id,
+        data: {
+          width: node.width,
+          height: node.height,
+          task: group,
+          instance,
+          isSelected,
+          latestDagRunId,
+          onToggleCollapse: () => {
+            let newGroupIds = [];
+            if (!node.value.isOpen) {
+              newGroupIds = [...openGroupIds, node.value.label];
+            } else {
+              newGroupIds = openGroupIds.filter((g) => g !== node.value.label);
+            }
+            onToggleGroups(newGroupIds);
+          },
+          ...node.value,
+        },
+        type: 'custom',
+        position: {
+          x: node.x || 0,
+          y: node.y || 0,
+        },
+        ...parentNode,
+      });
+      if (node.children) {
+        flattenNodes(node.children, node.id);
+      }
+    });
+  };
+
+  if (graphData?.children) {
+    flattenNodes(graphData.children);
+  }
+
+  const focusNode = () => {
+    const selectedNode = nodes.find((n) => n.id === taskId);
+    if (selectedNode) {
+      setCenter(selectedNode.position.x, selectedNode.position.y, { duration: 1000 });
+    }
+  };
+
+  const nodeTypes = useMemo(() => ({ custom: Node }), []);

Review Comment:
   Instead of using a useMemo can we just take it out of the component and drop the useMemo ? (same for `edgeTypes`)
   ```
   const nodeTypes = { custom: Node };
   const edgeTypes = { custom: Edge };
   ```



##########
airflow/www/static/js/dag/details/graph/index.tsx:
##########
@@ -0,0 +1,260 @@
+/*!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import React, {
+  useRef, useMemo, useState, useEffect,
+} from 'react';
+import Color from 'color';
+import {
+  Box,
+  useTheme,
+  Select,
+  Text,
+} from '@chakra-ui/react';
+import ReactFlow, {
+  ReactFlowProvider,
+  Controls,
+  Background,
+  MiniMap,
+  Node as ReactFlowNode,
+  useReactFlow,
+  ControlButton,
+  Panel,
+} from 'reactflow';
+import { MdCenterFocusStrong } from 'react-icons/md';
+
+import { useGraphData, useGridData } from 'src/api';
+import useOffsetHeight from 'src/utils/useOffsetHeight';
+import useSelection from 'src/dag/useSelection';
+import { getTask } from 'src/utils';
+import type { TaskInstance } from 'src/types';
+import { useGraphLayout } from 'src/utils/graph';
+import type { NodeType } from 'src/datasets/Graph/Node';
+
+import Edge from './Edge';
+import Node, { CustomNodeProps } from './Node';
+
+interface Props {
+  openGroupIds: string[];
+  onToggleGroups: (groupIds: string[]) => void;
+}
+
+const Graph = ({ openGroupIds, onToggleGroups }: Props) => {
+  const graphRef = useRef(null);
+  const { data } = useGraphData();
+  const [arrange, setArrange] = useState(data?.arrange || 'LR');
+
+  useEffect(() => {
+    setArrange(data?.arrange || 'LR');
+  }, [data?.arrange]);
+
+  const { data: graphData } = useGraphLayout({
+    edges: data?.edges, nodes: data?.nodes, openGroupIds, arrange,
+  });
+  const { selected: { runId, taskId } } = useSelection();
+  const { data: { dagRuns, groups } } = useGridData();
+  const { colors } = useTheme();
+  const { setCenter } = useReactFlow();
+  const latestDagRunId = dagRuns[dagRuns.length - 1]?.runId;
+
+  const offsetHeight = useOffsetHeight(graphRef, undefined, 750);
+
+  const nodes: ReactFlowNode<CustomNodeProps>[] = [];
+  const edges = (graphData?.edges || []).map((edge) => ({
+    id: edge.id,
+    source: edge.sources[0],
+    target: edge.targets[0],
+    data: { rest: edge },
+    type: 'custom',
+  }));
+
+  const flattenNodes = (children: NodeType[], parent?: string) => {
+    const parentNode = parent ? { parentNode: parent } : undefined;
+    children.forEach((node) => {
+      let instance: TaskInstance | undefined;
+      const group = getTask({ taskId: node.id, task: groups });
+      if (!node.id.includes('join_id') && runId) {
+        instance = group?.instances.find((ti) => ti.runId === runId);
+      }
+      const isSelected = node.id === taskId && !!instance;
+
+      nodes.push({
+        id: node.id,
+        data: {
+          width: node.width,
+          height: node.height,
+          task: group,
+          instance,
+          isSelected,
+          latestDagRunId,
+          onToggleCollapse: () => {
+            let newGroupIds = [];
+            if (!node.value.isOpen) {
+              newGroupIds = [...openGroupIds, node.value.label];
+            } else {
+              newGroupIds = openGroupIds.filter((g) => g !== node.value.label);
+            }
+            onToggleGroups(newGroupIds);
+          },
+          ...node.value,
+        },
+        type: 'custom',
+        position: {
+          x: node.x || 0,
+          y: node.y || 0,
+        },
+        ...parentNode,
+      });
+      if (node.children) {
+        flattenNodes(node.children, node.id);
+      }
+    });
+  };
+
+  if (graphData?.children) {
+    flattenNodes(graphData.children);
+  }
+
+  const focusNode = () => {
+    const selectedNode = nodes.find((n) => n.id === taskId);
+    if (selectedNode) {
+      setCenter(selectedNode.position.x, selectedNode.position.y, { duration: 1000 });
+    }
+  };
+
+  const nodeTypes = useMemo(() => ({ custom: Node }), []);
+  const edgeTypes = useMemo(() => ({ custom: Edge }), []);
+
+  const newEdges = edges.map((e) => {

Review Comment:
   I think we can extract this outside the component, with a function, maybe like `buildGraphEdges(gridDataEdges, taskId,)`. This will make the component a bit smaller and easier to maintain on top. Same for nodeColor and nodeStroke I believe also avoid redefining these functions on each render.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] bbovenzi commented on a diff in pull request #29413: Add a new graph inside of the grid view

Posted by "bbovenzi (via GitHub)" <gi...@apache.org>.
bbovenzi commented on code in PR #29413:
URL: https://github.com/apache/airflow/pull/29413#discussion_r1104764231


##########
airflow/www/static/js/dag/details/index.tsx:
##########
@@ -17,23 +17,89 @@
  * under the License.
  */
 
-import React from 'react';
+import React, { useEffect, useState } from 'react';
 import {
   Flex,
-  Box,
   Divider,
+  Tabs,
+  TabList,
+  TabPanel,
+  TabPanels,
+  Tab,
+  Text,
 } from '@chakra-ui/react';
+import { useSearchParams } from 'react-router-dom';
 
 import useSelection from 'src/dag/useSelection';
+import URLSearchParamsWrapper from 'src/utils/URLSearchParamWrapper';
+import { getTask, getMetaValue } from 'src/utils';
+import { useGridData, useTaskInstance } from 'src/api';
 
 import Header from './Header';
 import TaskInstanceContent from './taskInstance';
 import DagRunContent from './dagRun';
 import DagContent from './Dag';
+import Graph from './graph';
+import MappedInstances from './taskInstance/MappedInstances';
+import Logs from './taskInstance/Logs';
+import BackToTaskSummary from './taskInstance/BackToTaskSummary';
 
-const Details = () => {
+const SHOW_GRAPH_PARAM = 'show_graph';
+const dagId = getMetaValue('dag_id')!;
+
+interface Props {
+  openGroupIds: string[];
+  onToggleGroups: (groupIds: string[]) => void;
+}
+
+const Details = ({ openGroupIds, onToggleGroups }: Props) => {
   const { selected: { runId, taskId, mapIndex }, onSelect } = useSelection();
 
+  const [searchParams, setSearchParams] = useSearchParams();
+  const showGraph = !!searchParams.get(SHOW_GRAPH_PARAM);
+  const [tabIndex, setTabIndex] = useState(showGraph ? 1 : 0);
+
+  // TODO: restore preferred tab index logic

Review Comment:
   I'll do this as a follow up with moving task actions to a modal so they can be accessed from all tabs. When we do that, we may want to default to the graph tab instead of details



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] notatallshaw-gts commented on pull request #29413: Add a new graph inside of the grid view

Posted by "notatallshaw-gts (via GitHub)" <gi...@apache.org>.
notatallshaw-gts commented on PR #29413:
URL: https://github.com/apache/airflow/pull/29413#issuecomment-1422691322

   Looks great! I'm sure you've already thought of these use cases but just in case two things I'd be worried about with this style of straight lines is:
   
   1. How it looks when a task has a large number of downstream tasks (e.g. 20+)?
   2. Is it possible to visually distinguish lines when they have to cross each other?
   
   The current curve lines means even it's messy the angle always allows you to distinguish the lines, e.g.:
   
   ```python
   import datetime
   from airflow.models import DAG
   from airflow.operators.empty import EmptyOperator
   
   
   with DAG(
       dag_id="test",
       schedule_interval="@once",
       start_date=datetime.datetime(2000, 1, 1),
   ) as test:
       upstream_tasks = [EmptyOperator(task_id=f"upstream_{i}") for i in range(4)]
       downstream_tasks = [EmptyOperator(task_id=f"downstream_{i}") for i in range(4)]
       for upstream_task in upstream_tasks:
           upstream_task >> downstream_tasks
   
       other_tasks = [EmptyOperator(task_id=f"other_{i}") for i in range(16)]
       upstream_tasks[0] >> other_tasks
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] bbovenzi commented on a diff in pull request #29413: Add a new graph inside of the grid view

Posted by "bbovenzi (via GitHub)" <gi...@apache.org>.
bbovenzi commented on code in PR #29413:
URL: https://github.com/apache/airflow/pull/29413#discussion_r1119101266


##########
airflow/www/static/js/dag/details/index.tsx:
##########
@@ -17,23 +17,89 @@
  * under the License.
  */
 
-import React from 'react';
+import React, { useEffect, useState } from 'react';
 import {
   Flex,
-  Box,
   Divider,
+  Tabs,
+  TabList,
+  TabPanel,
+  TabPanels,
+  Tab,
+  Text,
 } from '@chakra-ui/react';
+import { useSearchParams } from 'react-router-dom';
 
 import useSelection from 'src/dag/useSelection';
+import URLSearchParamsWrapper from 'src/utils/URLSearchParamWrapper';
+import { getTask, getMetaValue } from 'src/utils';
+import { useGridData, useTaskInstance } from 'src/api';
 
 import Header from './Header';
 import TaskInstanceContent from './taskInstance';
 import DagRunContent from './dagRun';
 import DagContent from './Dag';
+import Graph from './graph';
+import MappedInstances from './taskInstance/MappedInstances';
+import Logs from './taskInstance/Logs';
+import BackToTaskSummary from './taskInstance/BackToTaskSummary';
 
-const Details = () => {
+const SHOW_GRAPH_PARAM = 'show_graph';
+const dagId = getMetaValue('dag_id')!;
+
+interface Props {
+  openGroupIds: string[];
+  onToggleGroups: (groupIds: string[]) => void;
+}
+
+const Details = ({ openGroupIds, onToggleGroups }: Props) => {
   const { selected: { runId, taskId, mapIndex }, onSelect } = useSelection();
 
+  const [searchParams, setSearchParams] = useSearchParams();
+  const showGraph = !!searchParams.get(SHOW_GRAPH_PARAM);
+  const [tabIndex, setTabIndex] = useState(showGraph ? 1 : 0);
+
+  // TODO: restore preferred tab index logic

Review Comment:
   I just updated this to store the tab location in the url params.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] bbovenzi commented on a diff in pull request #29413: Add a new graph inside of the grid view

Posted by "bbovenzi (via GitHub)" <gi...@apache.org>.
bbovenzi commented on code in PR #29413:
URL: https://github.com/apache/airflow/pull/29413#discussion_r1114762187


##########
airflow/www/webpack.config.js:
##########
@@ -136,6 +137,15 @@ const config = {
           'css-loader',
         ],
       },
+      // Extract css files
+      {
+        test: /\.css$/,

Review Comment:
   This allows us to import the css file for react-flow inside of react



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ashb commented on a diff in pull request #29413: Add a new graph inside of the grid view

Posted by "ashb (via GitHub)" <gi...@apache.org>.
ashb commented on code in PR #29413:
URL: https://github.com/apache/airflow/pull/29413#discussion_r1115866839


##########
airflow/www/static/js/dag/details/graph/Node.tsx:
##########
@@ -0,0 +1,156 @@
+/*!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import React from 'react';
+import {
+  Box,
+  Text,
+  Flex,
+} from '@chakra-ui/react';
+import { Handle, NodeProps, Position } from 'reactflow';
+
+import { SimpleStatus } from 'src/dag/StatusBox';
+import useSelection from 'src/dag/useSelection';
+import type { DagRun, Task, TaskInstance } from 'src/types';
+import { finalStatesMap, hoverDelay } from 'src/utils';
+import Tooltip from 'src/components/Tooltip';
+import InstanceTooltip from 'src/dag/InstanceTooltip';
+import { useContainerRef } from 'src/context/containerRef';
+
+export interface CustomNodeProps {
+  label: string;
+  height?: number;
+  width?: number;
+  isJoinNode?: boolean;
+  instance?: TaskInstance;
+  task?: Task | null;
+  isSelected: boolean;
+  latestDagRunId: DagRun['runId'];
+  childCount?: number;
+  onToggleCollapse: () => void;
+  isOpen?: boolean;
+}
+
+const Node = ({
+  id,
+  data: {
+    label,
+    childCount,
+    height,
+    width,
+    isJoinNode,
+    instance,
+    task,
+    isSelected,
+    latestDagRunId,
+    onToggleCollapse,
+    isOpen,
+  },
+}: NodeProps<CustomNodeProps>) => {
+  const { onSelect } = useSelection();
+  const containerRef = useContainerRef();
+
+  if (isJoinNode) {
+    return (
+      <>
+        <Handle type="target" position={Position.Top} style={{ visibility: 'hidden' }} />
+        <Box height={`${height}px`} width={`${width}px`} borderRadius={width} bg="gray.400" />
+        <Handle type="source" position={Position.Bottom} style={{ visibility: 'hidden' }} />
+      </>
+    );
+  }
+
+  const isMapped = task?.isMapped;
+  const mappedStates = instance?.mappedStates;
+
+  const numMap = finalStatesMap();
+  let numMapped = 0;
+
+  if (isMapped && mappedStates) {
+    Object.keys(mappedStates).forEach((stateKey) => {
+      const num = mappedStates[stateKey];
+      numMapped += num;
+      numMap.set(stateKey || 'no_status', num);
+    });

Review Comment:
   This feels general purpose/like I've seen it somewhere else before. Should this be pulled out in to a common function somewhere?



##########
airflow/www/static/js/dag/details/graph/Edge.tsx:
##########
@@ -0,0 +1,66 @@
+/*!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import React from 'react';
+import { Text } from '@chakra-ui/react';
+import type {
+  ElkEdgeSection, ElkLabel, ElkPoint, LayoutOptions,
+} from 'elkjs';
+
+import Edge from 'src/datasets/Graph/Edge';
+import { Group } from '@visx/group';
+
+interface EdgeProps {
+  data?: {
+    rest: {
+      isSelected: boolean | '' | null;

Review Comment:
   What's the reason we allow all three of these types here?



##########
airflow/www/static/js/dag/details/graph/utils.ts:
##########
@@ -0,0 +1,116 @@
+/*!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import Color from 'color';
+import type {
+  Node as ReactFlowNode,
+} from 'reactflow';
+import type { ElkExtendedEdge } from 'elkjs';
+
+import type { CustomNodeProps } from './Node';
+
+export const nodeColor = (
+  { data: { height, width, instance } }: ReactFlowNode<CustomNodeProps>,
+) => {
+  if (!height || !width) return '';
+  if (width > 200 || height > 60) {
+    return '#cccccc50';
+  }
+  if (instance?.state) {
+    return `${Color(stateColors[instance.state]).hex()}90`;
+  }
+  return '#cccccc90';
+};
+
+export const nodeStrokeColor = (
+  { data: { isSelected } }: ReactFlowNode<CustomNodeProps>,
+  colors: any,
+) => (isSelected ? colors.blue[500] : '');
+
+interface BuildEdgesProps {
+  edges?: ElkExtendedEdge[];
+  nodes: ReactFlowNode<CustomNodeProps>[];
+  taskId?: string | null;
+}
+
+export const buildEdges = ({

Review Comment:
   Nit: should this instead by in `graph/Edge.tsx`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] pierrejeambrun commented on a diff in pull request #29413: Add a new graph inside of the grid view

Posted by "pierrejeambrun (via GitHub)" <gi...@apache.org>.
pierrejeambrun commented on code in PR #29413:
URL: https://github.com/apache/airflow/pull/29413#discussion_r1103667180


##########
airflow/www/static/js/dag/details/graph/index.tsx:
##########
@@ -0,0 +1,260 @@
+/*!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import React, {
+  useRef, useMemo, useState, useEffect,
+} from 'react';
+import Color from 'color';
+import {
+  Box,
+  useTheme,
+  Select,
+  Text,
+} from '@chakra-ui/react';
+import ReactFlow, {
+  ReactFlowProvider,
+  Controls,
+  Background,
+  MiniMap,
+  Node as ReactFlowNode,
+  useReactFlow,
+  ControlButton,
+  Panel,
+} from 'reactflow';
+import { MdCenterFocusStrong } from 'react-icons/md';
+
+import { useGraphData, useGridData } from 'src/api';
+import useOffsetHeight from 'src/utils/useOffsetHeight';
+import useSelection from 'src/dag/useSelection';
+import { getTask } from 'src/utils';
+import type { TaskInstance } from 'src/types';
+import { useGraphLayout } from 'src/utils/graph';
+import type { NodeType } from 'src/datasets/Graph/Node';
+
+import Edge from './Edge';
+import Node, { CustomNodeProps } from './Node';
+
+interface Props {
+  openGroupIds: string[];
+  onToggleGroups: (groupIds: string[]) => void;
+}
+
+const Graph = ({ openGroupIds, onToggleGroups }: Props) => {
+  const graphRef = useRef(null);
+  const { data } = useGraphData();
+  const [arrange, setArrange] = useState(data?.arrange || 'LR');
+
+  useEffect(() => {
+    setArrange(data?.arrange || 'LR');
+  }, [data?.arrange]);
+
+  const { data: graphData } = useGraphLayout({
+    edges: data?.edges, nodes: data?.nodes, openGroupIds, arrange,
+  });
+  const { selected: { runId, taskId } } = useSelection();
+  const { data: { dagRuns, groups } } = useGridData();
+  const { colors } = useTheme();
+  const { setCenter } = useReactFlow();
+  const latestDagRunId = dagRuns[dagRuns.length - 1]?.runId;
+
+  const offsetHeight = useOffsetHeight(graphRef, undefined, 750);
+
+  const nodes: ReactFlowNode<CustomNodeProps>[] = [];
+  const edges = (graphData?.edges || []).map((edge) => ({
+    id: edge.id,
+    source: edge.sources[0],
+    target: edge.targets[0],
+    data: { rest: edge },
+    type: 'custom',
+  }));
+
+  const flattenNodes = (children: NodeType[], parent?: string) => {
+    const parentNode = parent ? { parentNode: parent } : undefined;
+    children.forEach((node) => {
+      let instance: TaskInstance | undefined;
+      const group = getTask({ taskId: node.id, task: groups });
+      if (!node.id.includes('join_id') && runId) {
+        instance = group?.instances.find((ti) => ti.runId === runId);
+      }
+      const isSelected = node.id === taskId && !!instance;
+
+      nodes.push({
+        id: node.id,
+        data: {
+          width: node.width,
+          height: node.height,
+          task: group,
+          instance,
+          isSelected,
+          latestDagRunId,
+          onToggleCollapse: () => {
+            let newGroupIds = [];
+            if (!node.value.isOpen) {
+              newGroupIds = [...openGroupIds, node.value.label];
+            } else {
+              newGroupIds = openGroupIds.filter((g) => g !== node.value.label);
+            }
+            onToggleGroups(newGroupIds);
+          },
+          ...node.value,
+        },
+        type: 'custom',
+        position: {
+          x: node.x || 0,
+          y: node.y || 0,
+        },
+        ...parentNode,
+      });
+      if (node.children) {
+        flattenNodes(node.children, node.id);
+      }
+    });
+  };
+
+  if (graphData?.children) {
+    flattenNodes(graphData.children);
+  }
+
+  const focusNode = () => {
+    const selectedNode = nodes.find((n) => n.id === taskId);
+    if (selectedNode) {
+      setCenter(selectedNode.position.x, selectedNode.position.y, { duration: 1000 });
+    }
+  };
+
+  const nodeTypes = useMemo(() => ({ custom: Node }), []);
+  const edgeTypes = useMemo(() => ({ custom: Edge }), []);
+
+  const newEdges = edges.map((e) => {

Review Comment:
   I think we can extract this outside the component, with a function, maybe like `buildGraphEdges(gridDataEdges, taskId,)`. This will make the component a bit smaller and easier to maintain. Same for nodeColor and nodeStroke, also avoid redefining these functions on each render.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] bbovenzi commented on pull request #29413: Add a new graph inside of the grid view

Posted by "bbovenzi (via GitHub)" <gi...@apache.org>.
bbovenzi commented on PR #29413:
URL: https://github.com/apache/airflow/pull/29413#issuecomment-1451065822

   Failing check is not because of this PR. Merging.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] pierrejeambrun commented on pull request #29413: Add a new graph inside of the grid view

Posted by "pierrejeambrun (via GitHub)" <gi...@apache.org>.
pierrejeambrun commented on PR #29413:
URL: https://github.com/apache/airflow/pull/29413#issuecomment-1445486187

   Forget my comment about asset recompilation error. We fixed it here https://github.com/apache/airflow/pull/29769.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on pull request #29413: Add a new graph inside of the grid view

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #29413:
URL: https://github.com/apache/airflow/pull/29413#issuecomment-1423193996

   :heart: :heart: :heart: :heart: :heart: :heart: 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] bbovenzi merged pull request #29413: Add a new graph inside of the grid view

Posted by "bbovenzi (via GitHub)" <gi...@apache.org>.
bbovenzi merged PR #29413:
URL: https://github.com/apache/airflow/pull/29413


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] jedcunningham commented on pull request #29413: Add a new graph inside of the grid view

Posted by "jedcunningham (via GitHub)" <gi...@apache.org>.
jedcunningham commented on PR #29413:
URL: https://github.com/apache/airflow/pull/29413#issuecomment-1451123006

   (#protm)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] pierrejeambrun commented on a diff in pull request #29413: Add a new graph inside of the grid view

Posted by "pierrejeambrun (via GitHub)" <gi...@apache.org>.
pierrejeambrun commented on code in PR #29413:
URL: https://github.com/apache/airflow/pull/29413#discussion_r1122340683


##########
airflow/www/static/js/utils/index.ts:
##########
@@ -49,6 +48,44 @@ const finalStatesMap = () =>
     ["no_status", 0],
   ]);
 
+interface GroupSummaryProps {
+  group: Task;
+  runId?: string;
+  mappedStates: TaskInstance["mappedStates"];
+}
+
+const getGroupAndMapSummary = ({
+  group,
+  runId,
+  mappedStates,
+}: GroupSummaryProps) => {
+  let totalTasks = 0;
+  const childTaskMap = finalStatesMap();
+  if (!!group.children && !group.isMapped) {
+    group.children?.forEach((child) => {
+      const taskInstance = child.instances.find((ti) => ti.runId === runId);
+      if (taskInstance) {
+        const stateKey =
+          taskInstance.state == null ? "no_status" : taskInstance.state;
+        if (childTaskMap.has(stateKey)) {
+          childTaskMap.set(stateKey, (childTaskMap.get(stateKey) || 0) + 1);
+        }
+      }
+    });
+  } else if (group.isMapped && mappedStates) {
+    Object.keys(mappedStates).forEach((stateKey) => {

Review Comment:
   Nit: If you need access to keys and values, Object.entries can be a little easier. It has less support than Object.keys but babel should be able to transform that. (We might want to check, I don't see the browserlist target in our package.json)



##########
airflow/www/static/js/dag/index.tsx:
##########
@@ -30,13 +31,24 @@ import Main from "./Main";
 // create shadowRoot
 const root = document.querySelector("#root");
 const shadowRoot = root?.attachShadow({ mode: "open" });
+
 const cache = createCache({
   container: shadowRoot,
   key: "c",
 });
 const mainElement = document.getElementById("react-container");
 
 if (mainElement) {
+  const styleTag = document.createElement("style");
+  // We need to set the z-index manually to make sure edges inside of groups still work
+  let style = reactFlowStyle.toString();
+  style += `

Review Comment:
   To avoid this we could set a `background-color: transparent` on nodes that are taskgroup when expended/isOpen.
   
   I tried playing around with z-index trying to avoid the `!important` but it looks like the stacking contexts are a bit tricky there.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] bbovenzi commented on a diff in pull request #29413: Add a new graph inside of the grid view

Posted by "bbovenzi (via GitHub)" <gi...@apache.org>.
bbovenzi commented on code in PR #29413:
URL: https://github.com/apache/airflow/pull/29413#discussion_r1119101266


##########
airflow/www/static/js/dag/details/index.tsx:
##########
@@ -17,23 +17,89 @@
  * under the License.
  */
 
-import React from 'react';
+import React, { useEffect, useState } from 'react';
 import {
   Flex,
-  Box,
   Divider,
+  Tabs,
+  TabList,
+  TabPanel,
+  TabPanels,
+  Tab,
+  Text,
 } from '@chakra-ui/react';
+import { useSearchParams } from 'react-router-dom';
 
 import useSelection from 'src/dag/useSelection';
+import URLSearchParamsWrapper from 'src/utils/URLSearchParamWrapper';
+import { getTask, getMetaValue } from 'src/utils';
+import { useGridData, useTaskInstance } from 'src/api';
 
 import Header from './Header';
 import TaskInstanceContent from './taskInstance';
 import DagRunContent from './dagRun';
 import DagContent from './Dag';
+import Graph from './graph';
+import MappedInstances from './taskInstance/MappedInstances';
+import Logs from './taskInstance/Logs';
+import BackToTaskSummary from './taskInstance/BackToTaskSummary';
 
-const Details = () => {
+const SHOW_GRAPH_PARAM = 'show_graph';
+const dagId = getMetaValue('dag_id')!;
+
+interface Props {
+  openGroupIds: string[];
+  onToggleGroups: (groupIds: string[]) => void;
+}
+
+const Details = ({ openGroupIds, onToggleGroups }: Props) => {
   const { selected: { runId, taskId, mapIndex }, onSelect } = useSelection();
 
+  const [searchParams, setSearchParams] = useSearchParams();
+  const showGraph = !!searchParams.get(SHOW_GRAPH_PARAM);
+  const [tabIndex, setTabIndex] = useState(showGraph ? 1 : 0);
+
+  // TODO: restore preferred tab index logic

Review Comment:
   I just updated this to store the tab location in the url params.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] jedcunningham commented on a diff in pull request #29413: Add a new graph inside of the grid view

Posted by "jedcunningham (via GitHub)" <gi...@apache.org>.
jedcunningham commented on code in PR #29413:
URL: https://github.com/apache/airflow/pull/29413#discussion_r1120613976


##########
airflow/www/views.py:
##########
@@ -3530,6 +3530,42 @@ def extra_links(self, *, session: Session = NEW_SESSION):
         else:
             return {"url": None, "error": f"No URL found for {link_name}"}, 404
 
+    @expose("/object/graph_data")
+    @auth.has_access(
+        [
+            (permissions.ACTION_CAN_READ, permissions.RESOURCE_DAG),
+            (permissions.ACTION_CAN_READ, permissions.RESOURCE_TASK_INSTANCE),
+            (permissions.ACTION_CAN_READ, permissions.RESOURCE_TASK_LOG),
+        ]
+    )
+    @gzipped
+    @action_logging
+    @provide_session
+    def graph_data(self, session=None):
+        """Get Graph Data"""
+        dag_id = request.args.get("dag_id")
+        dag = get_airflow_app().dag_bag.get_dag(dag_id, session=session)
+        root = request.args.get("root")
+        if root:
+            filter_upstream = True if request.args.get("filter_upstream") == "true" else False
+            filter_downstream = True if request.args.get("filter_downstream") == "true" else False

Review Comment:
   ```suggestion
               filter_upstream = request.args.get("filter_upstream") == "true"
               filter_downstream = request.args.get("filter_downstream") == "true"
   ```
   
   This already gives you a bool 👍



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] bbovenzi commented on pull request #29413: Add a new graph inside of the grid view

Posted by "bbovenzi (via GitHub)" <gi...@apache.org>.
bbovenzi commented on PR #29413:
URL: https://github.com/apache/airflow/pull/29413#issuecomment-1422981078

   Here's the two versions of that DAG:
   
   <img width="348" alt="Screenshot 2023-02-08 at 11 34 00 AM" src="https://user-images.githubusercontent.com/4600967/217592932-db5f20b8-ea27-4fa8-966c-561cddd30b29.png">
   <img width="182" alt="Screenshot 2023-02-08 at 11 34 09 AM" src="https://user-images.githubusercontent.com/4600967/217592940-80222a40-4cec-4c4a-aa32-16ceb687182d.png">
   
   With task selection working across grid/graph. You can highlight the upstream/downstream even when not hovering or explicitly filtering the whole graph:
   <img width="707" alt="Screenshot 2023-02-08 at 12 22 14 PM" src="https://user-images.githubusercontent.com/4600967/217605031-2e19f09c-ed57-400b-9ec9-4f4b2190e938.png">
   
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] notatallshaw-gts commented on pull request #29413: Add a new graph inside of the grid view

Posted by "notatallshaw-gts (via GitHub)" <gi...@apache.org>.
notatallshaw-gts commented on PR #29413:
URL: https://github.com/apache/airflow/pull/29413#issuecomment-1423208628

   > We could also experiment with concentrating edges, but there's always going to be trade-offs between different graphing strategies and various DAG structures.
   
   Yeah I didn't want to kick of a design discussion in this PR and slow work down, I'm sure once this lands people will have lots of suggestions.
   
   I just wanted to check this didn't introduce regressions in being able to follow edges in complicated graphs, and from that perspective it seems good!
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] bbovenzi commented on a diff in pull request #29413: Add a new graph inside of the grid view

Posted by "bbovenzi (via GitHub)" <gi...@apache.org>.
bbovenzi commented on code in PR #29413:
URL: https://github.com/apache/airflow/pull/29413#discussion_r1116075518


##########
airflow/www/static/js/dag/details/graph/Node.tsx:
##########
@@ -0,0 +1,156 @@
+/*!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import React from 'react';
+import {
+  Box,
+  Text,
+  Flex,
+} from '@chakra-ui/react';
+import { Handle, NodeProps, Position } from 'reactflow';
+
+import { SimpleStatus } from 'src/dag/StatusBox';
+import useSelection from 'src/dag/useSelection';
+import type { DagRun, Task, TaskInstance } from 'src/types';
+import { finalStatesMap, hoverDelay } from 'src/utils';
+import Tooltip from 'src/components/Tooltip';
+import InstanceTooltip from 'src/dag/InstanceTooltip';
+import { useContainerRef } from 'src/context/containerRef';
+
+export interface CustomNodeProps {
+  label: string;
+  height?: number;
+  width?: number;
+  isJoinNode?: boolean;
+  instance?: TaskInstance;
+  task?: Task | null;
+  isSelected: boolean;
+  latestDagRunId: DagRun['runId'];
+  childCount?: number;
+  onToggleCollapse: () => void;
+  isOpen?: boolean;
+}
+
+const Node = ({
+  id,
+  data: {
+    label,
+    childCount,
+    height,
+    width,
+    isJoinNode,
+    instance,
+    task,
+    isSelected,
+    latestDagRunId,
+    onToggleCollapse,
+    isOpen,
+  },
+}: NodeProps<CustomNodeProps>) => {
+  const { onSelect } = useSelection();
+  const containerRef = useContainerRef();
+
+  if (isJoinNode) {
+    return (
+      <>
+        <Handle type="target" position={Position.Top} style={{ visibility: 'hidden' }} />
+        <Box height={`${height}px`} width={`${width}px`} borderRadius={width} bg="gray.400" />
+        <Handle type="source" position={Position.Bottom} style={{ visibility: 'hidden' }} />
+      </>
+    );
+  }
+
+  const isMapped = task?.isMapped;
+  const mappedStates = instance?.mappedStates;
+
+  const numMap = finalStatesMap();
+  let numMapped = 0;
+
+  if (isMapped && mappedStates) {
+    Object.keys(mappedStates).forEach((stateKey) => {
+      const num = mappedStates[stateKey];
+      numMapped += num;
+      numMap.set(stateKey || 'no_status', num);
+    });

Review Comment:
   You're right. Let me do that.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] talnagar commented on pull request #29413: Add a new graph inside of the grid view

Posted by "talnagar (via GitHub)" <gi...@apache.org>.
talnagar commented on PR #29413:
URL: https://github.com/apache/airflow/pull/29413#issuecomment-1436416320

   this feature looks amazing! 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] bbovenzi commented on a diff in pull request #29413: Add a new graph inside of the grid view

Posted by "bbovenzi (via GitHub)" <gi...@apache.org>.
bbovenzi commented on code in PR #29413:
URL: https://github.com/apache/airflow/pull/29413#discussion_r1116070096


##########
airflow/www/static/js/dag/details/graph/utils.ts:
##########
@@ -0,0 +1,116 @@
+/*!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import Color from 'color';
+import type {
+  Node as ReactFlowNode,
+} from 'reactflow';
+import type { ElkExtendedEdge } from 'elkjs';
+
+import type { CustomNodeProps } from './Node';
+
+export const nodeColor = (
+  { data: { height, width, instance } }: ReactFlowNode<CustomNodeProps>,
+) => {
+  if (!height || !width) return '';
+  if (width > 200 || height > 60) {
+    return '#cccccc50';
+  }
+  if (instance?.state) {
+    return `${Color(stateColors[instance.state]).hex()}90`;
+  }
+  return '#cccccc90';
+};
+
+export const nodeStrokeColor = (
+  { data: { isSelected } }: ReactFlowNode<CustomNodeProps>,
+  colors: any,
+) => (isSelected ? colors.blue[500] : '');
+
+interface BuildEdgesProps {
+  edges?: ElkExtendedEdge[];
+  nodes: ReactFlowNode<CustomNodeProps>[];
+  taskId?: string | null;
+}
+
+export const buildEdges = ({

Review Comment:
   This is the edge data itself in a format that works for react-flow. Edge.tsx is the component itself. I can improve the names or add a comment to avoid confusion



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] bbovenzi commented on a diff in pull request #29413: Add a new graph inside of the grid view

Posted by "bbovenzi (via GitHub)" <gi...@apache.org>.
bbovenzi commented on code in PR #29413:
URL: https://github.com/apache/airflow/pull/29413#discussion_r1122384454


##########
airflow/www/static/js/dag/index.tsx:
##########
@@ -30,13 +31,24 @@ import Main from "./Main";
 // create shadowRoot
 const root = document.querySelector("#root");
 const shadowRoot = root?.attachShadow({ mode: "open" });
+
 const cache = createCache({
   container: shadowRoot,
   key: "c",
 });
 const mainElement = document.getElementById("react-container");
 
 if (mainElement) {
+  const styleTag = document.createElement("style");
+  // We need to set the z-index manually to make sure edges inside of groups still work
+  let style = reactFlowStyle.toString();
+  style += `

Review Comment:
   Fixed with your `defaultEdgeOptions={{ zIndex: 1 }}` suggestion.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] eladkal commented on pull request #29413: Add a new graph inside of the grid view

Posted by "eladkal (via GitHub)" <gi...@apache.org>.
eladkal commented on PR #29413:
URL: https://github.com/apache/airflow/pull/29413#issuecomment-1423190053

   > Just added the ability to zoom to a selected node to help navigate large graphs
   
   This is super cool!
   BTW a really useful feature would be to select 2 tasks in the list then showing graph view only for these two tasks (all nodes that are between the two) it would be extremely useful for zooming in on a specific flow. Basically what you show with the blue lines but without showing all the rest of the nodes.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] pierrejeambrun commented on a diff in pull request #29413: Add a new graph inside of the grid view

Posted by "pierrejeambrun (via GitHub)" <gi...@apache.org>.
pierrejeambrun commented on code in PR #29413:
URL: https://github.com/apache/airflow/pull/29413#discussion_r1103667180


##########
airflow/www/static/js/dag/details/graph/index.tsx:
##########
@@ -0,0 +1,260 @@
+/*!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import React, {
+  useRef, useMemo, useState, useEffect,
+} from 'react';
+import Color from 'color';
+import {
+  Box,
+  useTheme,
+  Select,
+  Text,
+} from '@chakra-ui/react';
+import ReactFlow, {
+  ReactFlowProvider,
+  Controls,
+  Background,
+  MiniMap,
+  Node as ReactFlowNode,
+  useReactFlow,
+  ControlButton,
+  Panel,
+} from 'reactflow';
+import { MdCenterFocusStrong } from 'react-icons/md';
+
+import { useGraphData, useGridData } from 'src/api';
+import useOffsetHeight from 'src/utils/useOffsetHeight';
+import useSelection from 'src/dag/useSelection';
+import { getTask } from 'src/utils';
+import type { TaskInstance } from 'src/types';
+import { useGraphLayout } from 'src/utils/graph';
+import type { NodeType } from 'src/datasets/Graph/Node';
+
+import Edge from './Edge';
+import Node, { CustomNodeProps } from './Node';
+
+interface Props {
+  openGroupIds: string[];
+  onToggleGroups: (groupIds: string[]) => void;
+}
+
+const Graph = ({ openGroupIds, onToggleGroups }: Props) => {
+  const graphRef = useRef(null);
+  const { data } = useGraphData();
+  const [arrange, setArrange] = useState(data?.arrange || 'LR');
+
+  useEffect(() => {
+    setArrange(data?.arrange || 'LR');
+  }, [data?.arrange]);
+
+  const { data: graphData } = useGraphLayout({
+    edges: data?.edges, nodes: data?.nodes, openGroupIds, arrange,
+  });
+  const { selected: { runId, taskId } } = useSelection();
+  const { data: { dagRuns, groups } } = useGridData();
+  const { colors } = useTheme();
+  const { setCenter } = useReactFlow();
+  const latestDagRunId = dagRuns[dagRuns.length - 1]?.runId;
+
+  const offsetHeight = useOffsetHeight(graphRef, undefined, 750);
+
+  const nodes: ReactFlowNode<CustomNodeProps>[] = [];
+  const edges = (graphData?.edges || []).map((edge) => ({
+    id: edge.id,
+    source: edge.sources[0],
+    target: edge.targets[0],
+    data: { rest: edge },
+    type: 'custom',
+  }));
+
+  const flattenNodes = (children: NodeType[], parent?: string) => {
+    const parentNode = parent ? { parentNode: parent } : undefined;
+    children.forEach((node) => {
+      let instance: TaskInstance | undefined;
+      const group = getTask({ taskId: node.id, task: groups });
+      if (!node.id.includes('join_id') && runId) {
+        instance = group?.instances.find((ti) => ti.runId === runId);
+      }
+      const isSelected = node.id === taskId && !!instance;
+
+      nodes.push({
+        id: node.id,
+        data: {
+          width: node.width,
+          height: node.height,
+          task: group,
+          instance,
+          isSelected,
+          latestDagRunId,
+          onToggleCollapse: () => {
+            let newGroupIds = [];
+            if (!node.value.isOpen) {
+              newGroupIds = [...openGroupIds, node.value.label];
+            } else {
+              newGroupIds = openGroupIds.filter((g) => g !== node.value.label);
+            }
+            onToggleGroups(newGroupIds);
+          },
+          ...node.value,
+        },
+        type: 'custom',
+        position: {
+          x: node.x || 0,
+          y: node.y || 0,
+        },
+        ...parentNode,
+      });
+      if (node.children) {
+        flattenNodes(node.children, node.id);
+      }
+    });
+  };
+
+  if (graphData?.children) {
+    flattenNodes(graphData.children);
+  }
+
+  const focusNode = () => {
+    const selectedNode = nodes.find((n) => n.id === taskId);
+    if (selectedNode) {
+      setCenter(selectedNode.position.x, selectedNode.position.y, { duration: 1000 });
+    }
+  };
+
+  const nodeTypes = useMemo(() => ({ custom: Node }), []);
+  const edgeTypes = useMemo(() => ({ custom: Edge }), []);
+
+  const newEdges = edges.map((e) => {

Review Comment:
   I think we can extract this outside the component, with a function, maybe like `buildGraphEdges(gridDataEdges, taskId,)`. This will make the component a bit smaller and easier to maintain. Same for nodeColor and nodeStroke I believe also avoid redefining these functions on each render.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] bbovenzi commented on a diff in pull request #29413: Add a new graph inside of the grid view

Posted by "bbovenzi (via GitHub)" <gi...@apache.org>.
bbovenzi commented on code in PR #29413:
URL: https://github.com/apache/airflow/pull/29413#discussion_r1104764231


##########
airflow/www/static/js/dag/details/index.tsx:
##########
@@ -17,23 +17,89 @@
  * under the License.
  */
 
-import React from 'react';
+import React, { useEffect, useState } from 'react';
 import {
   Flex,
-  Box,
   Divider,
+  Tabs,
+  TabList,
+  TabPanel,
+  TabPanels,
+  Tab,
+  Text,
 } from '@chakra-ui/react';
+import { useSearchParams } from 'react-router-dom';
 
 import useSelection from 'src/dag/useSelection';
+import URLSearchParamsWrapper from 'src/utils/URLSearchParamWrapper';
+import { getTask, getMetaValue } from 'src/utils';
+import { useGridData, useTaskInstance } from 'src/api';
 
 import Header from './Header';
 import TaskInstanceContent from './taskInstance';
 import DagRunContent from './dagRun';
 import DagContent from './Dag';
+import Graph from './graph';
+import MappedInstances from './taskInstance/MappedInstances';
+import Logs from './taskInstance/Logs';
+import BackToTaskSummary from './taskInstance/BackToTaskSummary';
 
-const Details = () => {
+const SHOW_GRAPH_PARAM = 'show_graph';
+const dagId = getMetaValue('dag_id')!;
+
+interface Props {
+  openGroupIds: string[];
+  onToggleGroups: (groupIds: string[]) => void;
+}
+
+const Details = ({ openGroupIds, onToggleGroups }: Props) => {
   const { selected: { runId, taskId, mapIndex }, onSelect } = useSelection();
 
+  const [searchParams, setSearchParams] = useSearchParams();
+  const showGraph = !!searchParams.get(SHOW_GRAPH_PARAM);
+  const [tabIndex, setTabIndex] = useState(showGraph ? 1 : 0);
+
+  // TODO: restore preferred tab index logic

Review Comment:
   I'll do this as a follow up with moving task actions to a modal so they can be accessed from all tabs



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] bbovenzi commented on a diff in pull request #29413: Add a new graph inside of the grid view

Posted by "bbovenzi (via GitHub)" <gi...@apache.org>.
bbovenzi commented on code in PR #29413:
URL: https://github.com/apache/airflow/pull/29413#discussion_r1114817390


##########
airflow/www/views.py:
##########
@@ -3591,6 +3591,42 @@ def extra_links(self, *, session: Session = NEW_SESSION):
         else:
             return {"url": None, "error": f"No URL found for {link_name}"}, 404
 
+    @expose("/object/graph_data")
+    @auth.has_access(
+        [
+            (permissions.ACTION_CAN_READ, permissions.RESOURCE_DAG),
+            (permissions.ACTION_CAN_READ, permissions.RESOURCE_TASK_INSTANCE),
+            (permissions.ACTION_CAN_READ, permissions.RESOURCE_TASK_LOG),
+        ]
+    )
+    @gzipped
+    @action_logging
+    @provide_session
+    def graph_data(self, session=None):
+        """Get Graph Data"""
+        dag_id = request.args.get("dag_id")
+        dag = get_airflow_app().dag_bag.get_dag(dag_id, session=session)
+        root = request.args.get("root")
+        if root:
+            filter_upstream = True if request.args.get("filter_upstream") == "true" else False
+            filter_downstream = True if request.args.get("filter_downstream") == "true" else False
+            dag = dag.partial_subset(
+                task_ids_or_regex=root, include_upstream=filter_upstream, include_downstream=filter_downstream
+            )
+
+        nodes = task_group_to_dict(dag.task_group)
+        edges = dag_edges(dag)

Review Comment:
   We already have tests that cover `task_group_to_dict()` and `dag_edges()`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] pierrejeambrun commented on a diff in pull request #29413: Add a new graph inside of the grid view

Posted by "pierrejeambrun (via GitHub)" <gi...@apache.org>.
pierrejeambrun commented on code in PR #29413:
URL: https://github.com/apache/airflow/pull/29413#discussion_r1122333211


##########
airflow/www/static/js/dag/index.tsx:
##########
@@ -30,13 +31,24 @@ import Main from "./Main";
 // create shadowRoot
 const root = document.querySelector("#root");
 const shadowRoot = root?.attachShadow({ mode: "open" });
+
 const cache = createCache({
   container: shadowRoot,
   key: "c",
 });
 const mainElement = document.getElementById("react-container");
 
 if (mainElement) {
+  const styleTag = document.createElement("style");
+  // We need to set the z-index manually to make sure edges inside of groups still work
+  let style = reactFlowStyle.toString();
+  style += `

Review Comment:
   To avoid this we could set a `background-color: transparent` on nodes that are taskgroup when expended/isOpen.
   
   I tried playing around with z-index trying to avoid the `!important` but it looks like the stacking contexts are a bit tricky there. (or I'm missing something)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] pierrejeambrun commented on a diff in pull request #29413: Add a new graph inside of the grid view

Posted by "pierrejeambrun (via GitHub)" <gi...@apache.org>.
pierrejeambrun commented on code in PR #29413:
URL: https://github.com/apache/airflow/pull/29413#discussion_r1122333211


##########
airflow/www/static/js/dag/index.tsx:
##########
@@ -30,13 +31,24 @@ import Main from "./Main";
 // create shadowRoot
 const root = document.querySelector("#root");
 const shadowRoot = root?.attachShadow({ mode: "open" });
+
 const cache = createCache({
   container: shadowRoot,
   key: "c",
 });
 const mainElement = document.getElementById("react-container");
 
 if (mainElement) {
+  const styleTag = document.createElement("style");
+  // We need to set the z-index manually to make sure edges inside of groups still work
+  let style = reactFlowStyle.toString();
+  style += `

Review Comment:
   I tried playing around with z-index trying to avoid this css trick but it looks like the stacking contexts are a bit tricky there. (or I'm missing something)
   
   To avoid this we could set a `background-color: transparent` on nodes that are taskgroup when expended/isOpen.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] blag commented on pull request #29413: Add a new graph inside of the grid view

Posted by "blag (via GitHub)" <gi...@apache.org>.
blag commented on PR #29413:
URL: https://github.com/apache/airflow/pull/29413#issuecomment-1430316952

   So looking forward to this. This looks great!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org