diff --git a/.github/workflows/interactive.yml b/.github/workflows/interactive.yml index b62a57073d28..3e9a6cda4d87 100644 --- a/.github/workflows/interactive.yml +++ b/.github/workflows/interactive.yml @@ -156,6 +156,9 @@ jobs: cp ${GITHUB_WORKSPACE}/flex/interactive/examples/new_graph_algo/import.yaml ${INTERACTIVE_WORKSPACE}/data/new_graph_algo/import.yaml mkdir -p ${INTERACTIVE_WORKSPACE}/data/modern_graph cp ${GITHUB_WORKSPACE}/flex/interactive/examples/modern_graph/graph.yaml ${INTERACTIVE_WORKSPACE}/data/modern_graph/graph.yaml + mkdir -p ${INTERACTIVE_WORKSPACE}/data/type_test + cp ${GITHUB_WORKSPACE}/interactive_engine/compiler/src/test/resources/flex_bench/modern.yaml ${INTERACTIVE_WORKSPACE}/data/type_test/graph.yaml + cp ${GITHUB_WORKSPACE}/interactive_engine/compiler/src/test/resources/flex_bench/import.yaml ${INTERACTIVE_WORKSPACE}/data/type_test/import.yaml # load graph cd ${GITHUB_WORKSPACE}/flex/build @@ -169,6 +172,15 @@ jobs: GLOG_v=10 ./bin/bulk_loader -g ${INTERACTIVE_WORKSPACE}/data/new_graph_algo/graph.yaml -l ${INTERACTIVE_WORKSPACE}/data/new_graph_algo/import.yaml -d ${INTERACTIVE_WORKSPACE}/data/new_graph_algo/indices/ export FLEX_DATA_DIR=../interactive/examples/modern_graph GLOG_v=10 ./bin/bulk_loader -g ${INTERACTIVE_WORKSPACE}/data/modern_graph/graph.yaml -l ../interactive/examples/modern_graph/bulk_load.yaml -d ${INTERACTIVE_WORKSPACE}/data/modern_graph/indices/ + export FLEX_DATA_DIR=${GITHUB_WORKSPACE}/interactive_engine/compiler/src/test/resources/flex_bench/data + GLOG_v=10 ./bin/bulk_loader -g ${INTERACTIVE_WORKSPACE}/data/type_test/graph.yaml -l ${INTERACTIVE_WORKSPACE}/data/type_test/import.yaml -d ${INTERACTIVE_WORKSPACE}/data/type_test/indices/ + + - name: Interactive Type Test + env: + INTERACTIVE_WORKSPACE: /tmp/interactive_workspace + run: | + cd ${GITHUB_WORKSPACE}/flex/tests/hqps + bash hqps_type_test.sh ${INTERACTIVE_WORKSPACE} ./interactive_config_test.yaml - name: Test HQPS admin http service env: diff --git a/flex/tests/hqps/hqps_type_test.sh b/flex/tests/hqps/hqps_type_test.sh new file mode 100644 index 000000000000..13a322f72c84 --- /dev/null +++ b/flex/tests/hqps/hqps_type_test.sh @@ -0,0 +1,94 @@ +#!/bin/bash +# Copyright 2020 Alibaba Group Holding Limited. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +set -e +SCRIPT_DIR=$(cd -- "$(dirname -- "${BASH_SOURCE[0]}")" &>/dev/null && pwd) +FLEX_HOME=${SCRIPT_DIR}/../../ +BULK_LOADER=${FLEX_HOME}/build/bin/bulk_loader +SERVER_BIN=${FLEX_HOME}/build/bin/interactive_server +GIE_HOME=${FLEX_HOME}/../interactive_engine/ + +# +if [ $# -ne 2 ]; then + echo "Receives: $# args, need 2 args" + echo "Usage: $0 " + exit 1 +fi + +INTERACTIVE_WORKSPACE=$1 +ENGINE_CONFIG_PATH=$2 + + +RED='\033[0;31m' +GREEN='\033[0;32m' +NC='\033[0m' # No Color +err() { + echo -e "${RED}[$(date +'%Y-%m-%d %H:%M:%S')] -ERROR- $* ${NC}" >&2 +} + +info() { + echo -e "${GREEN}[$(date +'%Y-%m-%d %H:%M:%S')] -INFO- $* ${NC}" +} + + +kill_service(){ + info "Kill Service first" + ps -ef | grep "com.alibaba.graphscope.GraphServer" | awk '{print $2}' | xargs kill -9 || true + ps -ef | grep "interactive_server" | awk '{print $2}' | xargs kill -9 || true + sleep 3 + # check if service is killed + info "Kill Service success" +} + +# kill service when exit +trap kill_service EXIT + + +# start engine service and load ldbc graph +start_engine_service(){ + # suppose graph has been loaded, check ${GRAPH_CSR_DATA_DIR} exists + + #check SERVER_BIN exists + if [ ! -f ${SERVER_BIN} ]; then + err "SERVER_BIN not found" + exit 1 + fi + cmd="${SERVER_BIN} -c ${ENGINE_CONFIG_PATH} --start-compiler true " + cmd="${cmd} -w ${INTERACTIVE_WORKSPACE} --enable-admin-service true > /tmp/engine.log 2>&1 &" + + info "Start engine service with command: ${cmd}" + ${cmd} & + sleep 5 + #check interactive_server is running, if not, exit + ps -ef | grep "interactive_server" | grep -v grep + + info "Start engine service success" +} + + + +run_type_test() { + echo "run type test" + pushd ${GIE_HOME}/compiler + cmd="mvn test -Dtest=com.alibaba.graphscope.cypher.integration.flex.bench.FlexTypeTest" + info "Run type test with command: ${cmd}" + ${cmd} + info "Run type test success" + popd +} + +kill_service +start_engine_service +run_type_test +kill_service diff --git a/flex/tests/hqps/interactive_config_test.yaml b/flex/tests/hqps/interactive_config_test.yaml index b27f35e45e13..b318d95dcb67 100644 --- a/flex/tests/hqps/interactive_config_test.yaml +++ b/flex/tests/hqps/interactive_config_test.yaml @@ -12,13 +12,17 @@ compute_engine: type: file # file/sqlite/etcd wal_uri: file://{GRAPH_DATA_DIR}/wal # Could be file://{GRAPH_DATA_DIR}/wal or other supported storage class. GRAPH_DATA_DIR is the placeholder for the graph data directory. compiler: + physical: + opt: + config: proto planner: is_on: true - opt: RBO + opt: CBO rules: - FilterIntoJoinRule - FilterMatchRule - - NotMatchToAntiJoinRule + - ExtendIntersectRule + - ExpandGetVFusionRule meta: reader: schema: diff --git a/interactive_engine/compiler/Makefile b/interactive_engine/compiler/Makefile index b1a12978074d..fea6a06948c3 100644 --- a/interactive_engine/compiler/Makefile +++ b/interactive_engine/compiler/Makefile @@ -25,6 +25,7 @@ physical:= procedure:= extra:= config.path:=conf/ir.compiler.properties +disable.expr.simplify:= build: cd $(CUR_DIR)/.. && \ @@ -63,6 +64,7 @@ run: -Dgraph.planner.rules=${graph.planner.rules} \ -Dgraph.planner.opt=${graph.planner.opt} \ -Dgraph.statistics=${graph.statistics} \ + -Ddisable.expr.simplify={disable.expr.simplify} \ com.alibaba.graphscope.GraphServer ${config.path} # make physical_plan config.path='' diff --git a/interactive_engine/compiler/pom.xml b/interactive_engine/compiler/pom.xml index 6392385974b2..dea48d62116d 100644 --- a/interactive_engine/compiler/pom.xml +++ b/interactive_engine/compiler/pom.xml @@ -362,6 +362,7 @@ **/IrPatternTest.java **/MovieTest.java **/GraphAlgoTest.java + **/FlexTypeTest.java diff --git a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/config/FrontendConfig.java b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/config/FrontendConfig.java index bbbbe19b7967..b01338db3b01 100644 --- a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/config/FrontendConfig.java +++ b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/config/FrontendConfig.java @@ -66,4 +66,7 @@ public class FrontendConfig { public static final Config METRICS_TOOL_INTERVAL_MS = Config.longConfig("metrics.tool.interval.ms", 5 * 60 * 1000L); + + public static final Config DISABLE_EXPR_SIMPLIFY = + Config.boolConfig("disable.expr.simplify", false); } diff --git a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/meta/fetcher/StaticIrMetaFetcher.java b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/meta/fetcher/StaticIrMetaFetcher.java index dae52f6113c8..4c21cefacf86 100644 --- a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/meta/fetcher/StaticIrMetaFetcher.java +++ b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/meta/fetcher/StaticIrMetaFetcher.java @@ -52,7 +52,7 @@ public StaticIrMetaFetcher(IrMetaReader dataReader, List tracker) this.metaStats = new IrMetaStats( meta.getSnapshotId(), meta.getSchema(), meta.getStoredProcedures(), stats); - if (tracker != null && stats != null) { + if (tracker != null) { tracker.forEach(t -> t.onStatsChanged(this.metaStats)); } } diff --git a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/meta/glogue/calcite/handler/GraphCollationHandler.java b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/meta/glogue/calcite/handler/GraphCollationHandler.java index a0cae6385f47..8853a61d90ef 100644 --- a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/meta/glogue/calcite/handler/GraphCollationHandler.java +++ b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/meta/glogue/calcite/handler/GraphCollationHandler.java @@ -1,19 +1,17 @@ /* + * Copyright 2020 Alibaba Group Holding Limited. * - * * Copyright 2020 Alibaba Group Holding Limited. - * * - * * Licensed under the Apache License, Version 2.0 (the "License"); - * * you may not use this file except in compliance with the License. - * * You may obtain a copy of the License at - * * - * * http://www.apache.org/licenses/LICENSE-2.0 - * * - * * Unless required by applicable law or agreed to in writing, software - * * distributed under the License is distributed on an "AS IS" BASIS, - * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * * See the License for the specific language governing permissions and - * * limitations under the License. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.alibaba.graphscope.common.ir.meta.glogue.calcite.handler; diff --git a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/meta/schema/IrDataTypeConvertor.java b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/meta/schema/IrDataTypeConvertor.java index d71a546b39be..aa98ac90167b 100644 --- a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/meta/schema/IrDataTypeConvertor.java +++ b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/meta/schema/IrDataTypeConvertor.java @@ -36,6 +36,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.math.BigDecimal; import java.util.Map; import java.util.Objects; @@ -45,6 +46,15 @@ public interface IrDataTypeConvertor { Logger logger = LoggerFactory.getLogger(IrDataTypeConvertor.class); + // support unsigned type as decimal type with fixed precision and scale + int UINT32_PRECISION = 10; + int UINT32_SCALE = 0; + int UINT64_PRECISION = 20; + int UINT64_SCALE = 0; + + BigDecimal UINT32_MAX = new BigDecimal("4294967295"); + BigDecimal UINT64_MAX = new BigDecimal("18446744073709551615"); + RelDataType convert(T dataFrom); T convert(RelDataType dataFrom); @@ -220,9 +230,15 @@ public RelDataType convert(GSDataTypeDesc from) { case "DT_ANY": // any type return typeFactory.createSqlType(SqlTypeName.ANY); + case "DT_UNSIGNED_INT32": + return typeFactory.createSqlType( + SqlTypeName.DECIMAL, UINT32_PRECISION, UINT32_SCALE); case "DT_SIGNED_INT32": // 4-bytes signed integer return typeFactory.createSqlType(SqlTypeName.INTEGER); + case "DT_UNSIGNED_INT64": + return typeFactory.createSqlType( + SqlTypeName.DECIMAL, UINT64_PRECISION, UINT64_SCALE); case "DT_SIGNED_INT64": // 8-bytes signed integer return typeFactory.createSqlType(SqlTypeName.BIGINT); @@ -435,12 +451,20 @@ public GSDataTypeDesc convert(RelDataType from) { ImmutableMap.of("key_type", keyType, "value_type", valueType)); break; case DECIMAL: - yamlDesc = - ImmutableMap.of( - "decimal", - ImmutableMap.of( - "precision", from.getPrecision(), - "scale", from.getScale())); + if (from.getPrecision() == UINT32_PRECISION + && from.getScale() == UINT32_SCALE) { + yamlDesc = ImmutableMap.of("primitive_type", "DT_UNSIGNED_INT32"); + } else if (from.getPrecision() == UINT64_PRECISION + && from.getScale() == UINT64_SCALE) { + yamlDesc = ImmutableMap.of("primitive_type", "DT_UNSIGNED_INT64"); + } else { + yamlDesc = + ImmutableMap.of( + "decimal", + ImmutableMap.of( + "precision", from.getPrecision(), + "scale", from.getScale())); + } break; default: if (throwsOnFail) { diff --git a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/runtime/proto/Utils.java b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/runtime/proto/Utils.java index 317cbfcadb4d..79574fd57555 100644 --- a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/runtime/proto/Utils.java +++ b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/runtime/proto/Utils.java @@ -46,6 +46,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.math.BigDecimal; import java.util.*; import java.util.stream.Collectors; @@ -113,7 +114,26 @@ public static final Common.Value protoValue(RexLiteral literal) { : (String) literal.getValue(); return Common.Value.newBuilder().setStr(valueStr).build(); case DECIMAL: + if (literal.getType().getPrecision() == IrDataTypeConvertor.UINT32_PRECISION + && literal.getType().getScale() == IrDataTypeConvertor.UINT32_SCALE) { + BigDecimal uint32Value = (BigDecimal) literal.getValue(); + return Common.Value.newBuilder().setU32(uint32Value.intValue()).build(); + } + if (literal.getType().getPrecision() == IrDataTypeConvertor.UINT64_PRECISION + && literal.getType().getScale() == IrDataTypeConvertor.UINT64_SCALE) { + BigDecimal uint32Value = (BigDecimal) literal.getValue(); + return Common.Value.newBuilder().setU64(uint32Value.longValue()).build(); + } + throw new UnsupportedOperationException( + "decimal type with precision=" + + literal.getType().getPrecision() + + ", scale=" + + literal.getType().getScale() + + " is unsupported yet"); case FLOAT: + return Common.Value.newBuilder() + .setF32(((Number) literal.getValue()).floatValue()) + .build(); case DOUBLE: return Common.Value.newBuilder() .setF64(((Number) literal.getValue()).doubleValue()) diff --git a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/tools/GraphBuilder.java b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/tools/GraphBuilder.java index b1ff07686c5f..ff4c03df037e 100644 --- a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/tools/GraphBuilder.java +++ b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/tools/GraphBuilder.java @@ -42,6 +42,7 @@ import com.alibaba.graphscope.common.ir.type.*; import com.alibaba.graphscope.gremlin.Utils; import com.alibaba.graphscope.proto.frontend.Code; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -96,6 +97,9 @@ protected GraphBuilder(Context context, GraphOptCluster cluster, RelOptSchema re new GraphRexSimplify( cluster.getRexBuilder(), RelOptPredicateList.EMPTY, RexUtil.EXECUTOR)); this.configs = context.unwrapOrThrow(Configs.class); + if (FrontendConfig.DISABLE_EXPR_SIMPLIFY.get(this.configs)) { + disableSimplify(); + } } /** @@ -109,6 +113,12 @@ public static GraphBuilder create( return new GraphBuilder(context, cluster, relOptSchema); } + @VisibleForTesting + public void disableSimplify() { + Config config = Utils.getFieldValue(RelBuilder.class, this, "config"); + config.withSimplify(false); + } + public Context getContext() { return this.configs; } @@ -1739,7 +1749,11 @@ public RexLiteral literal(@Nullable Object value) { return rexBuilder.makeLiteral((Boolean) value); } else if (value instanceof BigDecimal) { return rexBuilder.makeExactLiteral((BigDecimal) value); - } else if (value instanceof Float || value instanceof Double) { + } else if (value instanceof Float) { + return rexBuilder.makeApproxLiteral( + BigDecimal.valueOf(((Number) value).floatValue()), + getTypeFactory().createSqlType(SqlTypeName.FLOAT)); + } else if (value instanceof Double) { return rexBuilder.makeApproxLiteral(BigDecimal.valueOf(((Number) value).doubleValue())); } else if (value instanceof Long) { // convert long to BIGINT, i.e. 2l return rexBuilder.makeBigintLiteral(BigDecimal.valueOf(((Number) value).longValue())); diff --git a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/type/GraphTypeFactoryImpl.java b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/type/GraphTypeFactoryImpl.java index 881e8a6837fa..a7516fd5cb0c 100644 --- a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/type/GraphTypeFactoryImpl.java +++ b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/type/GraphTypeFactoryImpl.java @@ -18,12 +18,15 @@ import com.alibaba.graphscope.common.config.Configs; import com.alibaba.graphscope.common.config.FrontendConfig; +import com.alibaba.graphscope.common.ir.meta.schema.IrDataTypeConvertor; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.calcite.jdbc.JavaTypeFactoryImpl; import org.apache.calcite.rel.type.*; import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.sql.type.SqlTypeUtil; import org.apache.calcite.util.Util; import org.checkerframework.checker.nullness.qual.Nullable; @@ -36,7 +39,65 @@ public class GraphTypeFactoryImpl extends JavaTypeFactoryImpl { private final Configs configs; public GraphTypeFactoryImpl(Configs configs) { - super(); + super( + new RelDataTypeSystemImpl() { + @Override + public int getMaxNumericPrecision() { + return 20; + } + + @Override + public @Nullable RelDataType deriveDecimalPlusType( + RelDataTypeFactory typeFactory, RelDataType type1, RelDataType type2) { + if (!SqlTypeUtil.isExactNumeric(type1) + || !SqlTypeUtil.isExactNumeric(type2) + || !isDecimal(type1) && !isDecimal(type2)) { + return null; + } + return super.deriveDecimalPlusType(typeFactory, type1, type2); + } + + @Override + public @Nullable RelDataType deriveDecimalMultiplyType( + RelDataTypeFactory typeFactory, RelDataType type1, RelDataType type2) { + if (!SqlTypeUtil.isExactNumeric(type1) + || !SqlTypeUtil.isExactNumeric(type2) + || !isDecimal(type1) && !isDecimal(type2)) { + return null; + } + return super.deriveDecimalMultiplyType(typeFactory, type1, type2); + } + + @Override + public @Nullable RelDataType deriveDecimalDivideType( + RelDataTypeFactory typeFactory, RelDataType type1, RelDataType type2) { + if (!SqlTypeUtil.isExactNumeric(type1) + || !SqlTypeUtil.isExactNumeric(type2) + || !isDecimal(type1) && !isDecimal(type2)) { + return null; + } + return super.deriveDecimalDivideType(typeFactory, type1, type2); + } + + private boolean isDecimal(RelDataType type) { + SqlTypeName typeName = type.getSqlTypeName(); + return typeName == SqlTypeName.DECIMAL + && !isUint32(type) + && !isUint64(type); + } + + private boolean isUint32(RelDataType type) { + return type.getSqlTypeName() == SqlTypeName.DECIMAL + && type.getPrecision() == IrDataTypeConvertor.UINT32_PRECISION + && type.getScale() == IrDataTypeConvertor.UINT32_SCALE; + } + + private boolean isUint64(RelDataType type) { + return type.getSqlTypeName() == SqlTypeName.DECIMAL + && type.getPrecision() == IrDataTypeConvertor.UINT64_PRECISION + && type.getScale() == IrDataTypeConvertor.UINT64_SCALE; + } + }); this.configs = configs; } @@ -190,4 +251,14 @@ public RelDataType createArbitraryMapType( } return createArbitraryMapType(leastKeyValueType, isNullable); } + + @Override + public RelDataType createSqlType(SqlTypeName typeName, int precision, int scale) { + if (typeName == SqlTypeName.DECIMAL + && precision == typeSystem.getDefaultPrecision(SqlTypeName.BIGINT) + && scale == 0) { + return createSqlType(SqlTypeName.BIGINT); + } + return super.createSqlType(typeName, precision, scale); + } } diff --git a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/cypher/antlr4/visitor/ExpressionVisitor.java b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/cypher/antlr4/visitor/ExpressionVisitor.java index 70bc21515fd6..a0b7cdae81d9 100644 --- a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/cypher/antlr4/visitor/ExpressionVisitor.java +++ b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/cypher/antlr4/visitor/ExpressionVisitor.java @@ -20,6 +20,7 @@ import com.alibaba.graphscope.common.antlr4.ExprVisitorResult; import com.alibaba.graphscope.common.config.Configs; import com.alibaba.graphscope.common.ir.meta.function.GraphFunctions; +import com.alibaba.graphscope.common.ir.meta.schema.IrDataTypeConvertor; import com.alibaba.graphscope.common.ir.rel.type.group.GraphAggCall; import com.alibaba.graphscope.common.ir.rex.RexGraphVariable; import com.alibaba.graphscope.common.ir.rex.RexTmpVariable; @@ -38,15 +39,18 @@ import org.antlr.v4.runtime.tree.TerminalNode; import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rex.*; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.type.SqlTypeFamily; +import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.tools.RelBuilder; import org.apache.calcite.util.NlsString; import org.apache.commons.lang3.ObjectUtils; import org.checkerframework.checker.nullness.qual.Nullable; +import java.math.BigDecimal; import java.util.*; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; @@ -220,10 +224,47 @@ public ExprVisitorResult visitOC_MultiplyDivideModuloExpression( @Override public ExprVisitorResult visitOC_UnaryAddOrSubtractExpression( CypherGSParser.OC_UnaryAddOrSubtractExpressionContext ctx) { - ExprVisitorResult operand = visitOC_ListOperatorExpression(ctx.oC_ListOperatorExpression()); - List operators = - Utils.getOperators(ctx.children, ImmutableList.of("-", "+"), true); - return Utils.unaryCall(operators, operand, builder); + String text = ctx.getText().toLowerCase(); + if (text.startsWith("+")) { + text = text.substring(1); + } + Object integerValue; + try { + // check the expr is actual an integer value. + integerValue = LiteralVisitor.INSTANCE.parseInteger(text); + } catch (Exception e) { + // if not an integer, i.e. -a.age, then convert to the normal expression + ExprVisitorResult operand = + visitOC_ListOperatorExpression(ctx.oC_ListOperatorExpression()); + List operators = + Utils.getOperators(ctx.children, ImmutableList.of("-", "+"), true); + return Utils.unaryCall(operators, operand, builder); + } + // parse to unsigned types + if (ctx.getText().startsWith("+")) { + BigDecimal decimal = new BigDecimal(integerValue.toString()); + RelDataType type; + if (decimal.compareTo(IrDataTypeConvertor.UINT32_MAX) <= 0 && !text.endsWith("l")) { + type = + builder.getTypeFactory() + .createSqlType( + SqlTypeName.DECIMAL, + IrDataTypeConvertor.UINT32_PRECISION, + IrDataTypeConvertor.UINT32_SCALE); + } else if (decimal.compareTo(IrDataTypeConvertor.UINT64_MAX) <= 0) { + type = + builder.getTypeFactory() + .createSqlType( + SqlTypeName.DECIMAL, + IrDataTypeConvertor.UINT64_PRECISION, + IrDataTypeConvertor.UINT64_SCALE); + } else { + throw new IllegalArgumentException( + "value: " + decimal + " exceeds the range of uint64"); + } + return new ExprVisitorResult(builder.getRexBuilder().makeLiteral(decimal, type)); + } + return new ExprVisitorResult(builder.literal(integerValue)); } @Override diff --git a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/cypher/antlr4/visitor/LiteralVisitor.java b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/cypher/antlr4/visitor/LiteralVisitor.java index 346fa16e78ed..3d2d7889b55b 100644 --- a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/cypher/antlr4/visitor/LiteralVisitor.java +++ b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/cypher/antlr4/visitor/LiteralVisitor.java @@ -36,7 +36,11 @@ public Object visitOC_BooleanLiteral(CypherGSParser.OC_BooleanLiteralContext ctx @Override public Object visitOC_IntegerLiteral(CypherGSParser.OC_IntegerLiteralContext ctx) { - String integerLiteral = ctx.getText().toLowerCase(); + return parseInteger(ctx.getText()); + } + + public Object parseInteger(String text) { + String integerLiteral = text.toLowerCase(); try { if (integerLiteral.length() > 1) { char lastChar = integerLiteral.charAt(integerLiteral.length() - 1); @@ -75,6 +79,9 @@ public Object visitOC_IntegerLiteral(CypherGSParser.OC_IntegerLiteralContext ctx } else if (integerLiteral.charAt(startIndex) == '0') { radix = 8; } + if (integerLiteral.endsWith("l")) { + integerLiteral = integerLiteral.substring(0, integerLiteral.length() - 1); + } // create big integer return new BigInteger(integerLiteral, radix); } @@ -84,6 +91,9 @@ public Object visitOC_IntegerLiteral(CypherGSParser.OC_IntegerLiteralContext ctx @Override public Object visitOC_DoubleLiteral(CypherGSParser.OC_DoubleLiteralContext ctx) { String floatLiteral = ctx.getText().toLowerCase(); + if (floatLiteral.endsWith("f")) { + return Float.valueOf(floatLiteral); + } return Double.valueOf(floatLiteral); } diff --git a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/cypher/result/CypherRecordParser.java b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/cypher/result/CypherRecordParser.java index f9be5a894cd9..4c4ae5f545c1 100644 --- a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/cypher/result/CypherRecordParser.java +++ b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/cypher/result/CypherRecordParser.java @@ -313,6 +313,18 @@ protected AnyValue parseValue(Common.Value value, @Nullable RelDataType dataType return Values.intValue(value.getI32()); case I64: return Values.longValue(value.getI64()); + case U32: + // cypher does not support u32 directly, represent u32 as int, + // user need to convert int to u32 and handle the overflow cases at the + // client-side. + return Values.intValue(value.getU32()); + case U64: + // cypher does not support u64 directly, represent u64 as long, + // user need to convert long to u64 and handle the overflow cases at the + // client-side. + return Values.longValue(value.getU64()); + case F32: + return Values.floatValue(value.getF32()); case F64: return Values.doubleValue(value.getF64()); case STR: diff --git a/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/common/ir/QueryCacheTest.java b/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/common/ir/QueryCacheTest.java index 16540c2ab9e7..ea572960dd66 100644 --- a/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/common/ir/QueryCacheTest.java +++ b/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/common/ir/QueryCacheTest.java @@ -37,7 +37,9 @@ public class QueryCacheTest { // test hash code of query cache key @Test public void query_cache_1_test() { - Configs configs = new Configs(ImmutableMap.of("query.cache.size", "1")); + Configs configs = + new Configs( + ImmutableMap.of("query.cache.size", "1", "graph.physical.opt", "proto")); GraphPlanner graphPlanner = new GraphPlanner( configs, new LogicalPlanFactory.Cypher(), new GraphRelOptimizer(configs)); @@ -63,7 +65,9 @@ public void query_cache_1_test() { // test evict strategy of query cache @Test public void query_cache_2_test() throws Exception { - Configs configs = new Configs(ImmutableMap.of("query.cache.size", "1")); + Configs configs = + new Configs( + ImmutableMap.of("query.cache.size", "1", "graph.physical.opt", "proto")); GraphPlanner graphPlanner = new GraphPlanner( configs, new LogicalPlanFactory.Cypher(), new GraphRelOptimizer(configs)); @@ -86,7 +90,9 @@ public void query_cache_2_test() throws Exception { @Test public void query_cache_3_test() { - Configs configs = new Configs(ImmutableMap.of("query.cache.size", "10")); + Configs configs = + new Configs( + ImmutableMap.of("query.cache.size", "10", "graph.physical.opt", "proto")); GraphPlanner graphPlanner = new GraphPlanner( configs, new LogicalPlanFactory.Gremlin(), new GraphRelOptimizer(configs)); @@ -105,7 +111,7 @@ public void query_cache_3_test() { // test cache invalidation after schema update @Test public void query_cache_schema_update_test() throws Exception { - Configs configs = new Configs(ImmutableMap.of()); + Configs configs = new Configs(ImmutableMap.of("graph.physical.opt", "proto")); GraphPlanner graphPlanner = new GraphPlanner( configs, new LogicalPlanFactory.Cypher(), new GraphRelOptimizer(configs)); diff --git a/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/cypher/antlr4/MatchTest.java b/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/cypher/antlr4/MatchTest.java index 7c4189d46e88..3e0ff757a6fb 100644 --- a/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/cypher/antlr4/MatchTest.java +++ b/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/cypher/antlr4/MatchTest.java @@ -696,7 +696,7 @@ public void optional_shortest_path_test() { RelNode after = optimizer.optimize(node, new GraphIOProcessor(builder, irMeta)); Assert.assertEquals( "GraphLogicalProject(len=[len], isAppend=[false])\n" - + " GraphLogicalProject(len=[CASE(IS NULL(k), -(1), k.~len)]," + + " GraphLogicalProject(len=[CASE(IS NULL(k), -1, k.~len)]," + " isAppend=[false])\n" + " GraphLogicalGetV(tableConfig=[{isAll=false, tables=[person]}]," + " alias=[p2], fusedFilter=[[=(_.id, ?1)]], opt=[END])\n" diff --git a/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/cypher/antlr4/TypeTest.java b/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/cypher/antlr4/TypeTest.java new file mode 100644 index 000000000000..629c387f7f84 --- /dev/null +++ b/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/cypher/antlr4/TypeTest.java @@ -0,0 +1,233 @@ +/* + * Copyright 2020 Alibaba Group Holding Limited. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.alibaba.graphscope.cypher.integration.flex.bench; + +import com.alibaba.graphscope.common.config.Configs; +import com.alibaba.graphscope.common.ir.meta.IrMeta; +import com.alibaba.graphscope.common.ir.planner.GraphIOProcessor; +import com.alibaba.graphscope.common.ir.planner.GraphRelOptimizer; +import com.alibaba.graphscope.common.ir.runtime.proto.GraphRelProtoPhysicalBuilder; +import com.alibaba.graphscope.common.ir.tools.GraphBuilder; +import com.alibaba.graphscope.common.ir.tools.LogicalPlan; +import com.alibaba.graphscope.common.utils.FileUtils; +import com.alibaba.graphscope.cypher.antlr4.Utils; +import com.google.common.collect.ImmutableMap; + +import org.apache.calcite.rel.RelNode; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.math.BigDecimal; +import java.math.BigInteger; + +public class UnitTypeTest { + private static Configs configs; + private static IrMeta irMeta; + private static GraphRelOptimizer optimizer; + + @BeforeClass + public static void beforeClass() { + configs = + new Configs( + ImmutableMap.of( + "graph.planner.is.on", + "true", + "graph.planner.opt", + "CBO", + "graph.planner.rules", + "FilterIntoJoinRule, FilterMatchRule, ExtendIntersectRule," + + " ExpandGetVFusionRule")); + optimizer = new GraphRelOptimizer(configs); + irMeta = + com.alibaba.graphscope.common.ir.Utils.mockIrMeta( + "flex_bench/modern.yaml", "", optimizer); + } + + @Test + public void compare_uint64_uint64_test() { + GraphBuilder builder = + com.alibaba.graphscope.common.ir.Utils.mockGraphBuilder(optimizer, irMeta); + RelNode before = + com.alibaba.graphscope.cypher.antlr4.Utils.eval( + "MATCH (p:person)\n" + + " WHERE p.prop_uint64 = +18446744073709551602L\n" + + " RETURN p.prop_uint64", + builder) + .build(); + RelNode after = optimizer.optimize(before, new GraphIOProcessor(builder, irMeta)); + GraphRelProtoPhysicalBuilder builder1 = + new GraphRelProtoPhysicalBuilder(configs, irMeta, new LogicalPlan(after)); + Assert.assertEquals( + FileUtils.readJsonFromResource("proto/compare_uint64_uint64_test.json"), + builder1.build().explain().trim()); + } + + @Test + public void compare_double_float_test() { + GraphBuilder builder = + com.alibaba.graphscope.common.ir.Utils.mockGraphBuilder(optimizer, irMeta); + RelNode before = + com.alibaba.graphscope.cypher.antlr4.Utils.eval( + "MATCH (p:person)\n" + + " WHERE p.prop_double = 1.2f\n" + + " RETURN p.prop_double", + builder) + .build(); + RelNode after = optimizer.optimize(before, new GraphIOProcessor(builder, irMeta)); + GraphRelProtoPhysicalBuilder builder1 = + new GraphRelProtoPhysicalBuilder(configs, irMeta, new LogicalPlan(after)); + Assert.assertEquals( + FileUtils.readJsonFromResource("proto/compare_double_float_test.json"), + builder1.build().explain().trim()); + } + + @Test + public void divide_int32_int32_overflow_test() { + GraphBuilder builder = + com.alibaba.graphscope.common.ir.Utils.mockGraphBuilder(optimizer, irMeta); + builder.disableSimplify(); + RelNode before = + com.alibaba.graphscope.cypher.antlr4.Utils.eval( + "MATCH (p:person)\n" + " RETURN -2147483648 / -1", builder) + .build(); + RelNode after = optimizer.optimize(before, new GraphIOProcessor(builder, irMeta)); + Assert.assertEquals( + "GraphLogicalProject($f0=[/(-2147483648, -1)], isAppend=[false])\n" + + " GraphLogicalSource(tableConfig=[{isAll=false, tables=[person]}]," + + " alias=[p], opt=[VERTEX])", + after.explain().trim()); + } + + @Test + public void divide_int64_int32_overflow_test() { + GraphBuilder builder = + com.alibaba.graphscope.common.ir.Utils.mockGraphBuilder(optimizer, irMeta); + builder.disableSimplify(); + RelNode before = + com.alibaba.graphscope.cypher.antlr4.Utils.eval( + "MATCH (p:person)\n" + " RETURN -9223372036854775808L / -1", + builder) + .build(); + RelNode after = optimizer.optimize(before, new GraphIOProcessor(builder, irMeta)); + Assert.assertEquals( + "GraphLogicalProject($f0=[/(-9223372036854775808:BIGINT, -1)], isAppend=[false])\n" + + " GraphLogicalSource(tableConfig=[{isAll=false, tables=[person]}]," + + " alias=[p], opt=[VERTEX])", + after.explain().trim()); + } + + @Test + public void convert_int32_to_uint32_test() { + BigDecimal expected = new BigDecimal("4294967284"); + long signedVal = expected.longValue(); + BigDecimal unsignedVal = new BigDecimal(new BigInteger(1, Utils.longToBytes(signedVal))); + Assert.assertEquals(expected, unsignedVal); + } + + @Test + public void convert_int64_to_uint64_test() { + BigDecimal expected = new BigDecimal("18446744073709551602"); + long signedVal = expected.longValue(); + BigDecimal unsignedVal = new BigDecimal(new BigInteger(1, Utils.longToBytes(signedVal))); + Assert.assertEquals(expected, unsignedVal); + } + + /** + * When plus/minus/multiply/divide an int32 value with an uint32 value, the expected result range should be [INT32_MIN, UINT32_MAX], + * but there is not a type that can represent this range, currently we use uint32 instead. + * + * The return type is the least restrictive type of two parameters, specifically: + * + * uint32, uint32 -> uint32 + * uint32, int32 -> uint32 + * uint32, uint64 -> uint64 + * uint32, int64 -> int64 + * int32, int64 -> int64 + * int32, uint64 -> uint64 + */ + @Test + public void divide_int32_uint64_test() { + GraphBuilder builder = + com.alibaba.graphscope.common.ir.Utils.mockGraphBuilder(optimizer, irMeta); + builder.disableSimplify(); + RelNode before = + com.alibaba.graphscope.cypher.antlr4.Utils.eval( + "MATCH (p:person) RETURN 2 / +3L", builder) + .build(); + RelNode after = optimizer.optimize(before, new GraphIOProcessor(builder, irMeta)); + GraphRelProtoPhysicalBuilder builder1 = + new GraphRelProtoPhysicalBuilder(configs, irMeta, new LogicalPlan(after)); + Assert.assertEquals( + FileUtils.readJsonFromResource("proto/divide_int32_uint32_int32.json"), + builder1.build().explain().trim()); + } + + @Test + public void compare_date32_i32_test() { + GraphBuilder builder = + com.alibaba.graphscope.common.ir.Utils.mockGraphBuilder(optimizer, irMeta); + RelNode before = + com.alibaba.graphscope.cypher.antlr4.Utils.eval( + "MATCH (p:person)\n" + + " WHERE p.prop_date = 20132\n" + + " RETURN p.prop_date", + builder) + .build(); + RelNode after = optimizer.optimize(before, new GraphIOProcessor(builder, irMeta)); + Assert.assertEquals( + "GraphLogicalProject(prop_date=[p.prop_date], isAppend=[false])\n" + + " GraphLogicalSource(tableConfig=[{isAll=false, tables=[person]}]," + + " alias=[p], fusedFilter=[[=(_.prop_date, 20132)]], opt=[VERTEX])", + after.explain().trim()); + } + + @Test + public void compare_timestamp_i64_test() { + GraphBuilder builder = + com.alibaba.graphscope.common.ir.Utils.mockGraphBuilder(optimizer, irMeta); + RelNode before = + com.alibaba.graphscope.cypher.antlr4.Utils.eval( + "MATCH (p:person)\n" + + " WHERE p.prop_ts = 1739454301000L\n" + + " RETURN p.prop_ts", + builder) + .build(); + RelNode after = optimizer.optimize(before, new GraphIOProcessor(builder, irMeta)); + Assert.assertEquals( + "GraphLogicalProject(prop_ts=[p.prop_ts], isAppend=[false])\n" + + " GraphLogicalSource(tableConfig=[{isAll=false, tables=[person]}]," + + " alias=[p], fusedFilter=[[=(_.prop_ts, 1739454301000:BIGINT)]]," + + " opt=[VERTEX])", + after.explain().trim()); + } + + @Test + public void compare_int32_int64_array_test() { + String query = "MATCH (p:person) Where p.prop_int32 in [123L, 456] RETURN p.prop_int32"; + GraphBuilder builder = + com.alibaba.graphscope.common.ir.Utils.mockGraphBuilder(optimizer, irMeta); + RelNode before = com.alibaba.graphscope.cypher.antlr4.Utils.eval(query, builder).build(); + RelNode after = optimizer.optimize(before, new GraphIOProcessor(builder, irMeta)); + Assert.assertEquals( + "GraphLogicalProject(prop_int32=[p.prop_int32], isAppend=[false])\n" + + " GraphLogicalSource(tableConfig=[{isAll=false, tables=[person]}]," + + " alias=[p], opt=[VERTEX], uniqueKeyFilters=[SEARCH(_.prop_int32," + + " Sarg[123L:BIGINT, 456L:BIGINT]:BIGINT)])", + after.explain().trim()); + } +} diff --git a/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/cypher/antlr4/Utils.java b/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/cypher/antlr4/Utils.java index 01f5d916d97a..3ad777055b45 100644 --- a/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/cypher/antlr4/Utils.java +++ b/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/cypher/antlr4/Utils.java @@ -23,6 +23,9 @@ import com.alibaba.graphscope.cypher.antlr4.visitor.GraphBuilderVisitor; import com.alibaba.graphscope.cypher.antlr4.visitor.LogicalPlanVisitor; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + public abstract class Utils { public static final GraphBuilder eval(String query) { GraphBuilder graphBuilder = com.alibaba.graphscope.common.ir.Utils.mockGraphBuilder(); @@ -48,4 +51,18 @@ public static LogicalPlan evalLogicalPlan(String query, String schemaPath) { LogicalPlanVisitor logicalPlanVisitor = new LogicalPlanVisitor(graphBuilder, irMeta); return logicalPlanVisitor.visit(new CypherAntlr4Parser().parse(query)); } + + public static byte[] longToBytes(long value) { + ByteBuffer buffer = ByteBuffer.allocate(8); + buffer.order(ByteOrder.BIG_ENDIAN); + buffer.putLong(value); + return buffer.array(); + } + + public static byte[] intToBytes(int value) { + ByteBuffer buffer = ByteBuffer.allocate(4); + buffer.order(ByteOrder.BIG_ENDIAN); + buffer.putInt(value); + return buffer.array(); + } } diff --git a/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/cypher/integration/flex/bench/FlexTypeQueries.java b/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/cypher/integration/flex/bench/FlexTypeQueries.java new file mode 100644 index 000000000000..6e26c7d76b13 --- /dev/null +++ b/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/cypher/integration/flex/bench/FlexTypeQueries.java @@ -0,0 +1,1088 @@ +/* + * Copyright 2020 Alibaba Group Holding Limited. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.alibaba.graphscope.cypher.integration.flex.bench; + +import com.alibaba.graphscope.cypher.integration.suite.QueryContext; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +import org.apache.commons.io.FileUtils; + +import java.io.File; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Map; + +public class FlexTypeQueries { + private final Map queryParameters; + + public static class Parameter { + public final String name; + public final List parameters; + public final List results; + + public Parameter(String line) { + String[] parts = line.split("\\|"); + Preconditions.checkArgument(parts.length >= 3, "invalid parameter line: " + line); + this.name = parts[0].trim(); + this.parameters = parseParameters(parts[1].trim()); + this.results = parseParameters(parts[2].trim()); + } + + public String render(String template) { + for (int i = 1; i <= parameters.size(); i++) { + template = template.replaceAll("\\$" + i, parameters.get(i - 1)); + } + return template; + } + + private List parseParameters(String input) { + List result = Lists.newArrayList(); + StringBuilder token = new StringBuilder(); + int bracketLevel = 0; + + for (int i = 0; i < input.length(); i++) { + char ch = input.charAt(i); + if (ch == '[') { + if (bracketLevel == 0 && token.length() > 0) { + result.add(token.toString().trim()); + token.setLength(0); + } + bracketLevel++; + } else if (ch == ']') { + bracketLevel--; + } + + if (bracketLevel > 0 || ch != ',') { + token.append(ch); + } + + if (ch == ',' && bracketLevel == 0) { + if (token.length() > 0) { + result.add(token.toString().trim()); + token.setLength(0); + } + } + } + + if (token.length() > 0) { + result.add(token.toString()); + } + + return result; + } + } + + public FlexTypeQueries(String inputPath) throws Exception { + List parameters = FileUtils.readLines(new File(inputPath), StandardCharsets.UTF_8); + this.queryParameters = Maps.newHashMap(); + for (String parameter : parameters) { + if (parameter.trim().isEmpty() || parameter.startsWith("//")) continue; + Parameter param = new Parameter(parameter); + this.queryParameters.put(param.name, param); + } + } + + public CompareTest getCompare() { + return new CompareTest(); + } + + public PlusTest getPlus() { + return new PlusTest(); + } + + public MinusTest getMinus() { + return new MinusTest(); + } + + public MultiplyTest getMultiply() { + return new MultiplyTest(); + } + + public DivideTest getDivide() { + return new DivideTest(); + } + + // Assign type to the literal value with prefix or suffix characters. + // i.e. 1 denote int32, +1 denote uint32, 1L denote int64, +1L denote uint64, 1.0d denote + // double, 1.0f denote float. + public class CompareTest { + /** + * Compare an int32 type property with an int32 literal. + * + * Expected Result: + * The query should return the property value. + * @return + */ + public QueryContext compare_int32_int32_test() { + String query = + "MATCH (p:person)\n" + + " WHERE p.prop_int32 = $1\n" + + " RETURN p.prop_int32"; + Parameter parameter = queryParameters.get("compare_int32_int32"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Compare a uint32 type property with a uint32 literal. + * + * Expected Result: + * The query should return the property value. + * @return + */ + public QueryContext compare_uint32_uint32_test() { + String query = + "MATCH (p:person)\n" + + " WHERE p.prop_uint32 = $1\n" + + " RETURN p.prop_uint32"; + Parameter parameter = queryParameters.get("compare_uint32_uint32"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Compare a uint32 type property with an int32 literal. + * + * Expected Result: + * The query should return the property value. + * @return + */ + public QueryContext compare_uint32_int32_test() { + String query = + "MATCH (p:person)\n" + + " WHERE p.prop_uint32 = $1\n" + + " RETURN p.prop_uint32"; + Parameter parameter = queryParameters.get("compare_uint32_int32"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Compare a uint32 type property with an int32 literal, where the int32 literal is negative + * and has the same binary representation as the uint32 property value. + * + * Expected Result: + * The query should return empty due to integer overflow. + * @return + */ + public QueryContext compare_uint32_int32_overflow_test() { + String query = + "MATCH (p:person)\n" + + " WHERE p.prop_uint32 = $1\n" + + " RETURN p.prop_uint32"; + Parameter parameter = queryParameters.get("compare_uint32_int32_overflow"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Compare a uint64 type property with a uint64 literal. + * + * Expected Result: + * The query should return the property value. + * @return + */ + public QueryContext compare_uint64_uint64_test() { + String query = + "MATCH (p:person)\n" + + " WHERE p.prop_uint64 = $1\n" + + " RETURN p.prop_uint64"; + Parameter parameter = queryParameters.get("compare_uint64_uint64"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Compare an int64 type property with an int64 literal. + * + * Expected Result: + * The query should return the property value. + * @return + */ + public QueryContext compare_int64_int64_test() { + String query = + "MATCH (p:person)\n" + + " WHERE p.prop_int64 = $1\n" + + " RETURN p.prop_int64"; + Parameter parameter = queryParameters.get("compare_int64_int64"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Compare a uint64 type property with an int64 literal. + * + * Expected Result: + * The query should return the property value. + * @return + */ + public QueryContext compare_uint64_int64_test() { + String query = + "MATCH (p:person)\n" + + " WHERE p.prop_uint64 = $1\n" + + " RETURN p.prop_uint64"; + Parameter parameter = queryParameters.get("compare_uint64_int64"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Compare a uint64 type property with an int64 literal, where the int64 literal is negative + * and has the same binary representation as the uint64 property value. + * + * Expected Result: + * The query should return empty due to integer overflow. + * @return + */ + public QueryContext compare_uint64_int64_overflow_test() { + String query = + "MATCH (p:person)\n" + + " WHERE p.prop_uint64 = $1\n" + + " RETURN p.prop_uint64"; + Parameter parameter = queryParameters.get("compare_uint64_int64_overflow"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Compare an int32 type property with an int64 literal. + * + * Expected Result: + * The query should return the property value. + * @return + */ + public QueryContext compare_int32_int64_test() { + String query = + "MATCH (p:person)\n" + + " WHERE p.prop_int32 = $1\n" + + " RETURN p.prop_int32"; + Parameter parameter = queryParameters.get("compare_int32_int64"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Compare an int32 type property with an int64 literal that exceeds the range of int32. The int64 after overflow has the same binary representation as the int32 value. + * + * Expected Result: + * The query should return empty due to overflow. + * @return + */ + public QueryContext compare_int32_int64_overflow_test() { + String query = + "MATCH (p:person)\n" + + " WHERE p.prop_int32 = $1\n" + + " RETURN p.prop_int32"; + Parameter parameter = queryParameters.get("compare_int32_int64_overflow"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Compare a float type property with a float literal. + * + * Expected Result: + * The query should return the property value. + * @return + */ + public QueryContext compare_float_float_test() { + String query = + "MATCH (p:person)\n" + + " WHERE p.prop_float = $1\n" + + " RETURN p.prop_float"; + Parameter parameter = queryParameters.get("compare_float_float"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Compare a double type property with a double literal. + * + * Expected Result: + * The query should return the property value. + * @return + */ + public QueryContext compare_double_double_test() { + String query = + "MATCH (p:person)\n" + + " WHERE p.prop_double = $1\n" + + " RETURN p.prop_double"; + Parameter parameter = queryParameters.get("compare_double_double"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Compare a float type property with a double literal. + * + * Expected Result: + * The query should return the float value as the result. + * @return + */ + public QueryContext compare_float_double_test() { + String query = + "MATCH (p:person)\n" + + " WHERE p.prop_float = $1\n" + + " RETURN p.prop_float"; + Parameter parameter = queryParameters.get("compare_float_double"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Compare a float type property with a double literal, where the double literal has more precision + * than the float property, leading to precision loss. + * + * Expected Result: + * The query should return empty due to precision loss. + * @return + */ + public QueryContext compare_float_double_loss_test() { + String query = + "MATCH (p:person)\n" + + " WHERE p.prop_float = $1\n" + + " RETURN p.prop_float"; + Parameter parameter = queryParameters.get("compare_float_double_loss"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Compare an int32 type property with a double literal. + * + * Expected Result: + * The query should return the property value as an int32. + * @return + */ + public QueryContext compare_int32_double_test() { + String query = + "MATCH (p:person)\n" + + " WHERE p.prop_int32 = $1\n" + + " RETURN p.prop_int32"; + Parameter parameter = queryParameters.get("compare_int32_double"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Compare a characters type property with a string literal, the literal value is identical to the property value. + * + * Expected Results: + * The query should return the property value. + * @return + */ + public QueryContext compare_char_text_test() { + String query = + "MATCH (p:person)\n" + + " WHERE p.prop_char = $1\n" + + " RETURN p.prop_char"; + Parameter parameter = queryParameters.get("compare_char_text"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Compare a characters type property with a string literal, the literal is longer than the property value. + * + * Expected Results: + * The query should return the property value, the literal has been truncated to the length of the property value before comparison. + * @return + */ + public QueryContext compare_char_long_text_test() { + String query = + "MATCH (p:person)\n" + + " WHERE p.prop_char = $1\n" + + " RETURN p.prop_char"; + Parameter parameter = queryParameters.get("compare_char_long_text"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Compare a varchar type property with a string literal, the literal is longer than the property value. + * + * Expected Results: + * The query should return the property value, the literal has been truncated to the length of the property value before comparison. + * @return + */ + public QueryContext compare_varchar_long_text_test() { + String query = + "MATCH (p:person)\n" + + " WHERE p.prop_varchar = $1\n" + + " RETURN p.prop_varchar"; + Parameter parameter = queryParameters.get("compare_varchar_long_text"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Compare an unlimited length type property with a long text literal, the literal is identical to the property value. + * + * Expected Results: + * The query should return the property value. + * @return + */ + public QueryContext compare_string_long_text_test() { + String query = + "MATCH (p:person)\n" + + " WHERE p.prop_text = $1\n" + + " RETURN p.prop_text"; + Parameter parameter = queryParameters.get("compare_string_long_text"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Compare a date32 type property with an i32 literal. + * + * Expected Results: + * The query should return the property value. + * @return + */ + public QueryContext compare_date32_i32_test() { + String query = + "MATCH (p:person)\n" + + " WHERE p.prop_date = $1\n" + + " RETURN p.prop_date"; + Parameter parameter = queryParameters.get("compare_date32_i32"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Compare a timestamp type property with an i64 literal. + * + * Expected Results: + * The query should return the property value. + * @return + */ + public QueryContext compare_timestamp_i64_test() { + String query = + "MATCH (p:person)\n" + " WHERE p.prop_ts = $1\n" + " RETURN p.prop_ts"; + Parameter parameter = queryParameters.get("compare_timestamp_i64"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Check the property value is one of the element in the given i32 array. + * The prop_int32 is the primary key, which will be converted to index predicate in physical proto. + * The execution layer should handle the primary key of within in the index predicate correctly. + * @return + */ + public QueryContext compare_i32_within_i32_array_test() { + String query = "MATCH (p:person) Where p.prop_int32 in $1 RETURN p.prop_int32"; + Parameter parameter = queryParameters.get("compare_i32_within_i32_array"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Check the property value is one of the element in the given i64 array. + * @return + */ + public QueryContext compare_i32_within_i64_array_test() { + String query = "MATCH (p:person) Where p.prop_int32 in $1 RETURN p.prop_int32"; + Parameter parameter = queryParameters.get("compare_i32_within_i64_array"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Check the property value is one of the element in the given i32 array. + * @return + */ + public QueryContext compare_i64_within_i32_array_test() { + String query = "MATCH (p:person) Where p.prop_int64 in $1 RETURN p.prop_int64"; + Parameter parameter = queryParameters.get("compare_i64_within_i32_array"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Check the property value is one of the element in the given double array. + * @return + */ + public QueryContext compare_float_within_double_array_test() { + String query = "MATCH (p:person) Where p.prop_float in $1 RETURN p.prop_float"; + Parameter parameter = queryParameters.get("compare_float_within_double_array"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Check the property value is one of the element in the given string array. + * @return + */ + public QueryContext compare_char_within_string_array_test() { + String query = "MATCH (p:person) Where p.prop_char in $1 RETURN p.prop_char"; + Parameter parameter = queryParameters.get("compare_char_within_string_array"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + } + + public class PlusTest { + + // plus_int32_int32|12,13|25 + /** + * Plus an int32 value with an int32 value, the sum is within the range of int32. + * + * Expected Results: + * The query should return an int32 type value, identical to the sum of $1 and $2. + */ + public QueryContext plus_int32_int32_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 + $2"; + Parameter parameter = queryParameters.get("plus_int32_int32"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // plus_int32_int32_overflow|2147483647,1|overflow + /** + * Plus an int32 value with an int32 value, the sum exceeds the range of int32. + * + * Expected Results: + * The query should throw an overflow exception due to int32 overflow. + */ + public QueryContext plus_int32_int32_overflow_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 + $2"; + Parameter parameter = queryParameters.get("plus_int32_int32_overflow"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // plus_int32_uint32_int32|12,+13|25 + /** + * Plus an int32 value with an uint32 value, the sum is within the range of int32. + * + * Expected Results: + * The query should return an int32 type value, identical to the sum of $1 and $2. + */ + public QueryContext plus_int32_uint32_int32_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 + $2"; + Parameter parameter = queryParameters.get("plus_int32_uint32_int32"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // plus_int32_uint32_uint32|10,+2147483647|+2147483657 + /** + * Plus an int32 value with an uint32 value, the sum is out of range of int32 but within the range of uint32. + * + * Expected Results: + * The query should return a uint32 type value, identical to the sum of $1 and $2. + */ + public QueryContext plus_int32_uint32_uint32_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 + $2"; + Parameter parameter = queryParameters.get("plus_int32_uint32_uint32"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // plus_int32_uint32_overflow|1,+4294967295|overflow + /** + * Plus an int32 value with an uint32 value, the sum exceeds the range of uint32. + * + * Expected Results: + * The query should throw an overflow exception due to uint32 overflow. + */ + public QueryContext plus_int32_uint32_overflow_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 + $2"; + Parameter parameter = queryParameters.get("plus_int32_uint32_overflow"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // plus_int32_int64|12,14L|26L + /** + * Plus an int32 value with an int64 value, the sum is within the range of int64. + * + * Expected Results: + * The query should return an int64 type value, identical to the sum of $1 and $2. + */ + public QueryContext plus_int32_int64_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 + $2"; + Parameter parameter = queryParameters.get("plus_int32_int64"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // plus_int32_int64_overflow|10,9223372036854775807L|overflow + /** + * Plus an int32 value with an int64 value, the sum exceeds the range of int64. + * + * Expected Results: + * The query should throw an overflow exception due to int64 overflow. + */ + public QueryContext plus_int32_int64_overflow_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 + $2"; + Parameter parameter = queryParameters.get("plus_int32_int64_overflow"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Plus an int32 value with a double value. + * + * Expected Results: + * The query should return a double type value, identical to the sum of $1 and $2. + */ + public QueryContext plus_int32_double_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 + $2"; + Parameter parameter = queryParameters.get("plus_int32_double"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + /** + * Plus a float type value with a double value. + * + * Expected Results: + * The query should return a double type value, identical to the sum of $1 and $2. + */ + public QueryContext plus_float_double_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 + $2"; + Parameter parameter = queryParameters.get("plus_float_double"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + } + + public class MinusTest { + + // minus_int32_int32|12,13|-1 + /** + * Minus an int32 value with an int32 value, the result is within the range of int32. + * + * Expected Results: + * The query should return an int32 type value, identical to the difference of $1 and $2. + */ + public QueryContext minus_int32_int32_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 - $2"; + Parameter parameter = queryParameters.get("minus_int32_int32"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // minus_int32_int32_overflow|2147483647,-1|overflow + /** + * Minus an int32 value with an int32 value, the result exceeds the range of int32. + * + * Expected Results: + * The query should throw an overflow exception due to int32 overflow. + */ + public QueryContext minus_int32_int32_overflow_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 - $2"; + Parameter parameter = queryParameters.get("minus_int32_int32_overflow"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // minus_int32_uint32_int32|12,+13|-1 + /** + * Minus an int32 value with an uint32 value, the result is within the range of int32. + * + * Expected Results: + * The query should return an int32 type value, identical to the difference of $1 and $2. + */ + public QueryContext minus_int32_uint32_int32_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 - $2"; + Parameter parameter = queryParameters.get("minus_int32_uint32_int32"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // minus_uint32_int32_uint32|+2147483647,-10|+2147483657 + /** + * Minus an uint32 value with an int32 value, the result is within the range of uint32. + * + * Expected Results: + * The query should return an uint32 type value, identical to the difference of $1 and $2. + */ + public QueryContext minus_uint32_int32_uint32_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 - $2"; + Parameter parameter = queryParameters.get("minus_uint32_int32_uint32"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // minus_uint32_int32_overflow|+4294967295,-1|overflow + /** + * Minus an uint32 value with an int32 value, the result exceeds the range of uint32. + * + * Expected Results: + * The query should throw an overflow exception due to uint32 overflow. + */ + public QueryContext minus_uint32_int32_overflow_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 - $2"; + Parameter parameter = queryParameters.get("minus_uint32_int32_overflow"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // minus_int32_int64|12,13L|-1L + /** + * Minus an int32 value with an int64 value, the result is within the range of int64. + * + * Expected Results: + * The query should return an int64 type value, identical to the difference of $1 and $2. + */ + public QueryContext minus_int32_int64_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 - $2"; + Parameter parameter = queryParameters.get("minus_int32_int64"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // minus_int64_int32_overflow|9223372036854775807L,-10|overflow + /** + * Minus an int64 value with an int32 value, the result exceeds the range of int64. + * + * Expected Results: + * The query should throw an overflow exception due to int64 overflow. + */ + public QueryContext minus_int64_int32_overflow_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 - $2"; + Parameter parameter = queryParameters.get("minus_int64_int32_overflow"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // minus_int32_double|12,13.12d|-1.12d + /** + * Minus an int32 value with a double value, the result is within the range of double. + * + * Expected Results: + * The query should return a double type value, identical to the difference of $1 and $2. + */ + public QueryContext minus_int32_double_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 - $2"; + Parameter parameter = queryParameters.get("minus_int32_double"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // minus_float_double|12.0f,13.12d|-1.12d + /** + * Minus a float value with a double value, the result is within the range of double. + * + * Expected Results: + * The query should return a double type value, identical to the difference of $1 and $2. + */ + public QueryContext minus_float_double_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 - $2"; + Parameter parameter = queryParameters.get("minus_float_double"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + } + + public class MultiplyTest { + + // multiply_int32_int32|1,2|2 + /** + * Multiply an int32 value with an int32 value, the result is within the range of int32. + * + * Expected Results: + * The query should return an int32 type value, identical to the product of $1 and $2. + */ + public QueryContext multiply_int32_int32_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 * $2"; + Parameter parameter = queryParameters.get("multiply_int32_int32"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // multiply_int32_int32_overflow|2147483647,2|overflow + /** + * Multiply an int32 value with an int32 value, the result exceeds the range of int32. + * + * Expected Results: + * The query should throw an overflow exception due to int32 overflow. + */ + public QueryContext multiply_int32_int32_overflow_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 * $2"; + Parameter parameter = queryParameters.get("multiply_int32_int32_overflow"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // multiply_int32_uint32_int32|1,+2|2 + /** + * Multiply an int32 value with an uint32 value, the result is within the range of int32. + * + * Expected Results: + * The query should return an int32 type value, identical to the product of $1 and $2. + */ + public QueryContext multiply_int32_uint32_int32_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 * $2"; + Parameter parameter = queryParameters.get("multiply_int32_uint32_int32"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // multiply_int32_uint32_uint32|4,+1000000000|+4000000000 + /** + * Multiply an int32 value with an uint32 value, the result is out of the range of int32 but within the range of uint32. + * + * Expected Results: + * The query should return a uint32 type value, identical to the product of $1 and $2. + */ + public QueryContext multiply_int32_uint32_uint32_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 * $2"; + Parameter parameter = queryParameters.get("multiply_int32_uint32_uint32"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // multiply_int32_uint32_overflow|4,+2000000000|overflow + /** + * Multiply an int32 value with an uint32 value, the result exceeds the range of uint32. + * + * Expected Results: + * The query should throw an overflow exception due to uint32 overflow. + */ + public QueryContext multiply_int32_uint32_overflow_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 * $2"; + Parameter parameter = queryParameters.get("multiply_int32_uint32_overflow"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // multiply_int32_int64|1,2L|2L + /** + * Multiply an int32 value with an int64 value, the result is within the range of int64. + * + * Expected Results: + * The query should return an int64 type value, identical to the product of $1 and $2. + */ + public QueryContext multiply_int32_int64_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 * $2"; + Parameter parameter = queryParameters.get("multiply_int32_int64"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // multiply_int32_int64_overflow|2,9223372036854775807L|overflow + /** + * Multiply an int32 value with an int64 value, the result exceeds the range of int64. + * + * Expected Results: + * The query should throw an overflow exception due to int64 overflow. + */ + public QueryContext multiply_int32_int64_overflow_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 * $2"; + Parameter parameter = queryParameters.get("multiply_int32_int64_overflow"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // multiply_int32_double|2,2.12d|4.24d + /** + * Multiply an int32 value with a double value, the result is within the range of double. + * + * Expected Results: + * The query should return a double type value, identical to the product of $1 and $2. + */ + public QueryContext multiply_int32_double_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 * $2"; + Parameter parameter = queryParameters.get("multiply_int32_double"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // multiply_float_double|2.0f,2.12d|4.24d + /** + * Multiply a float value with a double value, the result is within the range of double. + * + * Expected Results: + * The query should return a double type value, identical to the product of $1 and $2. + */ + public QueryContext multiply_float_double_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 * $2"; + Parameter parameter = queryParameters.get("multiply_float_double"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + } + + public class DivideTest { + + // divide_int32_int32|2,1|2 + /** + * Divide an int32 value by an int32 value, the result is within the range of int32. + * + * Expected Results: + * The query should return an int32 type value, identical to the quotient of $1 and $2. + */ + public QueryContext divide_int32_int32_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 / $2"; + Parameter parameter = queryParameters.get("divide_int32_int32"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // divide_int32_int32_overflow|-2147483648,-1|overflow + /** + * Divide an int32 value by an int32 value, the result exceeds the range of int32. + * + * Expected Results: + * The query should throw an overflow exception due to int32 overflow. + */ + public QueryContext divide_int32_int32_overflow_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 / $2"; + Parameter parameter = queryParameters.get("divide_int32_int32_overflow"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // divide_int32_uint32_int32|2,+1|2 + /** + * Divide an int32 value by an uint32 value, the result is within the range of int32. + * + * Expected Results: + * The query should return an int32 type value, identical to the quotient of $1 and $2. + */ + public QueryContext divide_int32_uint32_int32_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 / $2"; + Parameter parameter = queryParameters.get("divide_int32_uint32_int32"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // divide_uint32_int32_uint32|+4294967295,1|+4294967295 + /** + * Divide an uint32 value by an int32 value, the result is out of the range of int32 but within the range of uint32. + * + * Expected Results: + * The query should return a uint32 type value, identical to the quotient of $1 and $2. + */ + public QueryContext divide_uint32_int32_uint32_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 / $2"; + Parameter parameter = queryParameters.get("divide_uint32_int32_uint32"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // divide_uint32_int32_overflow|+4294967295,-1|overflow + /** + * Divide an uint32 value by a negative int32 value, the result exceeds the range of int32 (INT32_MIN). + * + * Expected Results: + * The query should throw an overflow exception due to int32 overflow. + */ + public QueryContext divide_uint32_int32_overflow_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 / $2"; + Parameter parameter = queryParameters.get("divide_uint32_int32_overflow"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // divide_int32_int64|2,1L|2L + /** + * Divide an int32 value by an int64 value, the result is within the range of int64. + * + * Expected Results: + * The query should return an int64 type value, identical to the quotient of $1 and $2. + */ + public QueryContext divide_int32_int64_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 / $2"; + Parameter parameter = queryParameters.get("divide_int32_int64"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // divide_int64_int32_overflow|-9223372036854775808L,-1|overflow + /** + * Divide an int64 value by an int32 value, the result exceeds the range of int64. + * + * Expected Results: + * The query should throw an overflow exception due to int64 overflow. + */ + public QueryContext divide_int64_int32_overflow_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 / $2"; + Parameter parameter = queryParameters.get("divide_int64_int32_overflow"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // divide_int32_double|2,1.12d|1.77d + /** + * Divide an int32 value by a double value, the result is within the range of double. + * + * Expected Results: + * The query should return a double type value, identical to the quotient of $1 and $2. + */ + public QueryContext divide_int32_double_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 / $2"; + Parameter parameter = queryParameters.get("divide_int32_double"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // divide_float_double|2.0f,1.12d|1.78d + /** + * Divide a float value by a double value, the result is within the range of double. + * + * Expected Results: + * The query should return a double type value, identical to the quotient of $1 and $2. + */ + public QueryContext divide_float_double_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 / $2"; + Parameter parameter = queryParameters.get("divide_float_double"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + + // divide_float_double_NaN|2.0f,0.0d|NaN + /** + * Divide a float value by a double value, where the divisor is zero, resulting in NaN. + * + * Expected Results: + * The query should return NaN as the result of division by zero. + */ + public QueryContext divide_float_double_NaN_test() { + String query = "MATCH (p:person {prop_int32: 933})\n" + " RETURN $1 / $2"; + Parameter parameter = queryParameters.get("divide_float_double_NaN"); + query = parameter.render(query); + return new QueryContext(query, parameter.results); + } + } + + /** + * Find 'knows' edges between two persons of specific property values. + * + * Expected Results: + * The query should return the property values for the two persons involved in the 'knows' relationship. + * @return + */ + public QueryContext get_knows_between_two_persons_test() { + String query = + "MATCH (p1:person)-[r:knows]->(p2:person)\n" + + " WHERE p1.prop_int32 = $1 AND p2.prop_int32 = $2\n" + + " RETURN p1.prop_int32, p2.prop_int32"; + Parameter parameter = queryParameters.get("get_knows_between_two_persons"); + // render the query, replace the $1 and $2 with the actual values + for (int i = 0; i < parameter.parameters.size(); i++) { + query = query.replace("$" + (i + 1), parameter.parameters.get(i)); + } + return new QueryContext(query, parameter.results); + } +} diff --git a/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/cypher/integration/flex/bench/FlexTypeTest.java b/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/cypher/integration/flex/bench/FlexTypeTest.java new file mode 100644 index 000000000000..dbf4a27aeb27 --- /dev/null +++ b/interactive_engine/compiler/src/test/java/com/alibaba/graphscope/cypher/integration/flex/bench/FlexTypeTest.java @@ -0,0 +1,198 @@ +/* + * Copyright 2020 Alibaba Group Holding Limited. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.alibaba.graphscope.cypher.integration.flex.bench; + +import com.alibaba.graphscope.cypher.antlr4.Utils; +import com.alibaba.graphscope.cypher.integration.suite.QueryContext; + +import org.javatuples.Pair; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import org.neo4j.driver.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.reflect.Method; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.util.List; +import java.util.function.Supplier; + +public class FlexTypeTest { + private static final Logger logger = LoggerFactory.getLogger(FlexTypeTest.class); + private static Session session; + private static double DOUBLE_DELTA; + private static float FLOAT_DELTA; + + /** + * start compiler before the test: + * make run graph.schema=./src/test/resources/flex_bench/modern.yaml graph.planner.opt=CBO graph.physical.opt=proto disable.expr.simplify=true + */ + @BeforeClass + public static void setUp() { + String neo4jServerUrl = + System.getProperty("neo4j.bolt.server.url", "neo4j://localhost:7687"); + session = GraphDatabase.driver(neo4jServerUrl).session(); + DOUBLE_DELTA = Double.parseDouble(System.getProperty("double.delta", "0.01")); + FLOAT_DELTA = Float.parseFloat(System.getProperty("float.delta", "0.01")); + } + + @Test + public void run() throws Exception { + FlexTypeQueries queries = new FlexTypeQueries("src/test/resources/flex_bench/parameters"); + + // run comparison test, comparing numeric, string and temporal types + Pair compare = runComponent(queries.getCompare()); + + // run arithmetic plus test + Pair plus = runComponent(queries.getPlus()); + + // run arithmetic minus test + Pair minus = runComponent(queries.getMinus()); + + // run arithmetic multiply test + Pair multi = runComponent(queries.getMultiply()); + + // run arithmetic divide test + Pair divide = runComponent(queries.getDivide()); + + logger.warn( + "\n component: Compare, total tests {}, passed tests {}" + + "\n component: Plus, total tests {}, passed tests {}" + + "\n component: Minus, total tests {}, passed tests {}" + + "\n component: Multiply, total tests {}, passed tests {}" + + "\n component: Divide, total tests {}, passed tests {}", + compare.getValue0(), + compare.getValue1(), + plus.getValue0(), + plus.getValue1(), + minus.getValue0(), + minus.getValue1(), + multi.getValue0(), + multi.getValue1(), + divide.getValue0(), + divide.getValue1()); + + int total = + compare.getValue0() + + plus.getValue0() + + minus.getValue0() + + multi.getValue0() + + divide.getValue0(); + int passed = + compare.getValue1() + + plus.getValue1() + + minus.getValue1() + + multi.getValue1() + + divide.getValue1(); + Assert.assertEquals("total tests: " + total + ", passed tests: " + passed, total, passed); + } + + private Pair runComponent(Object component) { + Method[] methods = component.getClass().getDeclaredMethods(); + int totalTests = methods.length; + int passTests = 0; + for (Method method : methods) { + try { + QueryContext ctx = (QueryContext) method.invoke(component); + checkResult(() -> session.run(ctx.getQuery()), ctx); + logger.warn("test {} passed.", method.getName()); + ++passTests; + } catch (Throwable t) { + logger.error("test {} failed.", method.getName(), t); + } + } + return Pair.with(totalTests, passTests); + } + + private void checkResult(Supplier resultSupplier, QueryContext ctx) { + List expected = ctx.getExpectedResult(); + if (expected.size() == 1 && expected.get(0).equals("empty")) { + Assert.assertTrue(resultSupplier.get().list().isEmpty()); + } else if (expected.size() == 1 && expected.get(0).equals("overflow")) { + try { + resultSupplier.get().list(); + Assert.fail("overflow exception should have been thrown"); + } catch (Exception e) { + Assert.assertTrue( + "cannot catch overflow exception from execution message", + e.getMessage().contains("overflow")); + } + } else if (expected.size() == 1 && expected.get(0).equals("NaN")) { + try { + resultSupplier.get().list(); + Assert.fail("NaN exception should have been thrown"); + } catch (Exception e) { + Assert.assertTrue( + "cannot catch NaN exception from execution message", + e.getMessage().contains("NaN")); + } + } else { + List records = resultSupplier.get().list(); + Assert.assertTrue("records should not be empty", !records.isEmpty()); + Record single = records.get(0); + Assert.assertEquals(expected.size(), single.size()); + for (int i = 0; i < expected.size(); i++) { + Value actual = single.get(i); + String expectedValue = expected.get(i); + boolean unsigned = false; + if (expectedValue.startsWith("+")) { + expectedValue = expectedValue.substring(1); + unsigned = true; + } + boolean int32 = true; + if (expectedValue.toLowerCase().endsWith("l")) { + int32 = false; + } + String upperCase = expectedValue.toUpperCase(); + if (upperCase.endsWith("L") || upperCase.endsWith("D") || upperCase.endsWith("F")) { + expectedValue = expectedValue.substring(0, expectedValue.length() - 1); + } + if (upperCase.endsWith("F")) { + Assert.assertEquals( + Float.parseFloat(expectedValue), actual.asFloat(), FLOAT_DELTA); + } else if (upperCase.endsWith("D")) { + Assert.assertEquals( + Double.parseDouble(expectedValue), actual.asDouble(), DOUBLE_DELTA); + } else { + String actualValue = getActualValue(actual, unsigned, int32); + Assert.assertEquals(expectedValue, actualValue); + } + } + } + } + + @AfterClass + public static void tearDown() { + session.close(); + } + + public String getActualValue(Value actual, boolean unsigned, boolean int32) { + if (unsigned) { + if (int32) { + int value = actual.asInt(); + return new BigDecimal(new BigInteger(1, Utils.intToBytes(value))).toString(); + } else { + long value = actual.asLong(); + return new BigDecimal(new BigInteger(1, Utils.longToBytes(value))).toString(); + } + } + return actual.toString(); + } +} diff --git a/interactive_engine/compiler/src/test/resources/flex_bench/data/knows.csv b/interactive_engine/compiler/src/test/resources/flex_bench/data/knows.csv new file mode 100644 index 000000000000..feef611d8d67 --- /dev/null +++ b/interactive_engine/compiler/src/test/resources/flex_bench/data/knows.csv @@ -0,0 +1,4 @@ +source_person_id,destination_person_id +32,33 +33,34 +32,34 diff --git a/interactive_engine/compiler/src/test/resources/flex_bench/data/person.csv b/interactive_engine/compiler/src/test/resources/flex_bench/data/person.csv new file mode 100644 index 000000000000..2ce1f54db6dc --- /dev/null +++ b/interactive_engine/compiler/src/test/resources/flex_bench/data/person.csv @@ -0,0 +1,6 @@ +prop_int32,prop_uint32,prop_int64,prop_uint64,prop_bool,prop_float,prop_double,prop_text,prop_char,prop_varchar,prop_date,prop_ts +32,12345,1234567890,9876543210,true,3.14,3.14159,Sample text,abcd,abcd,2025-02-12,2025-02-12T12:34:56 +33,12346,1234567891,9876543211,false,2.71,2.71828,Another sample text,efgh,efgh,2025-02-13,2025-02-13T13:45:01 +34,12347,1234567892,9876543212,true,1.62,1.61803,Long text with no truncation,ijkl,ijkl,2025-02-14,2025-02-14T14:56:12 +-2147483648,4294967284,1234567890,18446744073709551602,true,3.14,3.14159,Sample text,abcd,abcd,2025-02-12,2025-02-12T12:34:56 +933,12347,1234567892,9876543212,true,1.62,1.61803,Long text with no truncation,ijkl,ijkl,2025-02-14,2025-02-14T14:56:12 diff --git a/interactive_engine/compiler/src/test/resources/flex_bench/import.yaml b/interactive_engine/compiler/src/test/resources/flex_bench/import.yaml new file mode 100644 index 000000000000..08768cf7e8ad --- /dev/null +++ b/interactive_engine/compiler/src/test/resources/flex_bench/import.yaml @@ -0,0 +1,40 @@ +graph: modern_graph +loading_config: + data_source: + scheme: file # file, oss, s3, hdfs; only file is supported now + #location: /home/graphscope/gs_interactive_default_graph/ + import_option: init # append, overwrite, only init is supported now + format: + type: csv + metadata: + delimiter: "," # other loading configuration places here + header_row: true # whether to use the first row as the header + quoting: false + quote_char: '"' + double_quote: true + escape_char: '\' + escaping: false + block_size: 4MB + batch_reader: true + null_values: [""] + +vertex_mappings: + - type_name: person # must align with the schema + inputs: + - person.csv +edge_mappings: + - type_triplet: + edge: knows + source_vertex: person + destination_vertex: person + inputs: + - knows.csv + source_vertex_mappings: + - column: + index: 0 + name: source_person_id + destination_vertex_mappings: + - column: + index: 1 + name: destination_person_id + \ No newline at end of file diff --git a/interactive_engine/compiler/src/test/resources/flex_bench/modern.yaml b/interactive_engine/compiler/src/test/resources/flex_bench/modern.yaml new file mode 100644 index 000000000000..3e6957050961 --- /dev/null +++ b/interactive_engine/compiler/src/test/resources/flex_bench/modern.yaml @@ -0,0 +1,70 @@ +schema: + vertex_types: + - type_name: person + type_id: 0 + properties: + - property_id: 0 + property_name: prop_int32 + property_type: + primitive_type: DT_SIGNED_INT32 + - property_id: 1 + property_name: prop_uint32 + property_type: + primitive_type: DT_UNSIGNED_INT32 + - property_id: 2 + property_name: prop_int64 + property_type: + primitive_type: DT_SIGNED_INT64 + - property_id: 3 + property_name: prop_uint64 + property_type: + primitive_type: DT_UNSIGNED_INT64 + - property_id: 4 + property_name: prop_bool + property_type: + primitive_type: DT_BOOL + - property_id: 5 + property_name: prop_float + property_type: + primitive_type: DT_FLOAT + - property_id: 6 + property_name: prop_double + property_type: + primitive_type: DT_DOUBLE + - property_id: 7 + property_name: prop_text + property_type: + string: + long_text: + - property_id: 8 + property_name: prop_char + property_type: + string: + char: + fixed_length: 4 + - property_id: 9 + property_name: prop_varchar + property_type: + string: + var_char: + max_length: 4 + - property_id: 10 + property_name: prop_date + property_type: + temporal: + date32: + - property_id: 11 + property_name: prop_ts + property_type: + temporal: + timestamp: + primary_keys: + - prop_int32 + edge_types: + - type_name: knows + type_id: 0 + vertex_type_pair_relations: + - source_vertex: person + destination_vertex: person + relation: MANY_TO_MANY + diff --git a/interactive_engine/compiler/src/test/resources/flex_bench/parameters b/interactive_engine/compiler/src/test/resources/flex_bench/parameters new file mode 100644 index 000000000000..ebe02ea722ca --- /dev/null +++ b/interactive_engine/compiler/src/test/resources/flex_bench/parameters @@ -0,0 +1,78 @@ +test_case_name|parameters|results +compare_int32_int32|-2147483648|-2147483648 +compare_uint32_uint32|+4294967284|+4294967284 +compare_uint32_int32|12345|+12345 +// 4294967284 +compare_uint32_int32_overflow|-12|empty +compare_uint64_uint64|+18446744073709551602L|+18446744073709551602L +compare_int64_int64|1234567890L|1234567890L +compare_uint64_int64|9876543210L|+9876543210L +// 18446744073709551602 +compare_uint64_int64_overflow|-14L|empty +compare_int32_int64|32L|32 +// -2147483648 +compare_int32_int64_overflow|2147483648L|empty +compare_float_float|3.14f|3.14f +compare_double_double|3.14159d|3.14159d +compare_float_double|3.14d|3.14f +compare_float_double_loss|3.14159d|empty +compare_int32_double|32.0d|32 +compare_char_text|"abcd"|"abcd" +compare_char_long_text|"abcde"|"abcd" +compare_varchar_long_text|"abcde"|"abcd" +compare_string_long_text|"Long text with no truncation"|"Long text with no truncation" +// 2025-02-13 +compare_date32_i32|20132|20132 +// 2025-02-13T13:45:01 +compare_timestamp_i64|1739454301000L|1739454301000L + +plus_int32_int32|12,13|25 +plus_int32_int32_overflow|2147483647,1|overflow +plus_int32_uint32_int32|12,+13|25 +plus_int32_uint32_uint32|10,+2147483647|+2147483657 +plus_int32_uint32_overflow|1,+4294967295|overflow +plus_int32_int64|12,14L|26L +plus_int32_int64_overflow|10,9223372036854775807L|overflow +plus_int32_double|12,16.23d|28.23d +plus_float_double|12.0f,16.23d|28.23d + +minus_int32_int32|12,13|-1 +minus_int32_int32_overflow|2147483647,-1|overflow +minus_int32_uint32_int32|13,+12|+1 +minus_uint32_int32_uint32|+2147483647,-10|+2147483657 +minus_uint32_int32_overflow|+4294967295,-1|overflow +minus_int32_int64|12,13L|-1L +minus_int64_int32_overflow|9223372036854775807L,-10|overflow +minus_int32_double|12,13.12d|-1.12d +minus_float_double|12.0f,13.12d|-1.12d + +multiply_int32_int32|1,2|2 +multiply_int32_int32_overflow|2147483647,2|overflow +multiply_int32_uint32_int32|1,+2|2 +multiply_int32_uint32_uint32|4,+1000000000|+4000000000 +multiply_int32_uint32_overflow|4,+2000000000|overflow +multiply_int32_int64|1,2L|2L +multiply_int32_int64_overflow|2,9223372036854775807L|overflow +multiply_int32_double|2,2.12d|4.24d +multiply_float_double|2.0f,2.12d|4.24d + +divide_int32_int32|2,1|2 +divide_int32_int32_overflow|-2147483648,-1|overflow +divide_int32_uint32_int32|2,+1|2 +divide_uint32_int32_uint32|+4294967295,1|+4294967295 +divide_uint32_int32_overflow|+4294967295,-1|overflow +divide_int32_int64|2,1L|2L +divide_int64_int32_overflow|-9223372036854775808L,-1|overflow +divide_int32_double|2,1.12d|1.79d +divide_float_double|2.0f,1.12d|1.79d +divide_float_double_NaN|2.0f,0.0d|NaN + +get_knows_between_two_persons|32,33|32,33 + +compare_i32_within_i32_array|[32,123]|32 +compare_i32_within_i64_array|[32L,123L]|32 +compare_i64_within_i32_array|[1234567890,12]|1234567890L +compare_float_within_double_array|[3.14d,6.71d,8.90d]|3.14f +compare_char_within_string_array|["abcde","cdefg"]|"abcd" + + diff --git a/interactive_engine/compiler/src/test/resources/proto/compare_double_float_test.json b/interactive_engine/compiler/src/test/resources/proto/compare_double_float_test.json new file mode 100644 index 000000000000..7c37de4adc25 --- /dev/null +++ b/interactive_engine/compiler/src/test/resources/proto/compare_double_float_test.json @@ -0,0 +1,216 @@ +{ + "plan": [{ + "opr": { + "scan": { + "alias": 0, + "params": { + "tables": [{ + "id": 0 + }], + "predicate": { + "operators": [{ + "var": { + "property": { + "key": { + "name": "prop_double" + } + }, + "nodeType": { + "dataType": { + "primitiveType": "DT_DOUBLE" + } + } + }, + "nodeType": { + "dataType": { + "primitiveType": "DT_DOUBLE" + } + } + }, { + "logical": "EQ", + "nodeType": { + "dataType": { + "primitiveType": "DT_BOOL" + } + } + }, { + "const": { + "f32": 1.2 + }, + "nodeType": { + "dataType": { + "primitiveType": "DT_FLOAT" + } + } + }] + }, + "sampleRatio": 1.0 + } + } + }, + "metaData": [{ + "type": { + "graphType": { + "graphDataType": [{ + "label": { + }, + "props": [{ + "propId": { + "name": "prop_int32" + }, + "type": { + "primitiveType": "DT_SIGNED_INT32" + } + }, { + "propId": { + "name": "prop_uint32" + }, + "type": { + "primitiveType": "DT_UNSIGNED_INT32" + } + }, { + "propId": { + "name": "prop_int64" + }, + "type": { + "primitiveType": "DT_SIGNED_INT64" + } + }, { + "propId": { + "name": "prop_uint64" + }, + "type": { + "primitiveType": "DT_UNSIGNED_INT64" + } + }, { + "propId": { + "name": "prop_bool" + }, + "type": { + "primitiveType": "DT_BOOL" + } + }, { + "propId": { + "name": "prop_float" + }, + "type": { + "primitiveType": "DT_FLOAT" + } + }, { + "propId": { + "name": "prop_double" + }, + "type": { + "primitiveType": "DT_DOUBLE" + } + }, { + "propId": { + "name": "prop_text" + }, + "type": { + "string": { + "longText": { + } + } + } + }, { + "propId": { + "name": "prop_char" + }, + "type": { + "string": { + "char": { + "fixedLength": 4 + } + } + } + }, { + "propId": { + "name": "prop_varchar" + }, + "type": { + "string": { + "varChar": { + "maxLength": 4 + } + } + } + }, { + "propId": { + "name": "prop_date" + }, + "type": { + "temporal": { + "date32": { + } + } + } + }, { + "propId": { + "name": "prop_ts" + }, + "type": { + "temporal": { + "timestamp": { + } + } + } + }] + }] + } + } + }] + }, { + "opr": { + "project": { + "mappings": [{ + "expr": { + "operators": [{ + "var": { + "tag": { + "id": 0 + }, + "property": { + "key": { + "name": "prop_double" + } + }, + "nodeType": { + "dataType": { + "primitiveType": "DT_DOUBLE" + } + } + }, + "nodeType": { + "dataType": { + "primitiveType": "DT_DOUBLE" + } + } + }] + }, + "alias": 1 + }] + } + }, + "metaData": [{ + "type": { + "dataType": { + "primitiveType": "DT_DOUBLE" + } + }, + "alias": 1 + }] + }, { + "opr": { + "sink": { + "tags": [{ + "tag": 1 + }], + "sinkTarget": { + "sinkDefault": { + } + } + } + } + }] +} diff --git a/interactive_engine/compiler/src/test/resources/proto/compare_uint64_uint64_test.json b/interactive_engine/compiler/src/test/resources/proto/compare_uint64_uint64_test.json new file mode 100644 index 000000000000..ac507ce23624 --- /dev/null +++ b/interactive_engine/compiler/src/test/resources/proto/compare_uint64_uint64_test.json @@ -0,0 +1,216 @@ +{ + "plan": [{ + "opr": { + "scan": { + "alias": 0, + "params": { + "tables": [{ + "id": 0 + }], + "predicate": { + "operators": [{ + "var": { + "property": { + "key": { + "name": "prop_uint64" + } + }, + "nodeType": { + "dataType": { + "primitiveType": "DT_UNSIGNED_INT64" + } + } + }, + "nodeType": { + "dataType": { + "primitiveType": "DT_UNSIGNED_INT64" + } + } + }, { + "logical": "EQ", + "nodeType": { + "dataType": { + "primitiveType": "DT_BOOL" + } + } + }, { + "const": { + "u64": "18446744073709551602" + }, + "nodeType": { + "dataType": { + "primitiveType": "DT_UNSIGNED_INT64" + } + } + }] + }, + "sampleRatio": 1.0 + } + } + }, + "metaData": [{ + "type": { + "graphType": { + "graphDataType": [{ + "label": { + }, + "props": [{ + "propId": { + "name": "prop_int32" + }, + "type": { + "primitiveType": "DT_SIGNED_INT32" + } + }, { + "propId": { + "name": "prop_uint32" + }, + "type": { + "primitiveType": "DT_UNSIGNED_INT32" + } + }, { + "propId": { + "name": "prop_int64" + }, + "type": { + "primitiveType": "DT_SIGNED_INT64" + } + }, { + "propId": { + "name": "prop_uint64" + }, + "type": { + "primitiveType": "DT_UNSIGNED_INT64" + } + }, { + "propId": { + "name": "prop_bool" + }, + "type": { + "primitiveType": "DT_BOOL" + } + }, { + "propId": { + "name": "prop_float" + }, + "type": { + "primitiveType": "DT_FLOAT" + } + }, { + "propId": { + "name": "prop_double" + }, + "type": { + "primitiveType": "DT_DOUBLE" + } + }, { + "propId": { + "name": "prop_text" + }, + "type": { + "string": { + "longText": { + } + } + } + }, { + "propId": { + "name": "prop_char" + }, + "type": { + "string": { + "char": { + "fixedLength": 4 + } + } + } + }, { + "propId": { + "name": "prop_varchar" + }, + "type": { + "string": { + "varChar": { + "maxLength": 4 + } + } + } + }, { + "propId": { + "name": "prop_date" + }, + "type": { + "temporal": { + "date32": { + } + } + } + }, { + "propId": { + "name": "prop_ts" + }, + "type": { + "temporal": { + "timestamp": { + } + } + } + }] + }] + } + } + }] + }, { + "opr": { + "project": { + "mappings": [{ + "expr": { + "operators": [{ + "var": { + "tag": { + "id": 0 + }, + "property": { + "key": { + "name": "prop_uint64" + } + }, + "nodeType": { + "dataType": { + "primitiveType": "DT_UNSIGNED_INT64" + } + } + }, + "nodeType": { + "dataType": { + "primitiveType": "DT_UNSIGNED_INT64" + } + } + }] + }, + "alias": 1 + }] + } + }, + "metaData": [{ + "type": { + "dataType": { + "primitiveType": "DT_UNSIGNED_INT64" + } + }, + "alias": 1 + }] + }, { + "opr": { + "sink": { + "tags": [{ + "tag": 1 + }], + "sinkTarget": { + "sinkDefault": { + } + } + } + } + }] +} diff --git a/interactive_engine/compiler/src/test/resources/proto/divide_int32_uint32_int32.json b/interactive_engine/compiler/src/test/resources/proto/divide_int32_uint32_int32.json new file mode 100644 index 000000000000..7d185c194268 --- /dev/null +++ b/interactive_engine/compiler/src/test/resources/proto/divide_int32_uint32_int32.json @@ -0,0 +1,183 @@ +{ + "plan": [{ + "opr": { + "scan": { + "alias": 0, + "params": { + "tables": [{ + "id": 0 + }], + "sampleRatio": 1.0 + } + } + }, + "metaData": [{ + "type": { + "graphType": { + "graphDataType": [{ + "label": { + }, + "props": [{ + "propId": { + "name": "prop_int32" + }, + "type": { + "primitiveType": "DT_SIGNED_INT32" + } + }, { + "propId": { + "name": "prop_uint32" + }, + "type": { + "primitiveType": "DT_UNSIGNED_INT32" + } + }, { + "propId": { + "name": "prop_int64" + }, + "type": { + "primitiveType": "DT_SIGNED_INT64" + } + }, { + "propId": { + "name": "prop_uint64" + }, + "type": { + "primitiveType": "DT_UNSIGNED_INT64" + } + }, { + "propId": { + "name": "prop_bool" + }, + "type": { + "primitiveType": "DT_BOOL" + } + }, { + "propId": { + "name": "prop_float" + }, + "type": { + "primitiveType": "DT_FLOAT" + } + }, { + "propId": { + "name": "prop_double" + }, + "type": { + "primitiveType": "DT_DOUBLE" + } + }, { + "propId": { + "name": "prop_text" + }, + "type": { + "string": { + "longText": { + } + } + } + }, { + "propId": { + "name": "prop_char" + }, + "type": { + "string": { + "char": { + "fixedLength": 4 + } + } + } + }, { + "propId": { + "name": "prop_varchar" + }, + "type": { + "string": { + "varChar": { + "maxLength": 4 + } + } + } + }, { + "propId": { + "name": "prop_date" + }, + "type": { + "temporal": { + "date32": { + } + } + } + }, { + "propId": { + "name": "prop_ts" + }, + "type": { + "temporal": { + "timestamp": { + } + } + } + }] + }] + } + } + }] + }, { + "opr": { + "project": { + "mappings": [{ + "expr": { + "operators": [{ + "const": { + "i32": 2 + }, + "nodeType": { + "dataType": { + "primitiveType": "DT_SIGNED_INT32" + } + } + }, { + "arith": "DIV", + "nodeType": { + "dataType": { + "primitiveType": "DT_UNSIGNED_INT64" + } + } + }, { + "const": { + "u64": "3" + }, + "nodeType": { + "dataType": { + "primitiveType": "DT_UNSIGNED_INT64" + } + } + }] + }, + "alias": 1 + }] + } + }, + "metaData": [{ + "type": { + "dataType": { + "primitiveType": "DT_UNSIGNED_INT64" + } + }, + "alias": 1 + }] + }, { + "opr": { + "sink": { + "tags": [{ + "tag": 1 + }], + "sinkTarget": { + "sinkDefault": { + } + } + } + } + }] +} diff --git a/interactive_engine/executor/ir/proto/common.proto b/interactive_engine/executor/ir/proto/common.proto index 14a07f03da4f..f3eb1f3d1d75 100644 --- a/interactive_engine/executor/ir/proto/common.proto +++ b/interactive_engine/executor/ir/proto/common.proto @@ -87,5 +87,8 @@ message Value { Date32 date = 14; Time32 time = 15; Timestamp timestamp = 16; + uint32 u32 = 17; + uint64 u64 = 18; + float f32 = 19; } }