You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by xt...@apache.org on 2022/07/15 03:32:12 UTC

[flink] branch master updated: [FLINK-28316][runtime-web] add external JM and TM log links under history server

This is an automated email from the ASF dual-hosted git repository.

xtsong pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new 4e7594a62cc [FLINK-28316][runtime-web] add external JM and TM log links under history server
4e7594a62cc is described below

commit 4e7594a62cc4712fa88d9f8c884bc4e3be2b5705
Author: yangjunhan <ya...@sina.com>
AuthorDate: Thu Jul 14 10:12:25 2022 +0800

    [FLINK-28316][runtime-web] add external JM and TM log links under history server
    
    This closes #20270
---
 .../modules/completed-job/completed-job.module.ts  | 39 ++++++++++-
 ...pleted-job-subtasks-table-action.component.html | 37 ++++++++++
 ...leted-job-subtasks-table-action.component.less} | 23 -------
 ...ompleted-job-subtasks-table-action.component.ts | 69 +++++++++++++++++++
 ...ed-job-taskmanagers-table-action.component.html | 51 ++++++++++++++
 ...d-job-taskmanagers-table-action.component.less} | 23 -------
 ...eted-job-taskmanagers-table-action.component.ts | 77 +++++++++++++++++++++
 .../src/app/services/job-manager.service.ts        | 28 +++++---
 .../src/app/services/task-manager.service.ts       | 22 +++---
 .../share/common/status/job-status.component.html  | 80 +++++++++++++---------
 .../share/common/status/job-status.component.less  | 61 +++--------------
 .../share/common/status/job-status.component.ts    | 43 ++++++++----
 .../customize/task-badge/task-badge.component.ts   |  4 +-
 .../web-dashboard/src/app/share/share.module.ts    |  4 +-
 .../web-dashboard/src/styles/rewrite.less          |  7 ++
 15 files changed, 400 insertions(+), 168 deletions(-)

diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/completed-job.module.ts b/flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/completed-job.module.ts
index bbd4687e457..a59376c9dc9 100644
--- a/flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/completed-job.module.ts
+++ b/flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/completed-job.module.ts
@@ -21,17 +21,40 @@ import { NgModule } from '@angular/core';
 
 import { JOB_MODULE_CONFIG, JOB_MODULE_DEFAULT_CONFIG, JobModuleConfig } from '@flink-runtime-web/pages/job/job.config';
 import { CompletedJobRoutingModule } from '@flink-runtime-web/pages/job/modules/completed-job/completed-job-routing.module';
+import {
+  JOB_OVERVIEW_MODULE_CONFIG,
+  JOB_OVERVIEW_MODULE_DEFAULT_CONFIG,
+  JobOverviewModuleConfig
+} from '@flink-runtime-web/pages/job/overview/job-overview.config';
 import { StatusService } from '@flink-runtime-web/services';
 import { ShareModule } from '@flink-runtime-web/share/share.module';
 import { NzAlertModule } from 'ng-zorro-antd/alert';
 import { NzCardModule } from 'ng-zorro-antd/card';
+import { NzDropDownModule } from 'ng-zorro-antd/dropdown';
 import { NzEmptyModule } from 'ng-zorro-antd/empty';
 import { NzIconModule } from 'ng-zorro-antd/icon';
+import { NzModalModule } from 'ng-zorro-antd/modal';
 import { NzPipesModule } from 'ng-zorro-antd/pipes';
 import { NzSkeletonModule } from 'ng-zorro-antd/skeleton';
 import { NzTableModule } from 'ng-zorro-antd/table';
+import { NzTabsModule } from 'ng-zorro-antd/tabs';
 
 import { ClusterConfigComponent } from './cluster-config/cluster-config.component';
