diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/asset_condition_evaluations.py b/python_modules/dagster-graphql/dagster_graphql/schema/asset_condition_evaluations.py index d5626cc10c1e4..427f9d3ecd072 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/asset_condition_evaluations.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/asset_condition_evaluations.py @@ -82,8 +82,8 @@ class GraphenePartitionedAssetConditionEvaluationNode(graphene.ObjectType): startTimestamp = graphene.Field(graphene.Float) endTimestamp = graphene.Field(graphene.Float) - numTrue = graphene.NonNull(graphene.Int) - numCandidates = graphene.Field(graphene.Int) + numTrue = graphene.NonNull(graphene.Int) # Can't have BigInt number of conditions + numCandidates = graphene.Field(graphene.Int) # Same childUniqueIds = non_null_list(graphene.String) @@ -216,8 +216,8 @@ class GrapheneAutomationConditionEvaluationNode(graphene.ObjectType): startTimestamp = graphene.Field(graphene.Float) endTimestamp = graphene.Field(graphene.Float) - numTrue = graphene.NonNull(graphene.Int) - numCandidates = graphene.Field(graphene.Int) + numTrue = graphene.NonNull(graphene.Int) # Same + numCandidates = graphene.Field(graphene.Int) # Same isPartitioned = graphene.NonNull(graphene.Boolean) @@ -252,7 +252,7 @@ class GrapheneAssetConditionEvaluationRecord(graphene.ObjectType): timestamp = graphene.NonNull(graphene.Float) assetKey = graphene.NonNull(GrapheneAssetKey) - numRequested = graphene.NonNull(graphene.Int) + numRequested = graphene.NonNull(graphene.Int) # Same startTimestamp = graphene.Field(graphene.Float) endTimestamp = graphene.Field(graphene.Float) diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/asset_graph.py b/python_modules/dagster-graphql/dagster_graphql/schema/asset_graph.py index 4b59f18050e30..a92c18d975bd3 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/asset_graph.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/asset_graph.py @@ -283,8 +283,8 @@ class GrapheneAssetNode(graphene.ObjectType): partitionKeys = non_null_list(graphene.String) partitionKeysByDimension = graphene.Field( non_null_list(GrapheneDimensionPartitionKeys), - startIdx=graphene.Int(), - endIdx=graphene.Int(), + startIdx=graphene.Int(), # Used to select a range of time partitions; shouldn't go into billions + endIdx=graphene.Int(), # Same ) repository = graphene.NonNull(lambda: external.GrapheneRepository) required_resources = non_null_list(GrapheneResourceRequirement) diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/auto_materialize_asset_evaluations.py b/python_modules/dagster-graphql/dagster_graphql/schema/auto_materialize_asset_evaluations.py index d1e9c4d7f0025..689bd3b080e4f 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/auto_materialize_asset_evaluations.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/auto_materialize_asset_evaluations.py @@ -211,9 +211,9 @@ def create_graphene_auto_materialize_rules_with_rule_evaluations( class GrapheneAutoMaterializeAssetEvaluationRecord(graphene.ObjectType): id = graphene.NonNull(graphene.ID) evaluationId = graphene.NonNull(graphene.ID) - numRequested = graphene.NonNull(graphene.Int) - numSkipped = graphene.NonNull(graphene.Int) - numDiscarded = graphene.NonNull(graphene.Int) + numRequested = graphene.NonNull(graphene.Int) # Makes no sense to have a BigInt quantity + numSkipped = graphene.NonNull(graphene.Int) # Same + numDiscarded = graphene.NonNull(graphene.Int) # Same rulesWithRuleEvaluations = non_null_list(GrapheneAutoMaterializeRuleWithRuleEvaluations) timestamp = graphene.NonNull(graphene.Float) runIds = non_null_list(graphene.String) diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/auto_materialize_policy.py b/python_modules/dagster-graphql/dagster_graphql/schema/auto_materialize_policy.py index 835666a77a371..343c3b8e23ede 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/auto_materialize_policy.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/auto_materialize_policy.py @@ -33,7 +33,9 @@ def __init__(self, description: str, decision_type: AutoMaterializeDecisionType) class GrapheneAutoMaterializePolicy(graphene.ObjectType): policyType = graphene.NonNull(graphene.Enum.from_enum(AutoMaterializePolicyType)) - maxMaterializationsPerMinute = graphene.Int() + maxMaterializationsPerMinute = ( + graphene.Int() + ) # I assume you have other problems if you need a BigInt limit here rules = non_null_list(GrapheneAutoMaterializeRule) class Meta: diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/backfill.py b/python_modules/dagster-graphql/dagster_graphql/schema/backfill.py index 81aba8d6a52e6..7da37dce95477 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/backfill.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/backfill.py @@ -320,8 +320,8 @@ class Meta: ) partitionNames = graphene.List(graphene.NonNull(graphene.String)) isValidSerialization = graphene.NonNull(graphene.Boolean) - numPartitions = graphene.Field(graphene.Int) - numCancelable = graphene.NonNull(graphene.Int) + numPartitions = graphene.Field(graphene.Int) # BigInt partitions would mean a problem + numCancelable = graphene.NonNull(graphene.Int) # Same fromFailure = graphene.NonNull(graphene.Boolean) reexecutionSteps = graphene.List(graphene.NonNull(graphene.String)) assetSelection = graphene.List(graphene.NonNull(GrapheneAssetKey)) @@ -345,14 +345,15 @@ class Meta: partitionSet = graphene.Field("dagster_graphql.schema.partition_sets.GraphenePartitionSet") runs = graphene.Field( non_null_list("dagster_graphql.schema.pipelines.pipeline.GrapheneRun"), - limit=graphene.Int(), + limit=graphene.Int(), # Don't see this needing BigInt ) unfinishedRuns = graphene.Field( non_null_list("dagster_graphql.schema.pipelines.pipeline.GrapheneRun"), - limit=graphene.Int(), + limit=graphene.Int(), # Same ) cancelableRuns = graphene.Field( - non_null_list("dagster_graphql.schema.pipelines.pipeline.GrapheneRun"), limit=graphene.Int() + non_null_list("dagster_graphql.schema.pipelines.pipeline.GrapheneRun"), + limit=graphene.Int(), # Same ) error = graphene.Field(GraphenePythonError) partitionStatuses = graphene.Field( @@ -746,7 +747,7 @@ class Meta: class GrapheneBackfillPolicy(graphene.ObjectType): - maxPartitionsPerRun = graphene.Field(graphene.Int()) + maxPartitionsPerRun = graphene.Field(graphene.Int()) # BigInt is too many partitions description = graphene.NonNull(graphene.String) policyType = graphene.NonNull(GrapheneBackfillPolicyType) diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/instance.py b/python_modules/dagster-graphql/dagster_graphql/schema/instance.py index 715f15b6b41cb..872498c8ffd6d 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/instance.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/instance.py @@ -114,7 +114,7 @@ class GraphenePendingConcurrencyStep(graphene.ObjectType): stepKey = graphene.NonNull(graphene.String) enqueuedTimestamp = graphene.NonNull(graphene.Float) assignedTimestamp = graphene.Float() - priority = graphene.Int() + priority = graphene.Int() # Can't imagine anybody needs to set BigInt priorities class Meta: name = "PendingConcurrencyStep" @@ -133,14 +133,18 @@ def __init__(self, pending_step_info: PendingStepInfo): class GrapheneConcurrencyKeyInfo(graphene.ObjectType): concurrencyKey = graphene.NonNull(graphene.String) - slotCount = graphene.NonNull(graphene.Int) + slotCount = graphene.NonNull( + graphene.Int + ) # I'm guessing lots of other things will go wrong if you have such massive slot counts that you need BigInt claimedSlots = non_null_list(GrapheneClaimedConcurrencySlot) pendingSteps = non_null_list(GraphenePendingConcurrencyStep) - activeSlotCount = graphene.NonNull(graphene.Int) + activeSlotCount = graphene.NonNull(graphene.Int) # Same activeRunIds = non_null_list(graphene.String) - pendingStepCount = graphene.NonNull(graphene.Int) + pendingStepCount = graphene.NonNull( + graphene.Int + ) # Again, if you have so many pending steps that you need BigInt, you have bigger problems pendingStepRunIds = non_null_list(graphene.String) - assignedStepCount = graphene.NonNull(graphene.Int) + assignedStepCount = graphene.NonNull(graphene.Int) # Same assignedStepRunIds = non_null_list(graphene.String) class Meta: @@ -195,7 +199,9 @@ def resolve_assignedStepRunIds(self, graphene_info: ResolveInfo): class GrapheneRunQueueConfig(graphene.ObjectType): - maxConcurrentRuns = graphene.NonNull(graphene.Int) + maxConcurrentRuns = graphene.NonNull( + graphene.Int + ) # I'm guessing you won't have so many concurrent runs that you need BigInt (without causing other issues) tagConcurrencyLimitsYaml = graphene.String() isOpConcurrencyAware = graphene.Boolean() @@ -235,8 +241,8 @@ class GrapheneInstance(graphene.ObjectType): hasInfo = graphene.NonNull(graphene.Boolean) autoMaterializePaused = graphene.NonNull(graphene.Boolean) supportsConcurrencyLimits = graphene.NonNull(graphene.Boolean) - minConcurrencyLimitValue = graphene.NonNull(graphene.Int) - maxConcurrencyLimitValue = graphene.NonNull(graphene.Int) + minConcurrencyLimitValue = graphene.NonNull(graphene.Int) # Same as above + maxConcurrencyLimitValue = graphene.NonNull(graphene.Int) # Same concurrencyLimits = non_null_list(GrapheneConcurrencyKeyInfo) concurrencyLimit = graphene.Field( graphene.NonNull(GrapheneConcurrencyKeyInfo), diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/instigation.py b/python_modules/dagster-graphql/dagster_graphql/schema/instigation.py index e58cc9b863cae..07bb11c1c2511 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/instigation.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/instigation.py @@ -248,7 +248,9 @@ class GrapheneInstigationTick(graphene.ObjectType): dynamicPartitionsRequestResults = non_null_list(GrapheneDynamicPartitionsRequestResult) endTimestamp = graphene.Field(graphene.Float) requestedAssetKeys = non_null_list(GrapheneAssetKey) - requestedAssetMaterializationCount = graphene.NonNull(graphene.Int) + requestedAssetMaterializationCount = graphene.NonNull( + graphene.Int + ) # Can't see why this could go into BigInt range, unless it's cumulative? requestedMaterializationsForAssets = non_null_list(GrapheneRequestedMaterializationsForAsset) autoMaterializeAssetEvaluationId = graphene.Field(graphene.ID) instigationType = graphene.NonNull(GrapheneInstigationType) @@ -538,15 +540,15 @@ class GrapheneInstigationState(graphene.ObjectType): non_null_list("dagster_graphql.schema.pipelines.pipeline.GrapheneRun"), limit=graphene.Int(), ) - runsCount = graphene.NonNull(graphene.Int) + runsCount = graphene.NonNull(graphene.Int) # Shouldn't go into billions on a single account tick = graphene.Field( graphene.NonNull(GrapheneInstigationTick), tickId=graphene.NonNull(graphene.ID), ) ticks = graphene.Field( non_null_list(GrapheneInstigationTick), - dayRange=graphene.Int(), - dayOffset=graphene.Int(), + dayRange=graphene.Int(), # Days can't go into BigInt range + dayOffset=graphene.Int(), # Same limit=graphene.Int(), cursor=graphene.String(), statuses=graphene.List(graphene.NonNull(GrapheneInstigationTickStatus)), @@ -554,7 +556,9 @@ class GrapheneInstigationState(graphene.ObjectType): afterTimestamp=graphene.Float(), ) nextTick = graphene.Field(GrapheneDryRunInstigationTick) - runningCount = graphene.NonNull(graphene.Int) # remove with cron scheduler + runningCount = graphene.NonNull( + graphene.Int + ) # remove with cron scheduler # Shouldn't go into billions on a single account hasStartPermission = graphene.NonNull(graphene.Boolean) hasStopPermission = graphene.NonNull(graphene.Boolean) diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/logs/events.py b/python_modules/dagster-graphql/dagster_graphql/schema/logs/events.py index 46a1dbc8bf0e2..60ea65ad546ca 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/logs/events.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/logs/events.py @@ -167,7 +167,9 @@ class Meta: class GrapheneExecutionStepUpForRetryEvent(graphene.ObjectType): - secondsToWait = graphene.Field(graphene.Int) + secondsToWait = graphene.Field( + graphene.Int + ) # Guess nobody would be waiting for more than 2.1 billion seconds/68 years class Meta: interfaces = (GrapheneMessageEvent, GrapheneStepEvent, GrapheneErrorEvent) @@ -309,7 +311,7 @@ class Meta: externalUrl = graphene.String() externalStdoutUrl = graphene.String() externalStderrUrl = graphene.String() - pid = graphene.Int() + pid = graphene.Int() # pids don't go into the billions # legacy name for compute log file key... required for back-compat reasons, but has been # renamed to fileKey for newer versions of the Dagster UI logKey = graphene.NonNull(graphene.String) diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/partition_sets.py b/python_modules/dagster-graphql/dagster_graphql/schema/partition_sets.py index 6610edfd4fcf4..af523e7900b7c 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/partition_sets.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/partition_sets.py @@ -124,7 +124,7 @@ class Meta: class GraphenePartitionStatusCounts(graphene.ObjectType): runStatus = graphene.NonNull(GrapheneRunStatus) - count = graphene.NonNull(graphene.Int) + count = graphene.NonNull(graphene.Int) # Can't have a BigInt number of partitions class Meta: name = "PartitionStatusCounts" @@ -148,10 +148,10 @@ class Meta: name = "AssetPartitionsStatusCounts" assetKey = graphene.NonNull(GrapheneAssetKey) - numPartitionsTargeted = graphene.NonNull(graphene.Int) - numPartitionsInProgress = graphene.NonNull(graphene.Int) - numPartitionsMaterialized = graphene.NonNull(graphene.Int) - numPartitionsFailed = graphene.NonNull(graphene.Int) + numPartitionsTargeted = graphene.NonNull(graphene.Int) # Same + numPartitionsInProgress = graphene.NonNull(graphene.Int) # Same + numPartitionsMaterialized = graphene.NonNull(graphene.Int) # Same + numPartitionsFailed = graphene.NonNull(graphene.Int) # Same class GrapheneUnpartitionedAssetStatus(graphene.ObjectType): @@ -237,7 +237,7 @@ class GraphenePartition(graphene.ObjectType): non_null_list(GrapheneRun), filter=graphene.Argument(GrapheneRunsFilter), cursor=graphene.String(), - limit=graphene.Int(), + limit=graphene.Int(), # I guess BigInt number of runs would be unreasonable? Not sure on this one ) status = graphene.Field(GrapheneRunStatus) @@ -332,7 +332,7 @@ class GraphenePartitionSet(graphene.ObjectType): partitionsOrError = graphene.Field( graphene.NonNull(GraphenePartitionsOrError), cursor=graphene.String(), - limit=graphene.Int(), + limit=graphene.Int(), # Can't have BigInt number of partitions reverse=graphene.Boolean(), ) partition = graphene.Field(GraphenePartition, partition_name=graphene.NonNull(graphene.String)) @@ -342,7 +342,7 @@ class GraphenePartitionSet(graphene.ObjectType): backfills = graphene.Field( non_null_list(GraphenePartitionBackfill), cursor=graphene.String(), - limit=graphene.Int(), + limit=graphene.Int(), # Can't have BigInt number of backfills ) class Meta: diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/config.py b/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/config.py index dda5af8fcd65c..bffd7fbb1050d 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/config.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/config.py @@ -26,7 +26,7 @@ class GrapheneEvaluationStackListItemEntry(graphene.ObjectType): - list_index = graphene.NonNull(graphene.Int) + list_index = graphene.NonNull(graphene.Int) # Hard to imagine list index in billions class Meta: name = "EvaluationStackListItemEntry" diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline.py b/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline.py index be0b522e2da82..ff55831bffb54 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline.py @@ -188,10 +188,12 @@ class Meta: class GraphenePartitionStats(graphene.ObjectType): - numMaterialized = graphene.NonNull(graphene.Int) - numPartitions = graphene.NonNull(graphene.Int) - numFailed = graphene.NonNull(graphene.Int) - numMaterializing = graphene.NonNull(graphene.Int) + numMaterialized = graphene.NonNull( + graphene.Int + ) # Unless this is over time, I'm not sure how you could get into the BigInt range + numPartitions = graphene.NonNull(graphene.Int) # Same as above + numFailed = graphene.NonNull(graphene.Int) # Same as above + numMaterializing = graphene.NonNull(graphene.Int) # Same as above class Meta: name = "PartitionStats" @@ -203,7 +205,7 @@ class GrapheneAsset(graphene.ObjectType): assetMaterializations = graphene.Field( non_null_list(GrapheneMaterializationEvent), partitions=graphene.List(graphene.NonNull(graphene.String)), - partitionInLast=graphene.Int(), + partitionInLast=graphene.Int(), # I assume num partitions can't really be so high to require BigInt beforeTimestampMillis=graphene.String(), afterTimestampMillis=graphene.String(), limit=graphene.Int(), @@ -211,7 +213,7 @@ class GrapheneAsset(graphene.ObjectType): assetObservations = graphene.Field( non_null_list(GrapheneObservationEvent), partitions=graphene.List(graphene.NonNull(graphene.String)), - partitionInLast=graphene.Int(), + partitionInLast=graphene.Int(), # Same as above beforeTimestampMillis=graphene.String(), afterTimestampMillis=graphene.String(), limit=graphene.Int(), @@ -921,7 +923,7 @@ class GraphenePipeline(GrapheneIPipelineSnapshotMixin, graphene.ObjectType): partitionKeysOrError = graphene.Field( graphene.NonNull(GraphenePartitionKeys), cursor=graphene.String(), - limit=graphene.Int(), + limit=graphene.Int(), # Can't see how partition keys could be so numerous to require BigInt reverse=graphene.Boolean(), selected_asset_keys=graphene.Argument( graphene.List(graphene.NonNull(GrapheneAssetKeyInput)) diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline_run_stats.py b/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline_run_stats.py index 3660e8a39be21..95b132862bd43 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline_run_stats.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline_run_stats.py @@ -8,10 +8,12 @@ class GraphenePipelineRunStatsSnapshot(graphene.Interface): id = graphene.NonNull(graphene.String) runId = graphene.NonNull(graphene.String) - stepsSucceeded = graphene.NonNull(graphene.Int) - stepsFailed = graphene.NonNull(graphene.Int) - materializations = graphene.NonNull(graphene.Int) - expectations = graphene.NonNull(graphene.Int) + stepsSucceeded = graphene.NonNull( + graphene.Int + ) # Can you really have a pipeline with so many steps that you need a BigInt? + stepsFailed = graphene.NonNull(graphene.Int) # Same + materializations = graphene.NonNull(graphene.Int) # Same + expectations = graphene.NonNull(graphene.Int) # Same enqueuedTime = graphene.Field(graphene.Float) launchTime = graphene.Field(graphene.Float) startTime = graphene.Field(graphene.Float) @@ -24,10 +26,10 @@ class Meta: class GrapheneRunStatsSnapshot(graphene.ObjectType): id = graphene.NonNull(graphene.String) runId = graphene.NonNull(graphene.String) - stepsSucceeded = graphene.NonNull(graphene.Int) - stepsFailed = graphene.NonNull(graphene.Int) - materializations = graphene.NonNull(graphene.Int) - expectations = graphene.NonNull(graphene.Int) + stepsSucceeded = graphene.NonNull(graphene.Int) # Same + stepsFailed = graphene.NonNull(graphene.Int) # Same + materializations = graphene.NonNull(graphene.Int) # Same + expectations = graphene.NonNull(graphene.Int) # Same enqueuedTime = graphene.Field(graphene.Float) launchTime = graphene.Field(graphene.Float) startTime = graphene.Field(graphene.Float) diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/roots/query.py b/python_modules/dagster-graphql/dagster_graphql/schema/roots/query.py index 48a244bee6457..0a41ad317732f 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/roots/query.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/roots/query.py @@ -616,8 +616,8 @@ class Meta: autoMaterializeTicks = graphene.Field( non_null_list(GrapheneInstigationTick), - dayRange=graphene.Int(), - dayOffset=graphene.Int(), + dayRange=graphene.Int(), # I assume days can't be in the BigInt range + dayOffset=graphene.Int(), # Same limit=graphene.Int(), cursor=graphene.String(), statuses=graphene.List(graphene.NonNull(GrapheneInstigationTickStatus)), diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/runs.py b/python_modules/dagster-graphql/dagster_graphql/schema/runs.py index 971f13d621efa..d9fcd3f60ddd4 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/runs.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/runs.py @@ -103,7 +103,7 @@ class Meta: class GraphenePipelineRuns(graphene.Interface): results = non_null_list("dagster_graphql.schema.pipelines.pipeline.GrapheneRun") - count = graphene.Int() + count = graphene.Int() # Won't be billions on a single account class Meta: name = "PipelineRuns" @@ -111,7 +111,7 @@ class Meta: class GrapheneRuns(graphene.ObjectType): results = non_null_list("dagster_graphql.schema.pipelines.pipeline.GrapheneRun") - count = graphene.Int() + count = graphene.Int() # Won't be billions on a single account class Meta: interfaces = (GraphenePipelineRuns,) diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/runs_feed.py b/python_modules/dagster-graphql/dagster_graphql/schema/runs_feed.py index 112ee46f89a03..dcad7a4fde144 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/runs_feed.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/runs_feed.py @@ -41,7 +41,7 @@ class GrapheneRunsFeedCount(graphene.ObjectType): class Meta: name = "RunsFeedCount" - count = graphene.NonNull(graphene.Int) + count = graphene.NonNull(graphene.Int) # Won't be billions on a single account class GrapheneRunsFeedCountOrError(graphene.Union): diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/schedules/schedules.py b/python_modules/dagster-graphql/dagster_graphql/schema/schedules/schedules.py index db052caa3277d..f2203026fac5d 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/schedules/schedules.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/schedules/schedules.py @@ -47,13 +47,16 @@ class GrapheneSchedule(graphene.ObjectType): until=graphene.Float(), ) futureTick = graphene.NonNull( - GrapheneDryRunInstigationTick, tick_timestamp=graphene.NonNull(graphene.Int) + GrapheneDryRunInstigationTick, + tick_timestamp=graphene.NonNull( + graphene.Int + ), # I'm assuming a timestamp can't require BigInt ) potentialTickTimestamps = graphene.NonNull( graphene.List(graphene.NonNull(graphene.Float)), start_timestamp=graphene.Float(), - upper_limit=graphene.Int(), - lower_limit=graphene.Int(), + upper_limit=graphene.Int(), # Same assumption as above + lower_limit=graphene.Int(), # Same assumption as above ) assetSelection = graphene.Field(GrapheneAssetSelection) tags = non_null_list(GrapheneDefinitionTag)