Skip to content

Commit

Permalink
fix workspace context overfetching on cascading updates
Browse files Browse the repository at this point in the history
  • Loading branch information
salazarm committed Jan 9, 2025
1 parent b39de84 commit fb8e87e
Show file tree
Hide file tree
Showing 3 changed files with 103 additions and 29 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,7 @@ interface RefreshLocationsIfNeededParams
React.SetStateAction<Record<string, WorkspaceLocationNodeFragment | PythonErrorFragment>>
>;
previousLocationVersionsRef: React.MutableRefObject<Record<string, string>>;
fetchingStatusesRef: React.MutableRefObject<Record<string, Promise<any>>>;
}

interface HandleDeletedLocationsParams
Expand All @@ -113,6 +114,8 @@ interface FetchLocationDataParams
setLocationEntryData: React.Dispatch<
React.SetStateAction<Record<string, WorkspaceLocationNodeFragment | PythonErrorFragment>>
>;
locationStatuses: Record<string, LocationStatusEntryFragment>;
fetchingStatusesRef: React.MutableRefObject<Record<string, Promise<any>>>;
}

const UNLOADED_CACHED_DATA = {};
Expand All @@ -129,6 +132,7 @@ export const WorkspaceProvider = ({children}: {children: React.ReactNode}) => {
);

const previousLocationVersionsRef = useRef<Record<string, string>>({});
const fetchingStatusesRef = useRef<Record<string, Promise<any>>>({});

const {locationStatuses, loading: loadingLocationStatuses} =
useCodeLocationStatuses(localCacheIdPrefix);
Expand Down Expand Up @@ -162,6 +166,7 @@ export const WorkspaceProvider = ({children}: {children: React.ReactNode}) => {
getData,
setLocationEntryData,
previousLocationVersionsRef,
fetchingStatusesRef,
};
refreshLocationsIfNeeded(params);
}, [locationStatuses, locationEntryData, client, localCacheIdPrefix, getData, loadingCachedData]);
Expand Down Expand Up @@ -197,6 +202,8 @@ export const WorkspaceProvider = ({children}: {children: React.ReactNode}) => {
localCacheIdPrefix,
getData,
setLocationEntryData,
locationStatuses,
fetchingStatusesRef,
};
return await fetchLocationData(params);
});
Expand Down Expand Up @@ -268,6 +275,7 @@ async function refreshLocationsIfNeeded(params: RefreshLocationsIfNeededParams):
getData,
setLocationEntryData,
previousLocationVersionsRef,
fetchingStatusesRef,
} = params;

const locationsToFetch = identifyStaleLocations(
Expand All @@ -281,18 +289,20 @@ async function refreshLocationsIfNeeded(params: RefreshLocationsIfNeededParams):
}

await Promise.all(
locationsToFetch.map((location) =>
fetchLocationData({
locationsToFetch.map(async (location) => {
await fetchLocationData({
name: location.name,
client,
localCacheIdPrefix,
getData,
setLocationEntryData,
}),
),
locationStatuses,
fetchingStatusesRef,
});
previousLocationVersionsRef.current[location.name] =
locationStatuses[location.name]?.versionKey ?? '';
}),
);

previousLocationVersionsRef.current = mapLocationVersions(locationStatuses);
}

