Skip to content

Commit

Permalink
Apply transforms to defer payloads (#579)
Browse files Browse the repository at this point in the history
* add error test for defer (#574)

* add initial defer transform tests (#576)

* initial defer logic (#577)

* get transforms actually working for deferred payloads (#578)

* improve JsonNodes class

* passing in NadelQueryPath instead of List<String> to allow better type safety

* remove unused if block

* add ability to log errors for defer transforms

* add extra defer tests for fields in lists and fields in hydrations

* move processing of graphql errors

* remove incorrect test

* remove unused test

* add logic to move transform functions next to each other

* add level of abstraction for reused code in transform functions

* add assertShouldNeverHappen when prefix does not match query path in JsonNodes

* remove empty file

* clean up defer transform code
  • Loading branch information
sbarker2 committed Sep 1, 2024
1 parent e8d5674 commit 5630e9a
Show file tree
Hide file tree
Showing 29 changed files with 1,953 additions and 53 deletions.
44 changes: 34 additions & 10 deletions lib/src/main/java/graphql/nadel/NextgenEngine.kt
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ import graphql.ExecutionResult
import graphql.GraphQLError
import graphql.execution.ExecutionIdProvider
import graphql.execution.instrumentation.InstrumentationState
import graphql.incremental.DeferPayload
import graphql.incremental.IncrementalExecutionResultImpl
import graphql.introspection.Introspection.TypeNameMetaFieldDef
import graphql.language.Document
Expand All @@ -25,6 +26,7 @@ import graphql.nadel.engine.transform.query.DynamicServiceResolution
import graphql.nadel.engine.transform.query.NadelFieldToService
import graphql.nadel.engine.transform.query.NadelQueryTransformer
import graphql.nadel.engine.transform.result.NadelResultTransformer
import graphql.nadel.engine.util.MutableJsonMap
import graphql.nadel.engine.util.beginExecute
import graphql.nadel.engine.util.compileToDocument
import graphql.nadel.engine.util.copy
Expand Down Expand Up @@ -62,6 +64,8 @@ import kotlinx.coroutines.awaitAll
import kotlinx.coroutines.cancel
import kotlinx.coroutines.coroutineScope
import kotlinx.coroutines.delay
import kotlinx.coroutines.flow.map
import kotlinx.coroutines.flow.onEach
import kotlinx.coroutines.future.asCompletableFuture
import kotlinx.coroutines.future.asDeferred
import kotlinx.coroutines.future.await
Expand Down Expand Up @@ -291,6 +295,29 @@ internal class NextgenEngine(
executionHydrationDetails = executionContext.hydrationDetails,
)
}
if (result is NadelIncrementalServiceExecutionResult) {
executionContext.incrementalResultSupport.defer(
result.incrementalItemPublisher
.asFlow()
.onEach {delayedIncrementalResult ->
// Transform
delayedIncrementalResult.incremental
?.filterIsInstance<DeferPayload>()
?.forEach {deferPayload ->
resultTransformer
.transform(
executionContext = executionContext,
serviceExecutionContext = serviceExecutionContext,
executionPlan = executionPlan,
artificialFields = queryTransform.artificialFields,
overallToUnderlyingFields = queryTransform.overallToUnderlyingFields,
service = service,
result = result,
deferPayload = deferPayload,
) }
}
)
}
val transformedResult: ServiceExecutionResult = when {
topLevelField.name.startsWith("__") -> result
else -> timer.time(step = RootStep.ResultTransforming) {
Expand Down Expand Up @@ -381,18 +408,15 @@ internal class NextgenEngine(
)
}

if (serviceExecResult is NadelIncrementalServiceExecutionResult) {
executionContext.incrementalResultSupport.defer(
serviceExecResult.incrementalItemPublisher.asFlow()
)
val transformedData: MutableJsonMap = serviceExecResult.data.let { data ->
data.takeIf { transformedQuery.resultKey in data }
?: mutableMapOf(transformedQuery.resultKey to null)
}

return serviceExecResult.copy(
data = serviceExecResult.data.let { data ->
data.takeIf { transformedQuery.resultKey in data }
?: mutableMapOf(transformedQuery.resultKey to null)
},
)
return when(serviceExecResult) {
is NadelServiceExecutionResultImpl -> serviceExecResult.copy(data = transformedData)
is NadelIncrementalServiceExecutionResult -> serviceExecResult.copy(data = transformedData)
}
}

private fun chooseServiceExecution(
Expand Down
4 changes: 2 additions & 2 deletions lib/src/main/java/graphql/nadel/ServiceExecutionResult.kt
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,7 @@ sealed class ServiceExecutionResult {
abstract val extensions: MutableMap<String, Any?>
}

class NadelIncrementalServiceExecutionResult(
data class NadelIncrementalServiceExecutionResult(
override val data: MutableMap<String, Any?> = LinkedHashMap(),
override val errors: MutableList<MutableMap<String, Any?>> = ArrayList(),
override val extensions: MutableMap<String, Any?> = LinkedHashMap(),
Expand All @@ -19,7 +19,7 @@ class NadelIncrementalServiceExecutionResult(
val hasNext: Boolean,
) : ServiceExecutionResult()

class NadelServiceExecutionResultImpl @JvmOverloads constructor(
data class NadelServiceExecutionResultImpl @JvmOverloads constructor(
override val data: MutableMap<String, Any?> = LinkedHashMap(),
override val errors: MutableList<MutableMap<String, Any?>> = ArrayList(),
override val extensions: MutableMap<String, Any?> = LinkedHashMap(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -216,6 +216,7 @@ internal class NadelDeepRenameTransform : NadelTransform<NadelDeepRenameTransfor
return NadelTransformUtil.makeTypeNameField(
aliasHelper = state.aliasHelper,
objectTypeNames = objectTypeNames,
deferredExecutions = field.deferredExecutions,
)
}

Expand Down Expand Up @@ -256,6 +257,7 @@ internal class NadelDeepRenameTransform : NadelTransform<NadelDeepRenameTransfor
queryPathToField = deepRename.queryPathToField,
fieldArguments = field.normalizedArguments,
fieldChildren = transformer.transform(field.children),
deferredExecutions = field.deferredExecutions,
),
)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -120,6 +120,7 @@ internal class NadelRenameTransform : NadelTransform<State> {
return NadelTransformUtil.makeTypeNameField(
aliasHelper = state.aliasHelper,
objectTypeNames = objectTypeNames,
deferredExecutions = field.deferredExecutions,
)
}

Expand Down Expand Up @@ -170,6 +171,7 @@ internal class NadelRenameTransform : NadelTransform<State> {
queryPathToField = NadelQueryPath(listOf(rename.underlyingName)),
fieldArguments = field.normalizedArguments,
fieldChildren = transformer.transform(field.children),
deferredExecutions = field.deferredExecutions,
)
)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ import graphql.nadel.engine.util.getField
import graphql.nadel.engine.util.makeFieldCoordinates
import graphql.normalized.ExecutableNormalizedField
import graphql.normalized.ExecutableNormalizedField.newNormalizedField
import graphql.normalized.incremental.NormalizedDeferredExecution
import graphql.schema.GraphQLFieldDefinition

object NadelTransformUtil {
Expand All @@ -36,11 +37,13 @@ object NadelTransformUtil {
fun makeTypeNameField(
aliasHelper: NadelAliasHelper,
objectTypeNames: List<String>,
deferredExecutions: LinkedHashSet<NormalizedDeferredExecution>
): ExecutableNormalizedField {
return newNormalizedField()
.alias(aliasHelper.typeNameResultKey)
.fieldName(TypeNameMetaFieldDef.name)
.objectTypeNames(objectTypeNames)
.deferredExecutions(deferredExecutions)
.build()
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -145,6 +145,7 @@ internal class NadelHydrationTransform(
return makeTypeNameField(
aliasHelper = state.aliasHelper,
objectTypeNames = objectTypeNames,
deferredExecutions = linkedSetOf(),
)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -140,6 +140,7 @@ internal class NadelBatchHydrationTransform(
return makeTypeNameField(
aliasHelper = state.aliasHelper,
objectTypeNames = objectTypeNames,
deferredExecutions = linkedSetOf(),
)
}
}
11 changes: 11 additions & 0 deletions lib/src/main/java/graphql/nadel/engine/transform/query/NFUtil.kt
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@ package graphql.nadel.engine.transform.query

import graphql.normalized.ExecutableNormalizedField
import graphql.normalized.NormalizedInputValue
import graphql.normalized.incremental.NormalizedDeferredExecution
import graphql.schema.GraphQLInterfaceType
import graphql.schema.GraphQLObjectType
import graphql.schema.GraphQLOutputType
Expand All @@ -18,6 +19,7 @@ object NFUtil {
pathToField: NadelQueryPath,
fieldArguments: Map<String, NormalizedInputValue>,
fieldChildren: List<ExecutableNormalizedField>,
deferredExecutions: LinkedHashSet<NormalizedDeferredExecution> = LinkedHashSet(),
): List<ExecutableNormalizedField> {
return createFieldRecursively(
schema,
Expand All @@ -26,6 +28,7 @@ object NFUtil {
fieldArguments,
fieldChildren,
pathToFieldIndex = 0,
deferredExecutions,
)
}

Expand All @@ -35,6 +38,7 @@ object NFUtil {
queryPathToField: NadelQueryPath,
fieldArguments: Map<String, NormalizedInputValue>,
fieldChildren: List<ExecutableNormalizedField>,
deferredExecutions: LinkedHashSet<NormalizedDeferredExecution> = LinkedHashSet(),
): ExecutableNormalizedField {
return createParticularField(
schema,
Expand All @@ -43,6 +47,7 @@ object NFUtil {
fieldArguments,
fieldChildren,
pathToFieldIndex = 0,
deferredExecutions,
)
}

Expand All @@ -53,6 +58,7 @@ object NFUtil {
fieldArguments: Map<String, NormalizedInputValue>,
fieldChildren: List<ExecutableNormalizedField>,
pathToFieldIndex: Int,
deferredExecutions: LinkedHashSet<NormalizedDeferredExecution>,
): List<ExecutableNormalizedField> {
// Note: remember that we are creating fields that do not exist in the original NF
// Thus, we need to handle interfaces and object types
Expand All @@ -65,6 +71,7 @@ object NFUtil {
fieldArguments,
fieldChildren,
pathToFieldIndex,
deferredExecutions,
)
}
is GraphQLObjectType -> listOf(
Expand All @@ -75,6 +82,7 @@ object NFUtil {
fieldArguments,
fieldChildren,
pathToFieldIndex,
deferredExecutions,
)
)
else -> error("Unsupported type '${parentType.javaClass.name}'")
Expand All @@ -88,6 +96,7 @@ object NFUtil {
fieldArguments: Map<String, NormalizedInputValue>,
fieldChildren: List<ExecutableNormalizedField>,
pathToFieldIndex: Int,
deferredExecutions: LinkedHashSet<NormalizedDeferredExecution>,
): ExecutableNormalizedField {
val fieldName = queryPathToField.segments[pathToFieldIndex]
val fieldDef = parentType.getFieldDefinition(fieldName)
Expand All @@ -96,6 +105,7 @@ object NFUtil {
return ExecutableNormalizedField.newNormalizedField()
.objectTypeNames(listOf(parentType.name))
.fieldName(fieldName)
.deferredExecutions(deferredExecutions)
.also { builder ->
if (pathToFieldIndex == queryPathToField.segments.lastIndex) {
builder.normalizedArguments(fieldArguments)
Expand All @@ -112,6 +122,7 @@ object NFUtil {
fieldArguments,
fieldChildren,
pathToFieldIndex = pathToFieldIndex + 1,
deferredExecutions,
)
}
)
Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,7 @@
package graphql.nadel.engine.transform.query

import graphql.Assert.assertShouldNeverHappen

data class NadelQueryPath(val segments: List<String>) {
val size: Int get() = segments.size

Expand All @@ -23,6 +25,21 @@ data class NadelQueryPath(val segments: List<String>) {
return segments.last()
}

fun startsWith(prefix: List<String>): Boolean {
if (prefix.size > segments.size) return false
for (i in 0..<prefix.size) {
if (prefix[i] != segments[i]) return false
}
return true
}

fun removePrefix(prefix: List<String>): NadelQueryPath {
if (this.startsWith(prefix)) {
return NadelQueryPath(segments.drop(prefix.size))
}
return assertShouldNeverHappen("NadelQueryPath did not start with prefix")
}

companion object {
val root = NadelQueryPath(emptyList())

Expand Down
Loading

0 comments on commit 5630e9a

Please sign in to comment.