Skip to content

Commit

Permalink
[2/n] Instrument sensors page for pageload + move workspace query to …
Browse files Browse the repository at this point in the history
…indexedb + speed up workspace pages (#21633)

## Summary & Motivation

- Instrument the sensors page by adding `useBlockTraceOnQueryResult`
where approriate
- Remove artificial query waterfall caused by WorkspaceRoot delaying
children from rendering until the workspace query is done loading
- I made the sensor page use the result from the workspace query if its
available, otherwise it makes its own query for now
- Improved a large workspace loading time A LOT (loading a sensor was
artificially waiting 40+ seconds, now we don't wait so thats -40 seconds
in all cases).
 

## How I Tested These Changes

Went through all the workspace pages and made sure they loaded fine.
  • Loading branch information
salazarm authored and cmpadden committed May 6, 2024
1 parent 5a39be0 commit e8207ba
Show file tree
Hide file tree
Showing 18 changed files with 142 additions and 45 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,8 @@ import {render, screen} from '@testing-library/react';
import userEvent from '@testing-library/user-event';
import {MemoryRouter} from 'react-router';

import {buildWorkspace} from '../../graphql/types';
import {buildWorkspaceContextMockedResponse} from '../../runs/__fixtures__/RunsFilterInput.fixtures';
import {WorkspaceProvider} from '../../workspace/WorkspaceContext';
import {AssetsCatalogTable} from '../AssetsCatalogTable';
import {
Expand All @@ -14,13 +16,16 @@ import {
SingleAssetQueryTrafficDashboard,
} from '../__fixtures__/AssetTables.fixtures';

const workspaceMock = buildWorkspaceContextMockedResponse(buildWorkspace({}));

const MOCKS = [
AssetCatalogTableMock,
AssetCatalogGroupTableMock,
SingleAssetQueryTrafficDashboard,
SingleAssetQueryMaterializedWithLatestRun,
SingleAssetQueryMaterializedStaleAndLate,
SingleAssetQueryLastRunFailed,
workspaceMock,
];

// This file must be mocked because Jest can't handle `import.meta.url`.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,12 +12,14 @@ import {
buildDimensionPartitionKeys,
buildMultiPartitionStatuses,
buildPartitionDefinition,
buildWorkspace,
} from '../../graphql/types';
import {CREATE_PARTITION_MUTATION} from '../../partitions/CreatePartitionDialog';
import {
AddDynamicPartitionMutation,
AddDynamicPartitionMutationVariables,
} from '../../partitions/types/CreatePartitionDialog.types';
import {buildWorkspaceContextMockedResponse} from '../../runs/__fixtures__/RunsFilterInput.fixtures';
import {buildMutationMock, buildQueryMock, getMockResultFn} from '../../testing/mocking';
import {buildRepoAddress} from '../../workspace/buildRepoAddress';
import {LaunchAssetChoosePartitionsDialog} from '../LaunchAssetChoosePartitionsDialog';
Expand All @@ -27,6 +29,8 @@ import {
} from '../types/usePartitionHealthData.types';
import {PARTITION_HEALTH_QUERY} from '../usePartitionHealthData';

const workspaceMock = buildWorkspaceContextMockedResponse(buildWorkspace({}));

describe('launchAssetChoosePartitionsDialog', () => {
it('Adding a dynamic partition when multiple assets selected', async () => {
const assetA = buildAsset('asset_a', ['test']);
Expand Down Expand Up @@ -88,6 +92,7 @@ describe('launchAssetChoosePartitionsDialog', () => {
assetASecondQueryMock,
assetBSecondQueryMock,
addPartitionMock,
workspaceMock,
]}
>
<LaunchAssetChoosePartitionsDialog
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,9 @@ import userEvent from '@testing-library/user-event';
import {CustomAlertProvider} from '../../app/CustomAlertProvider';
import {CustomConfirmationProvider} from '../../app/CustomConfirmationProvider';
import {displayNameForAssetKey} from '../../asset-graph/Utils';
import {buildWorkspace} from '../../graphql/types';
import {LaunchPartitionBackfillMutation} from '../../instance/backfill/types/BackfillUtils.types';
import {buildWorkspaceContextMockedResponse} from '../../runs/__fixtures__/RunsFilterInput.fixtures';
import {LaunchPipelineExecutionMutation} from '../../runs/types/RunUtils.types';
import {TestProvider} from '../../testing/TestProvider';
import * as WorkspaceContext from '../../workspace/WorkspaceContext';
Expand Down Expand Up @@ -44,6 +46,8 @@ import {
} from '../__fixtures__/LaunchAssetExecutionButton.fixtures';
import {asAssetKeyInput} from '../asInput';

const workspaceMock = buildWorkspaceContextMockedResponse(buildWorkspace({}));

// This file must be mocked because Jest can't handle `import.meta.url`.
jest.mock('../../graph/asyncGraphLayout', () => ({}));

Expand Down Expand Up @@ -649,6 +653,7 @@ function renderButton({
}),
buildLaunchAssetLoaderMock([MULTI_ASSET_OUT_1.assetKey, MULTI_ASSET_OUT_2.assetKey]),
buildLaunchAssetLoaderMock(assetKeys),
workspaceMock,
...(launchMock ? [launchMock] : []),
];

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import {Timestamp} from '../../app/time/Timestamp';
import {tokenForAssetKey} from '../../asset-graph/Utils';
import {AssetKeyInput, InstigationTickStatus} from '../../graphql/types';
import {TickDetailSummary} from '../../instigation/TickDetailsDialog';
import {useBlockTraceOnQueryResult} from '../../performance/TraceContext';
import {HeaderCell, HeaderRow, Inner, Row, RowCell} from '../../ui/VirtualizedTable';
import {buildRepoAddress} from '../../workspace/buildRepoAddress';
import {workspacePathFromAddress} from '../../workspace/workspacePath';
Expand Down Expand Up @@ -180,7 +181,7 @@ const AssetDetailRow = ({
evaluationId: number;
}) => {
const numMaterializations = partitionKeys?.length || 1;
const {data} = useQuery<AssetGroupAndLocationQuery, AssetGroupAndLocationQueryVariables>(
const queryResult = useQuery<AssetGroupAndLocationQuery, AssetGroupAndLocationQueryVariables>(
ASSET_GROUP_QUERY,
{
fetchPolicy: 'cache-and-network',
Expand All @@ -189,6 +190,9 @@ const AssetDetailRow = ({
},
},
);
const {data} = queryResult;
useBlockTraceOnQueryResult(queryResult, 'AssetGroupAndLocationQuery');

const asset = data?.assetOrError.__typename === 'Asset' ? data.assetOrError : null;
const definition = asset?.definition;
const repoAddress = definition
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@ import {Box, ButtonGroup, CursorHistoryControls} from '@dagster-io/ui-components
import styled from 'styled-components';

import {useQueryRefreshAtInterval} from '../../app/QueryRefresh';
import {useBlockTraceOnQueryResult} from '../../performance/TraceContext';
import {RunTable} from '../../runs/RunTable';
import {RUNS_ROOT_QUERY} from '../../runs/RunsRoot';
import {RunsRootQuery, RunsRootQueryVariables} from '../../runs/types/RunsRoot.types';
Expand Down Expand Up @@ -40,6 +41,7 @@ export const AutomaterializeRunHistoryTable = ({
query: RUNS_ROOT_QUERY,
pageSize: PAGE_SIZE,
});
useBlockTraceOnQueryResult(queryResult, 'RunsRootQuery');

useQueryRefreshAtInterval(queryResult, 15 * 1000);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ import {AssetDaemonTickFragment} from './types/AssetDaemonTicksQuery.types';
import {useQueryRefreshAtInterval} from '../../app/QueryRefresh';
import {InstigationTickStatus} from '../../graphql/types';
import {useQueryPersistedState} from '../../hooks/useQueryPersistedState';
import {useBlockTraceOnQueryResult} from '../../performance/TraceContext';
import {useCursorPaginatedQuery} from '../../runs/useCursorPaginatedQuery';
import {ASSET_SENSOR_TICKS_QUERY} from '../../sensors/AssetSensorTicksQuery';
import {
Expand Down Expand Up @@ -83,6 +84,8 @@ export const SensorAutomaterializationEvaluationHistoryTable = ({
pageSize: PAGE_SIZE,
});

useBlockTraceOnQueryResult(queryResult, 'AssetSensorTicksQuery');

// Only refresh if we're on the first page
useQueryRefreshAtInterval(queryResult, 10000, !paginationProps.hasPrevCursor);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ import userEvent from '@testing-library/user-event';
import {useContext} from 'react';
import {MemoryRouter} from 'react-router-dom';

import {__resetForJest} from '../../search/useIndexedDBCachedQuery';
import {
HIDDEN_REPO_KEYS,
WorkspaceContext,
Expand Down Expand Up @@ -41,6 +42,7 @@ describe('Repository options', () => {

afterEach(() => {
window.localStorage.clear();
__resetForJest();
});

it('Correctly displays the current repository state', async () => {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import {useQueryPersistedState} from '../hooks/useQueryPersistedState';
import {useSelectionReducer} from '../hooks/useSelectionReducer';
import {INSTANCE_HEALTH_FRAGMENT} from '../instance/InstanceHealthFragment';
import {filterPermissionedInstigationState} from '../instigation/filterPermissionedInstigationState';
import {useBlockTraceOnQueryResult} from '../performance/TraceContext';
import {SensorBulkActionMenu} from '../sensors/SensorBulkActionMenu';
import {SensorInfo} from '../sensors/SensorInfo';
import {makeSensorKey} from '../sensors/makeSensorKey';
Expand All @@ -33,6 +34,7 @@ import {WorkspaceContext} from '../workspace/WorkspaceContext';
import {buildRepoAddress} from '../workspace/buildRepoAddress';
import {repoAddressAsHumanString} from '../workspace/repoAddressAsString';
import {RepoAddress} from '../workspace/types';
import {RootWorkspaceQuery} from '../workspace/types/WorkspaceContext.types';

function toSetFilterValue(type: SensorType) {
const label = SENSOR_TYPE_META[type].name;
Expand All @@ -54,7 +56,13 @@ const SENSOR_TYPE_TO_FILTER: Partial<Record<SensorType, ReturnType<typeof toSetF
const ALL_SENSOR_TYPE_FILTERS = Object.values(SENSOR_TYPE_TO_FILTER);

export const OverviewSensors = () => {
const {allRepos, visibleRepos, loading: workspaceLoading} = useContext(WorkspaceContext);
const {
allRepos,
visibleRepos,
loading: workspaceLoading,
data: cachedData,
} = useContext(WorkspaceContext);

const repoCount = allRepos.length;
const [searchValue, setSearchValue] = useQueryPersistedState<string>({
queryKey: 'search',
Expand Down Expand Up @@ -96,7 +104,14 @@ export const OverviewSensors = () => {
notifyOnNetworkStatusChange: true,
},
);
const {data, loading} = queryResultOverview;
const {data: currentData, loading} = queryResultOverview;
const data =
currentData ??
(cachedData?.workspaceOrError.__typename === 'Workspace'
? (cachedData as Extract<typeof cachedData, {workspaceOrError: {__typename: 'Workspace'}}>)
: null);

useBlockTraceOnQueryResult(queryResultOverview, 'OverviewSensorsQuery');

const refreshState = useQueryRefreshAtInterval(queryResultOverview, FIFTEEN_SECONDS);

Expand Down Expand Up @@ -326,7 +341,7 @@ export const OverviewSensors = () => {
) : (
<>
<SensorInfo
daemonHealth={data?.instance.daemonHealth}
daemonHealth={currentData?.instance.daemonHealth}
padding={{vertical: 16, horizontal: 24}}
border="top"
/>
Expand All @@ -342,7 +357,7 @@ type RepoBucket = {
sensors: {name: string; sensorType: SensorType; sensorState: BasicInstigationStateFragment}[];
};

const buildBuckets = (data?: OverviewSensorsQuery): RepoBucket[] => {
const buildBuckets = (data?: null | OverviewSensorsQuery | RootWorkspaceQuery) => {
if (data?.workspaceOrError.__typename !== 'Workspace') {
return [];
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,10 @@ export class Dependency {
}

/** Use this to declare a dependency on an apollo query result */
export function useBlockTraceOnQueryResult(queryResult: QueryResult<any, any>, name: string) {
export function useBlockTraceOnQueryResult(
queryResult: Pick<QueryResult<any, any>, 'data' | 'error'>,
name: string,
) {
const dep = useTraceDependency(name);
const hasData = !!queryResult.data;
const hasError = !!queryResult.error;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import {
} from '../app/QueryRefresh';
import {useTrackPageView} from '../app/analytics';
import {usePageLoadTrace} from '../performance';
import {useBlockTraceOnQueryResult} from '../performance/TraceContext';
import {RUN_TABLE_RUN_FRAGMENT, RunTable} from '../runs/RunTable';
import {DagsterTag} from '../runs/RunTag';
import {RunsQueryRefetchContext} from '../runs/RunUtils';
Expand Down Expand Up @@ -114,6 +115,8 @@ export const PipelineRunsRoot = (props: Props) => {
},
});

useBlockTraceOnQueryResult(queryResult, 'PipelineRunsRootQuery');

const onAddTag = useCallback(
(token: RunFilterToken) => {
const tokenAsString = tokenToString(token);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,29 @@ import {
} from '../__fixtures__/RunsFilterInput.fixtures';
import {RunTagKeysQuery} from '../types/RunsFilterInput.types';

const workspaceMock = buildWorkspaceContextMockedResponse(
buildWorkspace({
locationEntries: [
buildWorkspaceLocationEntry({
name: 'some_workspace',
locationOrLoadError: buildRepositoryLocation({
name: 'some_location',
repositories: [
buildRepository({
name: 'some_repo',
pipelines: [
buildPipeline({
name: 'some_job',
}),
],
}),
],
}),
}),
],
}),
);

const runTagKeysMock: MockedResponse<RunTagKeysQuery> = {
request: {
query: RUN_TAG_KEYS_QUERY,
Expand Down Expand Up @@ -145,7 +168,7 @@ function TestRunsFilterInput({
);
}
return (
<MockedProvider mocks={mocks}>
<MockedProvider mocks={mocks?.length ? [workspaceMock, ...mocks] : [workspaceMock]}>
<WorkspaceProvider>
<RunsFilterInput tokens={tokens} onChange={onChange} enabledFilters={enabledFilters} />
</WorkspaceProvider>
Expand Down Expand Up @@ -196,30 +219,7 @@ describe('<RunFilterInput />', () => {
tokens={tokens}
onChange={onChange}
enabledFilters={['job']}
mocks={[
buildWorkspaceContextMockedResponse(
buildWorkspace({
locationEntries: [
buildWorkspaceLocationEntry({
name: 'some_workspace',
locationOrLoadError: buildRepositoryLocation({
name: 'some_location',
repositories: [
buildRepository({
name: 'some_repo',
pipelines: [
buildPipeline({
name: 'some_job',
}),
],
}),
],
}),
}),
],
}),
),
]}
mocks={[workspaceMock]}
/>,
);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,10 @@ jest.mock('idb-lru-cache', () => {
};
});

afterEach(() => {
jest.resetModules();
});

jest.mock('@apollo/client', () => {
const actual = jest.requireActual('@apollo/client');
const query = jest.fn().mockReturnValue({
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,7 @@ type CacheData<TQuery> = {
version: number;
};

const fetchState: Record<
let fetchState: Record<
string,
{
onFetched: ((value: any) => void)[];
Expand Down Expand Up @@ -84,9 +84,25 @@ export function useIndexedDBCachedQuery<TQuery, TVariables extends OperationVari
expiry: new Date('3000-01-01'), // never expire,
},
);
setData(data);
fetchState[key]?.onFetched.forEach((cb) => cb(queryResult));
delete fetchState[key];
const onFetched = fetchState[key]?.onFetched;
try {
setData(data);
} catch (e) {
setTimeout(() => {
throw e;
});
}
onFetched?.forEach((cb) => {
try {
cb(queryResult);
} catch (e) {
setTimeout(() => {
throw e;
});
}
});

return queryResult;
}, [client, key, lru, query, variables, version]);

Expand All @@ -96,3 +112,7 @@ export function useIndexedDBCachedQuery<TQuery, TVariables extends OperationVari
loading,
};
}

export const __resetForJest = () => {
fetchState = {};
};
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import {InstigationTickStatus} from '../graphql/types';
import {useQueryPersistedState} from '../hooks/useQueryPersistedState';
import {LiveTickTimeline} from '../instigation/LiveTickTimeline2';
import {isStuckStartedTick} from '../instigation/util';
import {useBlockTraceOnQueryResult} from '../performance/TraceContext';
import {DagsterTag} from '../runs/RunTag';
import {repoAddressAsTag} from '../workspace/repoAddressAsString';
import {RepoAddress} from '../workspace/types';
Expand Down Expand Up @@ -70,6 +71,7 @@ export const SensorPageAutomaterialize = (props: Props) => {
const [fetch, queryResult] = useLazyQuery<AssetSensorTicksQuery, AssetSensorTicksQueryVariables>(
ASSET_SENSOR_TICKS_QUERY,
);
useBlockTraceOnQueryResult(queryResult, 'AssetSensorTicksQuery');

const refresh = useCallback(
async () => await fetch({variables: getVariables()}),
Expand Down
Loading

0 comments on commit e8207ba

Please sign in to comment.