Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

refactor(interactive): Add Error Codes in CompilePlan JNI Results #4472

Merged
merged 13 commits into from
Feb 10, 2025
Merged
70 changes: 45 additions & 25 deletions docs/interactive_engine/graph_planner.md
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ Plan GraphPlannerWrapper::CompilePlan(const std::string &compiler_config_path,
const std::string &cypher_query_string,
const std::string &graph_schema_yaml,
const std::string &graph_statistic_json)

```

### Getting Started
Expand Down Expand Up @@ -74,31 +75,50 @@ To demonstrate the usage of the JNI interface, an example binary `test_graph_pla
bin/test_graph_planner libs native ./conf/graph.yaml ./conf/modern_statistics.json "MATCH (n) RETURN n, COUNT(n);" ./conf/gs_interactive_hiactor.yaml
```

The output consists of the physical plan (in byte format) and the result schema (in YAML format). The physical plan adheres to the specifications defined in the [protobuf]().

Below is an example of a result schema:

```yaml
schema:
name: default
description: default desc
mode: READ
extension: .so
library: libdefault.so
params: []
returns:
- name: n
type: {primitive_type: DT_UNKNOWN}
- name: $f1
type: {primitive_type: DT_SIGNED_INT64}
type: UNKNOWN
query: MATCH (n) RETURN n, COUNT(n);
```

The `returns` field defines the structure of the data returned by backend engines. Each nested entry in the returns field includes three components:
- the column name, which specifies the name of the result column;
- the entry’s ordinal position, which determines the column ID;
- the type, which enforces the data type constraint for the column.
The output consists of three key fields:

1. Error code for plan compilation. The following table outlines the error codes, along with their corresponding descriptions:
| Error Code | Description |
| -------------------------- | ------------------------------------------------------------------------------------------------ |
| OK | Query compilation succeeded. |
| GREMLIN_INVALID_SYNTAX | The provided Gremlin query contains syntax errors. |
| CYPHER_INVALID_SYNTAX | The provided Cypher query contains syntax errors. |
| TAG_NOT_FOUND | The specified tag is not found in the current query context. |
| LABEL_NOT_FOUND | The specified label is not found under the schema constraints. |
| PROPERTY_NOT_FOUND | The specified property is not found under the schema constraints. |
| TYPE_INFERENCE_FAILED | The query contains invalid graph patterns that violate schema constraints. |
| LOGICAL_PLAN_BUILD_FAILED | An error occurred during logical plan optimization. The error message provides specific details. |
| PHYSICAL_PLAN_BUILD_FAILED | An error occurred during physical plan construction. |
| GREMLIN_INVALID_RESULT | An error occurred while parsing the Gremlin query results. |
| CYPHER_INVALID_RESULT | An error occurred while parsing the Cypher query results. |
| ENGINE_UNAVAILABLE | The lower execution engine is unavailable. |
| QUERY_EXECUTION_TIMEOUT | The query execution time exceeded the predefined limit. |
| META_SCHEMA_NOT_READY | The schema metadata is not ready for querying. |
| META_STATISTICS_NOT_READY | The statistical metadata is not ready for querying. |
| EMPTY_RESULT | The compilation determines that the query will produce an empty result. |
2. Physical plan in byte format, adheres to the specifications defined in the [protobuf](https://github.com/alibaba/GraphScope/blob/main/interactive_engine/executor/ir/proto/physical.proto).
3. Result schema in YAML format. Below is an example of a result schema:
```yaml
schema:
name: default
description: default desc
mode: READ
extension: .so
library: libdefault.so
params: []
returns:
- name: n
type: {primitive_type: DT_UNKNOWN}
- name: $f1
type: {primitive_type: DT_SIGNED_INT64}
type: UNKNOWN
query: MATCH (n) RETURN n, COUNT(n);
```

The `returns` field defines the structure of the data returned by backend engines. Each nested entry in the returns field includes three components:
- the column name, which specifies the name of the result column;
- the entry’s ordinal position, which determines the column ID;
- the type, which enforces the data type constraint for the column.

## Restful API

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -322,6 +322,8 @@
* @param opt anti or optional
*/
public GraphBuilder match(RelNode single, GraphOpt.Match opt) {
// verify the hops in the sentence is valid
new QueryExecutionValidator(configs).validate(new LogicalPlan(single), true);
if (FrontendConfig.GRAPH_TYPE_INFERENCE_ENABLED.get(configs)) {
single =
new GraphTypeInference(
Expand Down Expand Up @@ -374,6 +376,9 @@
}
Preconditions.checkArgument(
sentences.size() > 1, "at least two sentences are required in multiple match");
// verify the hops in each sentence is valid
sentences.forEach(
k -> new QueryExecutionValidator(configs).validate(new LogicalPlan(k), true));
if (FrontendConfig.GRAPH_TYPE_INFERENCE_ENABLED.get(configs)) {
sentences =
new GraphTypeInference(
Expand Down Expand Up @@ -493,148 +498,148 @@
* @param property
* @return
*/
public RexGraphVariable variable(@Nullable String alias, String property) {
alias = AliasInference.isDefaultAlias(alias) ? AliasInference.DEFAULT_NAME : alias;
Objects.requireNonNull(property);
String varName = AliasInference.SIMPLE_NAME(alias) + AliasInference.DELIMITER + property;
List<ColumnField> columnFields = getAliasField(alias);
if (columnFields.size() != 1) {
throw new FrontendException(
Code.PROPERTY_NOT_FOUND,
"cannot get property="
+ property
+ " from alias="
+ alias
+ ", expected one column, but found "
+ columnFields.size());
}
ColumnField columnField = columnFields.get(0);
RelDataTypeField aliasField = columnField.right;
if (property.equals(GraphProperty.LEN_KEY)) {
if (!(aliasField.getType() instanceof GraphPathType)) {
throw new FrontendException(
Code.PROPERTY_NOT_FOUND,
"cannot get property='len' from type class ["
+ aliasField.getType().getClass()
+ "], should be ["
+ GraphPathType.class
+ "]");
} else {
return RexGraphVariable.of(
aliasField.getIndex(),
new GraphProperty(GraphProperty.Opt.LEN),
columnField.left,
varName,
getTypeFactory().createSqlType(SqlTypeName.INTEGER));
}
}
if (!(aliasField.getType() instanceof GraphSchemaType)) {
throw new FrontendException(
Code.PROPERTY_NOT_FOUND,
"cannot get property=['id', 'label', 'all', 'key'] from type class ["
+ aliasField.getType().getClass()
+ "], should be ["
+ GraphSchemaType.class
+ "]");
}
if (property.equals(GraphProperty.LABEL_KEY)) {
GraphSchemaType schemaType = (GraphSchemaType) aliasField.getType();
return RexGraphVariable.of(
aliasField.getIndex(),
new GraphProperty(GraphProperty.Opt.LABEL),
columnField.left,
varName,
schemaType.getLabelType());
} else if (property.equals(GraphProperty.ID_KEY)) {
return RexGraphVariable.of(
aliasField.getIndex(),
new GraphProperty(GraphProperty.Opt.ID),
columnField.left,
varName,
getTypeFactory().createSqlType(SqlTypeName.BIGINT));
} else if (property.equals(GraphProperty.ALL_KEY)) {
return RexGraphVariable.of(
aliasField.getIndex(),
new GraphProperty(GraphProperty.Opt.ALL),
columnField.left,
varName,
getTypeFactory().createSqlType(SqlTypeName.ANY));
} else if (property.equals(GraphProperty.START_V_KEY)) {
if (!(aliasField.getType() instanceof GraphPathType)) {
throw new FrontendException(
Code.PROPERTY_NOT_FOUND,
"cannot get property='start_v' from type class ["
+ aliasField.getType().getClass()
+ "], should be ["
+ GraphPathType.class
+ "]");
} else {
Preconditions.checkArgument(size() > 0, "frame stack is empty");
RelNode peek = peek();
Preconditions.checkArgument(
peek != null && !peek.getInputs().isEmpty(),
"path expand should have start vertex");
RelNode input = peek.getInput(0);
return RexGraphVariable.of(
aliasField.getIndex(),
new GraphProperty(GraphProperty.Opt.START_V),
columnField.left,
varName,
input.getRowType().getFieldList().get(0).getType());
}
} else if (property.equals(GraphProperty.END_V_KEY)) {
if (!(aliasField.getType() instanceof GraphPathType)) {
throw new FrontendException(
Code.PROPERTY_NOT_FOUND,
"cannot get property='end_v' from type class ["
+ aliasField.getType().getClass()
+ "], should be ["
+ GraphPathType.class
+ "]");
} else {
GraphPathType pathType = (GraphPathType) aliasField.getType();
return RexGraphVariable.of(
aliasField.getIndex(),
new GraphProperty(GraphProperty.Opt.END_V),
columnField.left,
varName,
pathType.getComponentType().getGetVType());
}
}
GraphSchemaType graphType = (GraphSchemaType) aliasField.getType();
List<String> properties = new ArrayList<>();
boolean isColumnId =
(relOptSchema instanceof GraphOptSchema)
? ((GraphOptSchema) relOptSchema).getRootSchema().isColumnId()
: false;
for (RelDataTypeField pField : graphType.getFieldList()) {
if (pField.getName().equals(property)) {
return RexGraphVariable.of(
aliasField.getIndex(),
isColumnId
? new GraphProperty(new GraphNameOrId(pField.getIndex()))
: new GraphProperty(new GraphNameOrId(pField.getName())),
columnField.left,
varName,
pField.getType());
}
properties.add(pField.getName());
}
throw new FrontendException(
Code.PROPERTY_NOT_FOUND,
"{property="
+ property
+ "} "
+ "not found; expected properties are: "
+ properties);
}

/**
* get {@code RelDataTypeField} by the given alias, for {@code RexGraphVariable} inference
*
* @param alias
* @return
*/

Check notice on line 642 in interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/tools/GraphBuilder.java

View check run for this annotation

codefactor.io / CodeFactor

interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/tools/GraphBuilder.java#L501-L642

Complex Method
private List<ColumnField> getAliasField(String alias) {
Objects.requireNonNull(alias);
if (alias.equals(AliasInference.STAR)) {
Expand Down Expand Up @@ -851,254 +856,254 @@
}
}

private boolean isCurrentSupported(SqlOperator operator) {
SqlKind sqlKind = operator.getKind();
return sqlKind.belongsTo(SqlKind.BINARY_ARITHMETIC)
|| sqlKind.belongsTo(SqlKind.COMPARISON)
|| sqlKind == SqlKind.AND
|| sqlKind == SqlKind.OR
|| sqlKind == SqlKind.DESCENDING
|| (sqlKind == SqlKind.OTHER_FUNCTION
&& (operator.getName().equals("POWER")
|| operator.getName().equals("<<")
|| operator.getName().equals(">>")))
|| (sqlKind == SqlKind.MINUS_PREFIX)
|| (sqlKind == SqlKind.CASE)
|| (sqlKind == SqlKind.PROCEDURE_CALL)
|| (sqlKind == SqlKind.NOT)
|| sqlKind == SqlKind.ARRAY_VALUE_CONSTRUCTOR
|| sqlKind == SqlKind.MAP_VALUE_CONSTRUCTOR
|| sqlKind == SqlKind.IS_NULL
|| sqlKind == SqlKind.IS_NOT_NULL
|| sqlKind == SqlKind.EXTRACT
|| sqlKind == SqlKind.SEARCH
|| sqlKind == SqlKind.POSIX_REGEX_CASE_SENSITIVE
|| sqlKind == SqlKind.AS
|| sqlKind == SqlKind.BIT_AND
|| sqlKind == SqlKind.BIT_OR
|| sqlKind == SqlKind.BIT_XOR
|| (sqlKind == SqlKind.OTHER
&& (operator.getName().equals("IN")
|| operator.getName().equals("DATETIME_MINUS")
|| operator.getName().equals("PATH_CONCAT")
|| operator.getName().equals("PATH_FUNCTION"))
|| operator.getName().startsWith(GraphFunctions.FUNCTION_PREFIX))
|| sqlKind == SqlKind.ARRAY_CONCAT;
}

Check notice on line 893 in interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/tools/GraphBuilder.java

View check run for this annotation

codefactor.io / CodeFactor

interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/tools/GraphBuilder.java#L859-L893

Complex Method
@Override
public GraphBuilder filter(RexNode... conditions) {
return filter(ImmutableList.copyOf(conditions));
}

@Override
public GraphBuilder filter(Iterable<? extends RexNode> conditions) {
conditions = flatExprs((Iterable<RexNode>) conditions);

RexVisitor propertyChecker = new RexPropertyChecker(true, this);
for (RexNode condition : conditions) {
RelDataType type = condition.getType();
// make sure all conditions have the Boolean return type
if (!(type instanceof BasicSqlType) || type.getSqlTypeName() != SqlTypeName.BOOLEAN) {
throw new IllegalArgumentException(
"filter condition "
+ condition
+ " should return Boolean value, but is "
+ type);
}
// check property existence for specific label
condition.accept(propertyChecker);
}

// specific implementation for gremlin, project will change the 'head' before the current
// filter, which need to be recovered later
RelDataTypeField recoverHead = null;

RexSubQueryPreComputer preComputer = new RexSubQueryPreComputer(this);
List<RexNode> newConditions = Lists.newArrayList();
for (RexNode condition : conditions) {
// if the condition has subquery as its operand, i.e. where(out().out().count() > 2),
// subquery(out().out().count()) needs to be precomputed in advance, the condition also
// needs to be adapted
newConditions.add(preComputer.precompute(condition));
}
// project subquery in advance
if (!preComputer.getSubQueryNodes().isEmpty()) {
RelNode input = requireNonNull(peek(), "frame stack is empty");
if (input.getRowType().getFieldList().size() == 1) {
RelDataTypeField field = input.getRowType().getFieldList().get(0);
// give a non-default alias to the head, so that it can be recovered later
if (field.getName() == AliasInference.DEFAULT_NAME) {
Set<String> uniqueAliases = AliasInference.getUniqueAliasList(input, true);
uniqueAliases.addAll(preComputer.getSubQueryAliases());
String nonDefault = AliasInference.inferAliasWithPrefix("$f", uniqueAliases);
// set the non default alias to the input
as(nonDefault);
recoverHead =
new RelDataTypeFieldImpl(
nonDefault, generateAliasId(nonDefault), field.getType());
} else {
recoverHead = field;
}
}
project(preComputer.getSubQueryNodes(), preComputer.getSubQueryAliases(), true);
conditions =
newConditions.stream()
.map(k -> k.accept(new RexTmpVariableConverter(true, this)))
.collect(Collectors.toList());
}
super.filter(ImmutableSet.of(), conditions);
// fuse filter with the previous table scan if meets the conditions
Filter filter = topFilter();
if (filter != null) {
GraphBuilder builder =
GraphBuilder.create(
this.configs, (GraphOptCluster) getCluster(), getRelOptSchema());
RexNode condition = filter.getCondition();
RelNode input = !filter.getInputs().isEmpty() ? filter.getInput(0) : null;
if (input instanceof AbstractBindableTableScan) {
AbstractBindableTableScan tableScan = (AbstractBindableTableScan) input;
List<Integer> aliasIds =
condition.accept(
new RexVariableAliasCollector<>(
true, RexGraphVariable::getAliasId));
// fuze all conditions into table scan
if (!aliasIds.isEmpty()
&& ImmutableList.of(AliasInference.DEFAULT_ID, tableScan.getAliasId())
.containsAll(aliasIds)) {
condition =
condition.accept(
new RexVariableAliasConverter(
true,
this,
AliasInference.SIMPLE_NAME(AliasInference.DEFAULT_NAME),
AliasInference.DEFAULT_ID));
// add condition into table scan
// pop the filter from the inner stack
replaceTop(fuseFilters(tableScan, condition, builder));
}
} else if (input instanceof AbstractLogicalMatch) {
List<RexNode> extraFilters = Lists.newArrayList();
AbstractLogicalMatch match =
fuseFilters((AbstractLogicalMatch) input, condition, extraFilters, builder);
if (!match.equals(input)) {
if (extraFilters.isEmpty()) {
replaceTop(match);
} else {
replaceTop(builder.push(match).filter(extraFilters).build());
}
}
}
}
if (recoverHead != null) {
project(ImmutableList.of(variable(recoverHead.getName())), ImmutableList.of(), true);
}
return this;
}

Check notice on line 1003 in interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/tools/GraphBuilder.java

View check run for this annotation

codefactor.io / CodeFactor

interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/tools/GraphBuilder.java#L899-L1003

Complex Method
private AbstractBindableTableScan fuseFilters(
AbstractBindableTableScan tableScan, RexNode condition, GraphBuilder builder) {
RexFilterClassifier classifier = new RexFilterClassifier(builder, tableScan);
ClassifiedFilter filterResult = classifier.classify(condition);
List<Comparable> labelValues = filterResult.getLabelValues();
List<RexNode> uniqueKeyFilters = Lists.newArrayList(filterResult.getUniqueKeyFilters());
List<RexNode> extraFilters = Lists.newArrayList(filterResult.getExtraFilters());
if (!labelValues.isEmpty()) {
GraphLabelType labelType =
((GraphSchemaType) tableScan.getRowType().getFieldList().get(0).getType())
.getLabelType();
List<String> labelsToKeep =
labelType.getLabelsEntry().stream()
.filter(k -> labelValues.contains(k.getLabel()))
.map(k -> k.getLabel())
.collect(Collectors.toList());
Preconditions.checkArgument(
!labelsToKeep.isEmpty(),
"cannot find common labels between values= " + labelValues + " and label=",
labelType);
if (labelsToKeep.size() < labelType.getLabelsEntry().size()) {
LabelConfig newLabelConfig = new LabelConfig(false);
labelsToKeep.forEach(k -> newLabelConfig.addLabel(k));
if (tableScan instanceof GraphLogicalSource) {
builder.source(
new SourceConfig(
((GraphLogicalSource) tableScan).getOpt(),
newLabelConfig,
tableScan.getAliasName()));
} else if (tableScan instanceof GraphLogicalExpand) {
builder.push(tableScan.getInput(0))
.expand(
new ExpandConfig(
((GraphLogicalExpand) tableScan).getOpt(),
newLabelConfig,
tableScan.getAliasName()));
} else if (tableScan instanceof GraphLogicalGetV) {
builder.push(tableScan.getInput(0))
.getV(
new GetVConfig(
((GraphLogicalGetV) tableScan).getOpt(),
newLabelConfig,
tableScan.getAliasName()));
}
if (builder.size() > 0) {
// check if the property still exist after updating the label type
RexVisitor propertyChecker = new RexPropertyChecker(true, builder);
if (tableScan instanceof GraphLogicalSource) {
RexNode originalUniqueKeyFilters =
((GraphLogicalSource) tableScan).getUniqueKeyFilters();
if (originalUniqueKeyFilters != null) {
originalUniqueKeyFilters.accept(propertyChecker);
builder.filter(originalUniqueKeyFilters);
}
if (!uniqueKeyFilters.isEmpty()) {
builder.filter(uniqueKeyFilters);
uniqueKeyFilters.clear();
}
}
ImmutableList originalFilters = tableScan.getFilters();
if (ObjectUtils.isNotEmpty(originalFilters)) {
originalFilters.forEach(k -> ((RexNode) k).accept(propertyChecker));
builder.filter(originalFilters);
}
if (!extraFilters.isEmpty()) {
extraFilters.forEach(k -> k.accept(propertyChecker));
builder.filter(extraFilters);
extraFilters.clear();
}
tableScan = (AbstractBindableTableScan) builder.build();
}
}
}
if (tableScan instanceof GraphLogicalSource && !uniqueKeyFilters.isEmpty()) {
GraphLogicalSource source = (GraphLogicalSource) tableScan;
if (source.getUniqueKeyFilters() != null || uniqueKeyFilters.size() > 1) {
extraFilters.addAll(uniqueKeyFilters);
} else {
source.setUniqueKeyFilters(uniqueKeyFilters.get(0));
}
}
if (!extraFilters.isEmpty()) {
ImmutableList originalFilters = tableScan.getFilters();
if (ObjectUtils.isNotEmpty(originalFilters)) {
for (int i = 0; i < originalFilters.size(); ++i) {
extraFilters.add(i, (RexNode) originalFilters.get(i));
}
}
tableScan.setFilters(
ImmutableList.of(
RexUtil.composeConjunction(this.getRexBuilder(), extraFilters)));
}
return tableScan;
}

/**
* fuse label filters into the {@code match} if possible
* @param match
* @param condition
* @param extraFilters
* @param builder
* @return
*/

Check notice on line 1106 in interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/tools/GraphBuilder.java

View check run for this annotation

codefactor.io / CodeFactor

interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/tools/GraphBuilder.java#L1004-L1106

Complex Method
private AbstractLogicalMatch fuseFilters(
AbstractLogicalMatch match,
RexNode condition,
Expand Down Expand Up @@ -1486,122 +1491,122 @@
* @param nodes build limit() if empty
* @return
*/
@Override
public GraphBuilder sortLimit(
@Nullable RexNode offsetNode,
@Nullable RexNode fetchNode,
Iterable<? extends RexNode> nodes) {
if (offsetNode != null && !(offsetNode instanceof RexLiteral)) {
throw new IllegalArgumentException("OFFSET node must be RexLiteral");
}
if (offsetNode != null && !(offsetNode instanceof RexLiteral)) {
throw new IllegalArgumentException("FETCH node must be RexLiteral");
}

nodes = flatExprs((Iterable<RexNode>) nodes);

RelNode input = requireNonNull(peek(), "frame stack is empty");

List<RelDataTypeField> originalFields = input.getRowType().getFieldList();

Registrar registrar = new Registrar(this, input, true);
List<RexNode> registerNodes = registrar.registerExpressions(ImmutableList.copyOf(nodes));

// expressions need to be projected in advance
if (!registrar.getExtraNodes().isEmpty()) {
if (input.getRowType().getFieldList().size() == 1) {
RelDataTypeField field = input.getRowType().getFieldList().get(0);
// give a non-default alias to the head, so that the tail project can preserve the
// head field
if (field.getName() == AliasInference.DEFAULT_NAME) {
Set<String> uniqueAliases = AliasInference.getUniqueAliasList(input, true);
uniqueAliases.addAll(registrar.getExtraAliases());
String nonDefault = AliasInference.inferAliasWithPrefix("$f", uniqueAliases);
// set the non default alias to the input
as(nonDefault);
originalFields =
Lists.newArrayList(
new RelDataTypeFieldImpl(
nonDefault,
generateAliasId(nonDefault),
field.getType()));
}
}
project(registrar.getExtraNodes(), registrar.getExtraAliases(), registrar.isAppend());
RexTmpVariableConverter converter = new RexTmpVariableConverter(true, this);
registerNodes =
registerNodes.stream()
.map(k -> k.accept(converter))
.collect(Collectors.toList());
input = requireNonNull(peek(), "frame stack is empty");
}

List<RelFieldCollation> fieldCollations = fieldCollations(registerNodes);
Config config = Utils.getFieldValue(RelBuilder.class, this, "config");

// limit 0 -> return empty value
if ((fetchNode != null && RexLiteral.intValue(fetchNode) == 0) && config.simplifyLimit()) {
return (GraphBuilder) empty();
}

// output all results without any order -> skip
if (offsetNode == null && fetchNode == null && fieldCollations.isEmpty()) {
return this; // sort is trivial
}
// sortLimit is actually limit if collations are empty
if (fieldCollations.isEmpty()) {
// fuse limit with the previous sort operator
// order + limit -> topK
if (input instanceof Sort) {
Sort sort2 = (Sort) input;
// output all results without any limitations
if (sort2.offset == null && sort2.fetch == null) {
RelNode sort =
GraphLogicalSort.create(
sort2.getInput(), sort2.collation, offsetNode, fetchNode);
replaceTop(sort);
return this;
}
}
// order + project + limit -> topK + project
if (input instanceof Project) {
Project project = (Project) input;
if (project.getInput() instanceof Sort) {
Sort sort2 = (Sort) project.getInput();
if (sort2.offset == null && sort2.fetch == null) {
RelNode sort =
GraphLogicalSort.create(
sort2.getInput(), sort2.collation, offsetNode, fetchNode);
replaceTop(
GraphLogicalProject.create(
(GraphOptCluster) project.getCluster(),
project.getHints(),
sort,
project.getProjects(),
project.getRowType(),
((GraphLogicalProject) project).isAppend()));
return this;
}
}
}
}
RelNode sort =
GraphLogicalSort.create(
input, GraphRelCollations.of(fieldCollations), offsetNode, fetchNode);
replaceTop(sort);
// to remove the extra columns we have added
if (!registrar.getExtraAliases().isEmpty()) {
List<RexNode> originalExprs = new ArrayList<>();
List<String> originalAliases = new ArrayList<>();
for (RelDataTypeField field : originalFields) {
originalExprs.add(variable(field.getName()));
originalAliases.add(field.getName());
}
project(originalExprs, originalAliases, false);
}
return this;
}

Check notice on line 1609 in interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/tools/GraphBuilder.java

View check run for this annotation

codefactor.io / CodeFactor

interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/tools/GraphBuilder.java#L1494-L1609

Complex Method
public GraphBuilder dedupBy(Iterable<? extends RexNode> nodes) {
RelNode input = requireNonNull(peek(), "frame stack is empty");

Expand Down Expand Up @@ -1825,104 +1830,104 @@
* @param alias
* @return
*/
@Override
public GraphBuilder as(String alias) {
RelNode top = requireNonNull(peek(), "frame stack is empty");
// skip intermediate operations which make no changes to the row type, i.e.
// filter/limit/dedup...
RelNode parent = null;
while (!top.getInputs().isEmpty() && top.getInput(0).getRowType() == top.getRowType()) {
parent = top;
top = top.getInput(0);
}
if (top instanceof AbstractBindableTableScan
|| top instanceof GraphLogicalPathExpand
|| top instanceof GraphLogicalProject
|| top instanceof GraphLogicalAggregate) {
RelDataType rowType = top.getRowType();
// we can assign the alias only if the top node has only one field, otherwise we skip
// the
// operation
if (rowType.getFieldList().size() != 1) {
return this;
}
build();
if (!top.getInputs().isEmpty()) {
push(top.getInput(0));
}
if (top instanceof GraphLogicalSource) {
GraphLogicalSource source = (GraphLogicalSource) top;
source(
new SourceConfig(
source.getOpt(), getLabelConfig(source.getTableConfig()), alias));
if (source.getUniqueKeyFilters() != null) {
filter(source.getUniqueKeyFilters());
}
if (ObjectUtils.isNotEmpty(source.getFilters())) {
filter(source.getFilters());
}
} else if (top instanceof GraphLogicalExpand) {
GraphLogicalExpand expand = (GraphLogicalExpand) top;
expand(
new ExpandConfig(
expand.getOpt(), getLabelConfig(expand.getTableConfig()), alias));
if (ObjectUtils.isNotEmpty(expand.getFilters())) {
filter(expand.getFilters());
}
} else if (top instanceof GraphLogicalGetV) {
GraphLogicalGetV getV = (GraphLogicalGetV) top;
getV(new GetVConfig(getV.getOpt(), getLabelConfig(getV.getTableConfig()), alias));
if (ObjectUtils.isNotEmpty(getV.getFilters())) {
filter(getV.getFilters());
}
} else if (top instanceof GraphLogicalPathExpand) {
GraphLogicalPathExpand pxdExpand = (GraphLogicalPathExpand) top;
GraphLogicalExpand expand = (GraphLogicalExpand) pxdExpand.getExpand();
GraphLogicalGetV getV = (GraphLogicalGetV) pxdExpand.getGetV();
PathExpandConfig.Builder pxdBuilder = PathExpandConfig.newBuilder(this);
RexNode offset = pxdExpand.getOffset(), fetch = pxdExpand.getFetch();
pxdBuilder
.expand(
new ExpandConfig(
expand.getOpt(),
getLabelConfig(expand.getTableConfig()),
expand.getAliasName()))
.getV(
new GetVConfig(
getV.getOpt(),
getLabelConfig(getV.getTableConfig()),
getV.getAliasName()))
.pathOpt(pxdExpand.getPathOpt())
.resultOpt(pxdExpand.getResultOpt())
.range(
offset == null
? 0
: ((RexLiteral) offset).getValueAs(Integer.class),
fetch == null ? -1 : ((RexLiteral) fetch).getValueAs(Integer.class))
.startAlias(pxdExpand.getStartAlias().getAliasName())
.alias(alias);
pathExpand(pxdBuilder.buildConfig());
} else if (top instanceof GraphLogicalProject) {
GraphLogicalProject project = (GraphLogicalProject) top;
project(project.getProjects(), Lists.newArrayList(alias), project.isAppend());
} else if (top instanceof GraphLogicalAggregate) {
GraphLogicalAggregate aggregate = (GraphLogicalAggregate) top;
// if group key is empty, we can assign the alias to the single aggregated value in
// group
if (aggregate.getGroupKey().groupKeyCount() == 0
&& aggregate.getAggCalls().size() == 1) {
GraphAggCall aggCall = aggregate.getAggCalls().get(0);
aggregate(aggregate.getGroupKey(), ImmutableList.of(aggCall.as(alias)));
}
}
if (parent != null && peek() != top) {
parent.replaceInput(0, build());
push(parent);
}
}
return this;
}

Check notice on line 1930 in interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/tools/GraphBuilder.java

View check run for this annotation

codefactor.io / CodeFactor

interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/tools/GraphBuilder.java#L1833-L1930

Complex Method
private LabelConfig getLabelConfig(TableConfig tableConfig) {
List<String> labels =
tableConfig.getTables().stream()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,6 +87,10 @@ public GraphPlanner(
this.validator = new QueryExecutionValidator(graphConfig);
}

public GraphRelOptimizer getOptimizer() {
return optimizer;
}

public PlannerInstance instance(String query, IrMeta irMeta) {
return instance(query, irMeta, null);
}
Expand Down
Loading
Loading