/**
Expand Down Expand Up @@ -339,29 +349,49 @@ function handleDeletedLocations(params: HandleDeletedLocationsParams): void {
async function fetchLocationData(
params: FetchLocationDataParams,
): Promise<LocationWorkspaceQuery | undefined> {
const {name, client, localCacheIdPrefix, getData, setLocationEntryData} = params;
try {
const {data} = await getData<LocationWorkspaceQuery, LocationWorkspaceQueryVariables>({
client,
query: LOCATION_WORKSPACE_QUERY,
key: `${localCacheIdPrefix}${locationWorkspaceKey(name)}`,
version: LocationWorkspaceQueryVersion,
variables: {name},
bypassCache: true,
});
const {
name,
client,
localCacheIdPrefix,
getData,
setLocationEntryData,
locationStatuses,
fetchingStatusesRef,
} = params;

const entry = data?.workspaceLocationEntryOrError;
if (entry) {
setLocationEntryData((prevData) => ({
...prevData,
[name]: entry,
}));
}
return data;
} catch (error) {
console.error(`Error fetching location data for ${name}:`, error);
const localKey = name + '@' + (locationStatuses[name]?.versionKey ?? '');
if (fetchingStatusesRef.current[localKey]) {
return fetchingStatusesRef.current[localKey];
}
return undefined;
fetchingStatusesRef.current[localKey] = new Promise(async (res) => {
try {
const {data} = await getData<LocationWorkspaceQuery, LocationWorkspaceQueryVariables>({
client,
query: LOCATION_WORKSPACE_QUERY,
key: `${localCacheIdPrefix}${locationWorkspaceKey(name)}`,
version: LocationWorkspaceQueryVersion,
variables: {name},
bypassCache: true,
});

const entry = data?.workspaceLocationEntryOrError;
if (entry) {
setLocationEntryData((prevData) => ({
...prevData,
[name]: entry,
}));
}
res(data);
} catch (error) {
console.error(`Error fetching location data for ${name}:`, error);
}
res(undefined);
});
const result = await fetchingStatusesRef.current[localKey];
requestAnimationFrame(() => {
delete fetchingStatusesRef.current[localKey];
});
return result;
}

/**
Expand Down Expand Up @@ -422,6 +452,7 @@ function identifyStaleLocations(
const entry = locationEntryData[statusEntry.name];
const locationEntry = entry?.__typename === 'WorkspaceLocationEntry' ? entry : null;
const dataVersion = locationEntry?.versionKey || '';

return currentVersion !== prevVersion || currentVersion !== dataVersion;
});
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,20 +33,23 @@ export const buildCodeLocationsStatusQuery = (

export const buildWorkspaceMocks = (
entries: WorkspaceLocationEntry[],
options: Partial<Omit<MockedResponse, 'result' | 'query' | 'variables' | 'data'>> = {},
options: Partial<Omit<MockedResponse, 'result' | 'query' | 'variables' | 'data'>> & {
cascadingUpdates?: boolean;
} = {},
) => {
return [
buildCodeLocationsStatusQuery(
entries.map((entry) =>
buildWorkspaceLocationStatusEntry({
...entry,
updateTimestamp: entry.updatedTimestamp,
versionKey: '' + entry.updatedTimestamp,
__typename: 'WorkspaceLocationStatusEntry',
}),
),
options,
),
...entries.map((entry) =>
...entries.map((entry, index) =>
buildQueryMock<LocationWorkspaceQuery, LocationWorkspaceQueryVariables>({
query: LOCATION_WORKSPACE_QUERY,
variables: {
Expand All @@ -56,6 +59,11 @@ export const buildWorkspaceMocks = (
workspaceLocationEntryOrError: entry,
},
...options,
...(options.cascadingUpdates
? {
delay: 100 * (1 + index),
}
: {}),
}),
),
];
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -510,4 +510,39 @@ describe('WorkspaceContext', () => {
expect(result.current.allRepos).toEqual([]);
expect(result.current.data).toEqual({});
});

it('Doesnt overfetch the same code location version on cascading location query responses', async () => {
const {location1, location2, location3, caches} = getLocationMocks(-1); // Initialize with outdated cache

caches.codeLocationStatusQuery.has.mockResolvedValue(false);
caches.location1.has.mockResolvedValue(false);

const mocks = buildWorkspaceMocks([location1, location2, location3], {
cascadingUpdates: true,
maxUsageCount: 9999,
});
const mockCbs = mocks.map(getMockResultFn);

// Include code location status mock a second time since we call runOnlyPendingTimersAsync twice
const {result} = renderWithMocks([...mocks, mocks[0]!]);

// Initial state
expect(result.current.allRepos).toEqual([]);
expect(result.current.data).toEqual({});
expect(result.current.loading).toEqual(true);

await waitFor(() => {
expect(result.current.loading).toEqual(false);
});

// Ensure no additional fetches were made
expect(mockCbs[1]).toHaveBeenCalledTimes(1);
expect(mockCbs[2]).toHaveBeenCalledTimes(1);
expect(mockCbs[3]).toHaveBeenCalledTimes(1);

await act(async () => {
// Exhaust any remaining tasks so they don't affect the next test.
await jest.runAllTicks();
});
});
});

0 comments on commit fb8e87e

Please sign in to comment.