[NIFI-12537] Open cluster/node dialog from Summary screen. (#8454)

* [NIFI-12537] - Summary screen cluster-related features

* common component status table

* cluster summary dialogs

* address review feedback

* ellisis for node columns in cluster dialogs, cluster node dropdown panel no longer wraps, shorted the dropdown width as well.

* prettier

This closes #8454
This commit is contained in:
Rob Fellows 2024-02-29 16:08:47 -05:00 committed by GitHub
parent c3c1b834c7
commit 455159f6ac
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
88 changed files with 4058 additions and 1012 deletions

View File

@ -19,8 +19,18 @@
<header class="nifi-header">
<navigation></navigation>
</header>
<div class="px-5 flex-1 flex flex-col">
<h3 class="text-xl bold summary-header">NiFi Summary</h3>
<div class="px-5 flex-1 flex flex-col gap-y-2">
<h3 class="text-xl bold summary-header">
@if (selectedClusterNode$ | async; as selectedNode) {
@if (selectedNode.id !== 'All') {
{{ selectedNode.address }} Summary
} @else {
NiFi Summary
}
} @else {
NiFi Summary
}
</h3>
<div class="summary-tabs">
<nav mat-tab-nav-bar color="primary" [tabPanel]="tabPanel">
@for (tab of tabLinks; track tab) {

View File

@ -20,6 +20,11 @@ import { Store } from '@ngrx/store';
import { NiFiState } from '../../../state';
import { startCurrentUserPolling, stopCurrentUserPolling } from '../../../state/current-user/current-user.actions';
import { loadSummaryListing, resetSummaryState } from '../state/summary-listing/summary-listing.actions';
import { loadClusterSummary, searchCluster } from '../../../state/cluster-summary/cluster-summary.actions';
import { selectClusterSummary } from '../../../state/cluster-summary/cluster-summary.selectors';
import { takeUntilDestroyed } from '@angular/core/rxjs-interop';
import { isDefinedAndNotNull } from '../../../state/shared';
import { selectSelectedClusterNode } from '../state/summary-listing/summary-listing.selectors';
interface TabLink {
label: string;
@ -41,11 +46,24 @@ export class Summary implements OnInit, OnDestroy {
{ label: 'Process Groups', link: 'process-groups' }
];
constructor(private store: Store<NiFiState>) {}
clusterSummary$ = this.store.select(selectClusterSummary);
selectedClusterNode$ = this.store.select(selectSelectedClusterNode).pipe(isDefinedAndNotNull());
constructor(private store: Store<NiFiState>) {
this.clusterSummary$.pipe(takeUntilDestroyed(), isDefinedAndNotNull()).subscribe((clusterSummary) => {
if (clusterSummary.connectedToCluster) {
this.store.dispatch(searchCluster({ request: {} }));
}
});
this.selectedClusterNode$.pipe(isDefinedAndNotNull(), takeUntilDestroyed()).subscribe(() => {
this.store.dispatch(loadSummaryListing({ recursive: true }));
});
}
ngOnInit(): void {
this.store.dispatch(startCurrentUserPolling());
this.store.dispatch(loadSummaryListing({ recursive: true }));
this.store.dispatch(loadClusterSummary());
}
ngOnDestroy(): void {

View File

@ -32,6 +32,7 @@ import { InputPortStatusListingModule } from '../ui/input-port-status-listing/in
import { SummaryListingEffects } from '../state/summary-listing/summary-listing.effects';
import { NgxSkeletonLoaderModule } from 'ngx-skeleton-loader';
import { Navigation } from '../../../ui/common/navigation/navigation.component';
import { ComponentClusterStatusEffects } from '../state/component-cluster-status/component-cluster-status.effects';
@NgModule({
declarations: [Summary],
@ -48,7 +49,7 @@ import { Navigation } from '../../../ui/common/navigation/navigation.component';
OutputPortStatusListingModule,
InputPortStatusListingModule,
StoreModule.forFeature(summaryFeatureKey, reducers),
EffectsModule.forFeature(SummaryListingEffects),
EffectsModule.forFeature(SummaryListingEffects, ComponentClusterStatusEffects),
NgxSkeletonLoaderModule,
Navigation
]

View File

@ -0,0 +1,64 @@
/*
* 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 { Injectable } from '@angular/core';
import { HttpClient, HttpParams } from '@angular/common/http';
import { Observable } from 'rxjs';
import { ComponentType } from '../../../state/shared';
@Injectable({ providedIn: 'root' })
export class ComponentClusterStatusService {
private static readonly API = '../nifi-api';
constructor(private httpClient: HttpClient) {}
getClusterStatus(componentId: string, componentType: ComponentType, nodewise: boolean = true): Observable<any> {
let componentPath: string;
switch (componentType) {
case ComponentType.Processor:
componentPath = 'processors';
break;
case ComponentType.ProcessGroup:
componentPath = 'process-groups';
break;
case ComponentType.Connection:
componentPath = 'connections';
break;
case ComponentType.RemoteProcessGroup:
componentPath = 'remote-process-groups';
break;
case ComponentType.InputPort:
componentPath = 'input-ports';
break;
case ComponentType.OutputPort:
componentPath = 'output-ports';
break;
default:
throw 'Unsupported component type';
}
let params: HttpParams = new HttpParams();
if (nodewise) {
params = params.set('nodewise', true);
}
return this.httpClient.get(
`${ComponentClusterStatusService.API}/flow/${componentPath}/${encodeURIComponent(componentId)}/status`,
{ params }
);
}
}

View File

@ -16,8 +16,9 @@
*/
import { Injectable } from '@angular/core';
import { HttpClient } from '@angular/common/http';
import { HttpClient, HttpParams } from '@angular/common/http';
import { Observable } from 'rxjs';
import { LoadSummaryRequest } from '../state/summary-listing';
@Injectable({ providedIn: 'root' })
export class ProcessGroupStatusService {
@ -25,13 +26,14 @@ export class ProcessGroupStatusService {
constructor(private httpClient: HttpClient) {}
getProcessGroupsStatus(recursive?: boolean): Observable<any> {
if (recursive) {
const params = {
recursive: true
};
return this.httpClient.get(`${ProcessGroupStatusService.API}/flow/process-groups/root/status`, { params });
getProcessGroupsStatus(request: LoadSummaryRequest): Observable<any> {
let params: HttpParams = new HttpParams();
if (request?.recursive) {
params = params.set('recursive', true);
}
return this.httpClient.get(`${ProcessGroupStatusService.API}/flow/process-groups/root/status`);
if (request?.clusterNodeId && request?.clusterNodeId !== 'All') {
params = params.set('clusterNodeId', request.clusterNodeId);
}
return this.httpClient.get(`${ProcessGroupStatusService.API}/flow/process-groups/root/status`, { params });
}
}

View File

@ -0,0 +1,43 @@
/*
* 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 { createAction, props } from '@ngrx/store';
import { ComponentClusterStatusRequest, ComponentClusterStatusResponse } from './index';
const CLUSTER_STATUS = '[Component Cluster Status]';
export const loadComponentClusterStatusAndOpenDialog = createAction(
`${CLUSTER_STATUS} Load Component Cluster Status and Open the Dialog`,
props<{ request: ComponentClusterStatusRequest }>()
);
export const loadComponentClusterStatus = createAction(
`${CLUSTER_STATUS} Load Component Cluster Status`,
props<{ request: ComponentClusterStatusRequest }>()
);
export const loadComponentClusterStatusSuccess = createAction(
`${CLUSTER_STATUS} Load Component Cluster Status Success`,
props<{ response: ComponentClusterStatusResponse }>()
);
export const resetComponentClusterStatusState = createAction(`${CLUSTER_STATUS} Reset Component Cluster Status State`);
export const openComponentClusterStatusDialog = createAction(
`${CLUSTER_STATUS} Open Component Cluster Status Dialog`,
props<{ response: ComponentClusterStatusResponse }>()
);

View File

@ -0,0 +1,114 @@
/*
* 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 { Injectable } from '@angular/core';
import { Actions, concatLatestFrom, createEffect, ofType } from '@ngrx/effects';
import { NiFiState } from '../../../../state';
import { Store } from '@ngrx/store';
import { ErrorHelper } from '../../../../service/error-helper.service';
import * as ClusterStatusActions from './component-cluster-status.actions';
import { catchError, from, map, of, switchMap, tap } from 'rxjs';
import { ComponentClusterStatusService } from '../../service/component-cluster-status.service';
import { MatDialog } from '@angular/material/dialog';
import { ClusterSummaryDialog } from '../../ui/common/cluster-summary-dialog/cluster-summary-dialog.component';
import { selectComponentClusterStatusLatestRequest } from './component-cluster-status.selectors';
import { isDefinedAndNotNull } from '../../../../state/shared';
import { HttpErrorResponse } from '@angular/common/http';
import * as ErrorActions from '../../../../state/error/error.actions';
@Injectable()
export class ComponentClusterStatusEffects {
constructor(
private actions$: Actions,
private store: Store<NiFiState>,
private errorHelper: ErrorHelper,
private clusterStatusService: ComponentClusterStatusService,
private dialog: MatDialog
) {}
loadComponentClusterStatusAndOpenDialog$ = createEffect(() =>
this.actions$.pipe(
ofType(ClusterStatusActions.loadComponentClusterStatusAndOpenDialog),
map((action) => action.request),
switchMap((request) => {
return from(this.clusterStatusService.getClusterStatus(request.id, request.componentType)).pipe(
map((response) => {
return ClusterStatusActions.openComponentClusterStatusDialog({
response: {
clusterStatusEntity: response,
componentType: request.componentType
}
});
}),
catchError((errorResponse: HttpErrorResponse) => {
if (this.errorHelper.showErrorInContext(errorResponse.status)) {
return of(ErrorActions.snackBarError({ error: errorResponse.error }));
}
return of(this.errorHelper.fullScreenError(errorResponse));
})
);
})
)
);
loadComponentClusterStatus$ = createEffect(() =>
this.actions$.pipe(
ofType(ClusterStatusActions.loadComponentClusterStatus),
map((action) => action.request),
switchMap((request) =>
from(this.clusterStatusService.getClusterStatus(request.id, request.componentType)).pipe(
map((response) => {
return ClusterStatusActions.loadComponentClusterStatusSuccess({
response: {
clusterStatusEntity: response,
componentType: request.componentType
}
});
}),
catchError((errorResponse: HttpErrorResponse) => {
if (this.errorHelper.showErrorInContext(errorResponse.status)) {
return of(ErrorActions.snackBarError({ error: errorResponse.error }));
}
return of(this.errorHelper.fullScreenError(errorResponse));
})
)
)
)
);
openComponentClusterStatusDialog$ = createEffect(
() =>
this.actions$.pipe(
ofType(ClusterStatusActions.openComponentClusterStatusDialog),
map((action) => action.response),
concatLatestFrom(() =>
this.store.select(selectComponentClusterStatusLatestRequest).pipe(isDefinedAndNotNull())
),
tap(([response, request]) => {
const dialogRef = this.dialog.open(ClusterSummaryDialog, {
panelClass: 'xl-dialog',
data: request
});
dialogRef.componentInstance.componentType = response.componentType;
dialogRef.afterClosed().subscribe(() => {
this.store.dispatch(ClusterStatusActions.resetComponentClusterStatusState());
});
})
),
{ dispatch: false }
);
}

View File

@ -0,0 +1,80 @@
/*
* 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 { createReducer, on } from '@ngrx/store';
import { ComponentClusterStatusState } from './index';
import * as ClusterStatusActions from './component-cluster-status.actions';
import { ComponentType } from '../../../../state/shared';
export const initialComponentClusterStatusState: ComponentClusterStatusState = {
clusterStatus: null,
status: 'pending',
loadedTimestamp: '',
latestRequest: null
};
export const componentClusterStatusReducer = createReducer(
initialComponentClusterStatusState,
on(
ClusterStatusActions.loadComponentClusterStatusAndOpenDialog,
ClusterStatusActions.loadComponentClusterStatus,
(state, { request }) => ({
...state,
status: 'loading' as const,
latestRequest: request
})
),
on(
ClusterStatusActions.loadComponentClusterStatusSuccess,
ClusterStatusActions.openComponentClusterStatusDialog,
(state, { response }) => {
let loadedTimestamp = '';
switch (response.componentType) {
case ComponentType.Processor:
loadedTimestamp = response.clusterStatusEntity.processorStatus?.statsLastRefreshed || '';
break;
case ComponentType.RemoteProcessGroup:
loadedTimestamp = response.clusterStatusEntity.remoteProcessGroupStatus?.statsLastRefreshed || '';
break;
case ComponentType.ProcessGroup:
loadedTimestamp = response.clusterStatusEntity.processGroupStatus?.statsLastRefreshed || '';
break;
case ComponentType.InputPort:
case ComponentType.OutputPort:
loadedTimestamp = response.clusterStatusEntity.portStatus?.statsLastRefreshed || '';
break;
case ComponentType.Connection:
loadedTimestamp = response.clusterStatusEntity.connectionStatus?.statsLastRefreshed || '';
break;
default:
loadedTimestamp = '';
}
return {
...state,
status: 'success' as const,
clusterStatus: response.clusterStatusEntity,
loadedTimestamp: loadedTimestamp
};
}
),
on(ClusterStatusActions.resetComponentClusterStatusState, () => ({
...initialComponentClusterStatusState
}))
);

View File

@ -0,0 +1,45 @@
/*
* 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 { createSelector } from '@ngrx/store';
import { selectSummaryState, SummaryState } from '../index';
import { componentClusterStatusFeatureKey, ComponentClusterStatusState } from './index';
export const selectComponentClusterStatusState = createSelector(
selectSummaryState,
(state: SummaryState) => state[componentClusterStatusFeatureKey]
);
export const selectComponentClusterStatusEntity = createSelector(
selectComponentClusterStatusState,
(state: ComponentClusterStatusState) => state.clusterStatus
);
export const selectComponentClusterStatusLatestRequest = createSelector(
selectComponentClusterStatusState,
(state: ComponentClusterStatusState) => state.latestRequest
);
export const selectComponentClusterStatusLoadingStatus = createSelector(
selectComponentClusterStatusState,
(state: ComponentClusterStatusState) => state.status
);
export const selectComponentClusterStatusLoadedTimestamp = createSelector(
selectComponentClusterStatusState,
(state: ComponentClusterStatusState) => state.loadedTimestamp
);

View File

@ -0,0 +1,47 @@
/*
* 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 { ConnectionStatus, PortStatus, ProcessGroupStatus, ProcessorStatus, RemoteProcessGroupStatus } from '../index';
import { ComponentType } from '../../../../state/shared';
export const componentClusterStatusFeatureKey = 'component-cluster-status';
export interface ComponentClusterStatusRequest {
id: string;
componentType: ComponentType;
}
export interface ComponentClusterStatusResponse {
clusterStatusEntity: ClusterStatusEntity;
componentType: ComponentType;
}
export interface ClusterStatusEntity {
canRead: boolean;
processorStatus?: ProcessorStatus;
portStatus?: PortStatus;
processGroupStatus?: ProcessGroupStatus;
remoteProcessGroupStatus?: RemoteProcessGroupStatus;
connectionStatus?: ConnectionStatus;
}
export interface ComponentClusterStatusState {
clusterStatus: ClusterStatusEntity | null;
latestRequest: ComponentClusterStatusRequest | null;
loadedTimestamp: string;
status: 'pending' | 'loading' | 'success';
}

View File

@ -18,17 +18,240 @@
import { Action, combineReducers, createFeatureSelector } from '@ngrx/store';
import { summaryListingFeatureKey, SummaryListingState } from './summary-listing';
import { summaryListingReducer } from './summary-listing/summary-listing.reducer';
import { componentClusterStatusFeatureKey, ComponentClusterStatusState } from './component-cluster-status';
import { componentClusterStatusReducer } from './component-cluster-status/component-cluster-status.reducer';
export const summaryFeatureKey = 'summary';
export interface SummaryState {
[summaryListingFeatureKey]: SummaryListingState;
[componentClusterStatusFeatureKey]: ComponentClusterStatusState;
}
export function reducers(state: SummaryState | undefined, action: Action) {
return combineReducers({
[summaryListingFeatureKey]: summaryListingReducer
[summaryListingFeatureKey]: summaryListingReducer,
[componentClusterStatusFeatureKey]: componentClusterStatusReducer
})(state, action);
}
export const selectSummaryState = createFeatureSelector<SummaryState>(summaryFeatureKey);
interface BaseSnapshot {
bytesIn: number;
bytesOut: number;
flowFilesIn: number;
flowFilesOut: number;
id: string;
input: string;
name: string;
output: string;
}
export interface BaseSnapshotEntity {
canRead: boolean;
id: string;
}
export interface ConnectionStatusSnapshot extends BaseSnapshot {
bytesQueued: number;
destinationName: string;
flowFileAvailability: string;
flowFilesQueued: number;
groupId: string;
percentUseCount: number;
percentUseBytes: number;
queued: string;
queuedCount: string;
queuedSize: string;
sourceName: string;
}
export interface RemoteProcessGroupStatusSnapshot {
activeThreadCount: number;
bytesReceived: number;
bytesSent: number;
flowFilesReceived: number;
flowFilesSent: number;
groupId: string;
id: string;
name: string;
received: string;
sent: string;
targetUri: string;
transmissionStatus: string;
}
export interface ConnectionStatusSnapshotEntity extends BaseSnapshotEntity {
connectionStatusSnapshot: ConnectionStatusSnapshot;
}
export interface RemoteProcessGroupStatusSnapshotEntity extends BaseSnapshotEntity {
remoteProcessGroupStatusSnapshot: RemoteProcessGroupStatusSnapshot;
}
export interface ProcessorStatusSnapshot extends BaseSnapshot {
activeThreadCount: number;
bytesRead: number;
bytesWritten: number;
executionNode: string;
groupId: string;
read: string;
runStatus: string;
taskCount: number;
tasks: string;
tasksDuration: string;
tasksDurationNanos: number;
terminatedThreadCount: number;
type: string;
written: string;
parentProcessGroupName: string;
processGroupNamePath: string;
}
export interface ProcessorStatusSnapshotEntity extends BaseSnapshotEntity {
processorStatusSnapshot: ProcessorStatusSnapshot;
}
export interface ProcessGroupStatusSnapshotEntity extends BaseSnapshotEntity {
processGroupStatusSnapshot: ProcessGroupStatusSnapshot;
}
export interface PortStatusSnapshotEntity extends BaseSnapshotEntity {
portStatusSnapshot: PortStatusSnapshot;
}
export enum VersionedFlowState {
SYNC_FAILURE = 'SYNC_FAILURE',
LOCALLY_MODIFIED = 'LOCALLY_MODIFIED',
STALE = 'STALE',
LOCALLY_MODIFIED_AND_STALE = 'LOCALLY_MODIFIED_AND_STALE',
UP_TO_DATE = 'UP_TO_DATE'
}
export interface ProcessGroupStatusSnapshot extends BaseSnapshot {
connectionStatusSnapshots: ConnectionStatusSnapshotEntity[];
processorStatusSnapshots: ProcessorStatusSnapshotEntity[];
processGroupStatusSnapshots: ProcessGroupStatusSnapshotEntity[];
remoteProcessGroupStatusSnapshots: any[];
inputPortStatusSnapshots: PortStatusSnapshotEntity[];
outputPortStatusSnapshots: PortStatusSnapshotEntity[];
bytesRead: number;
bytesReceived: number;
bytesSent: number;
bytesTransferred: number;
bytesWritten: number;
read: string;
received: string;
sent: string;
transferred: string;
written: string;
flowFilesReceived: number;
flowFilesTransferred: number;
flowFilesSent: number;
activeThreadCount: number;
processingNanos: number;
statelessActiveThreadCount: number;
terminatedThreadCount: number;
versionedFlowState?: VersionedFlowState;
}
export type AggregateSnapshot = ProcessGroupStatusSnapshot;
export interface PortStatusSnapshot extends BaseSnapshot {
runStatus: string;
groupId: string;
activeThreadCount: number;
}
export interface ProcessGroupStatusEntity {
canRead: boolean;
processGroupStatus: {
aggregateSnapshot: AggregateSnapshot;
id: string;
name: string;
statsLastRefreshed: string;
};
}
export interface NodeStatusSnapshot {
nodeId: string;
address: string;
apiPort: number;
}
export interface NodeProcessorStatusSnapshot extends NodeStatusSnapshot {
statusSnapshot: ProcessorStatusSnapshot;
}
export interface NodePortStatusSnapshot extends NodeStatusSnapshot {
statusSnapshot: PortStatusSnapshot;
}
export interface NodeProcessGroupStatusSnapshot extends NodeStatusSnapshot {
statusSnapshot: ProcessGroupStatusSnapshot;
}
export interface NodeRemoteProcessGroupStatusSnapshot extends NodeStatusSnapshot {
statusSnapshot: RemoteProcessGroupStatusSnapshot;
}
export interface NodeConnectionStatusSnapshot extends NodeStatusSnapshot {
statusSnapshot: ConnectionStatusSnapshot;
}
export interface ProcessorStatus {
groupId: string;
id: string;
name: string;
runStatus: string;
statsLastRefreshed: string;
aggregateSnapshot: ProcessorStatusSnapshot;
nodeSnapshots: NodeProcessorStatusSnapshot[];
}
export interface PortStatus {
groupId: string;
id: string;
name: string;
runStatus: string;
statsLastRefreshed: string;
aggregateSnapshot: PortStatusSnapshot;
nodeSnapshots: NodePortStatusSnapshot[];
}
export interface ProcessGroupStatus {
id: string;
name: string;
statsLastRefreshed: string;
aggregateSnapshot: ProcessGroupStatusSnapshot;
nodeSnapshots: NodeProcessGroupStatusSnapshot[];
}
export interface RemoteProcessGroupStatus {
groupId: string;
id: string;
name: string;
statsLastRefreshed: string;
targetUri: string;
transmissionStatus: string;
validationStatus: string;
aggregateSnapshot: RemoteProcessGroupStatusSnapshot;
nodeSnapshots: NodeRemoteProcessGroupStatusSnapshot[];
}
export interface ConnectionStatus {
groupId: string;
id: string;
name: string;
destinationId: string;
destinationName: string;
sourceId: string;
sourceName: string;
statsLastRefreshed: string;
aggregateSnapshot: ConnectionStatusSnapshot;
nodeSnapshots: NodeConnectionStatusSnapshot[];
}

View File

@ -15,157 +15,19 @@
* limitations under the License.
*/
import {
ConnectionStatusSnapshotEntity,
PortStatusSnapshotEntity,
ProcessGroupStatusEntity,
ProcessGroupStatusSnapshotEntity,
ProcessorStatusSnapshotEntity,
RemoteProcessGroupStatusSnapshotEntity
} from '../index';
import { NodeSearchResult } from '../../../../state/cluster-summary';
export const summaryListingFeatureKey = 'summary-listing';
export interface ClusterSummaryEntity {
clustered: boolean;
connectedNodeCount: number;
connectedToCluster: boolean;
totalNodeCount: number;
}
interface BaseSnapshot {
bytesIn: number;
bytesOut: number;
flowFilesIn: number;
flowFilesOut: number;
id: string;
input: string;
name: string;
output: string;
}
export interface BaseSnapshotEntity {
canRead: boolean;
id: string;
}
export interface ConnectionStatusSnapshot extends BaseSnapshot {
bytesQueued: number;
destinationName: string;
flowFileAvailability: string;
flowFilesQueued: number;
groupId: string;
percentUseCount: number;
percentUseBytes: number;
queued: string;
queuedCount: string;
queuedSize: string;
sourceName: string;
}
export interface RemoteProcessGroupStatusSnapshot {
activeThreadCount: number;
bytesReceived: number;
bytesSent: number;
flowFilesReceived: number;
flowFilesSent: number;
groupId: string;
id: string;
name: string;
received: string;
sent: string;
targetUri: string;
transmissionStatus: string;
}
export interface ConnectionStatusSnapshotEntity extends BaseSnapshotEntity {
connectionStatusSnapshot: ConnectionStatusSnapshot;
}
export interface RemoteProcessGroupStatusSnapshotEntity extends BaseSnapshotEntity {
remoteProcessGroupStatusSnapshot: RemoteProcessGroupStatusSnapshot;
}
export interface ProcessorStatusSnapshot extends BaseSnapshot {
activeThreadCount: number;
bytesRead: number;
bytesWritten: number;
executionNode: string;
groupId: string;
read: string;
runStatus: string;
taskCount: number;
tasks: string;
tasksDuration: string;
tasksDurationNanos: number;
terminatedThreadCount: number;
type: string;
written: string;
parentProcessGroupName: string;
processGroupNamePath: string;
}
export interface ProcessorStatusSnapshotEntity extends BaseSnapshotEntity {
processorStatusSnapshot: ProcessorStatusSnapshot;
}
export interface ProcessGroupStatusSnapshotEntity extends BaseSnapshotEntity {
processGroupStatusSnapshot: ProcessGroupStatusSnapshot;
}
export interface PortStatusSnapshotEntity extends BaseSnapshotEntity {
portStatusSnapshot: PortStatusSnapshot;
}
export enum VersionedFlowState {
SYNC_FAILURE = 'SYNC_FAILURE',
LOCALLY_MODIFIED = 'LOCALLY_MODIFIED',
STALE = 'STALE',
LOCALLY_MODIFIED_AND_STALE = 'LOCALLY_MODIFIED_AND_STALE',
UP_TO_DATE = 'UP_TO_DATE'
}
export interface ProcessGroupStatusSnapshot extends BaseSnapshot {
connectionStatusSnapshots: ConnectionStatusSnapshotEntity[];
processorStatusSnapshots: ProcessorStatusSnapshotEntity[];
processGroupStatusSnapshots: ProcessGroupStatusSnapshotEntity[];
remoteProcessGroupStatusSnapshots: any[];
inputPortStatusSnapshots: PortStatusSnapshotEntity[];
outputPortStatusSnapshots: PortStatusSnapshotEntity[];
bytesRead: number;
bytesReceived: number;
bytesSent: number;
bytesTransferred: number;
bytesWritten: number;
read: string;
received: string;
sent: string;
transferred: string;
written: string;
flowFilesReceived: number;
flowFilesTransferred: number;
flowFilesSent: number;
activeThreadCount: number;
processingNanos: number;
statelessActiveThreadCount: number;
terminatedThreadCount: number;
versionedFlowState?: VersionedFlowState;
}
export type AggregateSnapshot = ProcessGroupStatusSnapshot;
export interface PortStatusSnapshot extends BaseSnapshot {
runStatus: string;
groupId: string;
activeThreadCount: number;
}
export interface ProcessGroupStatusEntity {
canRead: boolean;
processGroupStatus: {
aggregateSnapshot: AggregateSnapshot;
id: string;
name: string;
statsLastRefreshed: string;
};
}
export interface SummaryListingResponse {
clusterSummary: ClusterSummaryEntity;
status: ProcessGroupStatusEntity;
}
@ -189,8 +51,12 @@ export interface SelectRemoteProcessGroupStatusRequest {
id: string;
}
export interface LoadSummaryRequest {
recursive?: boolean;
clusterNodeId?: string;
}
export interface SummaryListingState {
clusterSummary: ClusterSummaryEntity | null;
processGroupStatus: ProcessGroupStatusEntity | null;
processorStatusSnapshots: ProcessorStatusSnapshotEntity[];
processGroupStatusSnapshots: ProcessGroupStatusSnapshotEntity[];
@ -198,6 +64,7 @@ export interface SummaryListingState {
outputPortStatusSnapshots: PortStatusSnapshotEntity[];
connectionStatusSnapshots: ConnectionStatusSnapshotEntity[];
remoteProcessGroupStatusSnapshots: RemoteProcessGroupStatusSnapshotEntity[];
selectedClusterNode: NodeSearchResult | null;
loadedTimestamp: string;
status: 'pending' | 'loading' | 'success';
}

View File

@ -24,6 +24,7 @@ import {
SelectRemoteProcessGroupStatusRequest,
SummaryListingResponse
} from './index';
import { NodeSearchResult } from '../../../../state/cluster-summary';
const SUMMARY_LISTING_PREFIX = '[Summary Listing]';
@ -110,3 +111,8 @@ export const navigateToViewRemoteProcessGroupStatusHistory = createAction(
);
export const resetSummaryState = createAction(`${SUMMARY_LISTING_PREFIX} Reset Summary State`);
export const selectClusterNode = createAction(
`${SUMMARY_LISTING_PREFIX} Cluster Node Selected`,
props<{ clusterNode: NodeSearchResult }>()
);

View File

@ -19,24 +19,22 @@ import { Injectable } from '@angular/core';
import { Actions, concatLatestFrom, createEffect, ofType } from '@ngrx/effects';
import { Store } from '@ngrx/store';
import { NiFiState } from '../../../../state';
import { ClusterSummaryService } from '../../service/cluster-summary.service';
import { ProcessGroupStatusService } from '../../service/process-group-status.service';
import * as SummaryListingActions from './summary-listing.actions';
import * as StatusHistoryActions from '../../../../state/status-history/status-history.actions';
import { catchError, combineLatest, filter, map, of, switchMap, tap } from 'rxjs';
import { catchError, filter, from, map, of, switchMap, tap } from 'rxjs';
import { Router } from '@angular/router';
import { ComponentType } from '../../../../state/shared';
import { ErrorHelper } from '../../../../service/error-helper.service';
import { HttpErrorResponse } from '@angular/common/http';
import { selectSummaryListingStatus } from './summary-listing.selectors';
import { selectSelectedClusterNode, selectSummaryListingStatus } from './summary-listing.selectors';
import { LoadSummaryRequest } from './index';
@Injectable()
export class SummaryListingEffects {
constructor(
private actions$: Actions,
private store: Store<NiFiState>,
private clusterSummaryService: ClusterSummaryService,
private pgStatusService: ProcessGroupStatusService,
private errorHelper: ErrorHelper,
private router: Router
@ -46,16 +44,21 @@ export class SummaryListingEffects {
this.actions$.pipe(
ofType(SummaryListingActions.loadSummaryListing),
map((action) => action.recursive),
concatLatestFrom(() => this.store.select(selectSummaryListingStatus)),
switchMap(([recursive, listingStatus]) =>
combineLatest([
this.clusterSummaryService.getClusterSummary(),
this.pgStatusService.getProcessGroupsStatus(recursive)
]).pipe(
map(([clusterSummary, status]) =>
concatLatestFrom(() => [
this.store.select(selectSummaryListingStatus),
this.store.select(selectSelectedClusterNode)
]),
switchMap(([recursive, listingStatus, selectedClusterNode]) => {
const request: LoadSummaryRequest = {
recursive
};
if (selectedClusterNode) {
request.clusterNodeId = selectedClusterNode.id;
}
return from(this.pgStatusService.getProcessGroupsStatus(request)).pipe(
map((status) =>
SummaryListingActions.loadSummaryListingSuccess({
response: {
clusterSummary,
status
}
})
@ -63,8 +66,8 @@ export class SummaryListingEffects {
catchError((errorResponse: HttpErrorResponse) =>
of(this.errorHelper.handleLoadingError(listingStatus, errorResponse))
)
)
)
);
})
)
);

View File

@ -22,13 +22,17 @@ import {
ProcessGroupStatusSnapshot,
ProcessGroupStatusSnapshotEntity,
ProcessorStatusSnapshotEntity,
RemoteProcessGroupStatusSnapshotEntity,
SummaryListingState
} from './index';
import { loadSummaryListing, loadSummaryListingSuccess, resetSummaryState } from './summary-listing.actions';
RemoteProcessGroupStatusSnapshotEntity
} from '../index';
import {
loadSummaryListing,
loadSummaryListingSuccess,
resetSummaryState,
selectClusterNode
} from './summary-listing.actions';
import { SummaryListingState } from './index';
export const initialState: SummaryListingState = {
clusterSummary: null,
processGroupStatus: null,
processorStatusSnapshots: [],
processGroupStatusSnapshots: [],
@ -36,6 +40,7 @@ export const initialState: SummaryListingState = {
outputPortStatusSnapshots: [],
connectionStatusSnapshots: [],
remoteProcessGroupStatusSnapshots: [],
selectedClusterNode: null,
status: 'pending',
loadedTimestamp: ''
};
@ -82,7 +87,6 @@ export const summaryListingReducer = createReducer(
status: 'success' as const,
loadedTimestamp: response.status.processGroupStatus.statsLastRefreshed,
processGroupStatus: response.status,
clusterSummary: response.clusterSummary,
processorStatusSnapshots: processors,
processGroupStatusSnapshots: [root, ...childProcessGroups],
inputPortStatusSnapshots: inputPorts,
@ -94,6 +98,11 @@ export const summaryListingReducer = createReducer(
on(resetSummaryState, () => ({
...initialState
})),
on(selectClusterNode, (state, { clusterNode }) => ({
...state,
selectedClusterNode: clusterNode
}))
);

View File

@ -16,15 +16,15 @@
*/
import { createSelector } from '@ngrx/store';
import { selectSummaryState, SummaryState } from '../index';
import {
ConnectionStatusSnapshotEntity,
ProcessGroupStatusSnapshotEntity,
ProcessorStatusSnapshotEntity,
RemoteProcessGroupStatusSnapshotEntity,
summaryListingFeatureKey,
SummaryListingState
} from './index';
selectSummaryState,
SummaryState
} from '../index';
import { summaryListingFeatureKey, SummaryListingState } from './index';
import { selectCurrentRoute } from '../../../../state/router/router.selectors';
export const selectSummaryListing = createSelector(
@ -42,11 +42,6 @@ export const selectSummaryListingStatus = createSelector(
(state: SummaryListingState) => state.status
);
export const selectClusterSummary = createSelector(
selectSummaryListing,
(state: SummaryListingState) => state.clusterSummary
);
export const selectProcessGroupStatus = createSelector(
selectSummaryListing,
(state: SummaryListingState) => state.processGroupStatus
@ -149,3 +144,8 @@ export const selectRemoteProcessGroupStatusSnapshots = createSelector(
selectSummaryListing,
(state: SummaryListingState) => state.remoteProcessGroupStatusSnapshots
);
export const selectSelectedClusterNode = createSelector(
selectSummaryListing,
(state: SummaryListingState) => state.selectedClusterNode
);

View File

@ -0,0 +1,85 @@
<!--
~ 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.
-->
<div class="cluster-summary flex flex-col" tabindex="0">
<h2 mat-dialog-title>Cluster {{ componentType }} Summary</h2>
<mat-dialog-content class="cluster-summary-content flex flex-1 h-full grow">
<div class="dialog-content flex flex-1 flex-col grow">
<component-context
[type]="componentType"
[name]="componentHelper.getName()"
[id]="componentId"></component-context>
<div class="flex flex-1 flex-col cluster-summary-table-container">
@switch (componentType) {
@case (ComponentType.Processor) {
<processor-cluster-table
[components]="clusterStatusEntity?.processorStatus?.nodeSnapshots || []"
initialSortColumn="node"
initialSortDirection="asc"></processor-cluster-table>
}
@case (ComponentType.InputPort) {
<port-cluster-table
[components]="clusterStatusEntity?.portStatus?.nodeSnapshots || []"
portType="input"
initialSortColumn="node"
initialSortDirection="asc"></port-cluster-table>
}
@case (ComponentType.OutputPort) {
<port-cluster-table
[components]="clusterStatusEntity?.portStatus?.nodeSnapshots || []"
portType="output"
initialSortColumn="node"
initialSortDirection="asc"></port-cluster-table>
}
@case (ComponentType.RemoteProcessGroup) {
<remote-process-group-cluster-table
[components]="clusterStatusEntity?.remoteProcessGroupStatus?.nodeSnapshots || []"
initialSortColumn="node"
initialSortDirection="asc"></remote-process-group-cluster-table>
}
@case (ComponentType.Connection) {
<connection-cluster-table
[components]="clusterStatusEntity?.connectionStatus?.nodeSnapshots || []"
initialSortColumn="node"
initialSortDirection="asc"></connection-cluster-table>
}
@case (ComponentType.ProcessGroup) {
<process-group-cluster-table
[components]="clusterStatusEntity?.processGroupStatus?.nodeSnapshots || []"
initialSortColumn="node"
initialSortDirection="asc"></process-group-cluster-table>
}
}
</div>
</div>
</mat-dialog-content>
<mat-dialog-actions>
<div class="flex justify-between align-middle w-full">
<div class="refresh-container flex items-center gap-x-2">
<button class="nifi-button" (click)="refresh()">
<i class="fa fa-refresh" [class.fa-spin]="loading$ | async"></i>
</button>
<div>Last updated:</div>
<div class="refresh-timestamp">{{ loadedTimestamp$ | async }}</div>
</div>
<button color="primary" mat-raised-button mat-dialog-close tabindex="0">Close</button>
</div>
</mat-dialog-actions>
</div>

View File

@ -1,4 +1,4 @@
/*
/*!
* 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.
@ -15,17 +15,25 @@
* limitations under the License.
*/
import { Injectable } from '@angular/core';
import { HttpClient } from '@angular/common/http';
import { Observable } from 'rxjs';
@use '@angular/material' as mat;
@Injectable({ providedIn: 'root' })
export class ClusterSummaryService {
private static readonly API: string = '../nifi-api';
.cluster-summary {
@include mat.button-density(-1);
overflow-y: auto;
constructor(private httpClient: HttpClient) {}
.cluster-summary-content {
.mdc-dialog__content {
padding: 0 16px;
font-size: 14px;
getClusterSummary(): Observable<any> {
return this.httpClient.get(`${ClusterSummaryService.API}/flow/cluster/summary`);
.dialog-content {
min-height: 523px;
overflow-y: auto;
}
}
.cluster-summary-table-container {
min-height: 440px;
}
}
}

View File

@ -0,0 +1,62 @@
/*
* 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 { ComponentFixture, TestBed } from '@angular/core/testing';
import { ClusterSummaryDialog } from './cluster-summary-dialog.component';
import { MAT_DIALOG_DATA, MatDialogModule } from '@angular/material/dialog';
import { provideMockStore } from '@ngrx/store/testing';
import { initialComponentClusterStatusState } from '../../../state/component-cluster-status/component-cluster-status.reducer';
import { ComponentClusterStatusRequest, ComponentClusterStatusState } from '../../../state/component-cluster-status';
import { ComponentType } from '../../../../../state/shared';
import { NoopAnimationsModule } from '@angular/platform-browser/animations';
describe('ClusterSummaryDialog', () => {
let component: ClusterSummaryDialog;
let fixture: ComponentFixture<ClusterSummaryDialog>;
const data: ComponentClusterStatusRequest = {
id: 'id',
componentType: ComponentType.Processor
};
beforeEach(async () => {
await TestBed.configureTestingModule({
imports: [ClusterSummaryDialog, MatDialogModule, NoopAnimationsModule],
providers: [
{
provide: MAT_DIALOG_DATA,
useValue: data
},
provideMockStore({
initialState: {
...initialComponentClusterStatusState,
clusterStatus: {
canRead: true
}
} as ComponentClusterStatusState
})
]
}).compileComponents();
fixture = TestBed.createComponent(ClusterSummaryDialog);
component = fixture.componentInstance;
fixture.detectChanges();
});
it('should create', () => {
expect(component).toBeTruthy();
});
});

View File

@ -0,0 +1,140 @@
/*
* 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, Inject } from '@angular/core';
import {
MAT_DIALOG_DATA,
MatDialogActions,
MatDialogClose,
MatDialogContent,
MatDialogTitle
} from '@angular/material/dialog';
import { MatButton } from '@angular/material/button';
import { ComponentType, isDefinedAndNotNull } from '../../../../../state/shared';
import { ComponentContext } from '../../../../../ui/common/component-context/component-context.component';
import {
ClusterStatusEntity,
ComponentClusterStatusRequest,
ComponentClusterStatusState
} from '../../../state/component-cluster-status';
import { MatPaginator } from '@angular/material/paginator';
import { map, Observable } from 'rxjs';
import { AsyncPipe } from '@angular/common';
import {
selectComponentClusterStatusEntity,
selectComponentClusterStatusLoadedTimestamp,
selectComponentClusterStatusLoadingStatus
} from '../../../state/component-cluster-status/component-cluster-status.selectors';
import { Store } from '@ngrx/store';
import * as ClusterStatusActions from '../../../state/component-cluster-status/component-cluster-status.actions';
import { takeUntilDestroyed } from '@angular/core/rxjs-interop';
import { ProcessorClusterTable } from './processor-cluster-table/processor-cluster-table.component';
import { PortClusterTable } from './port-cluster-table/port-cluster-table.component';
import { RemoteProcessGroupClusterTable } from './remote-process-group-cluster-table/remote-process-group-cluster-table.component';
import { ConnectionClusterTable } from './connection-cluster-table/connection-cluster-table.component';
import { ProcessGroupClusterTable } from './process-group-cluster-table/process-group-cluster-table.component';
interface Helper {
getName: () => string;
}
@Component({
selector: 'cluster-summary-dialog',
standalone: true,
imports: [
MatDialogTitle,
MatDialogContent,
MatButton,
MatDialogActions,
MatDialogClose,
ComponentContext,
MatPaginator,
AsyncPipe,
ProcessorClusterTable,
PortClusterTable,
RemoteProcessGroupClusterTable,
ConnectionClusterTable,
ProcessGroupClusterTable
],
templateUrl: './cluster-summary-dialog.component.html',
styleUrl: './cluster-summary-dialog.component.scss'
})
export class ClusterSummaryDialog {
private _componentType: ComponentType = ComponentType.Processor;
loading$: Observable<boolean> = this.store
.select(selectComponentClusterStatusLoadingStatus)
.pipe(map((status) => status === 'loading'));
loadedTimestamp$: Observable<string> = this.store.select(selectComponentClusterStatusLoadedTimestamp);
clusterStatusEntity$: Observable<ClusterStatusEntity | null> = this.store.select(
selectComponentClusterStatusEntity
);
clusterStatusEntity: ClusterStatusEntity | null = null;
componentId!: string;
componentType!: ComponentType;
componentHelper: Helper = {
getName() {
return '';
}
};
constructor(
private store: Store<ComponentClusterStatusState>,
@Inject(MAT_DIALOG_DATA) private clusterStatusRequest: ComponentClusterStatusRequest
) {
this.componentId = clusterStatusRequest.id;
this.componentType = clusterStatusRequest.componentType;
this.clusterStatusEntity$.pipe(takeUntilDestroyed(), isDefinedAndNotNull()).subscribe((entity) => {
this.clusterStatusEntity = entity;
switch (this.componentType) {
case ComponentType.Processor:
this.componentHelper.getName = () => this.clusterStatusEntity?.processorStatus?.name || '';
break;
case ComponentType.RemoteProcessGroup:
this.componentHelper.getName = () => this.clusterStatusEntity?.remoteProcessGroupStatus?.name || '';
break;
case ComponentType.ProcessGroup:
this.componentHelper.getName = () => this.clusterStatusEntity?.processGroupStatus?.name || '';
break;
case ComponentType.InputPort:
case ComponentType.OutputPort:
this.componentHelper.getName = () => this.clusterStatusEntity?.portStatus?.name || '';
break;
case ComponentType.Connection:
this.componentHelper.getName = () => this.clusterStatusEntity?.connectionStatus?.name || '';
break;
default:
throw 'Unsupported Component Type';
}
});
}
refresh() {
this.store.dispatch(
ClusterStatusActions.loadComponentClusterStatus({
request: {
id: this.componentId,
componentType: this.componentType
}
})
);
}
protected readonly ComponentType = ComponentType;
}

View File

@ -0,0 +1,120 @@
/*
* 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, Input, OnChanges, SimpleChanges } from '@angular/core';
import { MultiSort } from '../../index';
import { MatSortModule, Sort, SortDirection } from '@angular/material/sort';
import { MatTableDataSource, MatTableModule } from '@angular/material/table';
import { NodeStatusSnapshot } from '../../../../state';
@Component({
standalone: true,
imports: [MatTableModule, MatSortModule],
template: ''
})
export abstract class ComponentClusterTable<T extends NodeStatusSnapshot> implements OnChanges {
private _initialSortColumn!: string;
private _initialSortDirection: SortDirection = 'asc';
selectedId: string | null = null;
multiSort: MultiSort = {
active: this._initialSortColumn,
direction: this._initialSortDirection,
sortValueIndex: 0,
totalValues: 2
};
dataSource: MatTableDataSource<T> = new MatTableDataSource<T>();
@Input() set initialSortColumn(initialSortColumn: string) {
this._initialSortColumn = initialSortColumn;
this.multiSort = { ...this.multiSort, active: initialSortColumn };
}
get initialSortColumn() {
return this._initialSortColumn;
}
@Input() set initialSortDirection(initialSortDirection: SortDirection) {
this._initialSortDirection = initialSortDirection;
this.multiSort = { ...this.multiSort, direction: initialSortDirection };
}
get initialSortDirection() {
return this._initialSortDirection;
}
abstract sortEntities(data: T[], sort: Sort): T[];
abstract supportsMultiValuedSort(sort: Sort): boolean;
@Input() components: T[] = [];
private setComponents(components: T[]) {
if (components) {
this.dataSource.data = this.sortEntities(components, this.multiSort);
}
}
ngOnChanges(changes: SimpleChanges) {
// Due to the generic nature of the component, handle the changes to the components in ngOnChanges
// rather than in a setter. This avoids IDE reporting @Input types cannot be converted to T[].
if (changes['components'].currentValue !== changes['components'].previousValue) {
this.setComponents(changes['components'].currentValue);
}
}
sortData(sort: Sort) {
this.setMultiSort(sort);
this.dataSource.data = this.sortEntities(this.dataSource.data, sort);
}
compare(a: number | string, b: number | string, isAsc: boolean) {
return (a < b ? -1 : a > b ? 1 : 0) * (isAsc ? 1 : -1);
}
select(item: T) {
this.selectedId = item.nodeId;
}
isSelected(item: T): boolean {
if (this.selectedId) {
return this.selectedId === item.nodeId;
}
return false;
}
private setMultiSort(sort: Sort) {
const { active, direction, sortValueIndex, totalValues } = this.multiSort;
if (this.supportsMultiValuedSort(sort)) {
if (active === sort.active) {
// previous sort was of the same column
if (direction === 'desc' && sort.direction === 'asc') {
// change from previous index to the next
const newIndex = sortValueIndex + 1 >= totalValues ? 0 : sortValueIndex + 1;
this.multiSort = { ...sort, sortValueIndex: newIndex, totalValues };
} else {
this.multiSort = { ...sort, sortValueIndex, totalValues };
}
} else {
// sorting a different column, just reset
this.multiSort = { ...sort, sortValueIndex: 0, totalValues };
}
} else {
this.multiSort = { ...sort, sortValueIndex: 0, totalValues };
}
}
}

View File

@ -0,0 +1,130 @@
<!--
~ 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.
-->
<div class="connection-cluster-table flex flex-1 h-full">
<div class="listing-table overflow-y-auto border flex-1">
<table
mat-table
[dataSource]="dataSource"
matSort
matSortDisableClear
(matSortChange)="sortData($event)"
[matSortActive]="initialSortColumn"
[matSortDirection]="initialSortDirection">
<!-- Node Column -->
<ng-container matColumnDef="node">
<th mat-header-cell *matHeaderCellDef mat-sort-header>
<div class="flex-1 overflow-ellipsis overflow-hidden whitespace-nowrap">Node</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatNode(item)">
<div class="overflow-ellipsis overflow-hidden whitespace-nowrap">{{ formatNode(item) }}</div>
</td>
</ng-container>
<!-- Queue column -->
<ng-container matColumnDef="queue">
<th mat-header-cell *matHeaderCellDef mat-sort-header title="Count / data size in the last 5 minutes">
<div class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1">
<span [ngClass]="{ underline: multiSort.active === 'queue' && multiSort.sortValueIndex === 0 }"
>Queue</span
>
<span [ngClass]="{ underline: multiSort.active === 'queue' && multiSort.sortValueIndex === 1 }"
>(Size)</span
>
<span class="font-light">5 min</span>
</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatQueue(item)">
{{ formatQueue(item) }}
</td>
</ng-container>
<!-- Threshold column -->
<ng-container matColumnDef="threshold">
<th
mat-header-cell
*matHeaderCellDef
mat-sort-header
title="Percent of threshold used for count and data size">
<div class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1">
<span>Threshold %:</span>
<span
[ngClass]="{
underline: multiSort.active === 'threshold' && multiSort.sortValueIndex === 0
}"
>Queue</span
>
<span>|</span>
<span
[ngClass]="{
underline: multiSort.active === 'threshold' && multiSort.sortValueIndex === 1
}"
>Size</span
>
</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatThreshold(item)">
{{ formatThreshold(item) }}
</td>
</ng-container>
<!-- Input column -->
<ng-container matColumnDef="in">
<th mat-header-cell *matHeaderCellDef mat-sort-header title="Count / data size in the last 5 minutes">
<div class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1">
<span [ngClass]="{ underline: multiSort.active === 'in' && multiSort.sortValueIndex === 0 }"
>In</span
>
<span [ngClass]="{ underline: multiSort.active === 'in' && multiSort.sortValueIndex === 1 }"
>(Size)</span
>
<span class="font-light">5 min</span>
</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatIn(item)">
{{ formatIn(item) }}
</td>
</ng-container>
<!-- Output column -->
<ng-container matColumnDef="out">
<th mat-header-cell *matHeaderCellDef mat-sort-header title="Count / data size in the last 5 minutes">
<div class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1">
<span [ngClass]="{ underline: multiSort.active === 'out' && multiSort.sortValueIndex === 0 }"
>Out</span
>
<span [ngClass]="{ underline: multiSort.active === 'out' && multiSort.sortValueIndex === 1 }"
>(Size)</span
>
<span class="font-light">5 min</span>
</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatOut(item)">
{{ formatOut(item) }}
</td>
</ng-container>
<tr mat-header-row *matHeaderRowDef="displayedColumns; sticky: true"></tr>
<tr
mat-row
*matRowDef="let row; let even = even; columns: displayedColumns"
[class.even]="even"
(click)="select(row)"
[class.selected]="isSelected(row)"></tr>
</table>
</div>
</div>

View File

@ -0,0 +1,25 @@
/*!
* 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.
*/
.connection-cluster-table {
.listing-table {
.mat-column-node {
min-width: 200px;
width: 30%;
}
}
}

View File

@ -0,0 +1,40 @@
/*
* 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 { ComponentFixture, TestBed } from '@angular/core/testing';
import { ConnectionClusterTable } from './connection-cluster-table.component';
import { NoopAnimationsModule } from '@angular/platform-browser/animations';
describe('ConnectionClusterTable', () => {
let component: ConnectionClusterTable;
let fixture: ComponentFixture<ConnectionClusterTable>;
beforeEach(async () => {
await TestBed.configureTestingModule({
imports: [ConnectionClusterTable, NoopAnimationsModule]
}).compileComponents();
fixture = TestBed.createComponent(ConnectionClusterTable);
component = fixture.componentInstance;
fixture.detectChanges();
});
it('should create', () => {
expect(component).toBeTruthy();
});
});

View File

@ -0,0 +1,109 @@
/*
* 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 } from '@angular/core';
import { ComponentClusterTable } from '../component-cluster-table/component-cluster-table.component';
import { NodeConnectionStatusSnapshot } from '../../../../state';
import { MatSortModule, Sort } from '@angular/material/sort';
import { MatTableModule } from '@angular/material/table';
import { NgClass } from '@angular/common';
@Component({
selector: 'connection-cluster-table',
standalone: true,
imports: [MatTableModule, MatSortModule, NgClass],
templateUrl: './connection-cluster-table.component.html',
styleUrl: './connection-cluster-table.component.scss'
})
export class ConnectionClusterTable extends ComponentClusterTable<NodeConnectionStatusSnapshot> {
displayedColumns: string[] = ['node', 'queue', 'threshold', 'in', 'out'];
constructor() {
super();
}
formatNode(processor: NodeConnectionStatusSnapshot): string {
return `${processor.address}:${processor.apiPort}`;
}
formatIn(connection: NodeConnectionStatusSnapshot): string {
return connection.statusSnapshot.input;
}
formatOut(connection: NodeConnectionStatusSnapshot): string {
return connection.statusSnapshot.output;
}
formatQueue(connection: NodeConnectionStatusSnapshot): string {
return connection.statusSnapshot.queued;
}
formatThreshold(connection: NodeConnectionStatusSnapshot): string {
return `${connection.statusSnapshot.percentUseCount}% | ${connection.statusSnapshot.percentUseBytes}%`;
}
override sortEntities(data: NodeConnectionStatusSnapshot[], sort: Sort): NodeConnectionStatusSnapshot[] {
if (!data) {
return [];
}
return data.slice().sort((a, b) => {
const isAsc: boolean = sort.direction === 'asc';
switch (sort.active) {
case 'node':
return this.compare(a.address, b.address, isAsc);
case 'in':
if (this.multiSort.sortValueIndex === 0) {
return this.compare(a.statusSnapshot.flowFilesIn, b.statusSnapshot.flowFilesIn, isAsc);
} else {
return this.compare(a.statusSnapshot.bytesIn, b.statusSnapshot.bytesIn, isAsc);
}
case 'out':
if (this.multiSort.sortValueIndex === 0) {
return this.compare(a.statusSnapshot.flowFilesOut, b.statusSnapshot.flowFilesOut, isAsc);
} else {
return this.compare(a.statusSnapshot.bytesOut, b.statusSnapshot.bytesOut, isAsc);
}
case 'queue':
if (this.multiSort.sortValueIndex === 0) {
return this.compare(a.statusSnapshot.flowFilesQueued, b.statusSnapshot.flowFilesQueued, isAsc);
} else {
return this.compare(a.statusSnapshot.bytesQueued, b.statusSnapshot.bytesQueued, isAsc);
}
case 'threshold':
if (this.multiSort.sortValueIndex === 0) {
return this.compare(a.statusSnapshot.percentUseCount, b.statusSnapshot.percentUseCount, isAsc);
} else {
return this.compare(a.statusSnapshot.percentUseBytes, b.statusSnapshot.percentUseBytes, isAsc);
}
default:
return 0;
}
});
}
override supportsMultiValuedSort(sort: Sort): boolean {
switch (sort.active) {
case 'in':
case 'out':
case 'threshold':
case 'queue':
return true;
default:
return false;
}
}
}

View File

@ -0,0 +1,107 @@
<!--
~ 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.
-->
<div class="port-cluster-table flex flex-1 h-full">
<div class="listing-table overflow-y-auto border flex-1">
<table
mat-table
[dataSource]="dataSource"
matSort
matSortDisableClear
(matSortChange)="sortData($event)"
[matSortActive]="initialSortColumn"
[matSortDirection]="initialSortDirection">
<!-- Node Column -->
<ng-container matColumnDef="node">
<th mat-header-cell *matHeaderCellDef mat-sort-header>
<div class="flex-1 overflow-ellipsis overflow-hidden whitespace-nowrap">Node</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatNode(item)">
<div class="overflow-ellipsis overflow-hidden whitespace-nowrap">{{ formatNode(item) }}</div>
</td>
</ng-container>
<!-- Run Status column -->
<ng-container matColumnDef="runStatus">
<th mat-header-cell *matHeaderCellDef mat-sort-header>Run Status</th>
<td mat-cell *matCellDef="let item">
<div
class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1.5 align-middle">
<span [ngClass]="getRunStatusIcon(item)"></span>
<span [title]="formatRunStatus(item)">{{ formatRunStatus(item) }}</span>
@if (item.statusSnapshot; as pg) {
@if (pg.terminatedThreadCount > 0) {
<span title="Threads: (Active / Terminated)"
>({{ pg.activeThreadCount }}/{{ pg.terminatedThreadCount }})</span
>
} @else {
@if (pg.activeThreadCount > 0) {
<span title="Active Threads">({{ pg.activeThreadCount }})</span>
}
}
}
</div>
</td>
</ng-container>
<!-- Input column -->
<ng-container matColumnDef="in">
<th mat-header-cell *matHeaderCellDef mat-sort-header title="Count / data size in the last 5 minutes">
<div class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1">
<span [ngClass]="{ underline: multiSort.active === 'in' && multiSort.sortValueIndex === 0 }"
>In</span
>
<span [ngClass]="{ underline: multiSort.active === 'in' && multiSort.sortValueIndex === 1 }"
>(Size)</span
>
<span class="font-light">5 min</span>
</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatIn(item)">
{{ formatIn(item) }}
</td>
</ng-container>
<!-- Output column -->
<ng-container matColumnDef="out">
<th mat-header-cell *matHeaderCellDef mat-sort-header title="Count / data size in the last 5 minutes">
<div class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1">
<span [ngClass]="{ underline: multiSort.active === 'out' && multiSort.sortValueIndex === 0 }"
>Out</span
>
<span [ngClass]="{ underline: multiSort.active === 'out' && multiSort.sortValueIndex === 1 }"
>(Size)</span
>
<span class="font-light">5 min</span>
</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatOut(item)">
{{ formatOut(item) }}
</td>
</ng-container>
<tr mat-header-row *matHeaderRowDef="displayedColumns; sticky: true"></tr>
<tr
mat-row
*matRowDef="let row; let even = even; columns: displayedColumns"
[class.even]="even"
(click)="select(row)"
[class.selected]="isSelected(row)"></tr>
</table>
</div>
</div>

View File

@ -0,0 +1,25 @@
/*!
* 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.
*/
.port-cluster-table {
.listing-table {
.mat-column-node {
min-width: 200px;
width: 50%;
}
}
}

View File

@ -0,0 +1,40 @@
/*
* 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 { ComponentFixture, TestBed } from '@angular/core/testing';
import { PortClusterTable } from './port-cluster-table.component';
import { NoopAnimationsModule } from '@angular/platform-browser/animations';
describe('PortClusterTable', () => {
let component: PortClusterTable;
let fixture: ComponentFixture<PortClusterTable>;
beforeEach(async () => {
await TestBed.configureTestingModule({
imports: [PortClusterTable, NoopAnimationsModule]
}).compileComponents();
fixture = TestBed.createComponent(PortClusterTable);
component = fixture.componentInstance;
fixture.detectChanges();
});
it('should create', () => {
expect(component).toBeTruthy();
});
});

View File

@ -0,0 +1,127 @@
/*
* 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, Input } from '@angular/core';
import { ComponentClusterTable } from '../component-cluster-table/component-cluster-table.component';
import { NodePortStatusSnapshot, NodeProcessorStatusSnapshot } from '../../../../state';
import { MatSortModule, Sort } from '@angular/material/sort';
import { MatTableModule } from '@angular/material/table';
import { NgClass } from '@angular/common';
@Component({
selector: 'port-cluster-table',
standalone: true,
imports: [MatSortModule, MatTableModule, NgClass],
templateUrl: './port-cluster-table.component.html',
styleUrl: './port-cluster-table.component.scss'
})
export class PortClusterTable extends ComponentClusterTable<NodePortStatusSnapshot> {
private _portType!: 'input' | 'output';
displayedColumns: string[] = [];
@Input() set portType(type: 'input' | 'output') {
if (type === 'input') {
this.displayedColumns = ['node', 'runStatus', 'out'];
} else {
this.displayedColumns = ['node', 'runStatus', 'in'];
}
this._portType = type;
}
get portType() {
return this._portType;
}
constructor() {
super();
}
sortEntities(data: NodePortStatusSnapshot[], sort: Sort): NodePortStatusSnapshot[] {
if (!data) {
return [];
}
return data.slice().sort((a, b) => {
const isAsc: boolean = sort.direction === 'asc';
switch (sort.active) {
case 'node':
return this.compare(a.address, b.address, isAsc);
case 'runStatus':
return this.compare(this.formatRunStatus(a), this.formatRunStatus(b), isAsc);
case 'in':
if (this.multiSort.sortValueIndex === 0) {
return this.compare(a.statusSnapshot.flowFilesIn, b.statusSnapshot.flowFilesIn, isAsc);
} else {
return this.compare(a.statusSnapshot.bytesIn, b.statusSnapshot.bytesIn, isAsc);
}
case 'out':
if (this.multiSort.sortValueIndex === 0) {
return this.compare(a.statusSnapshot.flowFilesOut, b.statusSnapshot.flowFilesOut, isAsc);
} else {
return this.compare(a.statusSnapshot.bytesOut, b.statusSnapshot.bytesOut, isAsc);
}
default:
return 0;
}
});
}
supportsMultiValuedSort(sort: Sort): boolean {
switch (sort.active) {
case 'in':
case 'out':
return true;
default:
return false;
}
}
formatNode(processor: NodeProcessorStatusSnapshot): string {
return `${processor.address}:${processor.apiPort}`;
}
formatRunStatus(port: NodePortStatusSnapshot): string {
return port.statusSnapshot.runStatus;
}
formatIn(port: NodePortStatusSnapshot): string {
return port.statusSnapshot.input;
}
formatOut(port: NodePortStatusSnapshot): string {
return port.statusSnapshot.output;
}
getRunStatusIcon(port: NodePortStatusSnapshot): string {
switch (port.statusSnapshot.runStatus.toLowerCase()) {
case 'running':
return 'fa fa-play running';
case 'stopped':
return 'fa fa-stop stopped';
case 'enabled':
return 'fa fa-flash enabled';
case 'disabled':
return 'icon icon-enable-false disabled';
case 'validating':
return 'fa fa-spin fa-circle-notch validating';
case 'invalid':
return 'fa fa-warning invalid';
default:
return '';
}
}
}

View File

@ -0,0 +1,172 @@
<!--
~ 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.
-->
<div class="process-group-cluster-table flex flex-1 h-full">
<div class="listing-table overflow-y-auto border flex-1">
<table
mat-table
[dataSource]="dataSource"
matSort
matSortDisableClear
(matSortChange)="sortData($event)"
[matSortActive]="initialSortColumn"
[matSortDirection]="initialSortDirection">
<!-- Node Column -->
<ng-container matColumnDef="node">
<th mat-header-cell *matHeaderCellDef mat-sort-header>
<div class="flex-1 overflow-ellipsis overflow-hidden whitespace-nowrap">Node</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatNode(item)">
<div class="overflow-ellipsis overflow-hidden whitespace-nowrap">{{ formatNode(item) }}</div>
</td>
</ng-container>
<!-- Transferred column -->
<ng-container matColumnDef="transferred">
<th mat-header-cell *matHeaderCellDef mat-sort-header>
<div
class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1"
title="Count / data size transferred to and from connections in the last 5 min">
<span
[ngClass]="{
underline: multiSort.active === 'transferred' && multiSort.sortValueIndex === 0
}"
>Transferred</span
>
<span
[ngClass]="{
underline: multiSort.active === 'transferred' && multiSort.sortValueIndex === 1
}"
>(Size)</span
>
<span class="font-light">5 min</span>
</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatTransferred(item)">
{{ formatTransferred(item) }}
</td>
</ng-container>
<!-- Input column -->
<ng-container matColumnDef="in">
<th mat-header-cell *matHeaderCellDef mat-sort-header title="Count / data size in the last 5 minutes">
<div class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1">
<span [ngClass]="{ underline: multiSort.active === 'in' && multiSort.sortValueIndex === 0 }"
>In</span
>
<span [ngClass]="{ underline: multiSort.active === 'in' && multiSort.sortValueIndex === 1 }"
>(Size)</span
>
<span class="font-light">5 min</span>
</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatIn(item)">
{{ formatIn(item) }}
</td>
</ng-container>
<!-- Read Write column -->
<ng-container matColumnDef="readWrite">
<th mat-header-cell *matHeaderCellDef mat-sort-header title="Data size in the last 5 minutes">
<div class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1">
<span
[ngClass]="{
underline: multiSort.active === 'readWrite' && multiSort.sortValueIndex === 0
}"
>Read</span
>
<span>|</span>
<span
[ngClass]="{
underline: multiSort.active === 'readWrite' && multiSort.sortValueIndex === 1
}"
>Write</span
>
<span class="font-light">5 min</span>
</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatReadWrite(item)">
{{ formatReadWrite(item) }}
</td>
</ng-container>
<!-- Output column -->
<ng-container matColumnDef="out">
<th mat-header-cell *matHeaderCellDef mat-sort-header title="Count / data size in the last 5 minutes">
<div class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1">
<span [ngClass]="{ underline: multiSort.active === 'out' && multiSort.sortValueIndex === 0 }"
>Out</span
>
<span [ngClass]="{ underline: multiSort.active === 'out' && multiSort.sortValueIndex === 1 }"
>(Size)</span
>
<span class="font-light">5 min</span>
</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatOut(item)">
{{ formatOut(item) }}
</td>
</ng-container>
<!-- Sent column -->
<ng-container matColumnDef="sent">
<th mat-header-cell *matHeaderCellDef mat-sort-header title="Count / data size in the last 5 minutes">
<div class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1">
<span [ngClass]="{ underline: multiSort.active === 'sent' && multiSort.sortValueIndex === 0 }"
>Sent</span
>
<span [ngClass]="{ underline: multiSort.active === 'sent' && multiSort.sortValueIndex === 1 }"
>(Size)</span
>
<span class="font-light">5 min</span>
</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatSent(item)">
{{ formatSent(item) }}
</td>
</ng-container>
<!-- Received column -->
<ng-container matColumnDef="received">
<th mat-header-cell *matHeaderCellDef mat-sort-header title="Count / data size in the last 5 minutes">
<div class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1">
<span
[ngClass]="{ underline: multiSort.active === 'received' && multiSort.sortValueIndex === 0 }"
>Received</span
>
<span
[ngClass]="{ underline: multiSort.active === 'received' && multiSort.sortValueIndex === 1 }"
>(Size)</span
>
<span class="font-light">5 min</span>
</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatReceived(item)">
{{ formatReceived(item) }}
</td>
</ng-container>
<tr mat-header-row *matHeaderRowDef="displayedColumns; sticky: true"></tr>
<tr
mat-row
*matRowDef="let row; let even = even; columns: displayedColumns"
[class.even]="even"
(click)="select(row)"
[class.selected]="isSelected(row)"></tr>
</table>
</div>
</div>

View File

@ -0,0 +1,25 @@
/*!
* 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.
*/
.process-group-cluster-table {
.listing-table {
.mat-column-node {
min-width: 200px;
width: 30%;
}
}
}

View File

@ -0,0 +1,40 @@
/*
* 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 { ComponentFixture, TestBed } from '@angular/core/testing';
import { ProcessGroupClusterTable } from './process-group-cluster-table.component';
import { NoopAnimationsModule } from '@angular/platform-browser/animations';
describe('ProcessGroupClusterTable', () => {
let component: ProcessGroupClusterTable;
let fixture: ComponentFixture<ProcessGroupClusterTable>;
beforeEach(async () => {
await TestBed.configureTestingModule({
imports: [ProcessGroupClusterTable, NoopAnimationsModule]
}).compileComponents();
fixture = TestBed.createComponent(ProcessGroupClusterTable);
component = fixture.componentInstance;
fixture.detectChanges();
});
it('should create', () => {
expect(component).toBeTruthy();
});
});

View File

@ -0,0 +1,144 @@
/*
* 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 } from '@angular/core';
import { ComponentClusterTable } from '../component-cluster-table/component-cluster-table.component';
import { NodeProcessGroupStatusSnapshot } from '../../../../state';
import { MatSortModule, Sort } from '@angular/material/sort';
import { MatTableModule } from '@angular/material/table';
import { NgClass } from '@angular/common';
@Component({
selector: 'process-group-cluster-table',
standalone: true,
imports: [MatTableModule, MatSortModule, NgClass],
templateUrl: './process-group-cluster-table.component.html',
styleUrl: './process-group-cluster-table.component.scss'
})
export class ProcessGroupClusterTable extends ComponentClusterTable<NodeProcessGroupStatusSnapshot> {
displayedColumns: string[] = ['node', 'transferred', 'in', 'readWrite', 'out', 'sent', 'received'];
constructor() {
super();
}
formatNode(processor: NodeProcessGroupStatusSnapshot): string {
return `${processor.address}:${processor.apiPort}`;
}
formatTransferred(pg: NodeProcessGroupStatusSnapshot): string {
return pg.statusSnapshot.transferred;
}
formatIn(pg: NodeProcessGroupStatusSnapshot): string {
return pg.statusSnapshot.input;
}
formatReadWrite(pg: NodeProcessGroupStatusSnapshot): string {
return `${pg.statusSnapshot.read} | ${pg.statusSnapshot.written}`;
}
formatOut(pg: NodeProcessGroupStatusSnapshot): string {
return pg.statusSnapshot.output;
}
formatSent(pg: NodeProcessGroupStatusSnapshot): string {
return pg.statusSnapshot.sent;
}
formatReceived(pg: NodeProcessGroupStatusSnapshot): string {
return pg.statusSnapshot.received;
}
override sortEntities(data: NodeProcessGroupStatusSnapshot[], sort: Sort): NodeProcessGroupStatusSnapshot[] {
if (!data) {
return [];
}
return data.slice().sort((a, b) => {
const isAsc = sort.direction === 'asc';
switch (sort.active) {
case 'node':
return this.compare(a.address, b.address, isAsc);
case 'transferred':
if (this.multiSort.sortValueIndex === 0) {
return this.compare(
a.statusSnapshot.flowFilesTransferred,
b.statusSnapshot.flowFilesTransferred,
isAsc
);
} else {
return this.compare(
a.statusSnapshot.bytesTransferred,
b.statusSnapshot.bytesTransferred,
isAsc
);
}
case 'in':
if (this.multiSort.sortValueIndex === 0) {
return this.compare(a.statusSnapshot.flowFilesIn, b.statusSnapshot.flowFilesIn, isAsc);
} else {
return this.compare(a.statusSnapshot.bytesIn, b.statusSnapshot.bytesIn, isAsc);
}
case 'out':
if (this.multiSort.sortValueIndex === 0) {
return this.compare(a.statusSnapshot.flowFilesOut, b.statusSnapshot.flowFilesOut, isAsc);
} else {
return this.compare(a.statusSnapshot.bytesOut, b.statusSnapshot.bytesOut, isAsc);
}
case 'readWrite':
if (this.multiSort.sortValueIndex === 0) {
return this.compare(a.statusSnapshot.bytesRead, b.statusSnapshot.bytesRead, isAsc);
} else {
return this.compare(a.statusSnapshot.bytesWritten, b.statusSnapshot.bytesWritten, isAsc);
}
case 'sent':
if (this.multiSort.sortValueIndex === 0) {
return this.compare(a.statusSnapshot.flowFilesSent, b.statusSnapshot.flowFilesSent, isAsc);
} else {
return this.compare(a.statusSnapshot.bytesSent, b.statusSnapshot.bytesSent, isAsc);
}
case 'received':
if (this.multiSort.sortValueIndex === 0) {
return this.compare(
a.statusSnapshot.flowFilesReceived,
b.statusSnapshot.flowFilesReceived,
isAsc
);
} else {
return this.compare(a.statusSnapshot.bytesReceived, b.statusSnapshot.bytesReceived, isAsc);
}
default:
return 0;
}
});
}
override supportsMultiValuedSort(sort: Sort): boolean {
switch (sort.active) {
case 'transferred':
case 'in':
case 'out':
case 'readWrite':
case 'received':
case 'sent':
return true;
default:
return false;
}
}
}

View File

@ -0,0 +1,151 @@
<!--
~ 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.
-->
<div class="processor-cluster-table flex flex-1 h-full">
<div class="listing-table overflow-y-auto border flex-1">
<table
mat-table
[dataSource]="dataSource"
matSort
matSortDisableClear
(matSortChange)="sortData($event)"
[matSortActive]="initialSortColumn"
[matSortDirection]="initialSortDirection">
<!-- Node Column -->
<ng-container matColumnDef="node">
<th mat-header-cell *matHeaderCellDef mat-sort-header>
<div class="flex-1 overflow-ellipsis overflow-hidden whitespace-nowrap">Node</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatNode(item)">
<div class="overflow-ellipsis overflow-hidden whitespace-nowrap">{{ formatNode(item) }}</div>
</td>
</ng-container>
<!-- Run Status column -->
<ng-container matColumnDef="runStatus">
<th mat-header-cell *matHeaderCellDef mat-sort-header>Run Status</th>
<td mat-cell *matCellDef="let item">
<div
class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1.5 align-middle">
<span [ngClass]="getRunStatusIcon(item)"></span>
<span [title]="formatRunStatus(item)">{{ formatRunStatus(item) }}</span>
@if (item.statusSnapshot; as pg) {
@if (pg.terminatedThreadCount > 0) {
<span title="Threads: (Active / Terminated)"
>({{ pg.activeThreadCount }}/{{ pg.terminatedThreadCount }})</span
>
} @else {
@if (pg.activeThreadCount > 0) {
<span title="Active Threads">({{ pg.activeThreadCount }})</span>
}
}
}
</div>
</td>
</ng-container>
<!-- Input column -->
<ng-container matColumnDef="in">
<th mat-header-cell *matHeaderCellDef mat-sort-header title="Count / data size in the last 5 minutes">
<div class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1">
<span [ngClass]="{ underline: multiSort.active === 'in' && multiSort.sortValueIndex === 0 }"
>Out</span
>
<span [ngClass]="{ underline: multiSort.active === 'in' && multiSort.sortValueIndex === 1 }"
>(Size)</span
>
<span class="font-light">5 min</span>
</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatIn(item)">
{{ formatIn(item) }}
</td>
</ng-container>
<!-- Read Write column -->
<ng-container matColumnDef="readWrite">
<th mat-header-cell *matHeaderCellDef mat-sort-header title="Data size in the last 5 minutes">
<div class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1">
<span
[ngClass]="{
underline: multiSort.active === 'readWrite' && multiSort.sortValueIndex === 0
}"
>Read</span
>
<span>|</span>
<span
[ngClass]="{
underline: multiSort.active === 'readWrite' && multiSort.sortValueIndex === 1
}"
>Write</span
>
<span class="font-light">5 min</span>
</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatReadWrite(item)">
{{ formatReadWrite(item) }}
</td>
</ng-container>
<!-- Output column -->
<ng-container matColumnDef="out">
<th mat-header-cell *matHeaderCellDef mat-sort-header title="Count / data size in the last 5 minutes">
<div class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1">
<span [ngClass]="{ underline: multiSort.active === 'out' && multiSort.sortValueIndex === 0 }"
>Out</span
>
<span [ngClass]="{ underline: multiSort.active === 'out' && multiSort.sortValueIndex === 1 }"
>(Size)</span
>
<span class="font-light">5 min</span>
</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatOut(item)">
{{ formatOut(item) }}
</td>
</ng-container>
<!-- Tasks column -->
<ng-container matColumnDef="tasks">
<th mat-header-cell *matHeaderCellDef mat-sort-header title="Count / duration in the last 5 minutes">
<div class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1">
<span [ngClass]="{ underline: multiSort.active === 'tasks' && multiSort.sortValueIndex === 0 }"
>Tasks</span
>
<span>|</span>
<span [ngClass]="{ underline: multiSort.active === 'tasks' && multiSort.sortValueIndex === 1 }"
>Time</span
>
<span class="font-light">5 min</span>
</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatTasks(item)">
{{ formatTasks(item) }}
</td>
</ng-container>
<tr mat-header-row *matHeaderRowDef="displayedColumns; sticky: true"></tr>
<tr
mat-row
*matRowDef="let row; let even = even; columns: displayedColumns"
[class.even]="even"
(click)="select(row)"
[class.selected]="isSelected(row)"></tr>
</table>
</div>
</div>

View File

@ -0,0 +1,25 @@
/*!
* 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.
*/
.processor-cluster-table {
.listing-table {
.mat-column-node {
min-width: 200px;
width: 30%;
}
}
}

View File

@ -0,0 +1,40 @@
/*
* 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 { ComponentFixture, TestBed } from '@angular/core/testing';
import { ProcessorClusterTable } from './processor-cluster-table.component';
import { NoopAnimationsModule } from '@angular/platform-browser/animations';
describe('ProcessorClusterTable', () => {
let component: ProcessorClusterTable;
let fixture: ComponentFixture<ProcessorClusterTable>;
beforeEach(async () => {
await TestBed.configureTestingModule({
imports: [ProcessorClusterTable, NoopAnimationsModule]
}).compileComponents();
fixture = TestBed.createComponent(ProcessorClusterTable);
component = fixture.componentInstance;
fixture.detectChanges();
});
it('should create', () => {
expect(component).toBeTruthy();
});
});

View File

@ -0,0 +1,138 @@
/*
* 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 } from '@angular/core';
import { NodeProcessorStatusSnapshot } from '../../../../state';
import { MatSortModule, Sort } from '@angular/material/sort';
import { ComponentClusterTable } from '../component-cluster-table/component-cluster-table.component';
import { MatTableModule } from '@angular/material/table';
import { NgClass } from '@angular/common';
@Component({
selector: 'processor-cluster-table',
standalone: true,
imports: [MatTableModule, MatSortModule, NgClass],
templateUrl: './processor-cluster-table.component.html',
styleUrl: './processor-cluster-table.component.scss'
})
export class ProcessorClusterTable extends ComponentClusterTable<NodeProcessorStatusSnapshot> {
displayedColumns: string[] = ['node', 'runStatus', 'in', 'readWrite', 'out', 'tasks'];
constructor() {
super();
}
override supportsMultiValuedSort(sort: Sort): boolean {
switch (sort.active) {
case 'in':
case 'out':
case 'readWrite':
case 'tasks':
return true;
default:
return false;
}
}
override sortEntities(data: NodeProcessorStatusSnapshot[], sort: Sort): NodeProcessorStatusSnapshot[] {
if (!data) {
return [];
}
return data.slice().sort((a, b) => {
const isAsc = sort.direction === 'asc';
switch (sort.active) {
case 'node':
return this.compare(a.address, b.address, isAsc);
case 'runStatus':
return this.compare(this.formatRunStatus(a), this.formatRunStatus(b), isAsc);
case 'in':
if (this.multiSort.sortValueIndex === 0) {
return this.compare(a.statusSnapshot.flowFilesIn, b.statusSnapshot.flowFilesIn, isAsc);
} else {
return this.compare(a.statusSnapshot.bytesIn, b.statusSnapshot.bytesIn, isAsc);
}
case 'out':
if (this.multiSort.sortValueIndex === 0) {
return this.compare(a.statusSnapshot.flowFilesOut, b.statusSnapshot.flowFilesOut, isAsc);
} else {
return this.compare(a.statusSnapshot.bytesOut, b.statusSnapshot.bytesOut, isAsc);
}
case 'readWrite':
if (this.multiSort.sortValueIndex === 0) {
return this.compare(a.statusSnapshot.bytesRead, b.statusSnapshot.bytesRead, isAsc);
} else {
return this.compare(a.statusSnapshot.bytesWritten, b.statusSnapshot.bytesWritten, isAsc);
}
case 'tasks':
if (this.multiSort.sortValueIndex === 0) {
return this.compare(a.statusSnapshot.taskCount, b.statusSnapshot.taskCount, isAsc);
} else {
return this.compare(
a.statusSnapshot.tasksDurationNanos,
b.statusSnapshot.tasksDurationNanos,
isAsc
);
}
default:
return 0;
}
});
}
formatNode(processor: NodeProcessorStatusSnapshot): string {
return `${processor.address}:${processor.apiPort}`;
}
formatRunStatus(processor: NodeProcessorStatusSnapshot): string {
return processor.statusSnapshot.runStatus;
}
formatIn(processor: NodeProcessorStatusSnapshot): string {
return processor.statusSnapshot.input;
}
formatOut(processor: NodeProcessorStatusSnapshot): string {
return processor.statusSnapshot.output;
}
formatReadWrite(processor: NodeProcessorStatusSnapshot): string {
return `${processor.statusSnapshot.read} | ${processor.statusSnapshot.written}`;
}
formatTasks(processor: NodeProcessorStatusSnapshot): string {
return `${processor.statusSnapshot.tasks} | ${processor.statusSnapshot.tasksDuration}`;
}
getRunStatusIcon(processor: NodeProcessorStatusSnapshot): string {
switch (processor.statusSnapshot.runStatus.toLowerCase()) {
case 'running':
return 'fa fa-play running';
case 'stopped':
return 'fa fa-stop stopped';
case 'enabled':
return 'fa fa-flash enabled';
case 'disabled':
return 'icon icon-enable-false disabled';
case 'validating':
return 'fa fa-spin fa-circle-notch validating';
case 'invalid':
return 'fa fa-warning invalid';
default:
return '';
}
}
}

View File

@ -0,0 +1,110 @@
<!--
~ 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.
-->
<div class="remote-process-group-cluster-table flex flex-1 h-full">
<div class="listing-table overflow-y-auto border flex-1">
<table
mat-table
[dataSource]="dataSource"
matSort
matSortDisableClear
(matSortChange)="sortData($event)"
[matSortActive]="initialSortColumn"
[matSortDirection]="initialSortDirection">
<!-- Node Column -->
<ng-container matColumnDef="node">
<th mat-header-cell *matHeaderCellDef mat-sort-header>
<div class="flex-1 overflow-ellipsis overflow-hidden whitespace-nowrap">Node</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatNode(item)">
<div class="overflow-ellipsis overflow-hidden whitespace-nowrap">{{ formatNode(item) }}</div>
</td>
</ng-container>
<!-- Target URI Column -->
<ng-container matColumnDef="uri">
<th mat-header-cell *matHeaderCellDef mat-sort-header>
<div class="flex-1 overflow-ellipsis overflow-hidden whitespace-nowrap">Target URI</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatUri(item)">
{{ formatUri(item) }}
</td>
</ng-container>
<!-- Transmission Status column -->
<ng-container matColumnDef="transmitting">
<th mat-header-cell *matHeaderCellDef mat-sort-header>Transmitting</th>
<td mat-cell *matCellDef="let item">
<div
class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1.5 align-middle">
<span [ngClass]="getTransmissionStatusIcon(item)"></span>
<span [title]="formatTransmitting(item)">{{ formatTransmitting(item) }}</span>
@if (item.activeThreadCount > 0) {
<span title="Active Threads">({{ item.activeThreadCount }})</span>
}
</div>
</td>
</ng-container>
<!-- Sent column -->
<ng-container matColumnDef="sent">
<th mat-header-cell *matHeaderCellDef mat-sort-header title="Count / data size in the last 5 minutes">
<div class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1">
<span [ngClass]="{ underline: multiSort.active === 'sent' && multiSort.sortValueIndex === 0 }"
>Sent</span
>
<span [ngClass]="{ underline: multiSort.active === 'sent' && multiSort.sortValueIndex === 1 }"
>(Size)</span
>
<span class="font-light">5 min</span>
</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatSent(item)">
{{ formatSent(item) }}
</td>
</ng-container>
<!-- Received column -->
<ng-container matColumnDef="received">
<th mat-header-cell *matHeaderCellDef mat-sort-header title="Count / data size in the last 5 minutes">
<div class="inline-block overflow-hidden overflow-ellipsis whitespace-nowrap space-x-1">
<span
[ngClass]="{ underline: multiSort.active === 'received' && multiSort.sortValueIndex === 0 }"
>Received</span
>
<span
[ngClass]="{ underline: multiSort.active === 'received' && multiSort.sortValueIndex === 1 }"
>(Size)</span
>
<span class="font-light">5 min</span>
</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatReceived(item)">
{{ formatReceived(item) }}
</td>
</ng-container>
<tr mat-header-row *matHeaderRowDef="displayedColumns; sticky: true"></tr>
<tr
mat-row
*matRowDef="let row; let even = even; columns: displayedColumns"
[class.even]="even"
(click)="select(row)"
[class.selected]="isSelected(row)"></tr>
</table>
</div>
</div>

View File

@ -0,0 +1,25 @@
/*!
* 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.
*/
.remote-process-group-cluster-table {
.listing-table {
.mat-column-node {
min-width: 200px;
width: 30%;
}
}
}

View File

@ -0,0 +1,40 @@
/*
* 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 { ComponentFixture, TestBed } from '@angular/core/testing';
import { NoopAnimationsModule } from '@angular/platform-browser/animations';
import { RemoteProcessGroupClusterTable } from './remote-process-group-cluster-table.component';
describe('RemoteProcessGroupClusterTable', () => {
let component: RemoteProcessGroupClusterTable;
let fixture: ComponentFixture<RemoteProcessGroupClusterTable>;
beforeEach(async () => {
await TestBed.configureTestingModule({
imports: [RemoteProcessGroupClusterTable, NoopAnimationsModule]
}).compileComponents();
fixture = TestBed.createComponent(RemoteProcessGroupClusterTable);
component = fixture.componentInstance;
fixture.detectChanges();
});
it('should create', () => {
expect(component).toBeTruthy();
});
});

View File

@ -0,0 +1,120 @@
/*
* 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 } from '@angular/core';
import { NodeRemoteProcessGroupStatusSnapshot } from '../../../../state';
import { MatSortModule, Sort } from '@angular/material/sort';
import { ComponentClusterTable } from '../component-cluster-table/component-cluster-table.component';
import { MatTableModule } from '@angular/material/table';
import { NgClass } from '@angular/common';
@Component({
selector: 'remote-process-group-cluster-table',
standalone: true,
imports: [MatTableModule, MatSortModule, NgClass],
templateUrl: './remote-process-group-cluster-table.component.html',
styleUrl: './remote-process-group-cluster-table.component.scss'
})
export class RemoteProcessGroupClusterTable extends ComponentClusterTable<NodeRemoteProcessGroupStatusSnapshot> {
displayedColumns: string[] = ['node', 'uri', 'transmitting', 'sent', 'received'];
constructor() {
super();
}
supportsMultiValuedSort(sort: Sort): boolean {
switch (sort.active) {
case 'sent':
case 'received':
return true;
default:
return false;
}
}
sortEntities(data: NodeRemoteProcessGroupStatusSnapshot[], sort: Sort): NodeRemoteProcessGroupStatusSnapshot[] {
if (!data) {
return [];
}
return data.slice().sort((a, b) => {
const isAsc: boolean = sort.direction === 'asc';
switch (sort.active) {
case 'node':
return this.compare(a.address, b.address, isAsc);
case 'transmitting':
return this.compare(
a.statusSnapshot.transmissionStatus,
b.statusSnapshot.transmissionStatus,
isAsc
);
case 'uri':
return this.compare(a.statusSnapshot.targetUri, b.statusSnapshot.targetUri, isAsc);
case 'sent':
if (this.multiSort.sortValueIndex === 0) {
return this.compare(a.statusSnapshot.flowFilesSent, b.statusSnapshot.flowFilesSent, isAsc);
} else {
return this.compare(a.statusSnapshot.bytesSent, b.statusSnapshot.bytesSent, isAsc);
}
case 'received':
if (this.multiSort.sortValueIndex === 0) {
return this.compare(
a.statusSnapshot.flowFilesReceived,
b.statusSnapshot.flowFilesReceived,
isAsc
);
} else {
return this.compare(a.statusSnapshot.bytesReceived, b.statusSnapshot.bytesReceived, isAsc);
}
default:
return 0;
}
});
}
formatNode(processor: NodeRemoteProcessGroupStatusSnapshot): string {
return `${processor.address}:${processor.apiPort}`;
}
formatTransmitting(rpg: NodeRemoteProcessGroupStatusSnapshot): string {
if (rpg.statusSnapshot.transmissionStatus === 'Transmitting') {
return rpg.statusSnapshot.transmissionStatus;
} else {
return 'Not Transmitting';
}
}
formatUri(rpg: NodeRemoteProcessGroupStatusSnapshot): string {
return rpg.statusSnapshot.targetUri;
}
formatSent(rpg: NodeRemoteProcessGroupStatusSnapshot): string {
return rpg.statusSnapshot.sent;
}
formatReceived(rpg: NodeRemoteProcessGroupStatusSnapshot): string {
return rpg.statusSnapshot.received;
}
getTransmissionStatusIcon(rpg: NodeRemoteProcessGroupStatusSnapshot): string {
if (rpg.statusSnapshot.transmissionStatus === 'Transmitting') {
return 'transmitting fa fa-bullseye';
} else {
return 'not-transmitting icon icon-transmit-false';
}
}
}

View File

@ -0,0 +1,236 @@
/*
* 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 { AfterViewInit, Component, EventEmitter, Input, Output, ViewChild } from '@angular/core';
import { BaseSnapshotEntity } from '../../../state';
import { MatTableDataSource, MatTableModule } from '@angular/material/table';
import { MatSortModule, Sort, SortDirection } from '@angular/material/sort';
import { MultiSort } from '../index';
import { SummaryTableFilterContext } from '../summary-table-filter/summary-table-filter.component';
import { MatPaginator } from '@angular/material/paginator';
import { NodeSearchResult } from '../../../../../state/cluster-summary';
@Component({
selector: 'component-status-table',
standalone: true,
imports: [MatTableModule, MatSortModule],
template: ''
})
export abstract class ComponentStatusTable<T extends BaseSnapshotEntity> implements AfterViewInit {
private _summaryListingStatus: string | null = null;
private _loadedTimestamp: string | null = null;
private _initialSortColumn!: string;
private _initialSortDirection: SortDirection = 'asc';
private _connectedToCluster: boolean = false;
private _clusterNodes: NodeSearchResult[] | null = null;
private _selectedClusterNode: NodeSearchResult | null = null;
private _selectedId: string | null = null;
totalCount = 0;
filteredCount = 0;
multiSort: MultiSort = {
active: this._initialSortColumn,
direction: this._initialSortDirection,
sortValueIndex: 0,
totalValues: 2
};
dataSource: MatTableDataSource<T> = new MatTableDataSource<T>();
@ViewChild(MatPaginator) paginator!: MatPaginator;
ngAfterViewInit(): void {
this.dataSource.paginator = this.paginator;
}
abstract sortEntities(data: T[], sort: Sort): T[];
abstract supportsMultiValuedSort(sort: Sort): boolean;
abstract filterPredicate(data: T, filter: string): boolean;
applyFilter(filter: SummaryTableFilterContext) {
if (!filter || !this.dataSource) {
return;
}
// determine if the filter changing is the selected cluster node, if so a new query needs issued to the backend
if (filter.changedField === 'clusterNode' && filter.clusterNode) {
// need to re-issue the query with the selected cluster node id
this.clusterNodeSelected.next(filter.clusterNode);
}
this.dataSource.filter = JSON.stringify(filter);
this.filteredCount = this.dataSource.filteredData.length;
this.resetPaginator();
this.selectNone();
}
@Input() set initialSortColumn(initialSortColumn: string) {
this._initialSortColumn = initialSortColumn;
this.multiSort = { ...this.multiSort, active: initialSortColumn };
}
get initialSortColumn() {
return this._initialSortColumn;
}
@Input() set initialSortDirection(initialSortDirection: SortDirection) {
this._initialSortDirection = initialSortDirection;
this.multiSort = { ...this.multiSort, direction: initialSortDirection };
}
get initialSortDirection() {
return this._initialSortDirection;
}
@Input({}) set components(components: T[]) {
if (components) {
this.dataSource.data = this.sortEntities(components, this.multiSort);
this.dataSource.filterPredicate = (data: T, filter: string) => this.filterPredicate(data, filter);
this.totalCount = components.length;
if (this.dataSource.filteredData.length > 0) {
this.filteredCount = this.dataSource.filteredData.length;
} else {
this.filteredCount = components.length;
}
}
}
@Input() set loadedTimestamp(value: string | null) {
this._loadedTimestamp = value;
}
get loadedTimestamp(): string | null {
return this._loadedTimestamp;
}
@Input() set summaryListingStatus(value: string | null) {
this._summaryListingStatus = value;
}
get summaryListingStatus(): string | null {
return this._summaryListingStatus;
}
@Input() set connectedToCluster(value: boolean) {
this._connectedToCluster = value;
}
get connectedToCluster(): boolean {
return this._connectedToCluster;
}
@Input() set clusterNodes(nodes: NodeSearchResult[] | null) {
this._clusterNodes = nodes;
}
get clusterNodes(): NodeSearchResult[] | null {
return this._clusterNodes;
}
@Input() set selectedClusterNode(selectedClusterNode: NodeSearchResult | null) {
this._selectedClusterNode = selectedClusterNode;
}
get selectedClusterNode(): NodeSearchResult | null {
return this._selectedClusterNode;
}
@Input() set selectedId(selectedId: string | null) {
this._selectedId = selectedId;
}
get selectedId(): string | null {
return this._selectedId;
}
@Output() refresh: EventEmitter<void> = new EventEmitter<void>();
@Output() viewStatusHistory: EventEmitter<T> = new EventEmitter<T>();
@Output() selectComponent: EventEmitter<T> = new EventEmitter<T>();
@Output() viewClusteredDetails: EventEmitter<T> = new EventEmitter<T>();
@Output() clearSelection: EventEmitter<void> = new EventEmitter<void>();
@Output() clusterNodeSelected: EventEmitter<NodeSearchResult> = new EventEmitter<NodeSearchResult>();
resetPaginator(): void {
if (this.dataSource.paginator) {
this.dataSource.paginator.firstPage();
}
}
paginationChanged(): void {
// clear out any selection
this.selectNone();
}
selectNone() {
this.clearSelection.next();
}
sortData(sort: Sort) {
this.setMultiSort(sort);
this.dataSource.data = this.sortEntities(this.dataSource.data, sort);
}
compare(a: number | string, b: number | string, isAsc: boolean) {
return (a < b ? -1 : a > b ? 1 : 0) * (isAsc ? 1 : -1);
}
select(item: T) {
this.selectComponent.next(item);
}
isSelected(item: T): boolean {
if (this.selectedId) {
return this.selectedId === item.id;
}
return false;
}
viewStatusHistoryClicked(event: MouseEvent, component: T): void {
event.stopPropagation();
this.viewStatusHistory.next(component);
}
viewClusteredDetailsClicked(event: MouseEvent, component: T): void {
event.stopPropagation();
this.viewClusteredDetails.next(component);
}
private setMultiSort(sort: Sort) {
const { active, direction, sortValueIndex, totalValues } = this.multiSort;
if (this.supportsMultiValuedSort(sort)) {
if (active === sort.active) {
// previous sort was of the same column
if (direction === 'desc' && sort.direction === 'asc') {
// change from previous index to the next
const newIndex = sortValueIndex + 1 >= totalValues ? 0 : sortValueIndex + 1;
this.multiSort = { ...sort, sortValueIndex: newIndex, totalValues };
} else {
this.multiSort = { ...sort, sortValueIndex, totalValues };
}
} else {
// sorting a different column, just reset
this.multiSort = { ...sort, sortValueIndex: 0, totalValues };
}
} else {
this.multiSort = { ...sort, sortValueIndex: 0, totalValues };
}
}
}

View File

@ -25,6 +25,8 @@
[filterableColumns]="filterableColumns"
[includeStatusFilter]="true"
[includePrimaryNodeOnlyFilter]="false"
[clusterNodes]="clusterNodes"
[selectedNode]="selectedClusterNode"
(filterChanged)="applyFilter($event)"></summary-table-filter>
<div class="flex-1 relative">
@ -143,7 +145,13 @@
class="pointer fa fa-long-arrow-right"
[routerLink]="getPortLink(item)"
(click)="$event.stopPropagation()"
title="Go to {{ portType }}} port"></div>
title="Go to {{ portType }} port"></div>
@if (connectedToCluster) {
<div
class="pointer fa fa-cubes"
title="View Clustered {{ portType | titlecase }} Port Details"
(click)="viewClusteredDetailsClicked($event, item)"></div>
}
</div>
</td>
</ng-container>

View File

@ -23,8 +23,8 @@
}
.mat-column-actions {
width: 72px;
min-width: 72px;
width: 80px;
min-width: 80px;
}
}
}

View File

@ -15,21 +15,18 @@
* limitations under the License.
*/
import { AfterViewInit, Component, EventEmitter, Input, Output, ViewChild } from '@angular/core';
import { Component, Input } from '@angular/core';
import { CommonModule } from '@angular/common';
import { MatSortModule, Sort, SortDirection } from '@angular/material/sort';
import { MultiSort } from '../index';
import { MatTableDataSource, MatTableModule } from '@angular/material/table';
import { PortStatusSnapshot, PortStatusSnapshotEntity } from '../../../state/summary-listing';
import { MatSortModule, Sort } from '@angular/material/sort';
import { MatTableModule } from '@angular/material/table';
import { SummaryTableFilterModule } from '../summary-table-filter/summary-table-filter.module';
import {
SummaryTableFilterArgs,
SummaryTableFilterColumn
} from '../summary-table-filter/summary-table-filter.component';
import { SummaryTableFilterColumn } from '../summary-table-filter/summary-table-filter.component';
import { ComponentType } from '../../../../../state/shared';
import { RouterLink } from '@angular/router';
import { NiFiCommon } from '../../../../../service/nifi-common.service';
import { MatPaginator, MatPaginatorModule } from '@angular/material/paginator';
import { MatPaginatorModule } from '@angular/material/paginator';
import { PortStatusSnapshot, PortStatusSnapshotEntity } from '../../../state';
import { ComponentStatusTable } from '../component-status-table/component-status-table.component';
export type SupportedColumns = 'name' | 'runStatus' | 'in' | 'out';
@ -40,40 +37,22 @@ export type SupportedColumns = 'name' | 'runStatus' | 'in' | 'out';
templateUrl: './port-status-table.component.html',
styleUrls: ['./port-status-table.component.scss']
})
export class PortStatusTable implements AfterViewInit {
private _initialSortColumn: SupportedColumns = 'name';
private _initialSortDirection: SortDirection = 'asc';
export class PortStatusTable extends ComponentStatusTable<PortStatusSnapshotEntity> {
private _portType!: 'input' | 'output';
filterableColumns: SummaryTableFilterColumn[] = [{ key: 'name', label: 'name' }];
totalCount = 0;
filteredCount = 0;
multiSort: MultiSort = {
active: this._initialSortColumn,
direction: this._initialSortDirection,
sortValueIndex: 0,
totalValues: 2
};
displayedColumns: string[] = [];
dataSource: MatTableDataSource<PortStatusSnapshotEntity> = new MatTableDataSource<PortStatusSnapshotEntity>();
@ViewChild(MatPaginator) paginator!: MatPaginator;
constructor(private nifiCommon: NiFiCommon) {}
ngAfterViewInit(): void {
this.dataSource.paginator = this.paginator;
constructor(private nifiCommon: NiFiCommon) {
super();
}
@Input() set portType(type: 'input' | 'output') {
if (type === 'input') {
this.displayedColumns = ['moreDetails', 'name', 'runStatus', 'in', 'actions'];
} else {
this.displayedColumns = ['moreDetails', 'name', 'runStatus', 'out', 'actions'];
} else {
this.displayedColumns = ['moreDetails', 'name', 'runStatus', 'in', 'actions'];
}
this._portType = type;
}
@ -82,74 +61,21 @@ export class PortStatusTable implements AfterViewInit {
return this._portType;
}
@Input() selectedPortId!: string;
override filterPredicate(data: PortStatusSnapshotEntity, filter: string): boolean {
const { filterTerm, filterColumn, filterStatus } = JSON.parse(filter);
const matchOnStatus: boolean = filterStatus !== 'All';
@Input() set initialSortColumn(initialSortColumn: SupportedColumns) {
this._initialSortColumn = initialSortColumn;
this.multiSort = { ...this.multiSort, active: initialSortColumn };
}
get initialSortColumn() {
return this._initialSortColumn;
}
@Input() set initialSortDirection(initialSortDirection: SortDirection) {
this._initialSortDirection = initialSortDirection;
this.multiSort = { ...this.multiSort, direction: initialSortDirection };
}
get initialSortDirection() {
return this._initialSortDirection;
}
@Input() set ports(ports: PortStatusSnapshotEntity[]) {
if (ports) {
this.dataSource.data = this.sortEntities(ports, this.multiSort);
this.dataSource.filterPredicate = (data: PortStatusSnapshotEntity, filter: string) => {
const { filterTerm, filterColumn, filterStatus } = JSON.parse(filter);
const matchOnStatus: boolean = filterStatus !== 'All';
if (matchOnStatus) {
if (data.portStatusSnapshot.runStatus !== filterStatus) {
return false;
}
}
if (filterTerm === '') {
return true;
}
const field: string = data.portStatusSnapshot[filterColumn as keyof PortStatusSnapshot] as string;
return this.nifiCommon.stringContains(field, filterTerm, true);
};
this.totalCount = ports.length;
this.filteredCount = ports.length;
if (matchOnStatus) {
if (data.portStatusSnapshot.runStatus !== filterStatus) {
return false;
}
}
}
@Input() summaryListingStatus: string | null = null;
@Input() loadedTimestamp: string | null = null;
@Output() refresh: EventEmitter<void> = new EventEmitter<void>();
@Output() selectPort: EventEmitter<PortStatusSnapshotEntity> = new EventEmitter<PortStatusSnapshotEntity>();
@Output() clearSelection: EventEmitter<void> = new EventEmitter<void>();
applyFilter(filter: SummaryTableFilterArgs) {
this.dataSource.filter = JSON.stringify(filter);
this.filteredCount = this.dataSource.filteredData.length;
this.resetPaginator();
this.selectNone();
}
resetPaginator(): void {
if (this.dataSource.paginator) {
this.dataSource.paginator.firstPage();
if (filterTerm === '') {
return true;
}
}
paginationChanged(): void {
// clear out any selection
this.selectNone();
const field: string = data.portStatusSnapshot[filterColumn as keyof PortStatusSnapshot] as string;
return this.nifiCommon.stringContains(field, filterTerm, true);
}
formatName(port: PortStatusSnapshotEntity): string {
@ -193,31 +119,11 @@ export class PortStatusTable implements AfterViewInit {
return ['/process-groups', port.portStatusSnapshot.groupId, componentType, port.id];
}
select(port: PortStatusSnapshotEntity): void {
this.selectPort.next(port);
}
private selectNone() {
this.clearSelection.next();
}
isSelected(port: PortStatusSnapshotEntity): boolean {
if (this.selectedPortId) {
return port.id === this.selectedPortId;
}
return false;
}
sortData(sort: Sort) {
this.setMultiSort(sort);
this.dataSource.data = this.sortEntities(this.dataSource.data, sort);
}
canRead(port: PortStatusSnapshotEntity) {
return port.canRead;
}
private supportsMultiValuedSort(sort: Sort): boolean {
override supportsMultiValuedSort(sort: Sort): boolean {
switch (sort.active) {
case 'in':
case 'out':
@ -227,29 +133,7 @@ export class PortStatusTable implements AfterViewInit {
}
}
private setMultiSort(sort: Sort) {
const { active, direction, sortValueIndex, totalValues } = this.multiSort;
if (this.supportsMultiValuedSort(sort)) {
if (active === sort.active) {
// previous sort was of the same column
if (direction === 'desc' && sort.direction === 'asc') {
// change from previous index to the next
const newIndex = sortValueIndex + 1 >= totalValues ? 0 : sortValueIndex + 1;
this.multiSort = { ...sort, sortValueIndex: newIndex, totalValues };
} else {
this.multiSort = { ...sort, sortValueIndex, totalValues };
}
} else {
// sorting a different column, just reset
this.multiSort = { ...sort, sortValueIndex: 0, totalValues };
}
} else {
this.multiSort = { ...sort, sortValueIndex: 0, totalValues };
}
}
private sortEntities(data: PortStatusSnapshotEntity[], sort: Sort): PortStatusSnapshotEntity[] {
override sortEntities(data: PortStatusSnapshotEntity[], sort: Sort): PortStatusSnapshotEntity[] {
if (!data) {
return [];
}

View File

@ -32,7 +32,7 @@
<mat-label>Filter By</mat-label>
<mat-select formControlName="filterColumn">
@for (option of filterableColumns; track option) {
<mat-option [value]="option.key"> {{ option.label }} </mat-option>
<mat-option [value]="option.key"> {{ option.label }}</mat-option>
}
</mat-select>
</mat-form-field>
@ -42,12 +42,12 @@
<mat-form-field>
<mat-label>Status</mat-label>
<mat-select formControlName="filterStatus">
<mat-option value="All"> All Statuses </mat-option>
<mat-option value="Running"> Running </mat-option>
<mat-option value="Stopped"> Stopped </mat-option>
<mat-option value="Validating"> Validating </mat-option>
<mat-option value="Disabled"> Disabled </mat-option>
<mat-option value="Invalid"> Invalid </mat-option>
<mat-option value="All"> All Statuses</mat-option>
<mat-option value="Running"> Running</mat-option>
<mat-option value="Stopped"> Stopped</mat-option>
<mat-option value="Validating"> Validating</mat-option>
<mat-option value="Disabled"> Disabled</mat-option>
<mat-option value="Invalid"> Invalid</mat-option>
</mat-select>
</mat-form-field>
</div>
@ -55,7 +55,20 @@
@if (includePrimaryNodeOnlyFilter) {
<div>
<mat-checkbox color="primary" formControlName="primaryOnly"></mat-checkbox>
<mat-label>Primary Node</mat-label>
<mat-label class="whitespace-nowrap">Primary Node</mat-label>
</div>
}
@if (clusterNodes && clusterNodes.length > 0) {
<div class="cluster-node-selection flex flex-1 flex-row-reverse">
<mat-form-field>
<mat-label>Cluster Node</mat-label>
<mat-select formControlName="clusterNode" panelWidth="">
@for (node of clusterNodes; track node) {
<mat-option [value]="node">{{ node.address }}</mat-option>
}
</mat-select>
</mat-form-field>
</div>
}
</div>

View File

@ -14,3 +14,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
.summary-table-filter-container {
.cluster-node-selection {
max-width: 100%;
.mat-mdc-form-field {
width: 300px;
}
}
}

View File

@ -19,6 +19,7 @@ import { AfterViewInit, Component, DestroyRef, EventEmitter, inject, Input, Outp
import { FormBuilder, FormGroup } from '@angular/forms';
import { debounceTime } from 'rxjs';
import { takeUntilDestroyed } from '@angular/core/rxjs-interop';
import { NodeSearchResult } from '../../../../../state/cluster-summary';
export interface SummaryTableFilterColumn {
key: string;
@ -30,6 +31,11 @@ export interface SummaryTableFilterArgs {
filterColumn: string;
filterStatus?: string;
primaryOnly?: boolean;
clusterNode?: NodeSearchResult;
}
export interface SummaryTableFilterContext extends SummaryTableFilterArgs {
changedField: string;
}
@Component({
@ -42,13 +48,71 @@ export class SummaryTableFilter implements AfterViewInit {
private _filteredCount = 0;
private _totalCount = 0;
private _initialFilterColumn = 'name';
private _filterableColumns: SummaryTableFilterColumn[] = [];
private destroyRef: DestroyRef = inject(DestroyRef);
showFilterMatchedLabel = false;
@Input() filterableColumns: SummaryTableFilterColumn[] = [];
showFilterMatchedLabel = false;
allNodes: NodeSearchResult = {
id: 'All',
address: 'All Nodes'
};
private _clusterNodes: NodeSearchResult[] = [];
private _selectedNode: NodeSearchResult | null = this.allNodes;
@Input() set filterableColumns(filterableColumns: SummaryTableFilterColumn[]) {
this._filterableColumns = filterableColumns;
}
get filterableColumns(): SummaryTableFilterColumn[] {
return this._filterableColumns;
}
@Input() includeStatusFilter = false;
@Input() includePrimaryNodeOnlyFilter = false;
@Output() filterChanged: EventEmitter<SummaryTableFilterArgs> = new EventEmitter<SummaryTableFilterArgs>();
@Input() set selectedNode(node: NodeSearchResult | null) {
const n: NodeSearchResult = node ? (node.id !== 'All' ? node : this.allNodes) : this.allNodes;
// find it in the available nodes
const found = this._clusterNodes.find((node) => node.id === n.id);
if (found) {
this.filterForm.get('clusterNode')?.setValue(found);
this._selectedNode = found;
}
}
@Input() set clusterNodes(nodes: NodeSearchResult[] | null) {
if (!nodes) {
this._clusterNodes = [];
} else {
// test if the nodes have changed
if (!this.areSame(this._clusterNodes, nodes)) {
this._clusterNodes = [this.allNodes, ...nodes];
if (this._selectedNode) {
this.selectedNode = this._selectedNode;
}
}
}
}
get clusterNodes(): NodeSearchResult[] {
return this._clusterNodes;
}
private areSame(a: NodeSearchResult[], b: NodeSearchResult[]) {
if (a.length !== b.length) {
return false;
}
const noMatch = a.filter((node) => b.findIndex((n) => n.id === node.id) < 0);
return noMatch.length === 0 || (noMatch.length === 1 && noMatch[0].id === 'All');
}
private areEqual(a: NodeSearchResult, b: NodeSearchResult) {
return a.id === b.id;
}
@Output() filterChanged: EventEmitter<SummaryTableFilterContext> = new EventEmitter<SummaryTableFilterContext>();
@Output() clusterFilterChanged: EventEmitter<SummaryTableFilterContext> =
new EventEmitter<SummaryTableFilterContext>();
@Input() set filterTerm(term: string) {
this.filterForm.get('filterTerm')?.value(term);
@ -94,7 +158,8 @@ export class SummaryTableFilter implements AfterViewInit {
filterTerm: '',
filterColumn: this._initialFilterColumn || 'name',
filterStatus: 'All',
primaryOnly: false
primaryOnly: false,
clusterNode: this.allNodes
});
}
@ -106,7 +171,8 @@ export class SummaryTableFilter implements AfterViewInit {
const filterColumn = this.filterForm.get('filterColumn')?.value;
const filterStatus = this.filterForm.get('filterStatus')?.value;
const primaryOnly = this.filterForm.get('primaryOnly')?.value;
this.applyFilter(filterTerm, filterColumn, filterStatus, primaryOnly);
const clusterNode = this.filterForm.get('clusterNode')?.value;
this.applyFilter(filterTerm, filterColumn, filterStatus, primaryOnly, clusterNode, 'filterTerm');
});
this.filterForm
@ -116,7 +182,8 @@ export class SummaryTableFilter implements AfterViewInit {
const filterTerm = this.filterForm.get('filterTerm')?.value;
const filterStatus = this.filterForm.get('filterStatus')?.value;
const primaryOnly = this.filterForm.get('primaryOnly')?.value;
this.applyFilter(filterTerm, filterColumn, filterStatus, primaryOnly);
const clusterNode = this.filterForm.get('clusterNode')?.value;
this.applyFilter(filterTerm, filterColumn, filterStatus, primaryOnly, clusterNode, 'filterColumn');
});
this.filterForm
@ -126,7 +193,8 @@ export class SummaryTableFilter implements AfterViewInit {
const filterTerm = this.filterForm.get('filterTerm')?.value;
const filterColumn = this.filterForm.get('filterColumn')?.value;
const primaryOnly = this.filterForm.get('primaryOnly')?.value;
this.applyFilter(filterTerm, filterColumn, filterStatus, primaryOnly);
const clusterNode = this.filterForm.get('clusterNode')?.value;
this.applyFilter(filterTerm, filterColumn, filterStatus, primaryOnly, clusterNode, 'filterStatus');
});
this.filterForm
@ -136,17 +204,45 @@ export class SummaryTableFilter implements AfterViewInit {
const filterTerm = this.filterForm.get('filterTerm')?.value;
const filterColumn = this.filterForm.get('filterColumn')?.value;
const filterStatus = this.filterForm.get('filterStatus')?.value;
this.applyFilter(filterTerm, filterColumn, filterStatus, primaryOnly);
const clusterNode = this.filterForm.get('clusterNode')?.value;
this.applyFilter(filterTerm, filterColumn, filterStatus, primaryOnly, clusterNode, 'primaryOnly');
});
this.filterForm
.get('clusterNode')
?.valueChanges.pipe(takeUntilDestroyed(this.destroyRef))
.subscribe((clusterNode) => {
if (this._selectedNode?.id !== clusterNode.id) {
this._selectedNode = clusterNode;
const filterTerm = this.filterForm.get('filterTerm')?.value;
const filterColumn = this.filterForm.get('filterColumn')?.value;
const filterStatus = this.filterForm.get('filterStatus')?.value;
const primaryOnly = this.filterForm.get('primaryOnly')?.value;
this.applyFilter(filterTerm, filterColumn, filterStatus, primaryOnly, clusterNode, 'clusterNode');
}
});
}
applyFilter(filterTerm: string, filterColumn: string, filterStatus: string, primaryOnly: boolean) {
applyFilter(
filterTerm: string,
filterColumn: string,
filterStatus: string,
primaryOnly: boolean,
clusterNode: NodeSearchResult,
changedField: string
) {
this.filterChanged.next({
filterColumn,
filterStatus,
filterTerm,
primaryOnly
primaryOnly,
clusterNode,
changedField
});
this.showFilterMatchedLabel = filterTerm?.length > 0 || filterStatus !== 'All' || primaryOnly;
this.showFilterMatchedLabel =
filterTerm?.length > 0 ||
filterStatus !== 'All' ||
primaryOnly ||
(clusterNode ? clusterNode.id !== 'All' : false);
}
}

View File

@ -22,14 +22,19 @@
</div>
} @else {
<connection-status-table
[connections]="(connectionStatusSnapshots$ | async)!"
[selectedConnectionId]="selectedConnectionId$ | async"
[components]="(connectionStatusSnapshots$ | async)!"
[selectedId]="selectedConnectionId$ | async"
[loadedTimestamp]="loadedTimestamp$ | async"
[summaryListingStatus]="summaryListingStatus$ | async"
(selectConnection)="selectConnection($event)"
[connectedToCluster]="(connectedToCluster$ | async) || false"
[clusterNodes]="clusterNodes$ | async"
[selectedClusterNode]="selectedClusterNode$ | async"
(selectComponent)="selectConnection($event)"
(clearSelection)="clearSelection()"
(viewStatusHistory)="viewStatusHistory($event)"
(refresh)="refreshSummaryListing()"
(clusterNodeSelected)="clusterNodeSelected($event)"
(viewClusteredDetails)="viewClusteredDetails($event)"
initialSortColumn="sourceName"
initialSortDirection="asc"></connection-status-table>
}

View File

@ -17,22 +17,31 @@
import { Component } from '@angular/core';
import { Store } from '@ngrx/store';
import { ConnectionStatusSnapshotEntity, SummaryListingState } from '../../state/summary-listing';
import { SummaryListingState } from '../../state/summary-listing';
import { initialState } from '../../state/summary-listing/summary-listing.reducer';
import * as SummaryListingActions from '../../state/summary-listing/summary-listing.actions';
import {
selectConnectionIdFromRoute,
selectConnectionStatus,
selectConnectionStatusSnapshots,
selectSelectedClusterNode,
selectSummaryListingLoadedTimestamp,
selectSummaryListingStatus,
selectViewStatusHistory
} from '../../state/summary-listing/summary-listing.selectors';
import { selectCurrentUser } from '../../../../state/current-user/current-user.selectors';
import { filter, switchMap, take } from 'rxjs';
import { filter, map, switchMap, take } from 'rxjs';
import { takeUntilDestroyed } from '@angular/core/rxjs-interop';
import { getStatusHistoryAndOpenDialog } from '../../../../state/status-history/status-history.actions';
import { ComponentType } from '../../../../state/shared';
import { ComponentType, isDefinedAndNotNull } from '../../../../state/shared';
import { loadClusterSummary } from '../../../../state/cluster-summary/cluster-summary.actions';
import { ConnectionStatusSnapshotEntity } from '../../state';
import {
selectClusterSearchResults,
selectClusterSummary
} from '../../../../state/cluster-summary/cluster-summary.selectors';
import * as ClusterStatusActions from '../../state/component-cluster-status/component-cluster-status.actions';
import { NodeSearchResult } from '../../../../state/cluster-summary';
@Component({
selector: 'connection-status-listing',
@ -45,6 +54,15 @@ export class ConnectionStatusListing {
currentUser$ = this.store.select(selectCurrentUser);
connectionStatusSnapshots$ = this.store.select(selectConnectionStatusSnapshots);
selectedConnectionId$ = this.store.select(selectConnectionIdFromRoute);
connectedToCluster$ = this.store.select(selectClusterSummary).pipe(
isDefinedAndNotNull(),
map((cluster) => cluster.connectedToCluster)
);
clusterNodes$ = this.store.select(selectClusterSearchResults).pipe(
isDefinedAndNotNull(),
map((results) => results.nodeResults)
);
selectedClusterNode$ = this.store.select(selectSelectedClusterNode);
constructor(private store: Store<SummaryListingState>) {
this.store
@ -80,6 +98,7 @@ export class ConnectionStatusListing {
refreshSummaryListing() {
this.store.dispatch(SummaryListingActions.loadSummaryListing({ recursive: true }));
this.store.dispatch(loadClusterSummary());
}
selectConnection(connection: ConnectionStatusSnapshotEntity): void {
@ -103,4 +122,19 @@ export class ConnectionStatusListing {
})
);
}
viewClusteredDetails(processor: ConnectionStatusSnapshotEntity): void {
this.store.dispatch(
ClusterStatusActions.loadComponentClusterStatusAndOpenDialog({
request: {
id: processor.id,
componentType: ComponentType.Connection
}
})
);
}
clusterNodeSelected(clusterNode: NodeSearchResult) {
this.store.dispatch(SummaryListingActions.selectClusterNode({ clusterNode }));
}
}

View File

@ -26,6 +26,8 @@
[includeStatusFilter]="false"
[includePrimaryNodeOnlyFilter]="false"
filterColumn="sourceName"
[clusterNodes]="clusterNodes"
[selectedNode]="selectedClusterNode"
(filterChanged)="applyFilter($event)"></summary-table-filter>
<div class="flex-1 relative">
@ -213,6 +215,13 @@
class="pointer fa fa-area-chart"
title="View Status History"
(click)="viewStatusHistoryClicked($event, item)"></div>
@if (connectedToCluster) {
<div
class="pointer fa fa-cubes"
title="View Clustered Connection Details"
(click)="viewClusteredDetailsClicked($event, item)"></div>
}
</div>
</td>
</ng-container>

View File

@ -15,21 +15,18 @@
* limitations under the License.
*/
import { AfterViewInit, Component, EventEmitter, Input, Output, ViewChild } from '@angular/core';
import { Component } from '@angular/core';
import { CommonModule } from '@angular/common';
import { SummaryTableFilterModule } from '../../common/summary-table-filter/summary-table-filter.module';
import { MatSortModule, Sort, SortDirection } from '@angular/material/sort';
import { MultiSort } from '../../common';
import { MatSortModule, Sort } from '@angular/material/sort';
import { NiFiCommon } from '../../../../../service/nifi-common.service';
import {
SummaryTableFilterArgs,
SummaryTableFilterColumn
} from '../../common/summary-table-filter/summary-table-filter.component';
import { ConnectionStatusSnapshot, ConnectionStatusSnapshotEntity } from '../../../state/summary-listing';
import { MatTableDataSource, MatTableModule } from '@angular/material/table';
import { SummaryTableFilterColumn } from '../../common/summary-table-filter/summary-table-filter.component';
import { MatTableModule } from '@angular/material/table';
import { ComponentType } from '../../../../../state/shared';
import { RouterLink } from '@angular/router';
import { MatPaginator, MatPaginatorModule } from '@angular/material/paginator';
import { MatPaginatorModule } from '@angular/material/paginator';
import { ConnectionStatusSnapshot, ConnectionStatusSnapshotEntity } from '../../../state';
import { ComponentStatusTable } from '../../common/component-status-table/component-status-table.component';
export type SupportedColumns = 'name' | 'queue' | 'in' | 'out' | 'threshold' | 'sourceName' | 'destinationName';
@ -40,26 +37,13 @@ export type SupportedColumns = 'name' | 'queue' | 'in' | 'out' | 'threshold' | '
templateUrl: './connection-status-table.component.html',
styleUrls: ['./connection-status-table.component.scss']
})
export class ConnectionStatusTable implements AfterViewInit {
private _initialSortColumn: SupportedColumns = 'sourceName';
private _initialSortDirection: SortDirection = 'asc';
export class ConnectionStatusTable extends ComponentStatusTable<ConnectionStatusSnapshotEntity> {
filterableColumns: SummaryTableFilterColumn[] = [
{ key: 'sourceName', label: 'source' },
{ key: 'name', label: 'name' },
{ key: 'destinationName', label: 'destination' }
];
totalCount = 0;
filteredCount = 0;
multiSort: MultiSort = {
active: this._initialSortColumn,
direction: this._initialSortDirection,
sortValueIndex: 0,
totalValues: 2
};
displayedColumns: string[] = [
'moreDetails',
'name',
@ -72,88 +56,19 @@ export class ConnectionStatusTable implements AfterViewInit {
'actions'
];
dataSource: MatTableDataSource<ConnectionStatusSnapshotEntity> =
new MatTableDataSource<ConnectionStatusSnapshotEntity>();
@ViewChild(MatPaginator) paginator!: MatPaginator;
constructor(private nifiCommon: NiFiCommon) {}
ngAfterViewInit(): void {
this.dataSource.paginator = this.paginator;
constructor(private nifiCommon: NiFiCommon) {
super();
}
@Input() set initialSortColumn(initialSortColumn: SupportedColumns) {
this._initialSortColumn = initialSortColumn;
this.multiSort = { ...this.multiSort, active: initialSortColumn };
}
override filterPredicate(data: ConnectionStatusSnapshotEntity, filter: string): boolean {
const { filterTerm, filterColumn } = JSON.parse(filter);
get initialSortColumn() {
return this._initialSortColumn;
}
@Input() set initialSortDirection(initialSortDirection: SortDirection) {
this._initialSortDirection = initialSortDirection;
this.multiSort = { ...this.multiSort, direction: initialSortDirection };
}
get initialSortDirection() {
return this._initialSortDirection;
}
@Input() selectedConnectionId!: string;
@Input() set connections(connections: ConnectionStatusSnapshotEntity[]) {
if (connections) {
this.dataSource.data = this.sortEntities(connections, this.multiSort);
this.dataSource.filterPredicate = (data: ConnectionStatusSnapshotEntity, filter: string) => {
const { filterTerm, filterColumn } = JSON.parse(filter);
if (filterTerm === '') {
return true;
}
const field: string = data.connectionStatusSnapshot[
filterColumn as keyof ConnectionStatusSnapshot
] as string;
return this.nifiCommon.stringContains(field, filterTerm, true);
};
this.totalCount = connections.length;
this.filteredCount = connections.length;
if (filterTerm === '') {
return true;
}
}
@Input() summaryListingStatus: string | null = null;
@Input() loadedTimestamp: string | null = null;
@Output() refresh: EventEmitter<void> = new EventEmitter<void>();
@Output() viewStatusHistory: EventEmitter<ConnectionStatusSnapshotEntity> =
new EventEmitter<ConnectionStatusSnapshotEntity>();
@Output() selectConnection: EventEmitter<ConnectionStatusSnapshotEntity> =
new EventEmitter<ConnectionStatusSnapshotEntity>();
@Output() clearSelection: EventEmitter<void> = new EventEmitter<void>();
resetPaginator(): void {
if (this.dataSource.paginator) {
this.dataSource.paginator.firstPage();
}
}
applyFilter(filter: SummaryTableFilterArgs) {
this.dataSource.filter = JSON.stringify(filter);
this.filteredCount = this.dataSource.filteredData.length;
this.resetPaginator();
this.selectNone();
}
paginationChanged(): void {
// clear out any selection
this.selectNone();
}
private selectNone() {
this.clearSelection.next();
const field: string = data.connectionStatusSnapshot[filterColumn as keyof ConnectionStatusSnapshot] as string;
return this.nifiCommon.stringContains(field, filterTerm, true);
}
getConnectionLink(connection: ConnectionStatusSnapshotEntity): string[] {
@ -165,54 +80,39 @@ export class ConnectionStatusTable implements AfterViewInit {
];
}
select(connection: ConnectionStatusSnapshotEntity): void {
this.selectConnection.next(connection);
}
isSelected(connection: ConnectionStatusSnapshotEntity): boolean {
if (this.selectedConnectionId) {
return connection.id === this.selectedConnectionId;
}
return false;
}
canRead(connection: ConnectionStatusSnapshotEntity): boolean {
return connection.canRead;
}
sortData(sort: Sort) {
this.setMultiSort(sort);
this.dataSource.data = this.sortEntities(this.dataSource.data, sort);
}
formatName(connection: ConnectionStatusSnapshotEntity): string {
return connection.connectionStatusSnapshot.name;
}
formatSource(connection: ConnectionStatusSnapshotEntity): string {
return connection.connectionStatusSnapshot.sourceName;
}
formatDestination(connection: ConnectionStatusSnapshotEntity): string {
return connection.connectionStatusSnapshot.destinationName;
}
formatIn(connection: ConnectionStatusSnapshotEntity): string {
return connection.connectionStatusSnapshot.input;
}
formatOut(connection: ConnectionStatusSnapshotEntity): string {
return connection.connectionStatusSnapshot.output;
}
formatQueue(connection: ConnectionStatusSnapshotEntity): string {
return connection.connectionStatusSnapshot.queued;
}
formatThreshold(connection: ConnectionStatusSnapshotEntity): string {
return `${connection.connectionStatusSnapshot.percentUseCount}% | ${connection.connectionStatusSnapshot.percentUseBytes}%`;
}
viewStatusHistoryClicked(event: MouseEvent, connection: ConnectionStatusSnapshotEntity): void {
event.stopPropagation();
this.viewStatusHistory.next(connection);
}
private supportsMultiValuedSort(sort: Sort): boolean {
override supportsMultiValuedSort(sort: Sort): boolean {
switch (sort.active) {
case 'in':
case 'out':
@ -224,29 +124,7 @@ export class ConnectionStatusTable implements AfterViewInit {
}
}
private setMultiSort(sort: Sort) {
const { active, direction, sortValueIndex, totalValues } = this.multiSort;
if (this.supportsMultiValuedSort(sort)) {
if (active === sort.active) {
// previous sort was of the same column
if (direction === 'desc' && sort.direction === 'asc') {
// change from previous index to the next
const newIndex = sortValueIndex + 1 >= totalValues ? 0 : sortValueIndex + 1;
this.multiSort = { ...sort, sortValueIndex: newIndex, totalValues };
} else {
this.multiSort = { ...sort, sortValueIndex, totalValues };
}
} else {
// sorting a different column, just reset
this.multiSort = { ...sort, sortValueIndex: 0, totalValues };
}
} else {
this.multiSort = { ...sort, sortValueIndex: 0, totalValues };
}
}
private sortEntities(data: ConnectionStatusSnapshotEntity[], sort: Sort): ConnectionStatusSnapshotEntity[] {
override sortEntities(data: ConnectionStatusSnapshotEntity[], sort: Sort): ConnectionStatusSnapshotEntity[] {
if (!data) {
return [];
}

View File

@ -22,14 +22,19 @@
</div>
} @else {
<port-status-table
[ports]="(portStatusSnapshots$ | async)!"
[selectedPortId]="selectedPortId$ | async"
[components]="(portStatusSnapshots$ | async)!"
[selectedId]="selectedPortId$ | async"
[loadedTimestamp]="loadedTimestamp$ | async"
[summaryListingStatus]="summaryListingStatus$ | async"
[connectedToCluster]="(connectedToCluster$ | async) || false"
[clusterNodes]="clusterNodes$ | async"
[selectedClusterNode]="selectedClusterNode$ | async"
portType="input"
(selectPort)="selectPort($event)"
(selectComponent)="selectPort($event)"
(clearSelection)="clearSelection()"
(refresh)="refreshSummaryListing()"
(clusterNodeSelected)="clusterNodeSelected($event)"
(viewClusteredDetails)="viewClusteredDetails($event)"
initialSortColumn="name"
initialSortDirection="asc"></port-status-table>
}

View File

@ -19,14 +19,25 @@ import { Component } from '@angular/core';
import {
selectInputPortIdFromRoute,
selectInputPortStatusSnapshots,
selectSelectedClusterNode,
selectSummaryListingLoadedTimestamp,
selectSummaryListingStatus
} from '../../state/summary-listing/summary-listing.selectors';
import { selectCurrentUser } from '../../../../state/current-user/current-user.selectors';
import { PortStatusSnapshotEntity, SummaryListingState } from '../../state/summary-listing';
import { SummaryListingState } from '../../state/summary-listing';
import { Store } from '@ngrx/store';
import { initialState } from '../../state/summary-listing/summary-listing.reducer';
import * as SummaryListingActions from '../../state/summary-listing/summary-listing.actions';
import { loadClusterSummary } from '../../../../state/cluster-summary/cluster-summary.actions';
import { PortStatusSnapshotEntity } from '../../state';
import {
selectClusterSearchResults,
selectClusterSummary
} from '../../../../state/cluster-summary/cluster-summary.selectors';
import { ComponentType, isDefinedAndNotNull } from '../../../../state/shared';
import { map } from 'rxjs';
import { NodeSearchResult } from '../../../../state/cluster-summary';
import * as ClusterStatusActions from '../../state/component-cluster-status/component-cluster-status.actions';
@Component({
selector: 'input-port-status-listing',
@ -39,6 +50,15 @@ export class InputPortStatusListing {
summaryListingStatus$ = this.store.select(selectSummaryListingStatus);
currentUser$ = this.store.select(selectCurrentUser);
selectedPortId$ = this.store.select(selectInputPortIdFromRoute);
connectedToCluster$ = this.store.select(selectClusterSummary).pipe(
isDefinedAndNotNull(),
map((cluster) => cluster.connectedToCluster)
);
clusterNodes$ = this.store.select(selectClusterSearchResults).pipe(
isDefinedAndNotNull(),
map((results) => results.nodeResults)
);
selectedClusterNode$ = this.store.select(selectSelectedClusterNode);
constructor(private store: Store<SummaryListingState>) {}
@ -48,6 +68,7 @@ export class InputPortStatusListing {
refreshSummaryListing() {
this.store.dispatch(SummaryListingActions.loadSummaryListing({ recursive: true }));
this.store.dispatch(loadClusterSummary());
}
selectPort(port: PortStatusSnapshotEntity): void {
@ -63,4 +84,19 @@ export class InputPortStatusListing {
clearSelection() {
this.store.dispatch(SummaryListingActions.clearInputPortStatusSelection());
}
clusterNodeSelected(clusterNode: NodeSearchResult) {
this.store.dispatch(SummaryListingActions.selectClusterNode({ clusterNode }));
}
viewClusteredDetails(port: PortStatusSnapshotEntity): void {
this.store.dispatch(
ClusterStatusActions.loadComponentClusterStatusAndOpenDialog({
request: {
id: port.id,
componentType: ComponentType.InputPort
}
})
);
}
}

View File

@ -22,14 +22,19 @@
</div>
} @else {
<port-status-table
[ports]="(portStatusSnapshots$ | async)!"
[selectedPortId]="selectedPortId$ | async"
[components]="(portStatusSnapshots$ | async)!"
[selectedId]="selectedPortId$ | async"
[loadedTimestamp]="loadedTimestamp$ | async"
[summaryListingStatus]="summaryListingStatus$ | async"
[connectedToCluster]="(connectedToCluster$ | async) || false"
[clusterNodes]="clusterNodes$ | async"
[selectedClusterNode]="selectedClusterNode$ | async"
portType="output"
(selectPort)="selectPort($event)"
(selectComponent)="selectPort($event)"
(clearSelection)="clearSelection()"
(refresh)="refreshSummaryListing()"
(clusterNodeSelected)="clusterNodeSelected($event)"
(viewClusteredDetails)="viewClusteredDetails($event)"
initialSortColumn="name"
initialSortDirection="asc"></port-status-table>
}

View File

@ -19,14 +19,25 @@ import { Component } from '@angular/core';
import {
selectOutputPortIdFromRoute,
selectOutputPortStatusSnapshots,
selectSelectedClusterNode,
selectSummaryListingLoadedTimestamp,
selectSummaryListingStatus
} from '../../state/summary-listing/summary-listing.selectors';
import { selectCurrentUser } from '../../../../state/current-user/current-user.selectors';
import { Store } from '@ngrx/store';
import { PortStatusSnapshotEntity, SummaryListingState } from '../../state/summary-listing';
import { SummaryListingState } from '../../state/summary-listing';
import { initialState } from '../../state/summary-listing/summary-listing.reducer';
import * as SummaryListingActions from '../../state/summary-listing/summary-listing.actions';
import { loadClusterSummary } from '../../../../state/cluster-summary/cluster-summary.actions';
import { PortStatusSnapshotEntity } from '../../state';
import {
selectClusterSearchResults,
selectClusterSummary
} from '../../../../state/cluster-summary/cluster-summary.selectors';
import { ComponentType, isDefinedAndNotNull } from '../../../../state/shared';
import { map } from 'rxjs';
import { NodeSearchResult } from '../../../../state/cluster-summary';
import * as ClusterStatusActions from '../../state/component-cluster-status/component-cluster-status.actions';
@Component({
selector: 'output-port-status-listing',
@ -39,6 +50,15 @@ export class OutputPortStatusListing {
summaryListingStatus$ = this.store.select(selectSummaryListingStatus);
currentUser$ = this.store.select(selectCurrentUser);
selectedPortId$ = this.store.select(selectOutputPortIdFromRoute);
connectedToCluster$ = this.store.select(selectClusterSummary).pipe(
isDefinedAndNotNull(),
map((cluster) => cluster.connectedToCluster)
);
clusterNodes$ = this.store.select(selectClusterSearchResults).pipe(
isDefinedAndNotNull(),
map((results) => results.nodeResults)
);
selectedClusterNode$ = this.store.select(selectSelectedClusterNode);
constructor(private store: Store<SummaryListingState>) {}
@ -48,6 +68,7 @@ export class OutputPortStatusListing {
refreshSummaryListing() {
this.store.dispatch(SummaryListingActions.loadSummaryListing({ recursive: true }));
this.store.dispatch(loadClusterSummary());
}
selectPort(port: PortStatusSnapshotEntity): void {
@ -63,4 +84,19 @@ export class OutputPortStatusListing {
clearSelection() {
this.store.dispatch(SummaryListingActions.clearOutputPortStatusSelection());
}
clusterNodeSelected(clusterNode: NodeSearchResult) {
this.store.dispatch(SummaryListingActions.selectClusterNode({ clusterNode }));
}
viewClusteredDetails(port: PortStatusSnapshotEntity): void {
this.store.dispatch(
ClusterStatusActions.loadComponentClusterStatusAndOpenDialog({
request: {
id: port.id,
componentType: ComponentType.OutputPort
}
})
);
}
}

View File

@ -22,15 +22,20 @@
</div>
} @else {
<process-group-status-table
[processGroups]="(processGroupStatusSnapshots$ | async)!"
[selectedProcessGroupId]="selectedProcessGroupId$ | async"
[components]="(processGroupStatusSnapshots$ | async)!"
[selectedId]="selectedProcessGroupId$ | async"
[rootProcessGroup]="(processGroupStatus$ | async)?.processGroupStatus?.aggregateSnapshot!"
[loadedTimestamp]="loadedTimestamp$ | async"
[summaryListingStatus]="summaryListingStatus$ | async"
[connectedToCluster]="(connectedToCluster$ | async) || false"
[clusterNodes]="clusterNodes$ | async"
[selectedClusterNode]="selectedClusterNode$ | async"
(viewStatusHistory)="viewStatusHistory($event)"
(selectProcessGroup)="selectProcessGroup($event)"
(selectComponent)="selectProcessGroup($event)"
(clearSelection)="clearSelection()"
(refresh)="refreshSummaryListing()"
(clusterNodeSelected)="clusterNodeSelected($event)"
(viewClusteredDetails)="viewClusteredDetails($event)"
initialSortColumn="name"
initialSortDirection="asc"></process-group-status-table>
}

View File

@ -18,22 +18,31 @@
import { Component } from '@angular/core';
import { initialState } from '../../state/summary-listing/summary-listing.reducer';
import * as SummaryListingActions from '../../state/summary-listing/summary-listing.actions';
import { ProcessGroupStatusSnapshotEntity, SummaryListingState } from '../../state/summary-listing';
import { SummaryListingState } from '../../state/summary-listing';
import { Store } from '@ngrx/store';
import {
selectProcessGroupIdFromRoute,
selectProcessGroupStatus,
selectProcessGroupStatusItem,
selectProcessGroupStatusSnapshots,
selectSelectedClusterNode,
selectSummaryListingLoadedTimestamp,
selectSummaryListingStatus,
selectViewStatusHistory
} from '../../state/summary-listing/summary-listing.selectors';
import { filter, switchMap, take } from 'rxjs';
import { filter, map, switchMap, take } from 'rxjs';
import { takeUntilDestroyed } from '@angular/core/rxjs-interop';
import { getStatusHistoryAndOpenDialog } from '../../../../state/status-history/status-history.actions';
import { ComponentType } from '../../../../state/shared';
import { ComponentType, isDefinedAndNotNull } from '../../../../state/shared';
import { selectCurrentUser } from '../../../../state/current-user/current-user.selectors';
import { loadClusterSummary } from '../../../../state/cluster-summary/cluster-summary.actions';
import { ProcessGroupStatusSnapshotEntity } from '../../state';
import * as ClusterStatusActions from '../../state/component-cluster-status/component-cluster-status.actions';
import { NodeSearchResult } from '../../../../state/cluster-summary';
import {
selectClusterSearchResults,
selectClusterSummary
} from '../../../../state/cluster-summary/cluster-summary.selectors';
@Component({
selector: 'process-group-status-listing',
@ -47,6 +56,15 @@ export class ProcessGroupStatusListing {
currentUser$ = this.store.select(selectCurrentUser);
selectedProcessGroupId$ = this.store.select(selectProcessGroupIdFromRoute);
processGroupStatus$ = this.store.select(selectProcessGroupStatus);
connectedToCluster$ = this.store.select(selectClusterSummary).pipe(
isDefinedAndNotNull(),
map((cluster) => cluster.connectedToCluster)
);
clusterNodes$ = this.store.select(selectClusterSearchResults).pipe(
isDefinedAndNotNull(),
map((results) => results.nodeResults)
);
selectedClusterNode$ = this.store.select(selectSelectedClusterNode);
constructor(private store: Store<SummaryListingState>) {
this.store
@ -82,6 +100,7 @@ export class ProcessGroupStatusListing {
refreshSummaryListing() {
this.store.dispatch(SummaryListingActions.loadSummaryListing({ recursive: true }));
this.store.dispatch(loadClusterSummary());
}
viewStatusHistory(pg: ProcessGroupStatusSnapshotEntity): void {
@ -105,4 +124,19 @@ export class ProcessGroupStatusListing {
clearSelection() {
this.store.dispatch(SummaryListingActions.clearProcessGroupStatusSelection());
}
viewClusteredDetails(pg: ProcessGroupStatusSnapshotEntity): void {
this.store.dispatch(
ClusterStatusActions.loadComponentClusterStatusAndOpenDialog({
request: {
id: pg.id,
componentType: ComponentType.ProcessGroup
}
})
);
}
clusterNodeSelected(clusterNode: NodeSearchResult) {
this.store.dispatch(SummaryListingActions.selectClusterNode({ clusterNode }));
}
}

View File

@ -20,10 +20,11 @@ import { ProcessGroupStatusListing } from './process-group-status-listing.compon
import { CommonModule } from '@angular/common';
import { NgxSkeletonLoaderModule } from 'ngx-skeleton-loader';
import { ProcessGroupStatusTable } from './process-group-status-table/process-group-status-table.component';
import { ProcessorStatusTable } from '../processor-status-listing/processor-status-table/processor-status-table.component';
@NgModule({
declarations: [ProcessGroupStatusListing],
exports: [ProcessGroupStatusListing],
imports: [CommonModule, NgxSkeletonLoaderModule, ProcessGroupStatusTable]
imports: [CommonModule, NgxSkeletonLoaderModule, ProcessGroupStatusTable, ProcessorStatusTable]
})
export class ProcessGroupStatusListingModule {}

View File

@ -25,6 +25,8 @@
[filterableColumns]="filterableColumns"
[includeStatusFilter]="false"
[includePrimaryNodeOnlyFilter]="false"
[clusterNodes]="clusterNodes"
[selectedNode]="selectedClusterNode"
(filterChanged)="applyFilter($event)"></summary-table-filter>
<div class="flex-1 relative">
@ -341,6 +343,13 @@
class="pointer fa fa-area-chart"
title="View Status History"
(click)="viewStatusHistoryClicked($event, item)"></div>
@if (connectedToCluster) {
<div
class="pointer fa fa-cubes"
title="View Clustered Process Group Details"
(click)="viewClusteredDetailsClicked($event, item)"></div>
}
</div>
</td>
</ng-container>

View File

@ -15,20 +15,17 @@
* limitations under the License.
*/
import { AfterViewInit, Component, EventEmitter, Input, Output, ViewChild } from '@angular/core';
import { Component, Input } from '@angular/core';
import { CommonModule } from '@angular/common';
import { MatSortModule, Sort, SortDirection } from '@angular/material/sort';
import { MultiSort } from '../../common';
import { MatTableDataSource, MatTableModule } from '@angular/material/table';
import { MatSortModule, Sort } from '@angular/material/sort';
import { MatTableModule } from '@angular/material/table';
import { SummaryTableFilterModule } from '../../common/summary-table-filter/summary-table-filter.module';
import { ProcessGroupStatusSnapshot, ProcessGroupStatusSnapshotEntity } from '../../../state/summary-listing';
import {
SummaryTableFilterArgs,
SummaryTableFilterColumn
} from '../../common/summary-table-filter/summary-table-filter.component';
import { SummaryTableFilterColumn } from '../../common/summary-table-filter/summary-table-filter.component';
import { NiFiCommon } from '../../../../../service/nifi-common.service';
import { RouterLink } from '@angular/router';
import { MatPaginator, MatPaginatorModule } from '@angular/material/paginator';
import { MatPaginatorModule } from '@angular/material/paginator';
import { ProcessGroupStatusSnapshot, ProcessGroupStatusSnapshotEntity } from '../../../state';
import { ComponentStatusTable } from '../../common/component-status-table/component-status-table.component';
export type SupportedColumns =
| 'name'
@ -49,20 +46,8 @@ export type SupportedColumns =
templateUrl: './process-group-status-table.component.html',
styleUrls: ['./process-group-status-table.component.scss']
})
export class ProcessGroupStatusTable implements AfterViewInit {
private _initialSortColumn: SupportedColumns = 'name';
private _initialSortDirection: SortDirection = 'asc';
export class ProcessGroupStatusTable extends ComponentStatusTable<ProcessGroupStatusSnapshotEntity> {
filterableColumns: SummaryTableFilterColumn[] = [{ key: 'name', label: 'name' }];
totalCount = 0;
filteredCount = 0;
multiSort: MultiSort = {
active: this._initialSortColumn,
direction: this._initialSortDirection,
sortValueIndex: 0,
totalValues: 2
};
displayedColumns: string[] = [
'moreDetails',
@ -79,87 +64,23 @@ export class ProcessGroupStatusTable implements AfterViewInit {
'actions'
];
dataSource: MatTableDataSource<ProcessGroupStatusSnapshotEntity> =
new MatTableDataSource<ProcessGroupStatusSnapshotEntity>();
constructor(private nifiCommon: NiFiCommon) {}
applyFilter(filter: SummaryTableFilterArgs) {
this.dataSource.filter = JSON.stringify(filter);
this.filteredCount = this.dataSource.filteredData.length;
this.resetPaginator();
this.selectNone();
}
@Input() selectedProcessGroupId!: string;
@Input() set initialSortColumn(initialSortColumn: SupportedColumns) {
this._initialSortColumn = initialSortColumn;
this.multiSort = { ...this.multiSort, active: initialSortColumn };
}
get initialSortColumn() {
return this._initialSortColumn;
}
@Input() set initialSortDirection(initialSortDirection: SortDirection) {
this._initialSortDirection = initialSortDirection;
this.multiSort = { ...this.multiSort, direction: initialSortDirection };
}
get initialSortDirection() {
return this._initialSortDirection;
constructor(private nifiCommon: NiFiCommon) {
super();
}
@Input() rootProcessGroup!: ProcessGroupStatusSnapshot;
@Input() set processGroups(processGroups: ProcessGroupStatusSnapshotEntity[]) {
if (processGroups) {
this.dataSource.data = this.sortEntities(processGroups, this.multiSort);
override filterPredicate(data: ProcessGroupStatusSnapshotEntity, filter: string): boolean {
const { filterTerm, filterColumn } = JSON.parse(filter);
this.dataSource.filterPredicate = (data: ProcessGroupStatusSnapshotEntity, filter: string): boolean => {
const { filterTerm, filterColumn } = JSON.parse(filter);
if (filterTerm === '') {
return true;
}
const field: string = data.processGroupStatusSnapshot[
filterColumn as keyof ProcessGroupStatusSnapshot
] as string;
return this.nifiCommon.stringContains(field, filterTerm, true);
};
this.totalCount = processGroups.length;
this.filteredCount = processGroups.length;
if (filterTerm === '') {
return true;
}
}
@Input() summaryListingStatus: string | null = null;
@Input() loadedTimestamp: string | null = null;
@Output() viewStatusHistory: EventEmitter<ProcessGroupStatusSnapshotEntity> =
new EventEmitter<ProcessGroupStatusSnapshotEntity>();
@Output() selectProcessGroup: EventEmitter<ProcessGroupStatusSnapshotEntity> =
new EventEmitter<ProcessGroupStatusSnapshotEntity>();
@Output() clearSelection: EventEmitter<void> = new EventEmitter<void>();
@Output() refresh: EventEmitter<void> = new EventEmitter<void>();
@ViewChild(MatPaginator) paginator!: MatPaginator;
ngAfterViewInit(): void {
this.dataSource.paginator = this.paginator;
}
resetPaginator(): void {
if (this.dataSource.paginator) {
this.dataSource.paginator.firstPage();
}
}
paginationChanged(): void {
// clear out any selection
this.selectNone();
const field: string = data.processGroupStatusSnapshot[
filterColumn as keyof ProcessGroupStatusSnapshot
] as string;
return this.nifiCommon.stringContains(field, filterTerm, true);
}
formatName(pg: ProcessGroupStatusSnapshotEntity): string {
@ -252,7 +173,7 @@ export class ProcessGroupStatusTable implements AfterViewInit {
return 0;
}
private supportsMultiValuedSort(sort: Sort): boolean {
override supportsMultiValuedSort(sort: Sort): boolean {
switch (sort.active) {
case 'transferred':
case 'in':
@ -268,34 +189,7 @@ export class ProcessGroupStatusTable implements AfterViewInit {
}
}
private setMultiSort(sort: Sort) {
const { active, direction, sortValueIndex, totalValues } = this.multiSort;
if (this.supportsMultiValuedSort(sort)) {
if (active === sort.active) {
// previous sort was of the same column
if (direction === 'desc' && sort.direction === 'asc') {
// change from previous index to the next
const newIndex = sortValueIndex + 1 >= totalValues ? 0 : sortValueIndex + 1;
this.multiSort = { ...sort, sortValueIndex: newIndex, totalValues };
} else {
this.multiSort = { ...sort, sortValueIndex, totalValues };
}
} else {
// sorting a different column, just reset
this.multiSort = { ...sort, sortValueIndex: 0, totalValues };
}
} else {
this.multiSort = { ...sort, sortValueIndex: 0, totalValues };
}
}
sortData(sort: Sort) {
this.setMultiSort(sort);
this.dataSource.data = this.sortEntities(this.dataSource.data, sort);
}
private sortEntities(data: ProcessGroupStatusSnapshotEntity[], sort: Sort): ProcessGroupStatusSnapshotEntity[] {
override sortEntities(data: ProcessGroupStatusSnapshotEntity[], sort: Sort): ProcessGroupStatusSnapshotEntity[] {
if (!data) {
return [];
}
@ -438,24 +332,4 @@ export class ProcessGroupStatusTable implements AfterViewInit {
getProcessGroupLink(pg: ProcessGroupStatusSnapshotEntity): string[] {
return ['/process-groups', pg.id];
}
select(pg: ProcessGroupStatusSnapshotEntity): void {
this.selectProcessGroup.next(pg);
}
isSelected(pg: ProcessGroupStatusSnapshotEntity): boolean {
if (this.selectedProcessGroupId) {
return pg.id === this.selectedProcessGroupId;
}
return false;
}
viewStatusHistoryClicked(event: MouseEvent, pg: ProcessGroupStatusSnapshotEntity): void {
event.stopPropagation();
this.viewStatusHistory.next(pg);
}
private selectNone() {
this.clearSelection.next();
}
}

View File

@ -21,14 +21,19 @@
</div>
} @else {
<processor-status-table
[processors]="(processorStatusSnapshots$ | async)!"
[selectedProcessorId]="selectedProcessorId$ | async"
[components]="(processorStatusSnapshots$ | async)!"
[selectedId]="selectedProcessorId$ | async"
[loadedTimestamp]="loadedTimestamp$ | async"
[summaryListingStatus]="summaryListingStatus$ | async"
[connectedToCluster]="(connectedToCluster$ | async) || false"
[clusterNodes]="clusterNodes$ | async"
[selectedClusterNode]="selectedClusterNode$ | async"
(viewStatusHistory)="viewStatusHistory($event)"
(selectProcessor)="selectProcessor($event)"
(selectComponent)="selectProcessor($event)"
(clearSelection)="clearSelection()"
(refresh)="refreshSummaryListing()"
(clusterNodeSelected)="clusterNodeSelected($event)"
(viewClusteredDetails)="viewClusteredDetails($event)"
initialSortColumn="name"
initialSortDirection="asc"></processor-status-table>
}

View File

@ -21,20 +21,29 @@ import {
selectProcessorIdFromRoute,
selectProcessorStatus,
selectProcessorStatusSnapshots,
selectSelectedClusterNode,
selectSummaryListingLoadedTimestamp,
selectSummaryListingStatus,
selectViewStatusHistory
} from '../../state/summary-listing/summary-listing.selectors';
import { ProcessorStatusSnapshotEntity, SummaryListingState } from '../../state/summary-listing';
import { SummaryListingState } from '../../state/summary-listing';
import { selectCurrentUser } from '../../../../state/current-user/current-user.selectors';
import { initialState } from '../../state/summary-listing/summary-listing.reducer';
import { getStatusHistoryAndOpenDialog } from '../../../../state/status-history/status-history.actions';
import { ComponentType } from '../../../../state/shared';
import { combineLatest, delay, filter, Subject, switchMap, take } from 'rxjs';
import { ComponentType, isDefinedAndNotNull } from '../../../../state/shared';
import { combineLatest, delay, filter, map, Subject, switchMap, take } from 'rxjs';
import { takeUntilDestroyed } from '@angular/core/rxjs-interop';
import * as SummaryListingActions from '../../state/summary-listing/summary-listing.actions';
import * as ClusterStatusActions from '../../state/component-cluster-status/component-cluster-status.actions';
import { MatPaginator } from '@angular/material/paginator';
import { ProcessorStatusTable } from './processor-status-table/processor-status-table.component';
import {
selectClusterSearchResults,
selectClusterSummary
} from '../../../../state/cluster-summary/cluster-summary.selectors';
import { loadClusterSummary } from '../../../../state/cluster-summary/cluster-summary.actions';
import { ProcessorStatusSnapshotEntity } from '../../state';
import { NodeSearchResult } from '../../../../state/cluster-summary';
@Component({
selector: 'processor-status-listing',
@ -46,6 +55,15 @@ export class ProcessorStatusListing implements AfterViewInit {
loadedTimestamp$ = this.store.select(selectSummaryListingLoadedTimestamp);
summaryListingStatus$ = this.store.select(selectSummaryListingStatus);
selectedProcessorId$ = this.store.select(selectProcessorIdFromRoute);
connectedToCluster$ = this.store.select(selectClusterSummary).pipe(
isDefinedAndNotNull(),
map((cluster) => cluster.connectedToCluster)
);
clusterNodes$ = this.store.select(selectClusterSearchResults).pipe(
isDefinedAndNotNull(),
map((results) => results.nodeResults)
);
selectedClusterNode$ = this.store.select(selectSelectedClusterNode);
currentUser$ = this.store.select(selectCurrentUser);
@ -104,6 +122,7 @@ export class ProcessorStatusListing implements AfterViewInit {
refreshSummaryListing() {
this.store.dispatch(SummaryListingActions.loadSummaryListing({ recursive: true }));
this.store.dispatch(loadClusterSummary());
}
viewStatusHistory(processor: ProcessorStatusSnapshotEntity): void {
@ -127,4 +146,19 @@ export class ProcessorStatusListing implements AfterViewInit {
clearSelection() {
this.store.dispatch(SummaryListingActions.clearProcessorStatusSelection());
}
viewClusteredDetails(processor: ProcessorStatusSnapshotEntity): void {
this.store.dispatch(
ClusterStatusActions.loadComponentClusterStatusAndOpenDialog({
request: {
id: processor.id,
componentType: ComponentType.Processor
}
})
);
}
clusterNodeSelected(clusterNode: NodeSearchResult) {
this.store.dispatch(SummaryListingActions.selectClusterNode({ clusterNode }));
}
}

View File

@ -0,0 +1,42 @@
/*!
* 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.
*/
@use 'sass:map';
@use '@angular/material' as mat;
@mixin nifi-theme($material-theme) {
// Get the color config from the theme.
$color-config: mat.get-color-config($material-theme);
// Get the color palette from the color-config.
$accent-palette: map.get($color-config, 'accent');
$primary-palette: map.get($color-config, 'primary');
// Get hues from palette
$accent-palette-A400: mat.get-color-from-palette($accent-palette, 'A400');
$primary-palette-contrast-300: mat.get-color-from-palette($primary-palette, '300-contrast');
$primary-palette-300: mat.get-color-from-palette($primary-palette, 300);
$primary-palette-900: mat.get-color-from-palette($primary-palette, 900);
.processor-status-table {
.primary-node-only {
color: $accent-palette-A400;
background-color: $primary-palette-contrast-300;
border: 1px solid $primary-palette-300;
}
}
}

View File

@ -25,6 +25,8 @@
[filterableColumns]="filterableColumns"
[includeStatusFilter]="true"
[includePrimaryNodeOnlyFilter]="true"
[clusterNodes]="clusterNodes"
[selectedNode]="selectedClusterNode"
(filterChanged)="applyFilter($event)"></summary-table-filter>
<div class="flex-1 relative">
@ -60,7 +62,16 @@
<div class="flex-1 overflow-ellipsis overflow-hidden whitespace-nowrap">Name</div>
</th>
<td mat-cell *matCellDef="let item" [title]="formatName(item)">
{{ formatName(item) }}
<div class="flex align-middle">
@if (item.processorStatusSnapshot.executionNode === 'PRIMARY') {
<div
class="primary-node-only mt-0.5 mr-1"
title="This component is only scheduled to execute on the Primary Node">
P
</div>
}
<div>{{ formatName(item) }}</div>
</div>
</td>
</ng-container>
@ -248,6 +259,13 @@
class="pointer fa fa-area-chart"
title="View Status History"
(click)="viewStatusHistoryClicked($event, item)"></div>
@if (connectedToCluster) {
<div
class="pointer fa fa-cubes"
title="View Clustered Processor Details"
(click)="viewClusteredDetailsClicked($event, item)"></div>
}
</div>
</td>
</ng-container>

View File

@ -22,8 +22,19 @@
}
.mat-column-actions {
width: 72px;
min-width: 72px;
width: 80px;
min-width: 80px;
}
}
.primary-node-only {
font-family: Roboto;
font-size: 10px;
font-weight: bold;
line-height: 14px;
width: 16px;
height: 16px;
border-radius: 8px;
float: left;
text-align: center;
}
}

View File

@ -15,21 +15,18 @@
* limitations under the License.
*/
import { AfterViewInit, Component, EventEmitter, Input, Output, ViewChild } from '@angular/core';
import { MatTableDataSource, MatTableModule } from '@angular/material/table';
import { ProcessorStatusSnapshot, ProcessorStatusSnapshotEntity } from '../../../state/summary-listing';
import { MatSortModule, Sort, SortDirection } from '@angular/material/sort';
import {
SummaryTableFilterArgs,
SummaryTableFilterColumn
} from '../../common/summary-table-filter/summary-table-filter.component';
import { Component } from '@angular/core';
import { MatTableModule } from '@angular/material/table';
import { MatSortModule, Sort } from '@angular/material/sort';
import { SummaryTableFilterColumn } from '../../common/summary-table-filter/summary-table-filter.component';
import { RouterLink } from '@angular/router';
import { SummaryTableFilterModule } from '../../common/summary-table-filter/summary-table-filter.module';
import { NgClass } from '@angular/common';
import { ComponentType } from '../../../../../state/shared';
import { MultiSort } from '../../common';
import { NiFiCommon } from '../../../../../service/nifi-common.service';
import { MatPaginator, MatPaginatorModule } from '@angular/material/paginator';
import { MatPaginatorModule } from '@angular/material/paginator';
import { ProcessorStatusSnapshot, ProcessorStatusSnapshotEntity } from '../../../state';
import { ComponentStatusTable } from '../../common/component-status-table/component-status-table.component';
export type SupportedColumns = 'name' | 'type' | 'processGroup' | 'runStatus' | 'in' | 'out' | 'readWrite' | 'tasks';
@ -40,23 +37,11 @@ export type SupportedColumns = 'name' | 'type' | 'processGroup' | 'runStatus' |
standalone: true,
imports: [RouterLink, SummaryTableFilterModule, MatTableModule, MatSortModule, NgClass, MatPaginatorModule]
})
export class ProcessorStatusTable implements AfterViewInit {
private _initialSortColumn: SupportedColumns = 'name';
private _initialSortDirection: SortDirection = 'asc';
export class ProcessorStatusTable extends ComponentStatusTable<ProcessorStatusSnapshotEntity> {
filterableColumns: SummaryTableFilterColumn[] = [
{ key: 'name', label: 'name' },
{ key: 'type', label: 'type' }
];
totalCount = 0;
filteredCount = 0;
multiSort: MultiSort = {
active: this._initialSortColumn,
direction: this._initialSortDirection,
sortValueIndex: 0,
totalValues: 2
};
displayedColumns: string[] = [
'moreDetails',
@ -70,95 +55,9 @@ export class ProcessorStatusTable implements AfterViewInit {
'tasks',
'actions'
];
dataSource: MatTableDataSource<ProcessorStatusSnapshotEntity> =
new MatTableDataSource<ProcessorStatusSnapshotEntity>();
@ViewChild(MatPaginator) paginator!: MatPaginator;
constructor(private nifiCommon: NiFiCommon) {}
ngAfterViewInit(): void {
this.dataSource.paginator = this.paginator;
}
applyFilter(filter: SummaryTableFilterArgs) {
this.dataSource.filter = JSON.stringify(filter);
this.filteredCount = this.dataSource.filteredData.length;
this.resetPaginator();
this.selectNone();
}
@Input() selectedProcessorId!: string;
@Input() set initialSortColumn(initialSortColumn: SupportedColumns) {
this._initialSortColumn = initialSortColumn;
this.multiSort = { ...this.multiSort, active: initialSortColumn };
}
get initialSortColumn() {
return this._initialSortColumn;
}
@Input() set initialSortDirection(initialSortDirection: SortDirection) {
this._initialSortDirection = initialSortDirection;
this.multiSort = { ...this.multiSort, direction: initialSortDirection };
}
get initialSortDirection() {
return this._initialSortDirection;
}
@Input() set processors(processors: ProcessorStatusSnapshotEntity[]) {
if (processors) {
this.dataSource.data = this.sortEntities(processors, this.multiSort);
this.dataSource.filterPredicate = (data: ProcessorStatusSnapshotEntity, filter: string): boolean => {
const { filterTerm, filterColumn, filterStatus, primaryOnly } = JSON.parse(filter);
const matchOnStatus: boolean = filterStatus !== 'All';
if (primaryOnly) {
if (data.processorStatusSnapshot.executionNode !== 'PRIMARY') {
return false;
}
}
if (matchOnStatus) {
if (data.processorStatusSnapshot.runStatus !== filterStatus) {
return false;
}
}
if (filterTerm === '') {
return true;
}
const field: string = data.processorStatusSnapshot[
filterColumn as keyof ProcessorStatusSnapshot
] as string;
return this.nifiCommon.stringContains(field, filterTerm, true);
};
this.totalCount = processors.length;
this.filteredCount = processors.length;
}
}
@Input() summaryListingStatus: string | null = null;
@Input() loadedTimestamp: string | null = null;
@Output() refresh: EventEmitter<void> = new EventEmitter<void>();
@Output() viewStatusHistory: EventEmitter<ProcessorStatusSnapshotEntity> =
new EventEmitter<ProcessorStatusSnapshotEntity>();
@Output() selectProcessor: EventEmitter<ProcessorStatusSnapshotEntity> =
new EventEmitter<ProcessorStatusSnapshotEntity>();
@Output() clearSelection: EventEmitter<void> = new EventEmitter<void>();
resetPaginator(): void {
if (this.dataSource.paginator) {
this.dataSource.paginator.firstPage();
}
}
paginationChanged(): void {
// clear out any selection
this.selectNone();
constructor(private nifiCommon: NiFiCommon) {
super();
}
formatName(processor: ProcessorStatusSnapshotEntity): string {
@ -220,12 +119,7 @@ export class ProcessorStatusTable implements AfterViewInit {
}
}
sortData(sort: Sort) {
this.setMultiSort(sort);
this.dataSource.data = this.sortEntities(this.dataSource.data, sort);
}
private sortEntities(data: ProcessorStatusSnapshotEntity[], sort: Sort): ProcessorStatusSnapshotEntity[] {
override sortEntities(data: ProcessorStatusSnapshotEntity[], sort: Sort): ProcessorStatusSnapshotEntity[] {
if (!data) {
return [];
}
@ -302,11 +196,7 @@ export class ProcessorStatusTable implements AfterViewInit {
});
}
private compare(a: number | string, b: number | string, isAsc: boolean) {
return (a < b ? -1 : a > b ? 1 : 0) * (isAsc ? 1 : -1);
}
private supportsMultiValuedSort(sort: Sort): boolean {
override supportsMultiValuedSort(sort: Sort): boolean {
switch (sort.active) {
case 'in':
case 'out':
@ -318,45 +208,25 @@ export class ProcessorStatusTable implements AfterViewInit {
}
}
private setMultiSort(sort: Sort) {
const { active, direction, sortValueIndex, totalValues } = this.multiSort;
override filterPredicate(data: ProcessorStatusSnapshotEntity, filter: string): boolean {
const { filterTerm, filterColumn, filterStatus, primaryOnly } = JSON.parse(filter);
const matchOnStatus: boolean = filterStatus !== 'All';
if (this.supportsMultiValuedSort(sort)) {
if (active === sort.active) {
// previous sort was of the same column
if (direction === 'desc' && sort.direction === 'asc') {
// change from previous index to the next
const newIndex = sortValueIndex + 1 >= totalValues ? 0 : sortValueIndex + 1;
this.multiSort = { ...sort, sortValueIndex: newIndex, totalValues };
} else {
this.multiSort = { ...sort, sortValueIndex, totalValues };
}
} else {
// sorting a different column, just reset
this.multiSort = { ...sort, sortValueIndex: 0, totalValues };
if (primaryOnly) {
if (data.processorStatusSnapshot.executionNode !== 'PRIMARY') {
return false;
}
} else {
this.multiSort = { ...sort, sortValueIndex: 0, totalValues };
}
}
select(processor: ProcessorStatusSnapshotEntity): void {
this.selectProcessor.next(processor);
}
isSelected(processor: ProcessorStatusSnapshotEntity): boolean {
if (this.selectedProcessorId) {
return processor.id === this.selectedProcessorId;
if (matchOnStatus) {
if (data.processorStatusSnapshot.runStatus !== filterStatus) {
return false;
}
}
if (filterTerm === '') {
return true;
}
return false;
}
viewStatusHistoryClicked(event: MouseEvent, processor: ProcessorStatusSnapshotEntity): void {
event.stopPropagation();
this.viewStatusHistory.next(processor);
}
private selectNone() {
this.clearSelection.next();
const field: string = data.processorStatusSnapshot[filterColumn as keyof ProcessorStatusSnapshot] as string;
return this.nifiCommon.stringContains(field, filterTerm, true);
}
}

View File

@ -22,14 +22,19 @@
</div>
} @else {
<remote-process-group-status-table
[remoteProcessGroups]="(rpgStatusSnapshots$ | async)!"
[selectedRemoteProcessGroupId]="selectedRpgId$ | async"
[components]="(rpgStatusSnapshots$ | async)!"
[selectedId]="selectedRpgId$ | async"
[loadedTimestamp]="loadedTimestamp$ | async"
[summaryListingStatus]="summaryListingStatus$ | async"
(selectRemoteProcessGroup)="selectRemoteProcessGroup($event)"
[connectedToCluster]="(connectedToCluster$ | async) || false"
[clusterNodes]="clusterNodes$ | async"
[selectedClusterNode]="selectedClusterNode$ | async"
(selectComponent)="selectRemoteProcessGroup($event)"
(clearSelection)="clearSelection()"
(viewStatusHistory)="viewStatusHistory($event)"
(refresh)="refreshSummaryListing()"
(clusterNodeSelected)="clusterNodeSelected($event)"
(viewClusteredDetails)="viewClusteredDetails($event)"
initialSortColumn="name"
initialSortDirection="asc"></remote-process-group-status-table>
}

View File

@ -20,19 +20,28 @@ import {
selectRemoteProcessGroupIdFromRoute,
selectRemoteProcessGroupStatus,
selectRemoteProcessGroupStatusSnapshots,
selectSelectedClusterNode,
selectSummaryListingLoadedTimestamp,
selectSummaryListingStatus,
selectViewStatusHistory
} from '../../state/summary-listing/summary-listing.selectors';
import { selectCurrentUser } from '../../../../state/current-user/current-user.selectors';
import { Store } from '@ngrx/store';
import { RemoteProcessGroupStatusSnapshotEntity, SummaryListingState } from '../../state/summary-listing';
import { filter, switchMap, take } from 'rxjs';
import { SummaryListingState } from '../../state/summary-listing';
import { filter, map, switchMap, take } from 'rxjs';
import { takeUntilDestroyed } from '@angular/core/rxjs-interop';
import { getStatusHistoryAndOpenDialog } from '../../../../state/status-history/status-history.actions';
import { ComponentType } from '../../../../state/shared';
import { ComponentType, isDefinedAndNotNull } from '../../../../state/shared';
import { initialState } from '../../state/summary-listing/summary-listing.reducer';
import * as SummaryListingActions from '../../state/summary-listing/summary-listing.actions';
import { loadClusterSummary } from '../../../../state/cluster-summary/cluster-summary.actions';
import { RemoteProcessGroupStatusSnapshotEntity } from '../../state';
import {
selectClusterSearchResults,
selectClusterSummary
} from '../../../../state/cluster-summary/cluster-summary.selectors';
import * as ClusterStatusActions from '../../state/component-cluster-status/component-cluster-status.actions';
import { NodeSearchResult } from '../../../../state/cluster-summary';
@Component({
selector: 'remote-process-group-status-listing',
@ -45,6 +54,15 @@ export class RemoteProcessGroupStatusListing {
currentUser$ = this.store.select(selectCurrentUser);
rpgStatusSnapshots$ = this.store.select(selectRemoteProcessGroupStatusSnapshots);
selectedRpgId$ = this.store.select(selectRemoteProcessGroupIdFromRoute);
connectedToCluster$ = this.store.select(selectClusterSummary).pipe(
isDefinedAndNotNull(),
map((cluster) => cluster.connectedToCluster)
);
clusterNodes$ = this.store.select(selectClusterSearchResults).pipe(
isDefinedAndNotNull(),
map((results) => results.nodeResults)
);
selectedClusterNode$ = this.store.select(selectSelectedClusterNode);
constructor(private store: Store<SummaryListingState>) {
this.store
@ -80,6 +98,7 @@ export class RemoteProcessGroupStatusListing {
refreshSummaryListing() {
this.store.dispatch(SummaryListingActions.loadSummaryListing({ recursive: true }));
this.store.dispatch(loadClusterSummary());
}
selectRemoteProcessGroup(rpg: RemoteProcessGroupStatusSnapshotEntity): void {
@ -103,4 +122,19 @@ export class RemoteProcessGroupStatusListing {
})
);
}
viewClusteredDetails(processor: RemoteProcessGroupStatusSnapshotEntity): void {
this.store.dispatch(
ClusterStatusActions.loadComponentClusterStatusAndOpenDialog({
request: {
id: processor.id,
componentType: ComponentType.RemoteProcessGroup
}
})
);
}
clusterNodeSelected(clusterNode: NodeSearchResult) {
this.store.dispatch(SummaryListingActions.selectClusterNode({ clusterNode }));
}
}

View File

@ -26,6 +26,8 @@
[filterableColumns]="filterableColumns"
[includeStatusFilter]="false"
[includePrimaryNodeOnlyFilter]="false"
[clusterNodes]="clusterNodes"
[selectedNode]="selectedClusterNode"
(filterChanged)="applyFilter($event)"></summary-table-filter>
<div class="flex-1 relative">
@ -155,6 +157,13 @@
class="pointer fa fa-area-chart"
title="View Status History"
(click)="viewStatusHistoryClicked($event, item)"></div>
@if (connectedToCluster) {
<div
class="pointer fa fa-cubes"
title="View Clustered Remote Process Group Details"
(click)="viewClusteredDetailsClicked($event, item)"></div>
}
</div>
</td>
</ng-container>

View File

@ -15,24 +15,18 @@
* limitations under the License.
*/
import { AfterViewInit, Component, EventEmitter, Input, Output, ViewChild } from '@angular/core';
import { Component } from '@angular/core';
import { CommonModule } from '@angular/common';
import { SummaryTableFilterModule } from '../../common/summary-table-filter/summary-table-filter.module';
import { MatSortModule, Sort, SortDirection } from '@angular/material/sort';
import {
SummaryTableFilterArgs,
SummaryTableFilterColumn
} from '../../common/summary-table-filter/summary-table-filter.component';
import { MultiSort } from '../../common';
import { MatTableDataSource, MatTableModule } from '@angular/material/table';
import {
RemoteProcessGroupStatusSnapshot,
RemoteProcessGroupStatusSnapshotEntity
} from '../../../state/summary-listing';
import { MatSortModule, Sort } from '@angular/material/sort';
import { SummaryTableFilterColumn } from '../../common/summary-table-filter/summary-table-filter.component';
import { MatTableModule } from '@angular/material/table';
import { NiFiCommon } from '../../../../../service/nifi-common.service';
import { ComponentType } from '../../../../../state/shared';
import { RouterLink } from '@angular/router';
import { MatPaginator, MatPaginatorModule } from '@angular/material/paginator';
import { MatPaginatorModule } from '@angular/material/paginator';
import { RemoteProcessGroupStatusSnapshot, RemoteProcessGroupStatusSnapshotEntity } from '../../../state';
import { ComponentStatusTable } from '../../common/component-status-table/component-status-table.component';
export type SupportedColumns = 'name' | 'uri' | 'transmitting' | 'sent' | 'received';
@ -43,105 +37,28 @@ export type SupportedColumns = 'name' | 'uri' | 'transmitting' | 'sent' | 'recei
templateUrl: './remote-process-group-status-table.component.html',
styleUrls: ['./remote-process-group-status-table.component.scss']
})
export class RemoteProcessGroupStatusTable implements AfterViewInit {
private _initialSortColumn: SupportedColumns = 'name';
private _initialSortDirection: SortDirection = 'asc';
export class RemoteProcessGroupStatusTable extends ComponentStatusTable<RemoteProcessGroupStatusSnapshotEntity> {
filterableColumns: SummaryTableFilterColumn[] = [
{ key: 'name', label: 'name' },
{ key: 'targetUri', label: 'uri' }
];
totalCount = 0;
filteredCount = 0;
multiSort: MultiSort = {
active: this._initialSortColumn,
direction: this._initialSortDirection,
sortValueIndex: 0,
totalValues: 2
};
displayedColumns: string[] = ['moreDetails', 'name', 'uri', 'transmitting', 'sent', 'received', 'actions'];
dataSource: MatTableDataSource<RemoteProcessGroupStatusSnapshotEntity> =
new MatTableDataSource<RemoteProcessGroupStatusSnapshotEntity>();
@ViewChild(MatPaginator) paginator!: MatPaginator;
ngAfterViewInit(): void {
this.dataSource.paginator = this.paginator;
constructor(private nifiCommon: NiFiCommon) {
super();
}
constructor(private nifiCommon: NiFiCommon) {}
override filterPredicate(data: RemoteProcessGroupStatusSnapshotEntity, filter: string): boolean {
const { filterTerm, filterColumn } = JSON.parse(filter);
@Input() set initialSortColumn(initialSortColumn: SupportedColumns) {
this._initialSortColumn = initialSortColumn;
this.multiSort = { ...this.multiSort, active: initialSortColumn };
}
get initialSortColumn() {
return this._initialSortColumn;
}
@Input() set initialSortDirection(initialSortDirection: SortDirection) {
this._initialSortDirection = initialSortDirection;
this.multiSort = { ...this.multiSort, direction: initialSortDirection };
}
get initialSortDirection() {
return this._initialSortDirection;
}
@Input() selectedRemoteProcessGroupId!: string;
@Input() set remoteProcessGroups(rpgs: RemoteProcessGroupStatusSnapshotEntity[]) {
if (rpgs) {
this.dataSource.data = this.sortEntities(rpgs, this.multiSort);
this.dataSource.filterPredicate = (data: RemoteProcessGroupStatusSnapshotEntity, filter: string) => {
const { filterTerm, filterColumn } = JSON.parse(filter);
if (filterTerm === '') {
return true;
}
const field: string = data.remoteProcessGroupStatusSnapshot[
filterColumn as keyof RemoteProcessGroupStatusSnapshot
] as string;
return this.nifiCommon.stringContains(field, filterTerm, true);
};
this.totalCount = rpgs.length;
this.filteredCount = rpgs.length;
if (filterTerm === '') {
return true;
}
}
@Input() summaryListingStatus: string | null = null;
@Input() loadedTimestamp: string | null = null;
@Output() refresh: EventEmitter<void> = new EventEmitter<void>();
@Output() viewStatusHistory: EventEmitter<RemoteProcessGroupStatusSnapshotEntity> =
new EventEmitter<RemoteProcessGroupStatusSnapshotEntity>();
@Output() selectRemoteProcessGroup: EventEmitter<RemoteProcessGroupStatusSnapshotEntity> =
new EventEmitter<RemoteProcessGroupStatusSnapshotEntity>();
@Output() clearSelection: EventEmitter<void> = new EventEmitter<void>();
applyFilter(filter: SummaryTableFilterArgs) {
this.dataSource.filter = JSON.stringify(filter);
this.filteredCount = this.dataSource.filteredData.length;
this.resetPaginator();
this.selectNone();
}
resetPaginator(): void {
if (this.dataSource.paginator) {
this.dataSource.paginator.firstPage();
}
}
paginationChanged(): void {
// clear out any selection
this.selectNone();
const field: string = data.remoteProcessGroupStatusSnapshot[
filterColumn as keyof RemoteProcessGroupStatusSnapshot
] as string;
return this.nifiCommon.stringContains(field, filterTerm, true);
}
getRemoteProcessGroupLink(rpg: RemoteProcessGroupStatusSnapshotEntity): string[] {
@ -153,35 +70,10 @@ export class RemoteProcessGroupStatusTable implements AfterViewInit {
];
}
select(rpg: RemoteProcessGroupStatusSnapshotEntity) {
this.selectRemoteProcessGroup.next(rpg);
}
private selectNone() {
this.clearSelection.next();
}
isSelected(rpg: RemoteProcessGroupStatusSnapshotEntity): boolean {
if (this.selectedRemoteProcessGroupId) {
return rpg.id === this.selectedRemoteProcessGroupId;
}
return false;
}
canRead(rpg: RemoteProcessGroupStatusSnapshotEntity): boolean {
return rpg.canRead;
}
sortData(sort: Sort) {
this.setMultiSort(sort);
this.dataSource.data = this.sortEntities(this.dataSource.data, sort);
}
viewStatusHistoryClicked(event: MouseEvent, rpg: RemoteProcessGroupStatusSnapshotEntity): void {
event.stopPropagation();
this.viewStatusHistory.next(rpg);
}
formatName(rpg: RemoteProcessGroupStatusSnapshotEntity): string {
return rpg.remoteProcessGroupStatusSnapshot.name;
}
@ -214,7 +106,7 @@ export class RemoteProcessGroupStatusTable implements AfterViewInit {
}
}
private supportsMultiValuedSort(sort: Sort): boolean {
override supportsMultiValuedSort(sort: Sort): boolean {
switch (sort.active) {
case 'sent':
case 'received':
@ -224,29 +116,7 @@ export class RemoteProcessGroupStatusTable implements AfterViewInit {
}
}
private setMultiSort(sort: Sort) {
const { active, direction, sortValueIndex, totalValues } = this.multiSort;
if (this.supportsMultiValuedSort(sort)) {
if (active === sort.active) {
// previous sort was of the same column
if (direction === 'desc' && sort.direction === 'asc') {
// change from previous index to the next
const newIndex = sortValueIndex + 1 >= totalValues ? 0 : sortValueIndex + 1;
this.multiSort = { ...sort, sortValueIndex: newIndex, totalValues };
} else {
this.multiSort = { ...sort, sortValueIndex, totalValues };
}
} else {
// sorting a different column, just reset
this.multiSort = { ...sort, sortValueIndex: 0, totalValues };
}
} else {
this.multiSort = { ...sort, sortValueIndex: 0, totalValues };
}
}
private sortEntities(
override sortEntities(
data: RemoteProcessGroupStatusSnapshotEntity[],
sort: Sort
): RemoteProcessGroupStatusSnapshotEntity[] {

View File

@ -0,0 +1,25 @@
/*
* 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 { ComponentTypeNamePipe } from './component-type-name.pipe';
describe('ComponentTypeNamePipe', () => {
it('create an instance', () => {
const pipe = new ComponentTypeNamePipe();
expect(pipe).toBeTruthy();
});
});

View File

@ -0,0 +1,60 @@
/*
* 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 { Pipe, PipeTransform } from '@angular/core';
import { ComponentType } from '../state/shared';
@Pipe({
name: 'componentTypeName',
standalone: true
})
export class ComponentTypeNamePipe implements PipeTransform {
transform(type: ComponentType): string {
switch (type) {
case ComponentType.Connection:
return 'Connection';
case ComponentType.Processor:
return 'Processor';
case ComponentType.OutputPort:
return 'Output Port';
case ComponentType.InputPort:
return 'Input Port';
case ComponentType.ProcessGroup:
return 'Process Group';
case ComponentType.ControllerService:
return 'Controller Service';
case ComponentType.Flow:
return 'Flow';
case ComponentType.FlowAnalysisRule:
return 'Flow Analysis Rule';
case ComponentType.FlowRegistryClient:
return 'Flow Registry Client';
case ComponentType.Funnel:
return 'Funnel';
case ComponentType.Label:
return 'Label';
case ComponentType.ParameterProvider:
return 'Parameter Provider';
case ComponentType.RemoteProcessGroup:
return 'Remote Process Group';
case ComponentType.ReportingTask:
return 'Reporting Task';
default:
return '';
}
}
}

View File

@ -16,7 +16,7 @@
*/
import { createAction, props } from '@ngrx/store';
import { LoadClusterSummaryResponse } from './index';
import { ClusterSearchRequest, ClusterSearchResults, LoadClusterSummaryResponse } from './index';
const CLUSTER_SUMMARY_STATE_PREFIX = '[Cluster Summary State]';
@ -37,3 +37,13 @@ export const clusterSummaryApiError = createAction(
);
export const clearClusterSummaryApiError = createAction(`${CLUSTER_SUMMARY_STATE_PREFIX} Clear About Api Error`);
export const searchCluster = createAction(
`${CLUSTER_SUMMARY_STATE_PREFIX} Search Cluster`,
props<{ request: ClusterSearchRequest }>()
);
export const searchClusterSuccess = createAction(
`${CLUSTER_SUMMARY_STATE_PREFIX} Search Cluster Success`,
props<{ response: ClusterSearchResults }>()
);

View File

@ -16,16 +16,23 @@
*/
import { Injectable } from '@angular/core';
import { Actions, createEffect, ofType } from '@ngrx/effects';
import { Actions, concatLatestFrom, createEffect, ofType } from '@ngrx/effects';
import * as ClusterSummaryActions from './cluster-summary.actions';
import { asyncScheduler, catchError, from, interval, map, of, switchMap, takeUntil } from 'rxjs';
import { asyncScheduler, catchError, filter, from, interval, map, of, switchMap, takeUntil } from 'rxjs';
import { ClusterService } from '../../service/cluster.service';
import { selectClusterSummary } from './cluster-summary.selectors';
import { isDefinedAndNotNull } from '../shared';
import { Store } from '@ngrx/store';
import { ClusterSummaryState } from './index';
import { HttpErrorResponse } from '@angular/common/http';
import * as ErrorActions from '../error/error.actions';
@Injectable()
export class ClusterSummaryEffects {
constructor(
private actions$: Actions,
private clusterService: ClusterService
private clusterService: ClusterService,
private store: Store<ClusterSummaryState>
) {}
loadClusterSummary$ = createEffect(() =>
@ -57,4 +64,29 @@ export class ClusterSummaryEffects {
switchMap(() => of(ClusterSummaryActions.loadClusterSummary()))
)
);
searchCluster$ = createEffect(() =>
this.actions$.pipe(
ofType(ClusterSummaryActions.searchCluster),
map((action) => action.request),
concatLatestFrom(() =>
this.store.select(selectClusterSummary).pipe(
isDefinedAndNotNull(),
filter((clusterSummary) => clusterSummary.connectedToCluster)
)
),
switchMap(([request]) => {
return from(this.clusterService.searchCluster(request.q)).pipe(
map((response) =>
ClusterSummaryActions.searchClusterSuccess({
response: response
})
),
catchError((errorResponse: HttpErrorResponse) =>
of(ErrorActions.snackBarError({ error: errorResponse.error }))
)
);
})
)
);
}

View File

@ -18,14 +18,16 @@
import { createReducer, on } from '@ngrx/store';
import { ClusterSummaryState } from './index';
import {
clusterSummaryApiError,
clearClusterSummaryApiError,
clusterSummaryApiError,
loadClusterSummary,
loadClusterSummarySuccess
loadClusterSummarySuccess,
searchClusterSuccess
} from './cluster-summary.actions';
export const initialState: ClusterSummaryState = {
clusterSummary: null,
searchResults: null,
error: null,
status: 'pending'
};
@ -51,5 +53,9 @@ export const clusterSummaryReducer = createReducer(
...state,
error: null,
status: 'pending' as const
})),
on(searchClusterSuccess, (state, { response }) => ({
...state,
searchResults: response
}))
);

View File

@ -24,3 +24,8 @@ export const selectClusterSummary = createSelector(
selectClusterSummaryState,
(state: ClusterSummaryState) => state.clusterSummary
);
export const selectClusterSearchResults = createSelector(
selectClusterSummaryState,
(state: ClusterSummaryState) => state.searchResults
);

View File

@ -40,6 +40,11 @@ export interface ClusterSearchResults {
export interface ClusterSummaryState {
clusterSummary: ClusterSummary | null;
searchResults: ClusterSearchResults | null;
error: string | null;
status: 'pending' | 'loading' | 'error' | 'success';
}
export interface ClusterSearchRequest {
q?: string;
}

View File

@ -0,0 +1,62 @@
/*!
* 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.
*/
@use 'sass:map';
@use '@angular/material' as mat;
@mixin nifi-theme($material-theme, $canvas-theme) {
// Get the color config from the theme.
$color-config: mat.get-color-config($material-theme);
$canvas-color-config: mat.get-color-config($canvas-theme);
// Get the color palette from the color-config.
$primary-palette: map.get($color-config, 'primary');
$accent-palette: map.get($color-config, 'accent');
$warn-palette: map.get($color-config, 'warn');
$canvas-primary-palette: map.get($canvas-color-config, 'primary');
// Get hues from palette
$primary-palette-700: mat.get-color-from-palette($primary-palette, 700);
$accent-palette-A400: mat.get-color-from-palette($accent-palette, 'A400');
$warn-palette-A200: mat.get-color-from-palette($warn-palette, 'A200');
$canvas-primary-palette-A200: mat.get-color-from-palette($canvas-primary-palette, 'A200');
.component-context {
.fa,
.icon {
color: $accent-palette-A400;
}
.component-context-logo {
.icon {
color: $warn-palette-A200;
}
}
.component-context-name {
color: $canvas-primary-palette-A200;
}
.component-context-type {
color: $primary-palette-700;
}
.component-context-id {
color: $warn-palette-A200;
}
}
}

View File

@ -0,0 +1,31 @@
<!--
~ 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.
-->
<div class="component-context flex flex-col">
<div class="flex gap-x-1 items-center">
<div class="component-context-logo flex flex-col">
<i class="icon" [class]="componentIconClass"></i>
</div>
<div class="flex flex-col flex-1">
<div class="component-context-name w-full">{{ name }}</div>
<div class="component-context-type w-full">{{ type | componentTypeName }}</div>
</div>
</div>
@if (id) {
<div class="component-context-id">{{ id }}</div>
}
</div>

View File

@ -0,0 +1,41 @@
/*!
* 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.
*/
.component-context {
.component-context-logo {
text-align: start;
.icon {
font-size: 32px;
}
}
.component-context-name {
font-size: 15px;
text-overflow: ellipsis;
overflow: hidden;
white-space: nowrap;
}
.component-context-type {
font-size: 12px;
line-height: 16px;
}
.component-context-id {
font-size: 12px;
}
}

View File

@ -0,0 +1,39 @@
/*
* 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 { ComponentFixture, TestBed } from '@angular/core/testing';
import { ComponentContext } from './component-context.component';
describe('ComponentContext', () => {
let component: ComponentContext;
let fixture: ComponentFixture<ComponentContext>;
beforeEach(async () => {
await TestBed.configureTestingModule({
imports: [ComponentContext]
}).compileComponents();
fixture = TestBed.createComponent(ComponentContext);
component = fixture.componentInstance;
fixture.detectChanges();
});
it('should create', () => {
expect(component).toBeTruthy();
});
});

View File

@ -0,0 +1,67 @@
/*
* 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, Input } from '@angular/core';
import { ComponentType } from '../../../state/shared';
import { ComponentTypeNamePipe } from '../../../pipes/component-type-name.pipe';
@Component({
selector: 'component-context',
standalone: true,
imports: [ComponentTypeNamePipe],
templateUrl: './component-context.component.html',
styleUrl: './component-context.component.scss'
})
export class ComponentContext {
private _componentType: ComponentType = ComponentType.Processor;
componentIconClass: string = '';
@Input() set type(type: ComponentType) {
this._componentType = type;
this.componentIconClass = this.getIconClassName(type);
}
get type(): ComponentType {
return this._componentType;
}
@Input() id: string | null = null;
@Input() name: string = '';
private getIconClassName(type: ComponentType) {
switch (type) {
case ComponentType.Connection:
return 'icon-connect';
case ComponentType.Processor:
return 'icon-processor';
case ComponentType.OutputPort:
return 'icon-port-out';
case ComponentType.InputPort:
return 'icon-port-in';
case ComponentType.ProcessGroup:
return 'icon-group';
case ComponentType.Funnel:
return 'icon-funnel';
case ComponentType.Label:
return 'icon-label';
case ComponentType.RemoteProcessGroup:
return 'icon-group-remote';
default:
return 'icon-connect';
}
}
}

View File

@ -54,6 +54,7 @@
th {
background-color: $primary-palette-500 !important;
color: $canvas-primary-palette-900;
user-select: none;
}
tr:hover {

View File

@ -62,6 +62,8 @@
@use 'app/ui/common/status-history/status-history.component-theme' as status-history;
@use 'app/ui/common/tooltips/property-hint-tip/property-hint-tip.component-theme' as property-hint-tip;
@use 'app/ui/common/provenance-event-dialog/provenance-event-dialog.component-theme' as provenance-event-dialog;
@use 'app/pages/summary/ui/processor-status-listing/processor-status-table/processor-status-table.component-theme' as processor-status-table;
@use 'app/ui/common/component-context/component-context.component-theme' as component-context;
// Plus imports for other components in your app.
@use 'roboto-fontface/css/roboto/roboto-fontface.css';
@ -513,6 +515,8 @@ $appFontPath: '~roboto-fontface/fonts';
@include status-history.nifi-theme($material-theme-light, $nifi-canvas-theme-light);
@include property-hint-tip.nifi-theme($material-theme-light);
@include provenance-event-dialog.nifi-theme($material-theme-light);
@include processor-status-table.nifi-theme($material-theme-light);
@include component-context.nifi-theme($material-theme-light, $nifi-canvas-theme-light);
.dark-theme {
// Include the dark theme color styles.
@ -563,4 +567,6 @@ $appFontPath: '~roboto-fontface/fonts';
@include status-history.nifi-theme($material-theme-dark, $nifi-canvas-theme-dark);
@include property-hint-tip.nifi-theme($material-theme-dark);
@include provenance-event-dialog.nifi-theme($material-theme-dark);
@include processor-status-table.nifi-theme($material-theme-dark);
@include component-context.nifi-theme($material-theme-dark, $nifi-canvas-theme-dark);
}