add ndc_explain to select node (#285)

Co-authored-by: Anon Ray <ecthiender@users.noreply.github.com>
V3_GIT_ORIGIN_REV_ID: e45510454fe299fb41690a8fb56bcdce7628cd67
This commit is contained in:
paritosh-08 2024-01-29 10:41:49 +05:30 committed by hasura-bot
parent 25472daa8a
commit a0d30feab9
28 changed files with 2246 additions and 2894 deletions

View File

@ -25,7 +25,7 @@ services:
timeout: 10s
retries: 20
postgres_connector:
image: ghcr.io/hasura/ndc-postgres:dev-main-5aec135c1
image: ghcr.io/hasura/ndc-postgres:dev-main-dd4172889
volumes:
- ./engine/tests/pg_ndc_config.json:/config.json
depends_on:

View File

@ -164,7 +164,7 @@ async fn get_capabilities() -> Json<models::CapabilitiesResponse> {
Json(models::CapabilitiesResponse {
versions: "^0.1.0".into(),
capabilities: models::Capabilities {
explain: Some(models::LeafCapability {}),
explain: None,
query: models::QueryCapabilities {
aggregates: Some(models::LeafCapability {}),
variables: Some(models::LeafCapability {}),

View File

@ -54,7 +54,7 @@ services:
COLLECTOR_OTLP_ENABLED: 'true'
COLLECTOR_ZIPKIN_HOST_PORT: '9411'
postgres_connector:
image: ghcr.io/hasura/ndc-postgres:dev-main-5aec135c1
image: ghcr.io/hasura/ndc-postgres:dev-main-dd4172889
ports:
- 8100:8100
volumes:

View File

@ -238,18 +238,27 @@ pub async fn execute_request_internal(
.await
}
RequestMode::Explain => {
tracer.in_span("explain", SpanVisibility::Internal, || {
// convert the query plan to explain step
let explain_response =
match crate::execute::explain::explain_query_plan(query_plan) {
Ok(step) => step.to_explain_response(),
Err(e) => explain::types::ExplainResponse::error(
e.to_graphql_error(None),
),
};
tracer
.in_span_async("explain", SpanVisibility::Internal, || {
Box::pin(async {
// convert the query plan to explain step
let explain_response =
match crate::execute::explain::explain_query_plan(
http_client,
query_plan,
)
.await
{
Ok(step) => step.make_explain_response(),
Err(e) => explain::types::ExplainResponse::error(
e.to_graphql_error(None),
),
};
ExecuteOrExplainResponse::Explain(explain_response)
})
ExecuteOrExplainResponse::Explain(explain_response)
})
})
.await
}
};
Ok(response)

View File

@ -1,16 +1,19 @@
use super::remote_joins::types::RemoteJoinType;
use super::ExecuteOrExplainResponse;
use crate::execute::query_plan::{NodeQueryPlan, ProcessResponseAs};
use crate::execute::remote_joins::types::{JoinId, JoinLocations, RemoteJoin};
use crate::execute::{error, query_plan};
use crate::metadata::resolved;
use crate::schema::GDS;
use async_recursion::async_recursion;
use hasura_authn_core::Session;
use lang_graphql as gql;
use lang_graphql::{http::RawRequest, schema::Schema};
use nonempty::NonEmpty;
use crate::execute::{error, query_plan};
use super::remote_joins::types::RemoteJoinType;
use super::ExecuteOrExplainResponse;
use tracing_util::SpanVisibility;
pub mod types;
use lang_graphql::ast::common as ast;
use ndc_client as ndc;
pub async fn execute_explain(
http_client: &reqwest::Client,
@ -49,7 +52,8 @@ pub async fn execute_explain_internal(
}
}
pub fn explain_query_plan(
pub(crate) async fn explain_query_plan(
http_client: &reqwest::Client,
query_plan: query_plan::QueryPlan<'_, '_, '_>,
) -> Result<types::Step, error::Error> {
let mut parallel_root_steps = vec![];
@ -58,20 +62,26 @@ pub fn explain_query_plan(
match node {
NodeQueryPlan::NDCQueryExecution(ndc_query_execution) => {
let sequence_steps = get_execution_steps(
http_client,
alias,
&ndc_query_execution.process_response_as,
ndc_query_execution.execution_tree.remote_executions,
ndc_query_execution.execution_tree.root_node.query,
);
ndc_query_execution.execution_tree.root_node.data_connector,
)
.await;
parallel_root_steps.push(Box::new(types::Step::Sequence(sequence_steps)));
}
NodeQueryPlan::RelayNodeSelect(Some(ndc_query_execution)) => {
let sequence_steps = get_execution_steps(
http_client,
alias,
&ndc_query_execution.process_response_as,
ndc_query_execution.execution_tree.remote_executions,
ndc_query_execution.execution_tree.root_node.query,
);
ndc_query_execution.execution_tree.root_node.data_connector,
)
.await;
parallel_root_steps.push(Box::new(types::Step::Sequence(sequence_steps)));
}
NodeQueryPlan::TypeName { .. } => {
@ -114,40 +124,58 @@ pub fn explain_query_plan(
}
}
fn get_execution_steps<'s>(
async fn get_execution_steps<'s>(
http_client: &reqwest::Client,
alias: gql::ast::common::Alias,
process_response_as: &ProcessResponseAs<'s>,
join_locations: JoinLocations<(RemoteJoin<'s, '_>, JoinId)>,
ndc_query_request: ndc_client::models::QueryRequest,
data_connector: &resolved::data_connector::DataConnector,
) -> NonEmpty<Box<types::Step>> {
let mut sequence_steps = match process_response_as {
ProcessResponseAs::CommandResponse { .. } => {
// A command execution node
let data_connector_explain = fetch_explain_from_data_connector(
http_client,
ndc_query_request.clone(),
data_connector,
)
.await;
NonEmpty::new(Box::new(types::Step::CommandSelect(
types::CommandSelectIR {
command_name: alias.to_string(),
query_request: ndc_query_request,
ndc_explain: data_connector_explain,
},
)))
}
ProcessResponseAs::Array { .. } | ProcessResponseAs::Object { .. } => {
// A model execution node
let data_connector_explain = fetch_explain_from_data_connector(
http_client,
ndc_query_request.clone(),
data_connector,
)
.await;
NonEmpty::new(Box::new(types::Step::ModelSelect(types::ModelSelectIR {
model_name: alias.to_string(),
query_request: ndc_query_request,
ndc_explain: data_connector_explain,
})))
}
};
if let Some(join_steps) = get_join_steps(alias.to_string(), join_locations) {
if let Some(join_steps) = get_join_steps(alias.to_string(), join_locations, http_client).await {
sequence_steps.push(Box::new(types::Step::Parallel(join_steps)));
sequence_steps.push(Box::new(types::Step::HashJoin));
};
sequence_steps
}
fn get_join_steps(
#[async_recursion]
async fn get_join_steps(
_root_field_name: String,
join_locations: JoinLocations<(RemoteJoin<'_, '_>, JoinId)>,
join_locations: JoinLocations<(RemoteJoin<'async_recursion, 'async_recursion>, JoinId)>,
http_client: &reqwest::Client,
) -> Option<NonEmpty<Box<types::Step>>> {
let mut parallel_join_steps = vec![];
for (alias, location) in join_locations.locations {
@ -155,24 +183,33 @@ fn get_join_steps(
if let Some((remote_join, _join_id)) = location.join_node {
let mut query_request = remote_join.target_ndc_ir;
query_request.variables = Some(vec![]);
let data_connector_explain = fetch_explain_from_data_connector(
http_client,
query_request.clone(),
remote_join.target_data_connector,
)
.await;
sequence_steps.push(Box::new(types::Step::ForEach(
// We don't support ndc_explain for for-each steps yet
match remote_join.remote_join_type {
RemoteJoinType::ToModel => {
types::ForEachStep::ModelSelect(types::ModelSelectIR {
model_name: alias.clone(),
query_request,
ndc_explain: data_connector_explain,
})
}
RemoteJoinType::ToCommand => {
types::ForEachStep::CommandSelect(types::CommandSelectIR {
command_name: alias.clone(),
query_request,
ndc_explain: data_connector_explain,
})
}
},
)))
};
if let Some(rest_join_steps) = get_join_steps(alias, location.rest) {
if let Some(rest_join_steps) = get_join_steps(alias, location.rest, http_client).await {
sequence_steps.push(Box::new(types::Step::Parallel(rest_join_steps)));
sequence_steps.push(Box::new(types::Step::HashJoin));
};
@ -212,32 +249,74 @@ fn simplify_step(step: Box<types::Step>) -> Box<types::Step> {
}
}
async fn fetch_explain_from_data_connector(
http_client: &reqwest::Client,
query_request: ndc_client::models::QueryRequest,
data_connector: &resolved::data_connector::DataConnector,
) -> types::NDCExplainResponse {
let tracer = tracing_util::global_tracer();
let response = tracer
.in_span_async(
"fetch_explain_from_data_connector",
SpanVisibility::Internal,
|| {
Box::pin(async {
let ndc_config = ndc::apis::configuration::Configuration {
base_path: data_connector.url.get_url(ast::OperationType::Query),
user_agent: None,
// This is isn't expensive, reqwest::Client is behind an Arc
client: http_client.clone(),
headers: data_connector.headers.0.clone(),
};
{
// TODO: use capabilities from the data connector context
let capabilities =
ndc::apis::default_api::capabilities_get(&ndc_config).await?;
match capabilities.capabilities.explain {
None => Ok(None),
Some(_) => {
ndc::apis::default_api::explain_post(&ndc_config, query_request)
.await
.map(Some)
.map_err(error::Error::from) // ndc_client::apis::Error -> InternalError -> Error
}
}
}
})
},
)
.await;
match response {
Ok(Some(response)) => types::NDCExplainResponse::success(response),
Ok(None) => types::NDCExplainResponse::not_supported(),
Err(e) => types::NDCExplainResponse::error(e),
}
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn test_simplify_steps() {
let step = types::Step::HashJoin;
let simplified_steps = simplify_step(Box::new(types::Step::Parallel(nonempty::nonempty![
Box::new(step.clone())
Box::new(types::Step::HashJoin)
])));
assert_eq!(*simplified_steps, step.clone());
assert_eq!(*simplified_steps, types::Step::HashJoin);
let simplified_steps = simplify_step(Box::new(types::Step::Sequence(nonempty::nonempty![
Box::new(step.clone())
Box::new(types::Step::HashJoin)
])));
assert_eq!(*simplified_steps, step.clone());
assert_eq!(*simplified_steps, types::Step::HashJoin);
let nested_step = types::Step::Parallel(nonempty::nonempty![Box::new(
types::Step::Sequence(nonempty::nonempty![Box::new(types::Step::Parallel(
nonempty::nonempty![Box::new(types::Step::Sequence(nonempty::nonempty![
Box::new(step.clone())
Box::new(types::Step::HashJoin)
]))]
))])
)]);
let simplified_steps = simplify_step(Box::new(nested_step));
assert_eq!(*simplified_steps, step.clone());
assert_eq!(*simplified_steps, types::Step::HashJoin);
}
}

View File

@ -1,3 +1,4 @@
use crate::execute::error;
use crate::execute::GraphQLErrors;
use lang_graphql::http::GraphQLError;
use nonempty::NonEmpty;
@ -15,9 +16,9 @@ pub enum RequestMode {
#[derive(Debug, Serialize)]
#[serde(rename_all = "camelCase")]
pub struct ExplainResponse {
pub explain: Option<Step>,
explain: Option<Step>,
#[serde(skip_serializing_if = "Option::is_none")]
pub errors: Option<nonempty::NonEmpty<GraphQLError>>,
errors: Option<nonempty::NonEmpty<GraphQLError>>,
}
impl Traceable for ExplainResponse {
@ -35,6 +36,10 @@ impl ExplainResponse {
errors: Some(nonempty::nonempty![error]),
}
}
pub fn does_contain_error(&self) -> bool {
self.errors.is_some()
}
}
impl axum::response::IntoResponse for ExplainResponse {
@ -51,10 +56,10 @@ impl axum::response::IntoResponse for ExplainResponse {
}
}
#[derive(Serialize, Debug, PartialEq, Clone)]
#[derive(Serialize, Debug, PartialEq)]
#[serde(rename_all = "camelCase")]
#[serde(tag = "type", content = "value")]
pub enum Step {
pub(crate) enum Step {
ModelSelect(ModelSelectIR),
CommandSelect(CommandSelectIR),
ForEach(ForEachStep),
@ -64,7 +69,7 @@ pub enum Step {
}
impl Step {
pub fn to_explain_response(self) -> ExplainResponse {
pub(crate) fn make_explain_response(self) -> ExplainResponse {
ExplainResponse {
explain: Some(self),
errors: None,
@ -72,24 +77,47 @@ impl Step {
}
}
#[derive(Serialize, Debug, PartialEq, Clone)]
#[derive(Serialize, Debug, PartialEq)]
#[serde(rename_all = "camelCase")]
pub struct ModelSelectIR {
pub model_name: String,
pub query_request: ndc_client::models::QueryRequest,
pub(crate) struct ModelSelectIR {
pub(crate) model_name: String,
pub(crate) query_request: ndc_client::models::QueryRequest,
pub(crate) ndc_explain: NDCExplainResponse,
}
#[derive(Serialize, Debug, PartialEq, Clone)]
#[derive(Serialize, Debug, PartialEq)]
#[serde(rename_all = "camelCase")]
pub struct CommandSelectIR {
pub command_name: String,
pub query_request: ndc_client::models::QueryRequest,
pub(crate) struct CommandSelectIR {
pub(crate) command_name: String,
pub(crate) query_request: ndc_client::models::QueryRequest,
pub(crate) ndc_explain: NDCExplainResponse,
}
#[derive(Serialize, Debug, PartialEq, Clone)]
#[derive(Serialize, Debug, PartialEq)]
#[serde(rename_all = "camelCase")]
#[serde(tag = "type", content = "value")]
pub enum ForEachStep {
pub(crate) enum ForEachStep {
ModelSelect(ModelSelectIR),
CommandSelect(CommandSelectIR),
}
#[derive(Serialize, Debug, PartialEq)]
#[serde(rename_all = "camelCase")]
#[serde(tag = "type", content = "value")]
pub(crate) enum NDCExplainResponse {
NotSupported,
Response(ndc_client::models::ExplainResponse),
Error(GraphQLError),
}
impl NDCExplainResponse {
pub(crate) fn error(error: error::Error) -> Self {
Self::Error(error.to_graphql_error(None))
}
pub(crate) fn success(response: ndc_client::models::ExplainResponse) -> Self {
Self::Response(response)
}
pub(crate) fn not_supported() -> Self {
Self::NotSupported
}
}

View File

@ -219,12 +219,25 @@ pub fn merge_with_common_metadata<T: Iterator<Item = PathBuf>>(
}
#[allow(dead_code)]
pub fn test_execute_explain(metadata_file_path: &str, test_dir_path: &str) {
pub fn test_execute_explain(
test_path_string: &str,
test_metadata_path: &str,
common_metadata_paths: &[&str],
) {
tokio_test::block_on(async {
let root_test_dir = PathBuf::from(env!("CARGO_MANIFEST_DIR")).join("tests");
let mut test_ctx = setup(&root_test_dir);
let test_path = root_test_dir.join(test_path_string);
let gql_request_file_path = test_path.join("request.gql");
let expected_response_file = test_path_string.to_string() + "/expected.json";
let metadata = read_json(&root_test_dir.join(metadata_file_path));
let test_metadata_path = root_test_dir.join(test_metadata_path);
let metadata = merge_with_common_metadata(
&test_metadata_path,
common_metadata_paths
.iter()
.map(|path| root_test_dir.join(path)),
);
let gds = GDS::new(serde_json::from_value(metadata).unwrap()).unwrap();
let schema = GDS::build_schema(&gds).unwrap();
let session = {
@ -235,8 +248,6 @@ pub fn test_execute_explain(metadata_file_path: &str, test_dir_path: &str) {
serde_json::from_str(session_variables_raw).unwrap();
resolve_session(session_variables)
};
let gql_request_file_path = test_dir_path.to_string() + "/request.gql";
let expected_response_file = test_dir_path.to_string() + "/expected.json";
let query = std::fs::read_to_string(root_test_dir.join(gql_request_file_path)).unwrap();
let raw_request = lang_graphql::http::RawRequest {
operation_name: None,

View File

@ -82,7 +82,10 @@
"author_id"
]
}
]
],
"selectMany": {
"queryRootField": "AuthorMany"
}
},
"filterableFields": [
{

View File

@ -298,7 +298,10 @@
"author_id"
]
}
]
],
"selectMany": {
"queryRootField": "AuthorMany"
}
},
"filterableFields": [
{

View File

@ -3,50 +3,63 @@ mod common;
#[test]
fn test_explain_introspection() {
common::test_execute_explain("explain/metadata.json", "explain/introspection_query/");
common::test_execute_explain(
"explain/introspection_query/",
"explain/introspection_query/metadata.json",
&[],
);
}
#[test]
fn test_multi_root_field_queries() {
common::test_execute_explain("explain/metadata.json", "explain/multi_root_field_queries/");
common::test_execute_explain(
"explain/multi_root_field_queries/",
"execute/multiple_root_fields/successful_execution/metadata.json",
&[],
);
}
#[test]
fn test_field_with_remote_relationship() {
common::test_execute_explain(
"explain/metadata.json",
"explain/field_with_remote_relationship/",
"execute/remote_relationships/array/metadata.json",
&["execute/common_metadata/two_postgres_connector_schema.json"],
);
}
#[test]
fn test_field_with_local_relationship() {
common::test_execute_explain(
"explain/metadata.json",
"explain/field_with_local_relationship/",
"execute/relationships/array/metadata.json",
&["execute/common_metadata/postgres_connector_schema.json"],
);
}
#[test]
fn test_field_with_multi_remote_relationship_subfields() {
common::test_execute_explain(
"explain/metadata.json",
"explain/field_with_multi_remote_relationship_subfields/",
"explain/field_with_multi_remote_relationship_subfields/metadata.json",
&["execute/common_metadata/two_postgres_connector_schema.json"],
);
}
#[test]
fn test_field_with_nested_remote_relationship_1() {
common::test_execute_explain(
"explain/metadata.json",
"explain/field_with_nested_remote_relationship_1/",
"explain/field_with_nested_remote_relationship_1/metadata.json",
&["execute/common_metadata/two_postgres_connector_schema.json"],
);
}
#[test]
fn test_field_with_nested_remote_relationship_2() {
common::test_execute_explain(
"explain/metadata.json",
"explain/field_with_nested_remote_relationship_2/",
"explain/field_with_nested_remote_relationship_2/metadata.json",
&["execute/common_metadata/two_postgres_connector_schema.json"],
);
}

View File

@ -2,45 +2,54 @@
"explain": {
"type": "modelSelect",
"value": {
"modelName": "cities",
"modelName": "AuthorMany",
"queryRequest": {
"collection": "city",
"collection": "authors",
"query": {
"fields": {
"code": {
"author_id": {
"type": "column",
"column": "code"
"column": "id"
},
"name": {
"first_name": {
"type": "column",
"column": "name"
"column": "first_name"
},
"state": {
"Articles": {
"type": "relationship",
"query": {
"fields": {
"name": {
"title": {
"type": "column",
"column": "name"
"column": "title"
}
}
},
"relationship": "[{\"subgraph\":\"default\",\"name\":\"city\"},\"state\"]",
"relationship": "[{\"subgraph\":\"default\",\"name\":\"author\"},\"Articles\"]",
"arguments": {}
}
}
},
"arguments": {},
"collection_relationships": {
"[{\"subgraph\":\"default\",\"name\":\"city\"},\"state\"]": {
"[{\"subgraph\":\"default\",\"name\":\"author\"},\"Articles\"]": {
"column_mapping": {
"state_code": "code"
"id": "author_id"
},
"relationship_type": "object",
"target_collection": "states",
"relationship_type": "array",
"target_collection": "articles",
"arguments": {}
}
}
},
"ndcExplain": {
"type": "response",
"value": {
"details": {
"Execution Plan": "Aggregate (cost=21328.02..21328.03 rows=1 width=32)\n -> Aggregate (cost=21328.00..21328.01 rows=1 width=32)\n -> Nested Loop Left Join (cost=25.03..21323.75 rows=850 width=68)\n -> Seq Scan on author \"%0_authors\" (cost=0.00..18.50 rows=850 width=36)\n -> Subquery Scan on \"%3_rows\" (cost=25.03..25.06 rows=1 width=32)\n -> Aggregate (cost=25.03..25.04 rows=1 width=32)\n -> Seq Scan on article \"%2_articles\" (cost=0.00..25.00 rows=6 width=32)\n Filter: (\"%0_authors\".id = author_id)",
"SQL Query": "EXPLAIN\nSELECT\n coalesce(json_agg(row_to_json(\"%5_universe\")), '[]') AS \"universe\"\nFROM\n (\n SELECT\n *\n FROM\n (\n SELECT\n coalesce(json_agg(row_to_json(\"%6_rows\")), '[]') AS \"rows\"\n FROM\n (\n SELECT\n \"%0_authors\".\"id\" AS \"author_id\",\n \"%0_authors\".\"first_name\" AS \"first_name\",\n \"%1_RELATIONSHIP_Articles\".\"Articles\" AS \"Articles\"\n FROM\n \"public\".\"author\" AS \"%0_authors\"\n LEFT OUTER JOIN LATERAL (\n SELECT\n row_to_json(\"%1_RELATIONSHIP_Articles\") AS \"Articles\"\n FROM\n (\n SELECT\n *\n FROM\n (\n SELECT\n coalesce(json_agg(row_to_json(\"%3_rows\")), '[]') AS \"rows\"\n FROM\n (\n SELECT\n \"%2_articles\".\"title\" AS \"title\"\n FROM\n \"public\".\"article\" AS \"%2_articles\"\n WHERE\n (\"%0_authors\".\"id\" = \"%2_articles\".\"author_id\")\n ) AS \"%3_rows\"\n ) AS \"%3_rows\"\n ) AS \"%1_RELATIONSHIP_Articles\"\n ) AS \"%1_RELATIONSHIP_Articles\" ON ('true')\n ) AS \"%6_rows\"\n ) AS \"%6_rows\"\n ) AS \"%5_universe\""
}
}
}
}
}

View File

@ -1,9 +1,9 @@
{
cities {
code
name
state {
name
AuthorMany {
author_id
first_name
Articles {
title
}
}
}

View File

@ -5,27 +5,36 @@
{
"type": "modelSelect",
"value": {
"modelName": "cities",
"modelName": "Album",
"queryRequest": {
"collection": "city",
"collection": "Album",
"query": {
"fields": {
"code": {
"ArtistId": {
"type": "column",
"column": "code"
"column": "ArtistId"
},
"name": {
"__hasura_phantom_field__ArtistId": {
"type": "column",
"column": "name"
"column": "ArtistId"
},
"__hasura_phantom_field__code": {
"__hasura_phantom_field__AlbumId": {
"type": "column",
"column": "code"
"column": "AlbumId"
}
}
},
"arguments": {},
"collection_relationships": {}
},
"ndcExplain": {
"type": "response",
"value": {
"details": {
"Execution Plan": "Aggregate (cost=13.17..13.18 rows=1 width=32)\n -> Aggregate (cost=13.15..13.16 rows=1 width=32)\n -> Seq Scan on \"Album\" \"%0_Album\" (cost=0.00..12.10 rows=210 width=8)",
"SQL Query": "EXPLAIN\nSELECT\n coalesce(json_agg(row_to_json(\"%1_universe\")), '[]') AS \"universe\"\nFROM\n (\n SELECT\n *\n FROM\n (\n SELECT\n coalesce(json_agg(row_to_json(\"%2_rows\")), '[]') AS \"rows\"\n FROM\n (\n SELECT\n \"%0_Album\".\"ArtistId\" AS \"ArtistId\",\n \"%0_Album\".\"ArtistId\" AS \"__hasura_phantom_field__ArtistId\",\n \"%0_Album\".\"AlbumId\" AS \"__hasura_phantom_field__AlbumId\"\n FROM\n \"public\".\"Album\" AS \"%0_Album\"\n ) AS \"%2_rows\"\n ) AS \"%2_rows\"\n ) AS \"%1_universe\""
}
}
}
}
},
@ -37,21 +46,25 @@
"value": {
"type": "modelSelect",
"value": {
"modelName": "weather",
"modelName": "Artist",
"queryRequest": {
"collection": "weather",
"collection": "Artist",
"query": {
"fields": {
"forecast": {
"ArtistId": {
"type": "column",
"column": "forecast"
"column": "ArtistId"
},
"Name": {
"type": "column",
"column": "Name"
}
},
"where": {
"type": "binary_comparison_operator",
"column": {
"type": "column",
"name": "code",
"name": "ArtistId",
"path": []
},
"operator": {
@ -59,13 +72,22 @@
},
"value": {
"type": "variable",
"name": "$code"
"name": "$ArtistId"
}
}
},
"arguments": {},
"collection_relationships": {},
"variables": []
},
"ndcExplain": {
"type": "response",
"value": {
"details": {
"Execution Plan": "",
"SQL Query": "EXPLAIN\nSELECT\n coalesce(json_agg(\"%5_universe_agg\".\"universe\"), '[]') AS \"universe\"\nFROM\n (\n SELECT\n row_to_json(\"%2_universe\") AS \"universe\"\n FROM\n json_to_recordset(cast($1 as json)) AS \"%0_%variables_table\"(\"%variable_order\" int)\n CROSS JOIN LATERAL (\n SELECT\n *\n FROM\n (\n SELECT\n coalesce(json_agg(row_to_json(\"%3_rows\")), '[]') AS \"rows\"\n FROM\n (\n SELECT\n \"%1_Artist\".\"ArtistId\" AS \"ArtistId\",\n \"%1_Artist\".\"Name\" AS \"Name\"\n FROM\n \"public\".\"Artist\" AS \"%1_Artist\"\n WHERE\n (\n \"%1_Artist\".\"ArtistId\" = cast(\"%0_%variables_table\".\"$ArtistId\" as int4)\n )\n ) AS \"%3_rows\"\n ) AS \"%3_rows\"\n ) AS \"%2_universe\"\n ORDER BY\n \"%0_%variables_table\".\"%variable_order\" ASC\n ) AS \"%5_universe_agg\""
}
}
}
}
}
@ -75,21 +97,21 @@
"value": {
"type": "modelSelect",
"value": {
"modelName": "census",
"modelName": "Tracks",
"queryRequest": {
"collection": "census",
"collection": "Track",
"query": {
"fields": {
"data": {
"TrackId": {
"type": "column",
"column": "data"
"column": "TrackId"
}
},
"where": {
"type": "binary_comparison_operator",
"column": {
"type": "column",
"name": "city_code",
"name": "AlbumId",
"path": []
},
"operator": {
@ -97,13 +119,22 @@
},
"value": {
"type": "variable",
"name": "$city_code"
"name": "$AlbumId"
}
}
},
"arguments": {},
"collection_relationships": {},
"variables": []
},
"ndcExplain": {
"type": "response",
"value": {
"details": {
"Execution Plan": "",
"SQL Query": "EXPLAIN\nSELECT\n coalesce(json_agg(\"%5_universe_agg\".\"universe\"), '[]') AS \"universe\"\nFROM\n (\n SELECT\n row_to_json(\"%2_universe\") AS \"universe\"\n FROM\n json_to_recordset(cast($1 as json)) AS \"%0_%variables_table\"(\"%variable_order\" int)\n CROSS JOIN LATERAL (\n SELECT\n *\n FROM\n (\n SELECT\n coalesce(json_agg(row_to_json(\"%3_rows\")), '[]') AS \"rows\"\n FROM\n (\n SELECT\n \"%1_Track\".\"TrackId\" AS \"TrackId\"\n FROM\n \"public\".\"Track\" AS \"%1_Track\"\n WHERE\n (\n \"%1_Track\".\"AlbumId\" = cast(\"%0_%variables_table\".\"$AlbumId\" as int4)\n )\n ) AS \"%3_rows\"\n ) AS \"%3_rows\"\n ) AS \"%2_universe\"\n ORDER BY\n \"%0_%variables_table\".\"%variable_order\" ASC\n ) AS \"%5_universe_agg\""
}
}
}
}
}

View File

@ -0,0 +1,451 @@
{
"version": "v2",
"subgraphs": [
{
"name": "connector_1",
"objects": [
{
"definition": {
"name": "Artist",
"fields": [
{
"name": "ArtistId",
"type": "Int"
},
{
"name": "Name",
"type": "String"
}
],
"graphql": {
"typeName": "Artist"
}
},
"version": "v1",
"kind": "ObjectType"
},
{
"definition": {
"name": "Track",
"fields": [
{
"name": "TrackId",
"type": "Int"
},
{
"name": "Name",
"type": "String"
},
{
"name": "AlbumId",
"type": "Int"
}
],
"graphql": {
"typeName": "Track"
}
},
"version": "v1",
"kind": "ObjectType"
},
{
"definition": {
"typeName": "Artist",
"permissions": [
{
"role": "admin",
"output": {
"allowedFields": [
"ArtistId",
"Name"
]
}
}
]
},
"version": "v1",
"kind": "TypePermissions"
},
{
"definition": {
"typeName": "Track",
"permissions": [
{
"role": "admin",
"output": {
"allowedFields": [
"TrackId",
"Name",
"AlbumId"
]
}
}
]
},
"version": "v1",
"kind": "TypePermissions"
},
{
"definition": {
"name": "Tracks",
"objectType": "Track",
"source": {
"dataConnectorName": "db",
"collection": "Track",
"typeMapping": {
"Track": {
"fieldMapping": {
"TrackId": {
"column": "TrackId"
},
"Name": {
"column": "Name"
},
"AlbumId": {
"column": "AlbumId"
}
}
}
}
},
"graphql": {
"selectUniques": [
]
},
"filterableFields": [
{
"fieldName": "TrackId",
"operators": {
"enableAll": true
}
},
{
"fieldName": "Name",
"operators": {
"enableAll": true
}
},
{
"fieldName": "AlbumId",
"operators": {
"enableAll": true
}
}
],
"orderableFields": [
{
"fieldName": "TrackId",
"orderByDirections": {
"enableAll": true
}
},
{
"fieldName": "Name",
"orderByDirections": {
"enableAll": true
}
},
{
"fieldName": "AlbumId",
"orderByDirections": {
"enableAll": true
}
}
]
},
"version": "v1",
"kind": "Model"
},
{
"definition": {
"modelName": "Tracks",
"permissions": [
{
"role": "admin",
"select": {
"filter": null
}
}
]
},
"version": "v1",
"kind": "ModelPermissions"
},
{
"definition": {
"name": "Artists",
"objectType": "Artist",
"source": {
"dataConnectorName": "db",
"collection": "Artist",
"typeMapping": {
"Artist": {
"fieldMapping": {
"ArtistId": {
"column": "ArtistId"
},
"Name": {
"column": "Name"
}
}
}
}
},
"graphql": {
"selectUniques": [
]
},
"filterableFields": [
{
"fieldName": "ArtistId",
"operators": {
"enableAll": true
}
},
{
"fieldName": "Name",
"operators": {
"enableAll": true
}
}
],
"orderableFields": [
{
"fieldName": "ArtistId",
"orderByDirections": {
"enableAll": true
}
},
{
"fieldName": "Name",
"orderByDirections": {
"enableAll": true
}
}
]
},
"version": "v1",
"kind": "Model"
},
{
"definition": {
"modelName": "Artists",
"permissions": [
{
"role": "admin",
"select": {
"filter": null
}
}
]
},
"version": "v1",
"kind": "ModelPermissions"
}
]
},
{
"name": "connector_2",
"objects": [
{
"definition": {
"name": "Album",
"fields": [
{
"name": "AlbumId",
"type": "Int"
},
{
"name": "Title",
"type": "String"
},
{
"name": "ArtistId",
"type": "Int"
}
],
"graphql": {
"typeName": "Album"
}
},
"version": "v1",
"kind": "ObjectType"
},
{
"definition": {
"typeName": "Album",
"permissions": [
{
"role": "admin",
"output": {
"allowedFields": [
"AlbumId",
"Title",
"ArtistId"
]
}
}
]
},
"version": "v1",
"kind": "TypePermissions"
},
{
"definition": {
"name": "Albums",
"objectType": "Album",
"source": {
"dataConnectorName": "db",
"collection": "Album",
"typeMapping": {
"Album": {
"fieldMapping": {
"AlbumId": {
"column": "AlbumId"
},
"Title": {
"column": "Title"
},
"ArtistId": {
"column": "ArtistId"
}
}
}
}
},
"graphql": {
"selectUniques": [
],
"selectMany": {
"queryRootField": "Album"
}
},
"filterableFields": [
{
"fieldName": "AlbumId",
"operators": {
"enableAll": true
}
},
{
"fieldName": "Title",
"operators": {
"enableAll": true
}
},
{
"fieldName": "ArtistId",
"operators": {
"enableAll": true
}
}
],
"orderableFields": [
{
"fieldName": "AlbumId",
"orderByDirections": {
"enableAll": true
}
},
{
"fieldName": "Title",
"orderByDirections": {
"enableAll": true
}
},
{
"fieldName": "ArtistId",
"orderByDirections": {
"enableAll": true
}
}
]
},
"version": "v1",
"kind": "Model"
},
{
"definition": {
"modelName": "Albums",
"permissions": [
{
"role": "admin",
"select": {
"filter": null
}
}
]
},
"version": "v1",
"kind": "ModelPermissions"
},
{
"definition": {
"source": "Album",
"name": "Tracks",
"target": {
"model": {
"name": "Tracks",
"subgraph": "connector_1",
"relationshipType": "Array"
}
},
"mapping": [
{
"source": {
"fieldPath": [
{
"fieldName": "AlbumId"
}
]
},
"target": {
"modelField": [
{
"fieldName": "AlbumId"
}
]
}
}
]
},
"version": "v1",
"kind": "Relationship"
},
{
"definition": {
"source": "Album",
"name": "Artist",
"target": {
"model": {
"name": "Artists",
"subgraph": "connector_1",
"relationshipType": "Object"
}
},
"mapping": [
{
"source": {
"fieldPath": [
{
"fieldName": "ArtistId"
}
]
},
"target": {
"modelField": [
{
"fieldName": "ArtistId"
}
]
}
}
]
},
"version": "v1",
"kind": "Relationship"
}
]
}
]
}

View File

@ -1,12 +1,12 @@
{
cities {
code
name
weather {
forecast
Album {
ArtistId
Artist {
ArtistId
Name
}
census {
data
Tracks {
TrackId
}
}
}
}

View File

@ -5,53 +5,62 @@
{
"type": "modelSelect",
"value": {
"modelName": "cities",
"modelName": "Album",
"queryRequest": {
"collection": "city",
"collection": "Album",
"query": {
"fields": {
"code": {
"AlbumId": {
"type": "column",
"column": "code"
"column": "AlbumId"
},
"name": {
"ArtistId": {
"type": "column",
"column": "name"
"column": "ArtistId"
},
"__hasura_phantom_field__code": {
"__hasura_phantom_field__ArtistId": {
"type": "column",
"column": "code"
"column": "ArtistId"
},
"state": {
"Tracks": {
"type": "relationship",
"query": {
"fields": {
"name": {
"TrackId": {
"type": "column",
"column": "name"
"column": "TrackId"
},
"__hasura_phantom_field__code": {
"__hasura_phantom_field__AlbumId": {
"type": "column",
"column": "code"
"column": "AlbumId"
}
}
},
"relationship": "[{\"subgraph\":\"default\",\"name\":\"city\"},\"state\"]",
"relationship": "[{\"subgraph\":\"connector_2\",\"name\":\"Album\"},\"Tracks\"]",
"arguments": {}
}
}
},
"arguments": {},
"collection_relationships": {
"[{\"subgraph\":\"default\",\"name\":\"city\"},\"state\"]": {
"[{\"subgraph\":\"connector_2\",\"name\":\"Album\"},\"Tracks\"]": {
"column_mapping": {
"state_code": "code"
"AlbumId": "AlbumId"
},
"relationship_type": "object",
"target_collection": "states",
"relationship_type": "array",
"target_collection": "Track",
"arguments": {}
}
}
},
"ndcExplain": {
"type": "response",
"value": {
"details": {
"Execution Plan": "Aggregate (cost=2342.72..2342.73 rows=1 width=32)\n -> Aggregate (cost=2342.70..2342.71 rows=1 width=32)\n -> Nested Loop Left Join (cost=11.06..2341.65 rows=210 width=40)\n -> Seq Scan on \"Album\" \"%0_Album\" (cost=0.00..12.10 rows=210 width=8)\n -> Subquery Scan on \"%3_rows\" (cost=11.06..11.08 rows=1 width=32)\n -> Aggregate (cost=11.06..11.07 rows=1 width=32)\n -> Bitmap Heap Scan on \"Track\" \"%2_Track\" (cost=4.29..11.05 rows=2 width=8)\n Recheck Cond: (\"%0_Album\".\"AlbumId\" = \"AlbumId\")\n -> Bitmap Index Scan on \"IFK_TrackAlbumId\" (cost=0.00..4.29 rows=2 width=0)\n Index Cond: (\"AlbumId\" = \"%0_Album\".\"AlbumId\")",
"SQL Query": "EXPLAIN\nSELECT\n coalesce(json_agg(row_to_json(\"%5_universe\")), '[]') AS \"universe\"\nFROM\n (\n SELECT\n *\n FROM\n (\n SELECT\n coalesce(json_agg(row_to_json(\"%6_rows\")), '[]') AS \"rows\"\n FROM\n (\n SELECT\n \"%0_Album\".\"AlbumId\" AS \"AlbumId\",\n \"%0_Album\".\"ArtistId\" AS \"ArtistId\",\n \"%0_Album\".\"ArtistId\" AS \"__hasura_phantom_field__ArtistId\",\n \"%1_RELATIONSHIP_Tracks\".\"Tracks\" AS \"Tracks\"\n FROM\n \"public\".\"Album\" AS \"%0_Album\"\n LEFT OUTER JOIN LATERAL (\n SELECT\n row_to_json(\"%1_RELATIONSHIP_Tracks\") AS \"Tracks\"\n FROM\n (\n SELECT\n *\n FROM\n (\n SELECT\n coalesce(json_agg(row_to_json(\"%3_rows\")), '[]') AS \"rows\"\n FROM\n (\n SELECT\n \"%2_Track\".\"TrackId\" AS \"TrackId\",\n \"%2_Track\".\"AlbumId\" AS \"__hasura_phantom_field__AlbumId\"\n FROM\n \"public\".\"Track\" AS \"%2_Track\"\n WHERE\n (\"%0_Album\".\"AlbumId\" = \"%2_Track\".\"AlbumId\")\n ) AS \"%3_rows\"\n ) AS \"%3_rows\"\n ) AS \"%1_RELATIONSHIP_Tracks\"\n ) AS \"%1_RELATIONSHIP_Tracks\" ON ('true')\n ) AS \"%6_rows\"\n ) AS \"%6_rows\"\n ) AS \"%5_universe\""
}
}
}
}
},
@ -63,21 +72,21 @@
"value": {
"type": "modelSelect",
"value": {
"modelName": "weather",
"modelName": "Artist",
"queryRequest": {
"collection": "weather",
"collection": "Artist",
"query": {
"fields": {
"forecast": {
"ArtistId": {
"type": "column",
"column": "forecast"
"column": "ArtistId"
}
},
"where": {
"type": "binary_comparison_operator",
"column": {
"type": "column",
"name": "code",
"name": "ArtistId",
"path": []
},
"operator": {
@ -85,13 +94,22 @@
},
"value": {
"type": "variable",
"name": "$code"
"name": "$ArtistId"
}
}
},
"arguments": {},
"collection_relationships": {},
"variables": []
},
"ndcExplain": {
"type": "response",
"value": {
"details": {
"Execution Plan": "",
"SQL Query": "EXPLAIN\nSELECT\n coalesce(json_agg(\"%5_universe_agg\".\"universe\"), '[]') AS \"universe\"\nFROM\n (\n SELECT\n row_to_json(\"%2_universe\") AS \"universe\"\n FROM\n json_to_recordset(cast($1 as json)) AS \"%0_%variables_table\"(\"%variable_order\" int)\n CROSS JOIN LATERAL (\n SELECT\n *\n FROM\n (\n SELECT\n coalesce(json_agg(row_to_json(\"%3_rows\")), '[]') AS \"rows\"\n FROM\n (\n SELECT\n \"%1_Artist\".\"ArtistId\" AS \"ArtistId\"\n FROM\n \"public\".\"Artist\" AS \"%1_Artist\"\n WHERE\n (\n \"%1_Artist\".\"ArtistId\" = cast(\"%0_%variables_table\".\"$ArtistId\" as int4)\n )\n ) AS \"%3_rows\"\n ) AS \"%3_rows\"\n ) AS \"%2_universe\"\n ORDER BY\n \"%0_%variables_table\".\"%variable_order\" ASC\n ) AS \"%5_universe_agg\""
}
}
}
}
}
@ -104,21 +122,21 @@
"value": {
"type": "modelSelect",
"value": {
"modelName": "weather",
"modelName": "Album",
"queryRequest": {
"collection": "weather",
"collection": "Album",
"query": {
"fields": {
"forecast": {
"AlbumId": {
"type": "column",
"column": "forecast"
"column": "AlbumId"
}
},
"where": {
"type": "binary_comparison_operator",
"column": {
"type": "column",
"name": "code",
"name": "AlbumId",
"path": []
},
"operator": {
@ -126,13 +144,22 @@
},
"value": {
"type": "variable",
"name": "$code"
"name": "$AlbumId"
}
}
},
"arguments": {},
"collection_relationships": {},
"variables": []
},
"ndcExplain": {
"type": "response",
"value": {
"details": {
"Execution Plan": "",
"SQL Query": "EXPLAIN\nSELECT\n coalesce(json_agg(\"%5_universe_agg\".\"universe\"), '[]') AS \"universe\"\nFROM\n (\n SELECT\n row_to_json(\"%2_universe\") AS \"universe\"\n FROM\n json_to_recordset(cast($1 as json)) AS \"%0_%variables_table\"(\"%variable_order\" int)\n CROSS JOIN LATERAL (\n SELECT\n *\n FROM\n (\n SELECT\n coalesce(json_agg(row_to_json(\"%3_rows\")), '[]') AS \"rows\"\n FROM\n (\n SELECT\n \"%1_Album\".\"AlbumId\" AS \"AlbumId\"\n FROM\n \"public\".\"Album\" AS \"%1_Album\"\n WHERE\n (\n \"%1_Album\".\"AlbumId\" = cast(\"%0_%variables_table\".\"$AlbumId\" as int4)\n )\n ) AS \"%3_rows\"\n ) AS \"%3_rows\"\n ) AS \"%2_universe\"\n ORDER BY\n \"%0_%variables_table\".\"%variable_order\" ASC\n ) AS \"%5_universe_agg\""
}
}
}
}
}

View File

@ -0,0 +1,612 @@
{
"version": "v2",
"subgraphs": [
{
"name": "connector_1",
"objects": [
{
"definition": {
"name": "Artist",
"fields": [
{
"name": "ArtistId",
"type": "Int"
},
{
"name": "Name",
"type": "String"
}
],
"graphql": {
"typeName": "Artist"
}
},
"version": "v1",
"kind": "ObjectType"
},
{
"definition": {
"typeName": "Artist",
"permissions": [
{
"role": "admin",
"output": {
"allowedFields": [
"ArtistId",
"Name"
]
}
}
]
},
"version": "v1",
"kind": "TypePermissions"
},
{
"definition": {
"name": "Artists",
"objectType": "Artist",
"source": {
"dataConnectorName": "db",
"collection": "Artist",
"typeMapping": {
"Artist": {
"fieldMapping": {
"ArtistId": {
"column": "ArtistId"
},
"Name": {
"column": "Name"
}
}
}
}
},
"graphql": {
"selectUniques": [
]
},
"filterableFields": [
{
"fieldName": "ArtistId",
"operators": {
"enableAll": true
}
},
{
"fieldName": "Name",
"operators": {
"enableAll": true
}
}
],
"orderableFields": [
{
"fieldName": "ArtistId",
"orderByDirections": {
"enableAll": true
}
},
{
"fieldName": "Name",
"orderByDirections": {
"enableAll": true
}
}
]
},
"version": "v1",
"kind": "Model"
},
{
"definition": {
"modelName": "Artists",
"permissions": [
{
"role": "admin",
"select": {
"filter": null
}
}
]
},
"version": "v1",
"kind": "ModelPermissions"
},
{
"definition": {
"name": "Album",
"fields": [
{
"name": "AlbumId",
"type": "Int"
},
{
"name": "Title",
"type": "String"
},
{
"name": "ArtistId",
"type": "Int"
}
],
"graphql": {
"typeName": "Album_2"
}
},
"version": "v1",
"kind": "ObjectType"
},
{
"definition": {
"typeName": "Album",
"permissions": [
{
"role": "admin",
"output": {
"allowedFields": [
"AlbumId",
"Title",
"ArtistId"
]
}
}
]
},
"version": "v1",
"kind": "TypePermissions"
},
{
"definition": {
"name": "Albums",
"objectType": "Album",
"source": {
"dataConnectorName": "db",
"collection": "Album",
"typeMapping": {
"Album": {
"fieldMapping": {
"AlbumId": {
"column": "AlbumId"
},
"Title": {
"column": "Title"
},
"ArtistId": {
"column": "ArtistId"
}
}
}
}
},
"graphql": {
"selectUniques": [
]
},
"filterableFields": [
{
"fieldName": "AlbumId",
"operators": {
"enableAll": true
}
},
{
"fieldName": "Title",
"operators": {
"enableAll": true
}
},
{
"fieldName": "ArtistId",
"operators": {
"enableAll": true
}
}
],
"orderableFields": [
{
"fieldName": "AlbumId",
"orderByDirections": {
"enableAll": true
}
},
{
"fieldName": "Title",
"orderByDirections": {
"enableAll": true
}
},
{
"fieldName": "ArtistId",
"orderByDirections": {
"enableAll": true
}
}
]
},
"version": "v1",
"kind": "Model"
},
{
"definition": {
"modelName": "Albums",
"permissions": [
{
"role": "admin",
"select": {
"filter": null
}
}
]
},
"version": "v1",
"kind": "ModelPermissions"
}
]
},
{
"name": "connector_2",
"objects": [
{
"definition": {
"name": "Album",
"fields": [
{
"name": "AlbumId",
"type": "Int"
},
{
"name": "Title",
"type": "String"
},
{
"name": "ArtistId",
"type": "Int"
}
],
"graphql": {
"typeName": "Album"
}
},
"version": "v1",
"kind": "ObjectType"
},
{
"definition": {
"typeName": "Album",
"permissions": [
{
"role": "admin",
"output": {
"allowedFields": [
"AlbumId",
"Title",
"ArtistId"
]
}
}
]
},
"version": "v1",
"kind": "TypePermissions"
},
{
"definition": {
"name": "Albums",
"objectType": "Album",
"source": {
"dataConnectorName": "db",
"collection": "Album",
"typeMapping": {
"Album": {
"fieldMapping": {
"AlbumId": {
"column": "AlbumId"
},
"Title": {
"column": "Title"
},
"ArtistId": {
"column": "ArtistId"
}
}
}
}
},
"graphql": {
"selectUniques": [
],
"selectMany": {
"queryRootField": "Album"
}
},
"filterableFields": [
{
"fieldName": "AlbumId",
"operators": {
"enableAll": true
}
},
{
"fieldName": "Title",
"operators": {
"enableAll": true
}
},
{
"fieldName": "ArtistId",
"operators": {
"enableAll": true
}
}
],
"orderableFields": [
{
"fieldName": "AlbumId",
"orderByDirections": {
"enableAll": true
}
},
{
"fieldName": "Title",
"orderByDirections": {
"enableAll": true
}
},
{
"fieldName": "ArtistId",
"orderByDirections": {
"enableAll": true
}
}
]
},
"version": "v1",
"kind": "Model"
},
{
"definition": {
"modelName": "Albums",
"permissions": [
{
"role": "admin",
"select": {
"filter": null
}
}
]
},
"version": "v1",
"kind": "ModelPermissions"
},
{
"definition": {
"name": "Track",
"fields": [
{
"name": "TrackId",
"type": "Int"
},
{
"name": "Name",
"type": "String"
},
{
"name": "AlbumId",
"type": "Int"
}
],
"graphql": {
"typeName": "Track"
}
},
"version": "v1",
"kind": "ObjectType"
},
{
"definition": {
"typeName": "Track",
"permissions": [
{
"role": "admin",
"output": {
"allowedFields": [
"TrackId",
"Name",
"AlbumId"
]
}
}
]
},
"version": "v1",
"kind": "TypePermissions"
},
{
"definition": {
"name": "Tracks",
"objectType": "Track",
"source": {
"dataConnectorName": "db",
"collection": "Track",
"typeMapping": {
"Track": {
"fieldMapping": {
"TrackId": {
"column": "TrackId"
},
"Name": {
"column": "Name"
},
"AlbumId": {
"column": "AlbumId"
}
}
}
}
},
"graphql": {
"selectUniques": [
]
},
"filterableFields": [
{
"fieldName": "TrackId",
"operators": {
"enableAll": true
}
},
{
"fieldName": "Name",
"operators": {
"enableAll": true
}
},
{
"fieldName": "AlbumId",
"operators": {
"enableAll": true
}
}
],
"orderableFields": [
{
"fieldName": "TrackId",
"orderByDirections": {
"enableAll": true
}
},
{
"fieldName": "Name",
"orderByDirections": {
"enableAll": true
}
},
{
"fieldName": "AlbumId",
"orderByDirections": {
"enableAll": true
}
}
]
},
"version": "v1",
"kind": "Model"
},
{
"definition": {
"modelName": "Tracks",
"permissions": [
{
"role": "admin",
"select": {
"filter": null
}
}
]
},
"version": "v1",
"kind": "ModelPermissions"
},
{
"definition": {
"source": "Album",
"name": "Tracks",
"target": {
"model": {
"name": "Tracks",
"relationshipType": "Array"
}
},
"mapping": [
{
"source": {
"fieldPath": [
{
"fieldName": "AlbumId"
}
]
},
"target": {
"modelField": [
{
"fieldName": "AlbumId"
}
]
}
}
]
},
"version": "v1",
"kind": "Relationship"
},
{
"definition": {
"source": "Track",
"name": "Album",
"target": {
"model": {
"name": "Albums",
"subgraph": "connector_1",
"relationshipType": "Object"
}
},
"mapping": [
{
"source": {
"fieldPath": [
{
"fieldName": "AlbumId"
}
]
},
"target": {
"modelField": [
{
"fieldName": "AlbumId"
}
]
}
}
]
},
"version": "v1",
"kind": "Relationship"
},
{
"definition": {
"source": "Album",
"name": "Artist",
"target": {
"model": {
"name": "Artists",
"subgraph": "connector_1",
"relationshipType": "Object"
}
},
"mapping": [
{
"source": {
"fieldPath": [
{
"fieldName": "ArtistId"
}
]
},
"target": {
"modelField": [
{
"fieldName": "ArtistId"
}
]
}
}
]
},
"version": "v1",
"kind": "Relationship"
}
]
}
]
}

View File

@ -1,14 +1,14 @@
{
cities {
code
name
weather {
forecast
Album {
AlbumId
ArtistId
Artist {
ArtistId
}
state {
name
weather {
forecast
Tracks {
TrackId
Album {
AlbumId
}
}
}

View File

@ -5,27 +5,40 @@
{
"type": "modelSelect",
"value": {
"modelName": "cities",
"modelName": "Album",
"queryRequest": {
"collection": "city",
"collection": "Album",
"query": {
"fields": {
"code": {
"AlbumId": {
"type": "column",
"column": "code"
"column": "AlbumId"
},
"name": {
"ArtistId": {
"type": "column",
"column": "name"
"column": "ArtistId"
},
"__hasura_phantom_field__code": {
"__hasura_phantom_field__AlbumId": {
"type": "column",
"column": "code"
"column": "AlbumId"
},
"__hasura_phantom_field__ArtistId": {
"type": "column",
"column": "ArtistId"
}
}
},
"arguments": {},
"collection_relationships": {}
},
"ndcExplain": {
"type": "response",
"value": {
"details": {
"Execution Plan": "Aggregate (cost=13.17..13.18 rows=1 width=32)\n -> Aggregate (cost=13.15..13.16 rows=1 width=32)\n -> Seq Scan on \"Album\" \"%0_Album\" (cost=0.00..12.10 rows=210 width=8)",
"SQL Query": "EXPLAIN\nSELECT\n coalesce(json_agg(row_to_json(\"%1_universe\")), '[]') AS \"universe\"\nFROM\n (\n SELECT\n *\n FROM\n (\n SELECT\n coalesce(json_agg(row_to_json(\"%2_rows\")), '[]') AS \"rows\"\n FROM\n (\n SELECT\n \"%0_Album\".\"AlbumId\" AS \"AlbumId\",\n \"%0_Album\".\"ArtistId\" AS \"ArtistId\",\n \"%0_Album\".\"AlbumId\" AS \"__hasura_phantom_field__AlbumId\",\n \"%0_Album\".\"ArtistId\" AS \"__hasura_phantom_field__ArtistId\"\n FROM\n \"public\".\"Album\" AS \"%0_Album\"\n ) AS \"%2_rows\"\n ) AS \"%2_rows\"\n ) AS \"%1_universe\""
}
}
}
}
},
@ -40,25 +53,25 @@
"value": {
"type": "modelSelect",
"value": {
"modelName": "departments",
"modelName": "Tracks",
"queryRequest": {
"collection": "departments",
"collection": "Track",
"query": {
"fields": {
"name": {
"TrackId": {
"type": "column",
"column": "name"
"column": "TrackId"
},
"__hasura_phantom_field__id": {
"__hasura_phantom_field__AlbumId": {
"type": "column",
"column": "id"
"column": "AlbumId"
}
},
"where": {
"type": "binary_comparison_operator",
"column": {
"type": "column",
"name": "city_code",
"name": "AlbumId",
"path": []
},
"operator": {
@ -66,13 +79,22 @@
},
"value": {
"type": "variable",
"name": "$city_code"
"name": "$AlbumId"
}
}
},
"arguments": {},
"collection_relationships": {},
"variables": []
},
"ndcExplain": {
"type": "response",
"value": {
"details": {
"Execution Plan": "",
"SQL Query": "EXPLAIN\nSELECT\n coalesce(json_agg(\"%5_universe_agg\".\"universe\"), '[]') AS \"universe\"\nFROM\n (\n SELECT\n row_to_json(\"%2_universe\") AS \"universe\"\n FROM\n json_to_recordset(cast($1 as json)) AS \"%0_%variables_table\"(\"%variable_order\" int)\n CROSS JOIN LATERAL (\n SELECT\n *\n FROM\n (\n SELECT\n coalesce(json_agg(row_to_json(\"%3_rows\")), '[]') AS \"rows\"\n FROM\n (\n SELECT\n \"%1_Track\".\"TrackId\" AS \"TrackId\",\n \"%1_Track\".\"AlbumId\" AS \"__hasura_phantom_field__AlbumId\"\n FROM\n \"public\".\"Track\" AS \"%1_Track\"\n WHERE\n (\n \"%1_Track\".\"AlbumId\" = cast(\"%0_%variables_table\".\"$AlbumId\" as int4)\n )\n ) AS \"%3_rows\"\n ) AS \"%3_rows\"\n ) AS \"%2_universe\"\n ORDER BY\n \"%0_%variables_table\".\"%variable_order\" ASC\n ) AS \"%5_universe_agg\""
}
}
}
}
}
@ -82,21 +104,21 @@
"value": {
"type": "modelSelect",
"value": {
"modelName": "ministers",
"modelName": "Album",
"queryRequest": {
"collection": "ministers",
"collection": "Album",
"query": {
"fields": {
"name": {
"AlbumId": {
"type": "column",
"column": "name"
"column": "AlbumId"
}
},
"where": {
"type": "binary_comparison_operator",
"column": {
"type": "column",
"name": "department_id",
"name": "AlbumId",
"path": []
},
"operator": {
@ -104,13 +126,22 @@
},
"value": {
"type": "variable",
"name": "$department_id"
"name": "$AlbumId"
}
}
},
"arguments": {},
"collection_relationships": {},
"variables": []
},
"ndcExplain": {
"type": "response",
"value": {
"details": {
"Execution Plan": "",
"SQL Query": "EXPLAIN\nSELECT\n coalesce(json_agg(\"%5_universe_agg\".\"universe\"), '[]') AS \"universe\"\nFROM\n (\n SELECT\n row_to_json(\"%2_universe\") AS \"universe\"\n FROM\n json_to_recordset(cast($1 as json)) AS \"%0_%variables_table\"(\"%variable_order\" int)\n CROSS JOIN LATERAL (\n SELECT\n *\n FROM\n (\n SELECT\n coalesce(json_agg(row_to_json(\"%3_rows\")), '[]') AS \"rows\"\n FROM\n (\n SELECT\n \"%1_Album\".\"AlbumId\" AS \"AlbumId\"\n FROM\n \"public\".\"Album\" AS \"%1_Album\"\n WHERE\n (\n \"%1_Album\".\"AlbumId\" = cast(\"%0_%variables_table\".\"$AlbumId\" as int4)\n )\n ) AS \"%3_rows\"\n ) AS \"%3_rows\"\n ) AS \"%2_universe\"\n ORDER BY\n \"%0_%variables_table\".\"%variable_order\" ASC\n ) AS \"%5_universe_agg\""
}
}
}
}
}
@ -125,21 +156,21 @@
"value": {
"type": "modelSelect",
"value": {
"modelName": "census",
"modelName": "Artist",
"queryRequest": {
"collection": "census",
"collection": "Artist",
"query": {
"fields": {
"data": {
"ArtistId": {
"type": "column",
"column": "data"
"column": "ArtistId"
}
},
"where": {
"type": "binary_comparison_operator",
"column": {
"type": "column",
"name": "city_code",
"name": "ArtistId",
"path": []
},
"operator": {
@ -147,13 +178,22 @@
},
"value": {
"type": "variable",
"name": "$city_code"
"name": "$ArtistId"
}
}
},
"arguments": {},
"collection_relationships": {},
"variables": []
},
"ndcExplain": {
"type": "response",
"value": {
"details": {
"Execution Plan": "",
"SQL Query": "EXPLAIN\nSELECT\n coalesce(json_agg(\"%5_universe_agg\".\"universe\"), '[]') AS \"universe\"\nFROM\n (\n SELECT\n row_to_json(\"%2_universe\") AS \"universe\"\n FROM\n json_to_recordset(cast($1 as json)) AS \"%0_%variables_table\"(\"%variable_order\" int)\n CROSS JOIN LATERAL (\n SELECT\n *\n FROM\n (\n SELECT\n coalesce(json_agg(row_to_json(\"%3_rows\")), '[]') AS \"rows\"\n FROM\n (\n SELECT\n \"%1_Artist\".\"ArtistId\" AS \"ArtistId\"\n FROM\n \"public\".\"Artist\" AS \"%1_Artist\"\n WHERE\n (\n \"%1_Artist\".\"ArtistId\" = cast(\"%0_%variables_table\".\"$ArtistId\" as int4)\n )\n ) AS \"%3_rows\"\n ) AS \"%3_rows\"\n ) AS \"%2_universe\"\n ORDER BY\n \"%0_%variables_table\".\"%variable_order\" ASC\n ) AS \"%5_universe_agg\""
}
}
}
}
}

View File

@ -0,0 +1,613 @@
{
"version": "v2",
"subgraphs": [
{
"name": "connector_1",
"objects": [
{
"definition": {
"name": "Artist",
"fields": [
{
"name": "ArtistId",
"type": "Int"
},
{
"name": "Name",
"type": "String"
}
],
"graphql": {
"typeName": "Artist"
}
},
"version": "v1",
"kind": "ObjectType"
},
{
"definition": {
"typeName": "Artist",
"permissions": [
{
"role": "admin",
"output": {
"allowedFields": [
"ArtistId",
"Name"
]
}
}
]
},
"version": "v1",
"kind": "TypePermissions"
},
{
"definition": {
"name": "Artists",
"objectType": "Artist",
"source": {
"dataConnectorName": "db",
"collection": "Artist",
"typeMapping": {
"Artist": {
"fieldMapping": {
"ArtistId": {
"column": "ArtistId"
},
"Name": {
"column": "Name"
}
}
}
}
},
"graphql": {
"selectUniques": [
]
},
"filterableFields": [
{
"fieldName": "ArtistId",
"operators": {
"enableAll": true
}
},
{
"fieldName": "Name",
"operators": {
"enableAll": true
}
}
],
"orderableFields": [
{
"fieldName": "ArtistId",
"orderByDirections": {
"enableAll": true
}
},
{
"fieldName": "Name",
"orderByDirections": {
"enableAll": true
}
}
]
},
"version": "v1",
"kind": "Model"
},
{
"definition": {
"modelName": "Artists",
"permissions": [
{
"role": "admin",
"select": {
"filter": null
}
}
]
},
"version": "v1",
"kind": "ModelPermissions"
},
{
"definition": {
"name": "Album",
"fields": [
{
"name": "AlbumId",
"type": "Int"
},
{
"name": "Title",
"type": "String"
},
{
"name": "ArtistId",
"type": "Int"
}
],
"graphql": {
"typeName": "Album_2"
}
},
"version": "v1",
"kind": "ObjectType"
},
{
"definition": {
"typeName": "Album",
"permissions": [
{
"role": "admin",
"output": {
"allowedFields": [
"AlbumId",
"Title",
"ArtistId"
]
}
}
]
},
"version": "v1",
"kind": "TypePermissions"
},
{
"definition": {
"name": "Albums",
"objectType": "Album",
"source": {
"dataConnectorName": "db",
"collection": "Album",
"typeMapping": {
"Album": {
"fieldMapping": {
"AlbumId": {
"column": "AlbumId"
},
"Title": {
"column": "Title"
},
"ArtistId": {
"column": "ArtistId"
}
}
}
}
},
"graphql": {
"selectUniques": [
]
},
"filterableFields": [
{
"fieldName": "AlbumId",
"operators": {
"enableAll": true
}
},
{
"fieldName": "Title",
"operators": {
"enableAll": true
}
},
{
"fieldName": "ArtistId",
"operators": {
"enableAll": true
}
}
],
"orderableFields": [
{
"fieldName": "AlbumId",
"orderByDirections": {
"enableAll": true
}
},
{
"fieldName": "Title",
"orderByDirections": {
"enableAll": true
}
},
{
"fieldName": "ArtistId",
"orderByDirections": {
"enableAll": true
}
}
]
},
"version": "v1",
"kind": "Model"
},
{
"definition": {
"modelName": "Albums",
"permissions": [
{
"role": "admin",
"select": {
"filter": null
}
}
]
},
"version": "v1",
"kind": "ModelPermissions"
},
{
"definition": {
"name": "Track",
"fields": [
{
"name": "TrackId",
"type": "Int"
},
{
"name": "Name",
"type": "String"
},
{
"name": "AlbumId",
"type": "Int"
}
],
"graphql": {
"typeName": "Track"
}
},
"version": "v1",
"kind": "ObjectType"
},
{
"definition": {
"typeName": "Track",
"permissions": [
{
"role": "admin",
"output": {
"allowedFields": [
"TrackId",
"Name",
"AlbumId"
]
}
}
]
},
"version": "v1",
"kind": "TypePermissions"
},
{
"definition": {
"name": "Tracks",
"objectType": "Track",
"source": {
"dataConnectorName": "db",
"collection": "Track",
"typeMapping": {
"Track": {
"fieldMapping": {
"TrackId": {
"column": "TrackId"
},
"Name": {
"column": "Name"
},
"AlbumId": {
"column": "AlbumId"
}
}
}
}
},
"graphql": {
"selectUniques": [
]
},
"filterableFields": [
{
"fieldName": "TrackId",
"operators": {
"enableAll": true
}
},
{
"fieldName": "Name",
"operators": {
"enableAll": true
}
},
{
"fieldName": "AlbumId",
"operators": {
"enableAll": true
}
}
],
"orderableFields": [
{
"fieldName": "TrackId",
"orderByDirections": {
"enableAll": true
}
},
{
"fieldName": "Name",
"orderByDirections": {
"enableAll": true
}
},
{
"fieldName": "AlbumId",
"orderByDirections": {
"enableAll": true
}
}
]
},
"version": "v1",
"kind": "Model"
},
{
"definition": {
"modelName": "Tracks",
"permissions": [
{
"role": "admin",
"select": {
"filter": null
}
}
]
},
"version": "v1",
"kind": "ModelPermissions"
},
{
"definition": {
"source": "Track",
"name": "Album",
"target": {
"model": {
"name": "Albums",
"subgraph": "connector_2",
"relationshipType": "Object"
}
},
"mapping": [
{
"source": {
"fieldPath": [
{
"fieldName": "AlbumId"
}
]
},
"target": {
"modelField": [
{
"fieldName": "AlbumId"
}
]
}
}
]
},
"version": "v1",
"kind": "Relationship"
}
]
},
{
"name": "connector_2",
"objects": [
{
"definition": {
"name": "Album",
"fields": [
{
"name": "AlbumId",
"type": "Int"
},
{
"name": "Title",
"type": "String"
},
{
"name": "ArtistId",
"type": "Int"
}
],
"graphql": {
"typeName": "Album"
}
},
"version": "v1",
"kind": "ObjectType"
},
{
"definition": {
"typeName": "Album",
"permissions": [
{
"role": "admin",
"output": {
"allowedFields": [
"AlbumId",
"Title",
"ArtistId"
]
}
}
]
},
"version": "v1",
"kind": "TypePermissions"
},
{
"definition": {
"name": "Albums",
"objectType": "Album",
"source": {
"dataConnectorName": "db",
"collection": "Album",
"typeMapping": {
"Album": {
"fieldMapping": {
"AlbumId": {
"column": "AlbumId"
},
"Title": {
"column": "Title"
},
"ArtistId": {
"column": "ArtistId"
}
}
}
}
},
"graphql": {
"selectUniques": [
],
"selectMany": {
"queryRootField": "Album"
}
},
"filterableFields": [
{
"fieldName": "AlbumId",
"operators": {
"enableAll": true
}
},
{
"fieldName": "Title",
"operators": {
"enableAll": true
}
},
{
"fieldName": "ArtistId",
"operators": {
"enableAll": true
}
}
],
"orderableFields": [
{
"fieldName": "AlbumId",
"orderByDirections": {
"enableAll": true
}
},
{
"fieldName": "Title",
"orderByDirections": {
"enableAll": true
}
},
{
"fieldName": "ArtistId",
"orderByDirections": {
"enableAll": true
}
}
]
},
"version": "v1",
"kind": "Model"
},
{
"definition": {
"modelName": "Albums",
"permissions": [
{
"role": "admin",
"select": {
"filter": null
}
}
]
},
"version": "v1",
"kind": "ModelPermissions"
},
{
"definition": {
"source": "Album",
"name": "Tracks",
"target": {
"model": {
"name": "Tracks",
"subgraph": "connector_1",
"relationshipType": "Array"
}
},
"mapping": [
{
"source": {
"fieldPath": [
{
"fieldName": "AlbumId"
}
]
},
"target": {
"modelField": [
{
"fieldName": "AlbumId"
}
]
}
}
]
},
"version": "v1",
"kind": "Relationship"
},
{
"definition": {
"source": "Album",
"name": "Artist",
"target": {
"model": {
"name": "Artists",
"subgraph": "connector_1",
"relationshipType": "Object"
}
},
"mapping": [
{
"source": {
"fieldPath": [
{
"fieldName": "ArtistId"
}
]
},
"target": {
"modelField": [
{
"fieldName": "ArtistId"
}
]
}
}
]
},
"version": "v1",
"kind": "Relationship"
}
]
}
]
}

View File

@ -1,15 +1,15 @@
{
cities {
code
name
departments {
name
ministers {
name
Album {
AlbumId
ArtistId
Tracks {
TrackId
Album {
AlbumId
}
}
census {
data
Artist {
ArtistId
}
}
}

View File

@ -5,27 +5,36 @@
{
"type": "modelSelect",
"value": {
"modelName": "cities",
"modelName": "AuthorMany",
"queryRequest": {
"collection": "city",
"collection": "authors",
"query": {
"fields": {
"code": {
"author_id": {
"type": "column",
"column": "code"
"column": "id"
},
"name": {
"first_name": {
"type": "column",
"column": "name"
"column": "first_name"
},
"__hasura_phantom_field__code": {
"__hasura_phantom_field__id": {
"type": "column",
"column": "code"
"column": "id"
}
}
},
"arguments": {},
"collection_relationships": {}
},
"ndcExplain": {
"type": "response",
"value": {
"details": {
"Execution Plan": "Aggregate (cost=22.77..22.78 rows=1 width=32)\n -> Aggregate (cost=22.75..22.76 rows=1 width=32)\n -> Seq Scan on author \"%0_authors\" (cost=0.00..18.50 rows=850 width=36)",
"SQL Query": "EXPLAIN\nSELECT\n coalesce(json_agg(row_to_json(\"%1_universe\")), '[]') AS \"universe\"\nFROM\n (\n SELECT\n *\n FROM\n (\n SELECT\n coalesce(json_agg(row_to_json(\"%2_rows\")), '[]') AS \"rows\"\n FROM\n (\n SELECT\n \"%0_authors\".\"id\" AS \"author_id\",\n \"%0_authors\".\"first_name\" AS \"first_name\",\n \"%0_authors\".\"id\" AS \"__hasura_phantom_field__id\"\n FROM\n \"public\".\"author\" AS \"%0_authors\"\n ) AS \"%2_rows\"\n ) AS \"%2_rows\"\n ) AS \"%1_universe\""
}
}
}
}
},
@ -34,21 +43,21 @@
"value": {
"type": "modelSelect",
"value": {
"modelName": "weather",
"modelName": "Articles",
"queryRequest": {
"collection": "weather",
"collection": "articles",
"query": {
"fields": {
"forecast": {
"title": {
"type": "column",
"column": "forecast"
"column": "title"
}
},
"where": {
"type": "binary_comparison_operator",
"column": {
"type": "column",
"name": "code",
"name": "author_id",
"path": []
},
"operator": {
@ -56,13 +65,22 @@
},
"value": {
"type": "variable",
"name": "$code"
"name": "$author_id"
}
}
},
"arguments": {},
"collection_relationships": {},
"variables": []
},
"ndcExplain": {
"type": "response",
"value": {
"details": {
"Execution Plan": "",
"SQL Query": "EXPLAIN\nSELECT\n coalesce(json_agg(\"%5_universe_agg\".\"universe\"), '[]') AS \"universe\"\nFROM\n (\n SELECT\n row_to_json(\"%2_universe\") AS \"universe\"\n FROM\n json_to_recordset(cast($1 as json)) AS \"%0_%variables_table\"(\"%variable_order\" int)\n CROSS JOIN LATERAL (\n SELECT\n *\n FROM\n (\n SELECT\n coalesce(json_agg(row_to_json(\"%3_rows\")), '[]') AS \"rows\"\n FROM\n (\n SELECT\n \"%1_articles\".\"title\" AS \"title\"\n FROM\n \"public\".\"article\" AS \"%1_articles\"\n WHERE\n (\n \"%1_articles\".\"author_id\" = cast(\"%0_%variables_table\".\"$author_id\" as integer)\n )\n ) AS \"%3_rows\"\n ) AS \"%3_rows\"\n ) AS \"%2_universe\"\n ORDER BY\n \"%0_%variables_table\".\"%variable_order\" ASC\n ) AS \"%5_universe_agg\""
}
}
}
}
}

View File

@ -1,9 +1,9 @@
{
cities {
code
name
weather {
forecast
AuthorMany {
author_id
first_name
Articles {
title
}
}
}

View File

@ -0,0 +1,15 @@
{
"version": "v2",
"supergraph": {
"objects": []
},
"subgraphs": [
{
"name": "default",
"objects": []
}
],
"flags": {
"require_ndc_capabilities": false
}
}

File diff suppressed because it is too large Load Diff

View File

@ -3,48 +3,99 @@
"type": "parallel",
"value": [
{
"type": "modelSelect",
"type": "commandSelect",
"value": {
"modelName": "cities",
"commandName": "getActorById",
"queryRequest": {
"collection": "city",
"collection": "get_actor_by_id",
"query": {
"fields": {
"code": {
"type": "column",
"column": "code"
},
"name": {
"type": "column",
"column": "name"
}
}
},
"arguments": {},
"arguments": {
"id": {
"type": "literal",
"value": 1
}
},
"collection_relationships": {}
},
"ndcExplain": {
"type": "notSupported"
}
}
},
{
"type": "modelSelect",
"value": {
"modelName": "countries",
"modelName": "AuthorMany",
"queryRequest": {
"collection": "country",
"collection": "authors",
"query": {
"fields": {
"code": {
"author_id": {
"type": "column",
"column": "code"
"column": "id"
},
"name": {
"first_name": {
"type": "column",
"column": "name"
"column": "first_name"
}
},
"order_by": {
"elements": [
{
"order_direction": "desc",
"target": {
"type": "column",
"name": "id",
"path": []
}
}
]
}
},
"arguments": {},
"collection_relationships": {}
},
"ndcExplain": {
"type": "response",
"value": {
"details": {
"Execution Plan": "Aggregate (cost=73.67..73.68 rows=1 width=32)\n -> Aggregate (cost=73.65..73.66 rows=1 width=32)\n -> Subquery Scan on \"%2_rows\" (cost=0.15..69.40 rows=850 width=60)\n -> Index Scan Backward using author_pkey on author \"%0_authors\" (cost=0.15..60.90 rows=850 width=36)",
"SQL Query": "EXPLAIN\nSELECT\n coalesce(json_agg(row_to_json(\"%1_universe\")), '[]') AS \"universe\"\nFROM\n (\n SELECT\n *\n FROM\n (\n SELECT\n coalesce(json_agg(row_to_json(\"%2_rows\")), '[]') AS \"rows\"\n FROM\n (\n SELECT\n \"%0_authors\".\"id\" AS \"author_id\",\n \"%0_authors\".\"first_name\" AS \"first_name\"\n FROM\n \"public\".\"author\" AS \"%0_authors\"\n ORDER BY\n \"%0_authors\".\"id\" DESC\n ) AS \"%2_rows\"\n ) AS \"%2_rows\"\n ) AS \"%1_universe\""
}
}
}
}
},
{
"type": "modelSelect",
"value": {
"modelName": "MovieMany",
"queryRequest": {
"collection": "movies",
"query": {
"fields": {
"title": {
"type": "column",
"column": "title"
},
"rating": {
"type": "column",
"column": "rating"
}
}
},
"arguments": {},
"collection_relationships": {}
},
"ndcExplain": {
"type": "notSupported"
}
}
}

View File

@ -1,10 +1,15 @@
{
cities {
code
getActorById(actor_id: 1){
name
}
countries {
code
name
AuthorMany(order_by: {author_id: Desc}){
author_id
first_name
}
}
MovieMany{
title
rating
}
}

View File

@ -32,7 +32,7 @@ pub type Path = Vec<PathSegment>;
/// A path segment is either a field name or an index into a list.
/// https://spec.graphql.org/October2021/#sel-HAPHRPJABABEyoB
#[derive(Serialize, Debug)]
#[derive(Serialize, Debug, PartialEq)]
#[serde(untagged)]
pub enum PathSegment {
/// Path segment that represent a field.
@ -53,7 +53,7 @@ impl PathSegment {
}
}
#[derive(Serialize, Debug)]
#[derive(Serialize, Debug, PartialEq)]
pub struct Extensions {
/// Details of any error
pub details: serde_json::Value,
@ -61,7 +61,7 @@ pub struct Extensions {
/// A GraphQL error as defined by the spec.
/// https://spec.graphql.org/October2021/#sec-Errors.Error-result-format
#[derive(Serialize, Debug)]
#[derive(Serialize, Debug, PartialEq)]
pub struct GraphQLError {
/// A string describing the error
pub message: String,