Skip to content

Commit

Permalink
ui/cluster-ui: fix polling in fingerprints pages
Browse files Browse the repository at this point in the history
Fixes: #85236

Previously, SQL statement and transaction stats would be refreshed
every 5 minutes via sagas in CC and the cached api reducer in
db-console. This method relied on a refresh data call that resided
in `shouldComponentUpdate`, which was ignored by the respective
polling managers when the time interval was not complete. This
pattern was hacky as (unguarded calls in `shouldComponentUpdate`
are typically avoided in React. Polling in these pages were removed
with the introduciton of persisted stats, however we would like to
reintroduce polling when the selected time interval is `Latest xx..'
(i.e. not a custom interval). The removal of this polling introduced
a bug in the CC fingerprint pages, as the saga effects for when the
data was received would invalidate the data after the polling interval.
Now that the data was never refreshed, the page would get stuck on
the 'loading data' page.

This commit reintroduces polling via a `setTimeout` in the page
components, rather than through cached data reducer and sagasn for CC.
Data in the fingerprints overview pages is  now refreshed every
5 minutes for non-custom time ranges. The data invalidation in
CC is also cleaned up such that we don't invalidate data after a
delay in the receive effect. When data is refreshed via polling, we
do not show the loading spinner.

Release note (bug fix): the statements and transaction fingerprint
will no longer get stuck on the loading page in CC after 5 minutes
idling on the page

Release note (ui change): the statements and transaction fingerprint
now refresh data every 5 minutes for non-custom time ranges

Release justification: bug fix
  • Loading branch information
xinhaoz committed Aug 16, 2022
1 parent aed288d commit bed986b
Show file tree
Hide file tree
Showing 15 changed files with 155 additions and 60 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -247,6 +247,7 @@ const diagnosticsReportsInProgress: IStatementDiagnosticsReport[] = [
];

const aggregatedTs = Date.parse("Sep 15 2021 01:00:00 GMT") * 1e-3;
const lastUpdated = moment("Sep 15 2021 01:30:00 GMT");
const aggregationInterval = 3600; // 1 hour

const statementsPagePropsFixture: StatementsPageProps = {
Expand Down Expand Up @@ -285,6 +286,7 @@ const statementsPagePropsFixture: StatementsPageProps = {
regions: "",
nodes: "",
},
lastUpdated,
// Aggregate key values in these statements will need to change if implementation
// of 'statementKey' in appStats.ts changes.
statements: [
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,11 @@ export interface StatementsSummaryData {
stats: StatementStatistics[];
}

export const selectStatementsLastUpdated = createSelector(
sqlStatsSelector,
sqlStats => sqlStats.lastUpdated,
);

// selectApps returns the array of all apps with statement statistics present
// in the data.
export const selectApps = createSelector(sqlStatsSelector, sqlStatsState => {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -122,6 +122,7 @@ export interface StatementsPageDispatchProps {

export interface StatementsPageStateProps {
statements: AggregateStatistics[];
lastUpdated: moment.Moment | null;
timeScale: TimeScale;
statementsError: Error | null;
apps: string[];
Expand Down Expand Up @@ -182,6 +183,8 @@ export class StatementsPage extends React.Component<
StatementsPageState
> {
activateDiagnosticsRef: React.RefObject<ActivateDiagnosticsModalRef>;
refreshDataTimeout: NodeJS.Timeout;

constructor(props: StatementsPageProps) {
super(props);
const defaultState = {
Expand Down Expand Up @@ -265,6 +268,7 @@ export class StatementsPage extends React.Component<
if (this.props.onTimeScaleChange) {
this.props.onTimeScaleChange(ts);
}
this.resetPolling(ts.key);
this.setState({
startRequest: new Date(),
});
Expand All @@ -281,10 +285,29 @@ export class StatementsPage extends React.Component<
});
};

clearRefreshDataTimeout() {
if (this.refreshDataTimeout != null) {
clearTimeout(this.refreshDataTimeout);
}
}

resetPolling(key: string) {
this.clearRefreshDataTimeout();
if (key !== "Custom") {
this.refreshDataTimeout = setTimeout(
this.refreshStatements,
300000, // 5 minutes
);
}
}

refreshStatements = (): void => {
const req = statementsRequestFromProps(this.props);
this.props.refreshStatements(req);

this.resetPolling(this.props.timeScale.key);
};

resetSQLStats = (): void => {
const req = statementsRequestFromProps(this.props);
this.props.resetSQLStats(req);
Expand All @@ -297,7 +320,23 @@ export class StatementsPage extends React.Component<
this.setState({
startRequest: new Date(),
});
this.refreshStatements();

// For the first data fetch for this page, we refresh if there are:
// - Last updated is null (no statements fetched previously)
// - The time interval is not custom, i.e. we have a moving window
// in which case we poll every 5 minutes. For the first fetch we will
// calculate the next time to refresh based on when the data was last
// updated.
if (this.props.timeScale.key !== "Custom" || !this.props.lastUpdated) {
const now = moment();
const nextRefresh =
this.props.lastUpdated?.clone().add(5, "minutes") || now;
setTimeout(
this.refreshStatements,
Math.max(0, nextRefresh.diff(now, "milliseconds")),
);
}

this.props.refreshUserSQLRoles();
if (!this.props.isTenant && !this.props.hasViewActivityRedactedRole) {
this.props.refreshStatementDiagnosticsRequests();
Expand Down Expand Up @@ -345,6 +384,7 @@ export class StatementsPage extends React.Component<

componentWillUnmount(): void {
this.props.dismissAlertMessage();
this.clearRefreshDataTimeout();
}

onChangePage = (current: number): void => {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import {
selectSortSetting,
selectFilters,
selectSearch,
selectStatementsLastUpdated,
} from "./statementsPage.selectors";
import {
selectIsTenant,
Expand Down Expand Up @@ -99,6 +100,7 @@ export const ConnectedStatementsPage = withRouter(
search: selectSearch(state),
sortSetting: selectSortSetting(state),
statements: selectStatements(state, props),
lastUpdated: selectStatementsLastUpdated(state),
statementsError: selectStatementsLastError(state),
totalFingerprints: selectTotalFingerprints(state),
},
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,19 +13,22 @@ import { cockroach } from "@cockroachlabs/crdb-protobuf-client";
import { DOMAIN_NAME } from "../utils";
import { StatementsRequest } from "src/api/statementsApi";
import { TimeScale } from "../../timeScaleDropdown";
import moment from "moment";

export type StatementsResponse = cockroach.server.serverpb.StatementsResponse;

export type SQLStatsState = {
data: StatementsResponse;
lastError: Error;
valid: boolean;
lastUpdated: moment.Moment | null;
};

const initialState: SQLStatsState = {
data: null,
lastError: null,
valid: true,
valid: false,
lastUpdated: null,
};

export type UpdateTimeScalePayload = {
Expand All @@ -40,10 +43,12 @@ const sqlStatsSlice = createSlice({
state.data = action.payload;
state.valid = true;
state.lastError = null;
state.lastUpdated = moment.utc();
},
failed: (state, action: PayloadAction<Error>) => {
state.valid = false;
state.lastError = action.payload;
state.lastUpdated = moment.utc();
},
invalidated: state => {
state.valid = false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,16 +20,27 @@ import { cockroach } from "@cockroachlabs/crdb-protobuf-client";
import { getCombinedStatements } from "src/api/statementsApi";
import { resetSQLStats } from "src/api/sqlStatsApi";
import {
receivedSQLStatsSaga,
refreshSQLStatsSaga,
requestSQLStatsSaga,
resetSQLStatsSaga,
} from "./sqlStats.sagas";
import { actions, reducer, SQLStatsState } from "./sqlStats.reducer";
import { actions as sqlDetailsStatsActions } from "../statementDetails/statementDetails.reducer";
import Long from "long";
import moment from "moment";

const lastUpdated = moment();

describe("SQLStats sagas", () => {
let spy: jest.SpyInstance;
beforeAll(() => {
spy = jest.spyOn(moment, "utc").mockImplementation(() => lastUpdated);
});

afterAll(() => {
spy.mockRestore();
});

const payload = new cockroach.server.serverpb.StatementsRequest({
start: Long.fromNumber(1596816675),
end: Long.fromNumber(1596820675),
Expand Down Expand Up @@ -70,6 +81,7 @@ describe("SQLStats sagas", () => {
data: sqlStatsResponse,
lastError: null,
valid: true,
lastUpdated,
})
.run();
});
Expand All @@ -84,46 +96,27 @@ describe("SQLStats sagas", () => {
data: null,
lastError: error,
valid: false,
lastUpdated,
})
.run();
});
});

describe("receivedSQLStatsSaga", () => {
it("sets valid status to false after specified period of time", () => {
const timeout = 500;
return expectSaga(receivedSQLStatsSaga, timeout)
.delay(timeout)
.put(actions.invalidated())
.withReducer(reducer, {
data: sqlStatsResponse,
lastError: null,
valid: true,
})
.hasFinalState<SQLStatsState>({
data: sqlStatsResponse,
lastError: null,
valid: false,
})
.run(1000);
});
});

describe("resetSQLStatsSaga", () => {
const resetSQLStatsResponse =
new cockroach.server.serverpb.ResetSQLStatsResponse();

it("successfully resets SQL stats", () => {
return expectSaga(resetSQLStatsSaga, payload)
.provide([[matchers.call.fn(resetSQLStats), resetSQLStatsResponse]])
.put(actions.invalidated())
.put(sqlDetailsStatsActions.invalidateAll())
.put(actions.refresh())
.withReducer(reducer)
.hasFinalState<SQLStatsState>({
data: null,
lastError: null,
valid: false,
lastUpdated: null,
})
.run();
});
Expand All @@ -138,6 +131,7 @@ describe("SQLStats sagas", () => {
data: null,
lastError: err,
valid: false,
lastUpdated,
})
.run();
});
Expand Down
36 changes: 5 additions & 31 deletions pkg/ui/workspaces/cluster-ui/src/store/sqlStats/sqlStats.sagas.ts
Original file line number Diff line number Diff line change
Expand Up @@ -9,14 +9,7 @@
// licenses/APL.txt.

import { PayloadAction } from "@reduxjs/toolkit";
import {
all,
call,
put,
delay,
takeLatest,
takeEvery,
} from "redux-saga/effects";
import { all, call, put, takeLatest, takeEvery } from "redux-saga/effects";
import Long from "long";
import { cockroach } from "@cockroachlabs/crdb-protobuf-client";
import {
Expand All @@ -30,8 +23,6 @@ import {
UpdateTimeScalePayload,
} from "./sqlStats.reducer";
import { actions as sqlDetailsStatsActions } from "../statementDetails/statementDetails.reducer";
import { rootActions } from "../reducers";
import { CACHE_INVALIDATION_PERIOD, throttleWithReset } from "src/store/utils";
import { toDateRange } from "../../timeScaleDropdown";

export function* refreshSQLStatsSaga(action: PayloadAction<StatementsRequest>) {
Expand All @@ -49,11 +40,6 @@ export function* requestSQLStatsSaga(
}
}

export function* receivedSQLStatsSaga(delayMs: number) {
yield delay(delayMs);
yield put(sqlStatsActions.invalidated());
}

export function* updateSQLStatsTimeScaleSaga(
action: PayloadAction<UpdateTimeScalePayload>,
) {
Expand All @@ -64,43 +50,31 @@ export function* updateSQLStatsTimeScaleSaga(
value: ts,
}),
);
yield put(sqlStatsActions.invalidated());
const [start, end] = toDateRange(ts);
const req = new cockroach.server.serverpb.StatementsRequest({
combined: true,
start: Long.fromNumber(start.unix()),
end: Long.fromNumber(end.unix()),
});
yield put(sqlStatsActions.invalidated());
yield put(sqlStatsActions.refresh(req));
}

export function* resetSQLStatsSaga(action: PayloadAction<StatementsRequest>) {
try {
yield call(resetSQLStats);
yield put(sqlStatsActions.invalidated());
yield put(sqlDetailsStatsActions.invalidateAll());
yield put(sqlStatsActions.invalidated());
yield put(sqlStatsActions.refresh(action.payload));
} catch (e) {
yield put(sqlStatsActions.failed(e));
}
}

export function* sqlStatsSaga(
cacheInvalidationPeriod: number = CACHE_INVALIDATION_PERIOD,
) {
export function* sqlStatsSaga() {
yield all([
throttleWithReset(
cacheInvalidationPeriod,
sqlStatsActions.refresh,
[sqlStatsActions.invalidated, rootActions.resetState],
refreshSQLStatsSaga,
),
takeLatest(sqlStatsActions.refresh, refreshSQLStatsSaga),
takeLatest(sqlStatsActions.request, requestSQLStatsSaga),
takeLatest(
sqlStatsActions.received,
receivedSQLStatsSaga,
cacheInvalidationPeriod,
),
takeLatest(sqlStatsActions.updateTimeScale, updateSQLStatsTimeScaleSaga),
takeEvery(sqlStatsActions.reset, resetSQLStatsSaga),
]);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,8 @@ export const filters: Filters = {
nodes: "",
};

export const lastUpdated = moment();

export const data: cockroach.server.serverpb.IStatementsResponse = {
statements: [
{
Expand Down
Loading

0 comments on commit bed986b

Please sign in to comment.