Skip to content

Commit

Permalink
[dagit] Fix job header “View assets” link, add missing graphQL types (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
bengotow committed Apr 20, 2022
1 parent d57d83b commit 725e6d9
Show file tree
Hide file tree
Showing 41 changed files with 431 additions and 284 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -23,14 +23,14 @@ import {RepoAddress} from '../workspace/types';

import {LiveDataForNode, displayNameForAssetKey} from './Utils';
import {SidebarAssetFragment} from './types/SidebarAssetFragment';
import {SidebarAssetQuery} from './types/SidebarAssetQuery';
import {SidebarAssetQuery, SidebarAssetQueryVariables} from './types/SidebarAssetQuery';

export const SidebarAssetInfo: React.FC<{
assetKey: AssetKey;
liveData: LiveDataForNode;
}> = ({assetKey, liveData}) => {
const partitionHealthData = usePartitionHealthData([assetKey]);
const {data} = useQuery<SidebarAssetQuery>(SIDEBAR_ASSET_QUERY, {
const {data} = useQuery<SidebarAssetQuery, SidebarAssetQueryVariables>(SIDEBAR_ASSET_QUERY, {
variables: {assetKey: {path: assetKey.path}},
fetchPolicy: 'cache-and-network',
});
Expand Down
11 changes: 7 additions & 4 deletions js_modules/dagit/packages/core/src/assets/AssetEntryRoot.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@ import {Loading} from '../ui/Loading';
import {AssetPageHeader} from './AssetPageHeader';
import {AssetView} from './AssetView';
import {AssetsCatalogTable} from './AssetsCatalogTable';
import {AssetEntryRootQuery} from './types/AssetEntryRootQuery';
import {AssetEntryRootQuery, AssetEntryRootQueryVariables} from './types/AssetEntryRootQuery';

export const AssetEntryRoot = () => {
const params = useParams();
Expand All @@ -17,9 +17,12 @@ export const AssetEntryRoot = () => {
.filter((x: string) => x)
.map(decodeURIComponent);

const queryResult = useQuery<AssetEntryRootQuery>(ASSET_ENTRY_ROOT_QUERY, {
variables: {assetKey: {path: currentPath}},
});
const queryResult = useQuery<AssetEntryRootQuery, AssetEntryRootQueryVariables>(
ASSET_ENTRY_ROOT_QUERY,
{
variables: {assetKey: {path: currentPath}},
},
);

return queryResult.loading ? (
<Page>
Expand Down
13 changes: 9 additions & 4 deletions js_modules/dagit/packages/core/src/assets/AssetWipeDialog.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,8 @@ import * as React from 'react';
import {PYTHON_ERROR_FRAGMENT} from '../app/PythonErrorInfo';
import {displayNameForAssetKey} from '../asset-graph/Utils';

import {AssetWipeMutation, AssetWipeMutationVariables} from './types/AssetWipeMutation';

interface AssetKey {
path: string[];
}
Expand All @@ -16,10 +18,13 @@ export const AssetWipeDialog: React.FC<{
onComplete: (assetKeys: AssetKey[]) => void;
requery?: RefetchQueriesFunction;
}> = ({assetKeys, isOpen, onClose, onComplete, requery}) => {
const [requestWipe] = useMutation(ASSET_WIPE_MUTATION, {
variables: {assetKeys: assetKeys.map((key) => ({path: key.path || []}))},
refetchQueries: requery,
});
const [requestWipe] = useMutation<AssetWipeMutation, AssetWipeMutationVariables>(
ASSET_WIPE_MUTATION,
{
variables: {assetKeys: assetKeys.map((key) => ({path: key.path || []}))},
refetchQueries: requery,
},
);

const wipe = async () => {
if (!assetKeys.length) {
Expand Down
14 changes: 9 additions & 5 deletions js_modules/dagit/packages/core/src/gantt/RunGroupPanel.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ import {RunStateSummary, RunTime, RUN_TIME_FRAGMENT} from '../runs/RunUtils';

import {
RunGroupPanelQuery,
RunGroupPanelQueryVariables,
RunGroupPanelQuery_runGroupOrError_RunGroup_runs,
} from './types/RunGroupPanelQuery';

Expand All @@ -27,11 +28,14 @@ export const RunGroupPanel: React.FC<{runId: string; runStatusLastChangedAt: num
runId,
runStatusLastChangedAt,
}) => {
const {data, refetch} = useQuery<RunGroupPanelQuery>(RUN_GROUP_PANEL_QUERY, {
variables: {runId},
fetchPolicy: 'cache-and-network',
pollInterval: 15000, // 15s
});
const {data, refetch} = useQuery<RunGroupPanelQuery, RunGroupPanelQueryVariables>(
RUN_GROUP_PANEL_QUERY,
{
variables: {runId},
fetchPolicy: 'cache-and-network',
pollInterval: 15000, // 15s
},
);

// Because the RunGroupPanel makes it's own query for the runs and their statuses,
// the log + gantt chart UI can show that the run is "completed" for up to 15s before
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@ import {doneStatuses} from '../runs/RunStatuses';
import {TerminationDialog} from '../runs/TerminationDialog';
import {BulkActionStatus} from '../types/globalTypes';

import {CancelBackfill, CancelBackfillVariables} from './types/CancelBackfill';
import {InstanceBackfillsQuery_partitionBackfillsOrError_PartitionBackfills_results} from './types/InstanceBackfillsQuery';

type Backfill = InstanceBackfillsQuery_partitionBackfillsOrError_PartitionBackfills_results;
Expand All @@ -17,7 +18,9 @@ interface Props {
onComplete: () => void;
}
export const BackfillTerminationDialog = ({backfill, onClose, onComplete}: Props) => {
const [cancelBackfill] = useMutation(CANCEL_BACKFILL_MUTATION);
const [cancelBackfill] = useMutation<CancelBackfill, CancelBackfillVariables>(
CANCEL_BACKFILL_MUTATION,
);
const [isSubmitting, setIsSubmitting] = React.useState(false);

if (!backfill) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@ import {
InstanceBackfillsQuery_partitionBackfillsOrError_PartitionBackfills_results_runs,
} from './types/InstanceBackfillsQuery';
import {InstanceHealthForBackfillsQuery} from './types/InstanceHealthForBackfillsQuery';
import {resumeBackfill, resumeBackfillVariables} from './types/resumeBackfill';

type Backfill = InstanceBackfillsQuery_partitionBackfillsOrError_PartitionBackfills_results;
type BackfillRun = InstanceBackfillsQuery_partitionBackfillsOrError_PartitionBackfills_results_runs;
Expand Down Expand Up @@ -170,7 +171,9 @@ const INSTANCE_HEALTH_FOR_BACKFILLS_QUERY = gql`

const BackfillTable = ({backfills, refetch}: {backfills: Backfill[]; refetch: () => void}) => {
const [terminationBackfill, setTerminationBackfill] = React.useState<Backfill>();
const [resumeBackfill] = useMutation(RESUME_BACKFILL_MUTATION);
const [resumeBackfill] = useMutation<resumeBackfill, resumeBackfillVariables>(
RESUME_BACKFILL_MUTATION,
);
const [cancelRunBackfill, setCancelRunBackfill] = React.useState<Backfill>();
const {canCancelPartitionBackfill} = usePermissions();

Expand Down Expand Up @@ -199,7 +202,7 @@ const BackfillTable = ({backfills, refetch}: {backfills: Backfill[]; refetch: ()
icon: 'error',
intent: 'danger',
});
} else {
} else if (data && data.resumePartitionBackfill.__typename === 'PythonError') {
const error = data.resumePartitionBackfill;
SharedToaster.show({
message: <div>An unexpected error occurred. This backfill was not retried.</div>,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -7,15 +7,21 @@ import {Link} from 'react-router-dom';
import {failedStatuses, inProgressStatuses} from '../runs/RunStatuses';
import {StepEventStatus} from '../types/globalTypes';

import {StepSummaryForRunQuery} from './types/StepSummaryForRunQuery';
import {
StepSummaryForRunQuery,
StepSummaryForRunQueryVariables,
} from './types/StepSummaryForRunQuery';

interface Props {
runId: string;
}

export const StepSummaryForRun = (props: Props) => {
const {runId} = props;
const {data} = useQuery<StepSummaryForRunQuery>(STEP_SUMMARY_FOR_RUN_QUERY, {variables: {runId}});
const {data} = useQuery<StepSummaryForRunQuery, StepSummaryForRunQueryVariables>(
STEP_SUMMARY_FOR_RUN_QUERY,
{variables: {runId}},
);

const run = data?.pipelineRunOrError;
const status = run?.__typename === 'Run' ? run.status : null;
Expand Down
25 changes: 14 additions & 11 deletions js_modules/dagit/packages/core/src/instigation/Unloadable.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -8,9 +8,9 @@ import {
STOP_SCHEDULE_MUTATION,
} from '../schedules/ScheduleMutations';
import {humanCronString} from '../schedules/humanCronString';
import {StopSchedule} from '../schedules/types/StopSchedule';
import {StopSchedule, StopScheduleVariables} from '../schedules/types/StopSchedule';
import {displaySensorMutationErrors, STOP_SENSOR_MUTATION} from '../sensors/SensorMutations';
import {StopSensor} from '../sensors/types/StopSensor';
import {StopSensor, StopSensorVariables} from '../sensors/types/StopSensor';
import {InstigationStatus, InstigationType} from '../types/globalTypes';
import {InstigatorSelectorInformation} from '../workspace/RepositoryInformation';

Expand Down Expand Up @@ -125,9 +125,12 @@ const UnloadableScheduleInfo = () => (
const SensorStateRow = ({sensorState}: {sensorState: InstigationStateFragment}) => {
const {id, selectorId, name, status, ticks} = sensorState;

const [stopSensor, {loading: toggleOffInFlight}] = useMutation<StopSensor>(STOP_SENSOR_MUTATION, {
onCompleted: displaySensorMutationErrors,
});
const [stopSensor, {loading: toggleOffInFlight}] = useMutation<StopSensor, StopSensorVariables>(
STOP_SENSOR_MUTATION,
{
onCompleted: displaySensorMutationErrors,
},
);
const confirm = useConfirmation();

const onChangeSwitch = async () => {
Expand Down Expand Up @@ -180,12 +183,12 @@ const SensorStateRow = ({sensorState}: {sensorState: InstigationStateFragment})
const ScheduleStateRow: React.FC<{
scheduleState: InstigationStateFragment;
}> = ({scheduleState}) => {
const [stopSchedule, {loading: toggleOffInFlight}] = useMutation<StopSchedule>(
STOP_SCHEDULE_MUTATION,
{
onCompleted: displayScheduleMutationErrors,
},
);
const [stopSchedule, {loading: toggleOffInFlight}] = useMutation<
StopSchedule,
StopScheduleVariables
>(STOP_SCHEDULE_MUTATION, {
onCompleted: displayScheduleMutationErrors,
});
const confirm = useConfirmation();
const {id, selectorId, name, ticks, status, typeSpecificData} = scheduleState;
const latestTick = ticks.length > 0 ? ticks[0] : null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import {
} from './types/ConfigEditorGeneratorPipelineFragment';
import {
ConfigPartitionsQuery,
ConfigPartitionsQueryVariables,
ConfigPartitionsQuery_partitionSetOrError_PartitionSet_partitionsOrError_Partitions_results,
} from './types/ConfigPartitionsQuery';

Expand Down Expand Up @@ -146,10 +147,13 @@ const ConfigEditorPartitionPicker: React.FC<ConfigEditorPartitionPickerProps> =
(props) => {
const {partitionSetName, value, onSelect, repoAddress} = props;
const repositorySelector = repoAddressToSelector(repoAddress);
const {data, loading} = useQuery<ConfigPartitionsQuery>(CONFIG_PARTITIONS_QUERY, {
variables: {repositorySelector, partitionSetName},
fetchPolicy: 'network-only',
});
const {data, loading} = useQuery<ConfigPartitionsQuery, ConfigPartitionsQueryVariables>(
CONFIG_PARTITIONS_QUERY,
{
variables: {repositorySelector, partitionSetName},
fetchPolicy: 'network-only',
},
);

const [sortOrder, setSortOrder] = React.useState('asc');

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,9 +25,10 @@ interface LaunchRootExecutionButtonProps {

export const LaunchRootExecutionButton: React.FC<LaunchRootExecutionButtonProps> = (props) => {
const {canLaunchPipelineExecution} = usePermissions();
const [launchPipelineExecution] = useMutation<LaunchPipelineExecution>(
LAUNCH_PIPELINE_EXECUTION_MUTATION,
);
const [launchPipelineExecution] = useMutation<
LaunchPipelineExecution,
LaunchPipelineExecutionVariables
>(LAUNCH_PIPELINE_EXECUTION_MUTATION);
const logTelemetry = useTelemetryAction();
const history = useHistory();

Expand Down
15 changes: 9 additions & 6 deletions js_modules/dagit/packages/core/src/launchpad/LaunchpadRoot.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@ import {
} from './ConfigEditorConfigPicker';
import {LaunchpadSessionError} from './LaunchpadSessionError';
import {LaunchpadSessionLoading} from './LaunchpadSessionLoading';
import {LaunchpadRootQuery} from './types/LaunchpadRootQuery';
import {LaunchpadRootQuery, LaunchpadRootQueryVariables} from './types/LaunchpadRootQuery';

const LaunchpadSessionContainer = React.lazy(() => import('./LaunchpadSessionContainer'));

Expand Down Expand Up @@ -49,11 +49,14 @@ const LaunchpadAllowedRoot: React.FC<Props> = (props) => {

const {name: repositoryName, location: repositoryLocationName} = repoAddress;

const result = useQuery<LaunchpadRootQuery>(PIPELINE_EXECUTION_ROOT_QUERY, {
variables: {repositoryName, repositoryLocationName, pipelineName},
fetchPolicy: 'cache-and-network',
partialRefetch: true,
});
const result = useQuery<LaunchpadRootQuery, LaunchpadRootQueryVariables>(
PIPELINE_EXECUTION_ROOT_QUERY,
{
variables: {repositoryName, repositoryLocationName, pipelineName},
fetchPolicy: 'cache-and-network',
partialRefetch: true,
},
);

const pipelineOrError = result?.data?.pipelineOrError;
const partitionSetsOrError = result?.data?.partitionSetsOrError;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,10 @@ import {
} from './types/ConfigPartitionSelectionQuery';
import {LaunchpadSessionContainerPartitionSetsFragment} from './types/LaunchpadSessionContainerPartitionSetsFragment';
import {LaunchpadSessionContainerPipelineFragment} from './types/LaunchpadSessionContainerPipelineFragment';
import {PipelineExecutionConfigSchemaQuery} from './types/PipelineExecutionConfigSchemaQuery';
import {
PipelineExecutionConfigSchemaQuery,
PipelineExecutionConfigSchemaQueryVariables,
} from './types/PipelineExecutionConfigSchemaQuery';
import {PreviewConfigQuery, PreviewConfigQueryVariables} from './types/PreviewConfigQuery';

const YAML_SYNTAX_INVALID = `The YAML you provided couldn't be parsed. Please fix the syntax errors and try again.`;
Expand Down Expand Up @@ -181,14 +184,14 @@ const LaunchpadSessionContainer: React.FC<LaunchpadSessionContainerProps> = (pro
solidSelection: currentSession?.solidSelection || undefined,
};

const configResult = useQuery<PipelineExecutionConfigSchemaQuery>(
PIPELINE_EXECUTION_CONFIG_SCHEMA_QUERY,
{
variables: {selector: pipelineSelector, mode: currentSession?.mode},
fetchPolicy: 'cache-and-network',
partialRefetch: true,
},
);
const configResult = useQuery<
PipelineExecutionConfigSchemaQuery,
PipelineExecutionConfigSchemaQueryVariables
>(PIPELINE_EXECUTION_CONFIG_SCHEMA_QUERY, {
variables: {selector: pipelineSelector, mode: currentSession?.mode},
fetchPolicy: 'cache-and-network',
partialRefetch: true,
});

const configSchemaOrError = configResult?.data?.runConfigSchemaOrError;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@ import {workspacePathFromAddress} from '../workspace/workspacePath';

import {LaunchpadSessionError} from './LaunchpadSessionError';
import {LaunchpadSessionLoading} from './LaunchpadSessionLoading';
import {ConfigForRunQuery} from './types/ConfigForRunQuery';
import {ConfigForRunQuery, ConfigForRunQueryVariables} from './types/ConfigForRunQuery';

export const LaunchpadSetupFromRunRoot: React.FC<{repoAddress: RepoAddress}> = (props) => {
const {repoAddress} = props;
Expand Down Expand Up @@ -64,7 +64,10 @@ const LaunchpadSetupFromRunAllowedRoot: React.FC<Props> = (props) => {

const [storageData, onSave] = useExecutionSessionStorage(repoAddress, pipelineName);

const {data, loading} = useQuery<ConfigForRunQuery>(CONFIG_FOR_RUN_QUERY, {variables: {runId}});
const {data, loading} = useQuery<ConfigForRunQuery, ConfigForRunQueryVariables>(
CONFIG_FOR_RUN_QUERY,
{variables: {runId}},
);
const runOrError = data?.runOrError;
const run = runOrError?.__typename === 'Run' ? runOrError : null;

Expand Down
13 changes: 8 additions & 5 deletions js_modules/dagit/packages/core/src/launchpad/OpSelector.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@ import {isThisThingAJob, useRepository} from '../workspace/WorkspaceContext';
import {repoAddressToSelector} from '../workspace/repoAddressToSelector';
import {RepoAddress} from '../workspace/types';

import {OpSelectorQuery} from './types/OpSelectorQuery';
import {OpSelectorQuery, OpSelectorQueryVariables} from './types/OpSelectorQuery';

interface IOpSelectorProps {
pipelineName: string;
Expand Down Expand Up @@ -69,10 +69,13 @@ export const OpSelector = (props: IOpSelectorProps) => {
const selector = {...repoAddressToSelector(repoAddress), pipelineName};
const repo = useRepository(repoAddress);
const isJob = isThisThingAJob(repo, pipelineName);
const {data, loading} = useQuery<OpSelectorQuery>(SOLID_SELECTOR_QUERY, {
variables: {selector, requestScopeHandleID: flattenGraphs ? undefined : ''},
fetchPolicy: 'cache-and-network',
});
const {data, loading} = useQuery<OpSelectorQuery, OpSelectorQueryVariables>(
SOLID_SELECTOR_QUERY,
{
variables: {selector, requestScopeHandleID: flattenGraphs ? undefined : ''},
fetchPolicy: 'cache-and-network',
},
);

const query = props.query || '*';

Expand Down
9 changes: 6 additions & 3 deletions js_modules/dagit/packages/core/src/nav/JobMetadata.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@ import {RepoAddress} from '../workspace/types';
import {LatestRunTag} from './LatestRunTag';
import {ScheduleOrSensorTag} from './ScheduleOrSensorTag';
import {JobMetadataFragment as Job} from './types/JobMetadataFragment';
import {JobMetadataQuery} from './types/JobMetadataQuery';
import {JobMetadataQuery, JobMetadataQueryVariables} from './types/JobMetadataQuery';
import {RunMetadataFragment} from './types/RunMetadataFragment';

interface Props {
Expand All @@ -22,8 +22,11 @@ interface Props {
export const JobMetadata: React.FC<Props> = (props) => {
const {pipelineName, repoAddress} = props;

const {data} = useQuery<JobMetadataQuery>(JOB_METADATA_QUERY, {
const {data} = useQuery<JobMetadataQuery, JobMetadataQueryVariables>(JOB_METADATA_QUERY, {
variables: {
runsFilter: {
pipelineName,
},
params: {
pipelineName,
repositoryName: repoAddress.name,
Expand Down Expand Up @@ -191,7 +194,7 @@ const JOB_METADATA_FRAGMENT = gql`
`;

const JOB_METADATA_QUERY = gql`
query JobMetadataQuery($params: PipelineSelector!, $runsFilter: RunsFilter) {
query JobMetadataQuery($params: PipelineSelector!, $runsFilter: RunsFilter!) {
pipelineOrError(params: $params) {
... on Pipeline {
id
Expand Down

0 comments on commit 725e6d9

Please sign in to comment.