Skip to content

Commit

Permalink
ui/cluster-ui: stop unwanted fetch unbounded date range in fingerprin…
Browse files Browse the repository at this point in the history
…ts pages

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 behaviour was left over from when we only surfaced
in memory stats, thus there were previously no request parameters
required. Now that we surface persisted and in-memory stats, the
request can include  the start and end time, which is created in the
page components based on props. Due to the previous behaviour, the
automatic polling was creating empty request objects that would attempt
to fetch all data in the table. This commit moves the polling logic
directly to the page components in order to construct the correct
request object, and also prevents polling when a custom, fixed time
window is in effect.

Release note (bug fix): the statements and transaction fingerprint
no longer attempt to fetch all data every 5m when left open.

Release justification: bug fix
  • Loading branch information
xinhaoz committed Aug 15, 2022
1 parent 4561f22 commit 1d85377
Show file tree
Hide file tree
Showing 13 changed files with 119 additions and 58 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 @@ -281,10 +284,26 @@ export class StatementsPage extends React.Component<
});
};

clearRefreshDataTimeout() {
if (this.refreshDataTimeout) {
clearTimeout(this.refreshDataTimeout);
}
}

refreshStatements = (): void => {
this.clearRefreshDataTimeout();

const req = statementsRequestFromProps(this.props);
this.props.refreshStatements(req);

if (this.props.timeScale.key !== "Custom") {
this.refreshDataTimeout = setTimeout(
this.refreshStatements,
300000, // 5 minutes
);
}
};

resetSQLStats = (): void => {
const req = statementsRequestFromProps(this.props);
this.props.resetSQLStats(req);
Expand All @@ -297,7 +316,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
// udpated.
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 +380,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,23 @@ 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,
// Data is valid if not in flight and no error was encountered.
valid: true,
lastUpdated: null,
};

export type UpdateTimeScalePayload = {
Expand All @@ -40,10 +44,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,7 +20,6 @@ import { cockroach } from "@cockroachlabs/crdb-protobuf-client";
import { getCombinedStatements } from "src/api/statementsApi";
import { resetSQLStats } from "src/api/sqlStatsApi";
import {
receivedSQLStatsSaga,
refreshSQLStatsSaga,
requestSQLStatsSaga,
resetSQLStatsSaga,
Expand Down Expand Up @@ -89,41 +88,20 @@ describe("SQLStats sagas", () => {
});
});

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,
valid: true,
})
.run();
});
Expand Down
35 changes: 4 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 @@ -41,6 +32,7 @@ export function* refreshSQLStatsSaga(action: PayloadAction<StatementsRequest>) {
export function* requestSQLStatsSaga(
action: PayloadAction<StatementsRequest>,
): any {
yield put(sqlStatsActions.invalidated());
try {
const result = yield call(getCombinedStatements, action.payload);
yield put(sqlStatsActions.received(result));
Expand All @@ -49,11 +41,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,7 +51,6 @@ export function* updateSQLStatsTimeScaleSaga(
value: ts,
}),
);
yield put(sqlStatsActions.invalidated());
const [start, end] = toDateRange(ts);
const req = new cockroach.server.serverpb.StatementsRequest({
combined: true,
Expand All @@ -77,30 +63,17 @@ export function* updateSQLStatsTimeScaleSaga(
export function* resetSQLStatsSaga(action: PayloadAction<StatementsRequest>) {
try {
yield call(resetSQLStats);
yield put(sqlStatsActions.invalidated());
yield put(sqlDetailsStatsActions.invalidateAll());
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 @@ -85,6 +85,7 @@ interface TState {
export interface TransactionsPageStateProps {
columns: string[];
data: IStatementsResponse;
lastUpdated: moment.Moment | null;
timeScale: TimeScale;
error?: Error | null;
filters: Filters;
Expand Down Expand Up @@ -127,6 +128,8 @@ export class TransactionsPage extends React.Component<
TransactionsPageProps,
TState
> {
refreshDataTimeout: NodeJS.Timeout;

constructor(props: TransactionsPageProps) {
super(props);
this.state = {
Expand Down Expand Up @@ -185,17 +188,51 @@ export class TransactionsPage extends React.Component<
};
};

clearRefreshDataTimeout() {
if (this.refreshDataTimeout) {
clearTimeout(this.refreshDataTimeout);
}
}

refreshData = (): void => {
this.clearRefreshDataTimeout();

const req = statementsRequestFromProps(this.props);
this.props.refreshData(req);

if (this.props.timeScale.key !== "Custom") {
this.refreshDataTimeout = setTimeout(
this.refreshData,
300000, // 5 minutes
);
}
};

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

componentDidMount(): void {
this.refreshData();
// 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
// udpated.
if (this.props.timeScale.key !== "Custom" || !this.props.lastUpdated) {
const now = moment();
const nextRefresh =
this.props.lastUpdated?.clone().add(5, "minutes") || now;
setTimeout(
this.refreshData,
Math.max(0, nextRefresh.diff(now, "milliseconds")),
);
}
}

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

updateQueryParams(): void {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,10 @@ import {
} from "./transactionsPage.selectors";
import { selectIsTenant } from "../store/uiConfig";
import { nodeRegionsByIDSelector } from "../store/nodes";
import { selectTimeScale } from "src/statementsPage/statementsPage.selectors";
import {
selectTimeScale,
selectStatementsLastUpdated,
} from "src/statementsPage/statementsPage.selectors";
import { StatementsRequest } from "src/api/statementsApi";
import { actions as localStorageActions } from "../store/localStorage";
import { Filters } from "../queryFilter";
Expand Down Expand Up @@ -69,6 +72,7 @@ export const TransactionsPageConnected = withRouter(
...props,
columns: selectTxnColumns(state),
data: selectTransactionsData(state),
lastUpdated: selectStatementsLastUpdated(state),
timeScale: selectTimeScale(state),
error: selectTransactionsLastError(state),
filters: selectFilters(state),
Expand Down
2 changes: 1 addition & 1 deletion pkg/ui/workspaces/db-console/src/redux/apiReducers.ts
Original file line number Diff line number Diff line change
Expand Up @@ -315,7 +315,7 @@ export const refreshStores = storesReducerObj.refresh;
const queriesReducerObj = new CachedDataReducer(
api.getCombinedStatements,
"statements",
moment.duration(5, "m"),
null,
moment.duration(30, "m"),
);
export const invalidateStatements = queriesReducerObj.invalidateData;
Expand Down
Loading

0 comments on commit 1d85377

Please sign in to comment.