+import { CompletedJobSubtasksTableActionComponent } from './subtasks-table-action/completed-job-subtasks-table-action.component';
+import { CompletedJobTaskmanagersTableActionComponent } from './taskmanagers-table-action/completed-job-taskmanagers-table-action.component';
+
+const OVERRIDE_JOB_OVERVIEW_MODULE_CONFIG_FACTORY = (statusService: StatusService): JobOverviewModuleConfig => {
+  const isHistoryServer = statusService.configuration.features['web-history'];
+  return {
+    customComponents: isHistoryServer
+      ? {
+          ...JOB_OVERVIEW_MODULE_DEFAULT_CONFIG.customComponents,
+          subtaskActionComponent: CompletedJobSubtasksTableActionComponent,
+          taskManagerActionComponent: CompletedJobTaskmanagersTableActionComponent
+        }
+      : JOB_OVERVIEW_MODULE_DEFAULT_CONFIG.customComponents
+  };
+};
 
 const OVERRIDE_JOB_MODULE_CONFIG_FACTORY = (statusService: StatusService): JobModuleConfig => {
   const isHistoryServer = statusService.configuration.features['web-history'];
@@ -50,7 +73,11 @@ const OVERRIDE_JOB_MODULE_CONFIG_FACTORY = (statusService: StatusService): JobMo
 };
 
 @NgModule({
-  declarations: [ClusterConfigComponent],
+  declarations: [
+    ClusterConfigComponent,
+    CompletedJobSubtasksTableActionComponent,
+    CompletedJobTaskmanagersTableActionComponent
+  ],
   imports: [
     CommonModule,
     CompletedJobRoutingModule,
@@ -61,9 +88,17 @@ const OVERRIDE_JOB_MODULE_CONFIG_FACTORY = (statusService: StatusService): JobMo
     NzCardModule,
     NzTableModule,
     NzEmptyModule,
-    NzPipesModule
+    NzPipesModule,
+    NzDropDownModule,
+    NzModalModule,
+    NzTabsModule
   ],
   providers: [
+    {
+      provide: JOB_OVERVIEW_MODULE_CONFIG,
+      useFactory: OVERRIDE_JOB_OVERVIEW_MODULE_CONFIG_FACTORY,
+      deps: [StatusService]
+    },
     {
       provide: JOB_MODULE_CONFIG,
       useFactory: OVERRIDE_JOB_MODULE_CONFIG_FACTORY,
diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/subtasks-table-action/completed-job-subtasks-table-action.component.html b/flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/subtasks-table-action/completed-job-subtasks-table-action.component.html
new file mode 100644
index 00000000000..f73ab3e3fd2
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/subtasks-table-action/completed-job-subtasks-table-action.component.html
@@ -0,0 +1,37 @@
+<!--
+  ~ 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.
+  -->
+<span
+  *ngIf="
+    !subtask || !subtask['taskmanager-id'] || subtask['taskmanager-id'] === '(unassigned)';
+    else hrefTpl
+  "
+>
+  -
+</span>
+<ng-template #hrefTpl>
+  <a nz-dropdown nzPlacement="bottomRight" [nzDropdownMenu]="menu">
+    <i nz-icon nzType="ellipsis" nzTheme="outline"></i>
+  </a>
+  <nz-dropdown-menu #menu="nzDropdownMenu">
+    <ul nz-menu>
+      <li nz-menu-item [nzDisabled]="loading">
+        <a [href]="logUrl" target="_blank">View External Taskmanager Log</a>
+      </li>
+    </ul>
+  </nz-dropdown-menu>
+</ng-template>
diff --git a/flink-runtime-web/web-dashboard/src/app/share/customize/task-badge/task-badge.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/subtasks-table-action/completed-job-subtasks-table-action.component.less
similarity index 52%
copy from flink-runtime-web/web-dashboard/src/app/share/customize/task-badge/task-badge.component.ts
copy to flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/subtasks-table-action/completed-job-subtasks-table-action.component.less
index 9d5573421de..2379ddac12c 100644
--- a/flink-runtime-web/web-dashboard/src/app/share/customize/task-badge/task-badge.component.ts
+++ b/flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/subtasks-table-action/completed-job-subtasks-table-action.component.less
@@ -15,26 +15,3 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-import { ChangeDetectionStrategy, Component, Input } from '@angular/core';
-
-import { TaskStatus } from '@flink-runtime-web/interfaces';
-import { ConfigService } from '@flink-runtime-web/services';
-
-@Component({
-  selector: 'flink-task-badge',
-  templateUrl: './task-badge.component.html',
-  styleUrls: ['./task-badge.component.less'],
-  changeDetection: ChangeDetectionStrategy.OnPush
-})
-export class TaskBadgeComponent {
-  @Input() tasks: TaskStatus;
-  statusList = Object.keys(this.configService.COLOR_MAP);
-
-  constructor(private readonly configService: ConfigService) {}
-
-  // eslint-disable-next-line @typescript-eslint/explicit-function-return-type
-  get colorMap() {
-    return this.configService.COLOR_MAP;
-  }
-}
diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/subtasks-table-action/completed-job-subtasks-table-action.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/subtasks-table-action/completed-job-subtasks-table-action.component.ts
new file mode 100644
index 00000000000..c60997eaf4f
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/subtasks-table-action/completed-job-subtasks-table-action.component.ts
@@ -0,0 +1,69 @@
+/*
+ * 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 { Component, ChangeDetectionStrategy, ChangeDetectorRef, Input, OnInit, OnDestroy } from '@angular/core';
+import { Subject } from 'rxjs';
+import { filter, mergeMap, take, takeUntil } from 'rxjs/operators';
+
+import { JobVertexSubTask } from '@flink-runtime-web/interfaces';
+import { JobLocalService } from '@flink-runtime-web/pages/job/job-local.service';
+import { JobOverviewSubtasksTableAction } from '@flink-runtime-web/pages/job/overview/subtasks/table-action/subtasks-table-action.component';
+import { TaskManagerService } from '@flink-runtime-web/services';
+
+@Component({
+  selector: 'flink-completed-job-subtasks-table-action',
+  templateUrl: './completed-job-subtasks-table-action.component.html',
+  styleUrls: ['./completed-job-subtasks-table-action.component.less'],
+  changeDetection: ChangeDetectionStrategy.OnPush
+})
+export class CompletedJobSubtasksTableActionComponent implements OnInit, OnDestroy, JobOverviewSubtasksTableAction {
+  @Input() subtask?: JobVertexSubTask;
+  loading = true;
+  logUrl = '';
+
+  private destroy$ = new Subject<void>();
+
+  constructor(
+    private jobLocalService: JobLocalService,
+    private taskManagerService: TaskManagerService,
+    private cdr: ChangeDetectorRef
+  ) {}
+
+  ngOnInit(): void {
+    this.jobLocalService
+      .jobDetailChanges()
+      .pipe(
+        take(1),
+        filter(() => !!this.subtask?.['taskmanager-id'] && this.subtask['taskmanager-id'] !== '(unassigned)'),
+        mergeMap(job =>
+          this.taskManagerService.loadHistoryServerTaskManagerLogUrl(job.jid, this.subtask!['taskmanager-id'])
+        ),
+        takeUntil(this.destroy$)
+      )
+      .subscribe(url => {
+        this.loading = false;
+        this.logUrl = url;
+        this.cdr.markForCheck();
+      });
+  }
+
+  ngOnDestroy(): void {
+    this.destroy$.next();
+    this.destroy$.complete();
+  }
+}
diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/taskmanagers-table-action/completed-job-taskmanagers-table-action.component.html b/flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/taskmanagers-table-action/completed-job-taskmanagers-table-action.component.html
new file mode 100644
index 00000000000..1a2a7defa68
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/taskmanagers-table-action/completed-job-taskmanagers-table-action.component.html
@@ -0,0 +1,51 @@
+<!--
+  ~ 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.
+  -->
+
+<span *ngIf="!taskManager; else hrefTpl">-</span>
+<ng-template #hrefTpl>
+  <a nz-dropdown nzPlacement="bottomRight" [nzDropdownMenu]="menu">
+    <i nz-icon nzType="ellipsis" nzTheme="outline"></i>
+  </a>
+  <nz-dropdown-menu #menu="nzDropdownMenu">
+    <ul nz-menu>
+      <li nz-menu-item>
+        <a (click)="setModalVisible(true)">View Aggregated Metrics</a>
+        <nz-modal
+          [nzTitle]="taskManager['taskmanager-id']"
+          [(nzVisible)]="visible"
+          [nzFooter]="null"
+          [nzWidth]="800"
+          (nzOnCancel)="setModalVisible(false)"
+        >
+          <ng-container *nzModalContent>
+            <flink-table-aggregated-metrics
+              [aggregated]="taskManager.aggregated"
+            ></flink-table-aggregated-metrics>
+          </ng-container>
+        </nz-modal>
+      </li>
+      <li
+        nz-menu-item
+        [nzDisabled]="loading"
+        *ngIf="taskManager['taskmanager-id'] && taskManager['taskmanager-id'] !== '(unassigned)'"
+      >
+        <a [href]="logUrl" target="_blank">View External Taskmanager Log</a>
+      </li>
+    </ul>
+  </nz-dropdown-menu>
+</ng-template>
diff --git a/flink-runtime-web/web-dashboard/src/app/share/customize/task-badge/task-badge.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/taskmanagers-table-action/completed-job-taskmanagers-table-action.component.less
similarity index 52%
copy from flink-runtime-web/web-dashboard/src/app/share/customize/task-badge/task-badge.component.ts
copy to flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/taskmanagers-table-action/completed-job-taskmanagers-table-action.component.less
index 9d5573421de..2379ddac12c 100644
--- a/flink-runtime-web/web-dashboard/src/app/share/customize/task-badge/task-badge.component.ts
+++ b/flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/taskmanagers-table-action/completed-job-taskmanagers-table-action.component.less
@@ -15,26 +15,3 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-import { ChangeDetectionStrategy, Component, Input } from '@angular/core';
-
-import { TaskStatus } from '@flink-runtime-web/interfaces';
-import { ConfigService } from '@flink-runtime-web/services';
-
-@Component({
-  selector: 'flink-task-badge',
-  templateUrl: './task-badge.component.html',
-  styleUrls: ['./task-badge.component.less'],
-  changeDetection: ChangeDetectionStrategy.OnPush
-})
-export class TaskBadgeComponent {
-  @Input() tasks: TaskStatus;
-  statusList = Object.keys(this.configService.COLOR_MAP);
-
-  constructor(private readonly configService: ConfigService) {}
-
-  // eslint-disable-next-line @typescript-eslint/explicit-function-return-type
-  get colorMap() {
-    return this.configService.COLOR_MAP;
-  }
-}
diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/taskmanagers-table-action/completed-job-taskmanagers-table-action.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/taskmanagers-table-action/completed-job-taskmanagers-table-action.component.ts
new file mode 100644
index 00000000000..0de32939837
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/src/app/pages/job/modules/completed-job/taskmanagers-table-action/completed-job-taskmanagers-table-action.component.ts
@@ -0,0 +1,77 @@
+/*
+ * 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 { Component, ChangeDetectionStrategy, Input, ChangeDetectorRef, OnInit, OnDestroy } from '@angular/core';
+import { Subject } from 'rxjs';
+import { filter, mergeMap, take, takeUntil } from 'rxjs/operators';
+
+import { VertexTaskManagerDetail } from '@flink-runtime-web/interfaces';
+import { JobLocalService } from '@flink-runtime-web/pages/job/job-local.service';
+import { JobOverviewTaskManagersTableAction } from '@flink-runtime-web/pages/job/overview/taskmanagers/table-action/taskmanagers-table-action.component';
+import { TaskManagerService } from '@flink-runtime-web/services';
+
+@Component({
+  selector: 'flink-completed-job-taskmanagers-table-action',
+  templateUrl: './completed-job-taskmanagers-table-action.component.html',
+  styleUrls: ['./completed-job-taskmanagers-table-action.component.less'],
+  changeDetection: ChangeDetectionStrategy.OnPush
+})
+export class CompletedJobTaskmanagersTableActionComponent
+  implements OnInit, OnDestroy, JobOverviewTaskManagersTableAction
+{
+  @Input() taskManager?: VertexTaskManagerDetail;
+  visible = false;
+  loading = true;
+  logUrl = '';
+
+  private destroy$ = new Subject<void>();
+
+  constructor(
+    private jobLocalService: JobLocalService,
+    private taskManagerService: TaskManagerService,
+    private cdr: ChangeDetectorRef
+  ) {}
+
+  ngOnInit(): void {
+    this.jobLocalService
+      .jobDetailChanges()
+      .pipe(
+        take(1),
+        filter(() => !!this.taskManager?.['taskmanager-id'] && this.taskManager['taskmanager-id'] !== '(unassigned)'),
+        mergeMap(job =>
+          this.taskManagerService.loadHistoryServerTaskManagerLogUrl(job.jid, this.taskManager!['taskmanager-id'])
+        ),
+        takeUntil(this.destroy$)
+      )
+      .subscribe(url => {
+        this.loading = false;
+        this.logUrl = url;
+        this.cdr.markForCheck();
+      });
+  }
+
+  ngOnDestroy(): void {
+    this.destroy$.next();
+    this.destroy$.complete();
+  }
+
+  setModalVisible(visible: boolean): void {
+    this.visible = visible;
+    this.cdr.markForCheck();
+  }
+}
diff --git a/flink-runtime-web/web-dashboard/src/app/services/job-manager.service.ts b/flink-runtime-web/web-dashboard/src/app/services/job-manager.service.ts
index 842f746c45f..f99a48bd956 100644
--- a/flink-runtime-web/web-dashboard/src/app/services/job-manager.service.ts
+++ b/flink-runtime-web/web-dashboard/src/app/services/job-manager.service.ts
@@ -39,35 +39,35 @@ import { ConfigService } from './config.service';
 export class JobManagerService {
   constructor(private readonly httpClient: HttpClient, private readonly configService: ConfigService) {}
 
-  public loadConfig(): Observable<ClusterConfiguration[]> {
+  loadConfig(): Observable<ClusterConfiguration[]> {
     return this.httpClient.get<ClusterConfiguration[]>(`${this.configService.BASE_URL}/jobmanager/config`);
   }
 
-  public loadEnvironment(): Observable<EnvironmentInfo> {
+  loadEnvironment(): Observable<EnvironmentInfo> {
     return this.httpClient.get<EnvironmentInfo>(`${this.configService.BASE_URL}/jobmanager/environment`);
   }
 
-  public loadLogs(): Observable<string> {
+  loadLogs(): Observable<string> {
     return this.httpClient.get(`${this.configService.BASE_URL}/jobmanager/log`, {
       responseType: 'text',
       headers: new HttpHeaders().append('Cache-Control', 'no-cache')
     });
   }
 
-  public loadStdout(): Observable<string> {
+  loadStdout(): Observable<string> {
     return this.httpClient.get(`${this.configService.BASE_URL}/jobmanager/stdout`, {
       responseType: 'text',
       headers: new HttpHeaders().append('Cache-Control', 'no-cache')
     });
   }
 
-  public loadLogList(): Observable<JobManagerLogItem[]> {
+  loadLogList(): Observable<JobManagerLogItem[]> {
     return this.httpClient
       .get<{ logs: JobManagerLogItem[] }>(`${this.configService.BASE_URL}/jobmanager/logs`)
       .pipe(map(data => data.logs));
   }
 
-  public loadLog(logName: string): Observable<JobManagerLogDetail> {
+  loadLog(logName: string): Observable<JobManagerLogDetail> {
     const url = `${this.configService.BASE_URL}/jobmanager/logs/${logName}`;
     return this.httpClient
       .get(url, { responseType: 'text', headers: new HttpHeaders().append('Cache-Control', 'no-cache') })
@@ -81,7 +81,7 @@ export class JobManagerService {
       );
   }
 
-  public loadThreadDump(): Observable<string> {
+  loadThreadDump(): Observable<string> {
     return this.httpClient.get<JobManagerThreadDump>(`${this.configService.BASE_URL}/jobmanager/thread-dump`).pipe(
       map(JobManagerThreadDump => {
         return JobManagerThreadDump.threadInfos.map(threadInfo => threadInfo.stringifiedThreadInfo).join('');
@@ -89,13 +89,13 @@ export class JobManagerService {
     );
   }
 
-  public loadMetricsName(): Observable<string[]> {
+  loadMetricsName(): Observable<string[]> {
     return this.httpClient
       .get<Array<{ id: string }>>(`${this.configService.BASE_URL}/jobmanager/metrics`)
       .pipe(map(arr => arr.map(item => item.id)));
   }
 
-  public loadMetrics(listOfMetricName: string[]): Observable<MetricMap> {
+  loadMetrics(listOfMetricName: string[]): Observable<MetricMap> {
     const metricName = listOfMetricName.join(',');
     return this.httpClient.get<JobMetric[]>(`${this.configService.BASE_URL}/jobmanager/metrics?get=${metricName}`).pipe(
       map(arr => {
@@ -108,13 +108,19 @@ export class JobManagerService {
     );
   }
 
-  public loadHistoryServerConfig(jobId: string): Observable<ClusterConfiguration[]> {
+  loadHistoryServerConfig(jobId: string): Observable<ClusterConfiguration[]> {
     return this.httpClient.get<ClusterConfiguration[]>(
       `${this.configService.BASE_URL}/jobs/${jobId}/jobmanager/config`
     );
   }
 
-  public loadHistoryServerEnvironment(jobId: string): Observable<EnvironmentInfo> {
+  loadHistoryServerEnvironment(jobId: string): Observable<EnvironmentInfo> {
     return this.httpClient.get<EnvironmentInfo>(`${this.configService.BASE_URL}/jobs/${jobId}/jobmanager/environment`);
   }
+
+  loadHistoryServerJobManagerLogUrl(jobId: string): Observable<string> {
+    return this.httpClient
+      .get<{ url: string }>(`${this.configService.BASE_URL}/jobs/${jobId}/jobmanager/log-url`)
+      .pipe(map(data => data.url));
+  }
 }
diff --git a/flink-runtime-web/web-dashboard/src/app/services/task-manager.service.ts b/flink-runtime-web/web-dashboard/src/app/services/task-manager.service.ts
index 78a7ce44a15..e0b6fb0ca9e 100644
--- a/flink-runtime-web/web-dashboard/src/app/services/task-manager.service.ts
+++ b/flink-runtime-web/web-dashboard/src/app/services/task-manager.service.ts
@@ -40,26 +40,26 @@ import { ConfigService } from './config.service';
 export class TaskManagerService {
   constructor(private readonly httpClient: HttpClient, private readonly configService: ConfigService) {}
 
-  public loadManagers(): Observable<TaskManagersItem[]> {
+  loadManagers(): Observable<TaskManagersItem[]> {
     return this.httpClient.get<TaskManagerList>(`${this.configService.BASE_URL}/taskmanagers`).pipe(
       map(data => data.taskmanagers || []),
       catchError(() => of([]))
     );
   }
 
-  public loadManager(taskManagerId: string): Observable<TaskManagerDetail> {
+  loadManager(taskManagerId: string): Observable<TaskManagerDetail> {
     return this.httpClient
       .get<TaskManagerDetail>(`${this.configService.BASE_URL}/taskmanagers/${taskManagerId}`)
       .pipe(catchError(() => EMPTY));
   }
 
-  public loadLogList(taskManagerId: string): Observable<TaskManagerLogItem[]> {
+  loadLogList(taskManagerId: string): Observable<TaskManagerLogItem[]> {
     return this.httpClient
       .get<{ logs: TaskManagerLogItem[] }>(`${this.configService.BASE_URL}/taskmanagers/${taskManagerId}/logs`)
       .pipe(map(data => data.logs));
   }
 
-  public loadLog(taskManagerId: string, logName: string): Observable<TaskManagerLogDetail> {
+  loadLog(taskManagerId: string, logName: string): Observable<TaskManagerLogDetail> {
     const url = `${this.configService.BASE_URL}/taskmanagers/${taskManagerId}/logs/${logName}`;
     return this.httpClient
       .get(url, { responseType: 'text', headers: new HttpHeaders().append('Cache-Control', 'no-cache') })
@@ -73,7 +73,7 @@ export class TaskManagerService {
       );
   }
 
-  public loadThreadDump(taskManagerId: string): Observable<string> {
+  loadThreadDump(taskManagerId: string): Observable<string> {
     return this.httpClient
       .get<TaskManagerThreadDump>(`${this.configService.BASE_URL}/taskmanagers/${taskManagerId}/thread-dump`)
       .pipe(
@@ -83,21 +83,21 @@ export class TaskManagerService {
       );
   }
 
-  public loadLogs(taskManagerId: string): Observable<string> {
+  loadLogs(taskManagerId: string): Observable<string> {
     return this.httpClient.get(`${this.configService.BASE_URL}/taskmanagers/${taskManagerId}/log`, {
       responseType: 'text',
       headers: new HttpHeaders().append('Cache-Control', 'no-cache')
     });
   }
 
-  public loadStdout(taskManagerId: string): Observable<string> {
+  loadStdout(taskManagerId: string): Observable<string> {
     return this.httpClient.get(`${this.configService.BASE_URL}/taskmanagers/${taskManagerId}/stdout`, {
       responseType: 'text',
       headers: new HttpHeaders().append('Cache-Control', 'no-cache')
     });
   }
 
-  public loadMetrics(taskManagerId: string, listOfMetricName: string[]): Observable<MetricMap> {
+  loadMetrics(taskManagerId: string, listOfMetricName: string[]): Observable<MetricMap> {
     const metricName = listOfMetricName.join(',');
     return this.httpClient
       .get<JobMetric[]>(`${this.configService.BASE_URL}/taskmanagers/${taskManagerId}/metrics?get=${metricName}`)
@@ -111,4 +111,10 @@ export class TaskManagerService {
         })
       );
   }
+
+  loadHistoryServerTaskManagerLogUrl(jobId: string, taskManagerId: string): Observable<string> {
+    return this.httpClient
+      .get<{ url: string }>(`${this.configService.BASE_URL}/jobs/${jobId}/taskmanagers/${taskManagerId}/log-url`)
+      .pipe(map(data => data.url));
+  }
 }
diff --git a/flink-runtime-web/web-dashboard/src/app/share/common/status/job-status.component.html b/flink-runtime-web/web-dashboard/src/app/share/common/status/job-status.component.html
index 76fec6314d7..6dfcac81d3c 100644
--- a/flink-runtime-web/web-dashboard/src/app/share/common/status/job-status.component.html
+++ b/flink-runtime-web/web-dashboard/src/app/share/common/status/job-status.component.html
@@ -16,39 +16,8 @@
   ~ limitations under the License.
   -->
 
-<div *ngIf="jobDetail && !isLoading" class="wrapper">
-  <div class="name">
-    <h2>{{ jobDetail.name }}</h2>
-    <nz-divider [nzType]="'vertical'"></nz-divider>
-    <flink-job-badge [state]="jobDetail.state"></flink-job-badge>
-    <nz-divider [nzType]="'vertical'"></nz-divider>
-    <flink-task-badge [tasks]="jobDetail['status-counts']"></flink-task-badge>
-  </div>
-  <div class="status">
-    <div class="field">
-      <div>ID:</div>
-      <div>{{ jobDetail.jid }}</div>
-    </div>
-    <nz-divider [nzType]="'vertical'"></nz-divider>
-    <div class="field">
-      <div>Start Time:</div>
-      <div>{{ jobDetail['start-time'] | date: 'yyyy-MM-dd HH:mm:ss' }}</div>
-    </div>
-    <nz-divider [nzType]="'vertical'"></nz-divider>
-    <ng-container *ngIf="jobDetail['end-time'] > -1">
-      <div class="field">
-        <div>End Time:</div>
-        <div>{{ jobDetail['end-time'] | date: 'yyyy-MM-dd HH:mm:ss' }}</div>
-      </div>
-      <nz-divider [nzType]="'vertical'"></nz-divider>
-    </ng-container>
-    <div class="field">
-      <div>Duration:</div>
-      <div>{{ jobDetail.duration | humanizeDuration }}</div>
-    </div>
-  </div>
-  <flink-navigation [listOfNavigation]="listOfNavigation"></flink-navigation>
-  <div class="operate">
+<ng-template #extraTpl>
+  <div class="operate-action">
     <span *ngIf="statusTips">{{ statusTips }}</span>
     <ng-container *ngIf="!statusTips">
       <a
@@ -68,5 +37,48 @@
       </a>
     </ng-container>
   </div>
-</div>
+</ng-template>
+
+<ng-container *ngIf="jobDetail && !isLoading">
+  <nz-descriptions
+    [nzTitle]="jobDetail.name"
+    [nzExtra]="extraTpl"
+    nzBordered
+    nzSize="small"
+    [nzColumn]="{ xxl: 3, xl: 3, lg: 2, md: 2, sm: 1, xs: 1 }"
+  >
+    <nz-descriptions-item nzTitle="Job ID">{{ jobDetail.jid }}</nz-descriptions-item>
+    <nz-descriptions-item nzTitle="Job State">
+      <div class="status-wrapper">
+        <flink-job-badge [state]="jobDetail.state"></flink-job-badge>
+        <nz-divider nzType="vertical"></nz-divider>
+        <flink-task-badge [tasks]="jobDetail['status-counts']"></flink-task-badge>
+      </div>
+    </nz-descriptions-item>
+    <nz-descriptions-item nzTitle="Actions">
+      <a
+        *ngIf="!isHistoryServer; else historyServerActions"
+        [routerLink]="['/job-manager', 'logs']"
+        [queryParamsHandling]="'preserve'"
+      >
+        Job Manager Log
+      </a>
+      <ng-template #historyServerActions>
+        <a [href]="jmLogUrl" *ngIf="!urlLoading">Job Manager Log</a>
+        <span *ngIf="urlLoading">-</span>
+      </ng-template>
+    </nz-descriptions-item>
+    <nz-descriptions-item nzTitle="Start Time">
+      {{ jobDetail['start-time'] | date: 'yyyy-MM-dd HH:mm:ss' }}
+    </nz-descriptions-item>
+    <nz-descriptions-item nzTitle="End Time" *ngIf="jobDetail['end-time'] > -1">
+      {{ jobDetail['end-time'] | date: 'yyyy-MM-dd HH:mm:ss' }}
+    </nz-descriptions-item>
+    <nz-descriptions-item nzTitle="Duration">
+      {{ jobDetail.duration | humanizeDuration }}
+    </nz-descriptions-item>
+  </nz-descriptions>
+
+  <flink-navigation [listOfNavigation]="listOfNavigation"></flink-navigation>
+</ng-container>
 <nz-skeleton [nzActive]="true" *ngIf="isLoading"></nz-skeleton>
diff --git a/flink-runtime-web/web-dashboard/src/app/share/common/status/job-status.component.less b/flink-runtime-web/web-dashboard/src/app/share/common/status/job-status.component.less
index 545fdab95d8..776a1de5d9d 100644
--- a/flink-runtime-web/web-dashboard/src/app/share/common/status/job-status.component.less
+++ b/flink-runtime-web/web-dashboard/src/app/share/common/status/job-status.component.less
@@ -19,66 +19,23 @@
 @import "theme";
 
 :host {
-  display: block;
+  display: flex;
+  flex-direction: column;
   margin: -24px -24px 0;
   padding: 16px 32px 0;
   border-bottom: 1px solid @border-color-split;
   background: @component-background;
-}
 
-.wrapper {
-  position: relative;
-}
-
-.name {
-  h2,
-  flink-job-badge {
-    display: inline-block;
+  nz-descriptions {
+    margin-bottom: @margin-xs;
   }
 
-  nz-divider {
-    margin: 0 12px;
+  .operate-action {
+    font-size: 24px;
   }
-}
 
-.field {
-  display: flex;
-  font-size: 14px;
-  line-height: 22px;
-
-  div {
-    line-height: 22px;
-
-    &:last-child {
-      margin-left: 8px;
-      color: @heading-color;
-      font-weight: 700;
-    }
+  .status-wrapper {
+    display: flex;
+    align-items: center;
   }
 }
-
-.status {
-  display: flex;
-  align-items: center;
-  margin-bottom: 12px;
-
-  nz-divider {
-    margin: 0 24px;
-  }
-}
-
-.operate {
-  position: absolute;
-  top: 24px;
-  right: 16px;
-  font-size: 24px;
-}
-
-flink-navigation {
-  display: block;
-  margin-bottom: -1px;
-}
-
-flink-task-badge {
-  display: inline-block;
-}
diff --git a/flink-runtime-web/web-dashboard/src/app/share/common/status/job-status.component.ts b/flink-runtime-web/web-dashboard/src/app/share/common/status/job-status.component.ts
index 067803f0795..e4f66d84a47 100644
--- a/flink-runtime-web/web-dashboard/src/app/share/common/status/job-status.component.ts
+++ b/flink-runtime-web/web-dashboard/src/app/share/common/status/job-status.component.ts
@@ -18,13 +18,13 @@
 
 import { ChangeDetectionStrategy, ChangeDetectorRef, Component, Inject, Input, OnDestroy, OnInit } from '@angular/core';
 import { merge, Subject } from 'rxjs';
-import { distinctUntilKeyChanged, takeUntil, tap } from 'rxjs/operators';
+import { distinctUntilKeyChanged, mergeMap, take, takeUntil, tap } from 'rxjs/operators';
 
 import { RouterTab } from '@flink-runtime-web/core/module-config';
 import { JobDetailCorrect } from '@flink-runtime-web/interfaces';
 import { JobLocalService } from '@flink-runtime-web/pages/job/job-local.service';
 import { JOB_MODULE_CONFIG, JOB_MODULE_DEFAULT_CONFIG, JobModuleConfig } from '@flink-runtime-web/pages/job/job.config';
-import { JobService, StatusService } from '@flink-runtime-web/services';
+import { JobManagerService, JobService, StatusService } from '@flink-runtime-web/services';
 
 @Component({
   selector: 'flink-job-status',
@@ -33,20 +33,24 @@ import { JobService, StatusService } from '@flink-runtime-web/services';
   changeDetection: ChangeDetectionStrategy.OnPush
 })
 export class JobStatusComponent implements OnInit, OnDestroy {
-  @Input() public isLoading = true;
-  public statusTips: string;
-  public jobDetail: JobDetailCorrect;
-  public readonly listOfNavigation: RouterTab[];
+  @Input() isLoading = true;
+  statusTips: string;
+  jobDetail: JobDetailCorrect;
+  jmLogUrl = '';
+  urlLoading = true;
+  readonly listOfNavigation: RouterTab[];
   private readonly checkpointIndexOfNavigation: number;
 
-  public webCancelEnabled = this.statusService.configuration.features['web-cancel'];
+  webCancelEnabled = this.statusService.configuration.features['web-cancel'];
+  isHistoryServer = this.statusService.configuration.features['web-history'];
 
   private destroy$ = new Subject<void>();
 
   constructor(
     private readonly jobService: JobService,
     private readonly jobLocalService: JobLocalService,
-    public readonly statusService: StatusService,
+    private readonly jobManagerService: JobManagerService,
+    private readonly statusService: StatusService,
     private readonly cdr: ChangeDetectorRef,
     @Inject(JOB_MODULE_CONFIG) readonly moduleConfig: JobModuleConfig
   ) {
@@ -54,7 +58,22 @@ export class JobStatusComponent implements OnInit, OnDestroy {
     this.checkpointIndexOfNavigation = this.checkpointIndexOfNav();
   }
 
-  public ngOnInit(): void {
+  ngOnInit(): void {
+    if (this.isHistoryServer) {
+      this.jobLocalService
+        .jobDetailChanges()
+        .pipe(
+          take(1),
+          mergeMap(job => this.jobManagerService.loadHistoryServerJobManagerLogUrl(job.jid)),
+          takeUntil(this.destroy$)
+        )
+        .subscribe(url => {
+          this.urlLoading = false;
+          this.jmLogUrl = url;
+          this.cdr.markForCheck();
+        });
+    }
+
     const updateList$ = this.jobLocalService.jobDetailChanges().pipe(tap(data => this.handleJobDetailChanged(data)));
     const updateTip$ = this.jobLocalService.jobDetailChanges().pipe(
       distinctUntilKeyChanged('state'),
@@ -67,19 +86,19 @@ export class JobStatusComponent implements OnInit, OnDestroy {
     merge(updateList$, updateTip$).pipe(takeUntil(this.destroy$)).subscribe();
   }
 
-  public ngOnDestroy(): void {
+  ngOnDestroy(): void {
     this.destroy$.next();
     this.destroy$.complete();
   }
 
-  public cancelJob(): void {
+  cancelJob(): void {
     this.jobService.cancelJob(this.jobDetail.jid).subscribe(() => {
       this.statusTips = 'Cancelling...';
       this.cdr.markForCheck();
     });
   }
 
-  public checkpointIndexOfNav(): number {
+  checkpointIndexOfNav(): number {
     return this.listOfNavigation.findIndex(item => item.path === 'checkpoints');
   }
 
diff --git a/flink-runtime-web/web-dashboard/src/app/share/customize/task-badge/task-badge.component.ts b/flink-runtime-web/web-dashboard/src/app/share/customize/task-badge/task-badge.component.ts
index 9d5573421de..5ca21adb679 100644
--- a/flink-runtime-web/web-dashboard/src/app/share/customize/task-badge/task-badge.component.ts
+++ b/flink-runtime-web/web-dashboard/src/app/share/customize/task-badge/task-badge.component.ts
@@ -18,7 +18,7 @@
 
 import { ChangeDetectionStrategy, Component, Input } from '@angular/core';
 
-import { TaskStatus } from '@flink-runtime-web/interfaces';
+import { JobStatusCounts, TaskStatus } from '@flink-runtime-web/interfaces';
 import { ConfigService } from '@flink-runtime-web/services';
 
 @Component({
@@ -28,7 +28,7 @@ import { ConfigService } from '@flink-runtime-web/services';
   changeDetection: ChangeDetectionStrategy.OnPush
 })
 export class TaskBadgeComponent {
-  @Input() tasks: TaskStatus;
+  @Input() tasks: TaskStatus | JobStatusCounts;
   statusList = Object.keys(this.configService.COLOR_MAP);
 
   constructor(private readonly configService: ConfigService) {}
diff --git a/flink-runtime-web/web-dashboard/src/app/share/share.module.ts b/flink-runtime-web/web-dashboard/src/app/share/share.module.ts
index dfcf5f36954..25eeb51fab6 100644
--- a/flink-runtime-web/web-dashboard/src/app/share/share.module.ts
+++ b/flink-runtime-web/web-dashboard/src/app/share/share.module.ts
@@ -38,6 +38,7 @@ import { TaskBadgeComponent } from '@flink-runtime-web/share/customize/task-badg
 import { PipeModule } from '@flink-runtime-web/share/pipes/pipe.module';
 import { NzButtonModule } from 'ng-zorro-antd/button';
 import { NzCardModule } from 'ng-zorro-antd/card';
+import { NzDescriptionsModule } from 'ng-zorro-antd/descriptions';
 import { NzDividerModule } from 'ng-zorro-antd/divider';
 import { NzEmptyModule } from 'ng-zorro-antd/empty';
 import { NzIconModule } from 'ng-zorro-antd/icon';
@@ -72,7 +73,8 @@ import { FlameGraphComponent } from './customize/flame-graph/flame-graph.compone
     RouterModule,
     NzSkeletonModule,
     NzEmptyModule,
-    NzPopconfirmModule
+    NzPopconfirmModule,
+    NzDescriptionsModule
   ],
   declarations: [
     JobBadgeComponent,
diff --git a/flink-runtime-web/web-dashboard/src/styles/rewrite.less b/flink-runtime-web/web-dashboard/src/styles/rewrite.less
index f78136cd487..c7b49da7d0e 100644
--- a/flink-runtime-web/web-dashboard/src/styles/rewrite.less
+++ b/flink-runtime-web/web-dashboard/src/styles/rewrite.less
@@ -92,3 +92,10 @@ nz-table {
 .ant-table-thead > tr > th {
   word-break: break-word;
 }
+
+li.ant-dropdown-menu-item-disabled {
+  a {
+    color: @text-color-secondary;
+    cursor: not-allowed;
+  }
+}