From 4af5260df2ce007500cbba8c1d1c3d90331a98bc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=A2=E5=81=A5?= Date: Mon, 15 Jan 2024 15:26:16 +0800 Subject: [PATCH 001/200] [fix](Nereids): fix wrong api calling (#29977) --- .../doris/nereids/rules/exploration/mv/HyperGraphAggTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/HyperGraphAggTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/HyperGraphAggTest.java index 29d3e0edc36d05..6032d000407db7 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/HyperGraphAggTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/HyperGraphAggTest.java @@ -87,9 +87,9 @@ void testIJWithAgg() { } LogicalCompatibilityContext constructContext(Plan p1, Plan p2) { - StructInfo st1 = AbstractMaterializedViewRule.extractStructInfo(p1, + StructInfo st1 = MaterializedViewUtils.extractStructInfo(p1, null).get(0); - StructInfo st2 = AbstractMaterializedViewRule.extractStructInfo(p2, + StructInfo st2 = MaterializedViewUtils.extractStructInfo(p2, null).get(0); RelationMapping rm = RelationMapping.generate(st1.getRelations(), st2.getRelations()).get(0); SlotMapping sm = SlotMapping.generate(rm); From cd2cf954968180369dd5b6d8f926ac8c85b4f75d Mon Sep 17 00:00:00 2001 From: Gabriel Date: Mon, 15 Jan 2024 15:30:40 +0800 Subject: [PATCH 002/200] [pipelineX](improvement) Improve data distribution for streaming agg (#29969) --- be/src/pipeline/exec/streaming_aggregation_sink_operator.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/be/src/pipeline/exec/streaming_aggregation_sink_operator.h b/be/src/pipeline/exec/streaming_aggregation_sink_operator.h index 16f28bbeb76821..766d2c2fbbef48 100644 --- a/be/src/pipeline/exec/streaming_aggregation_sink_operator.h +++ b/be/src/pipeline/exec/streaming_aggregation_sink_operator.h @@ -121,7 +121,7 @@ class StreamingAggSinkOperatorX final : public AggSinkOperatorX::required_data_distribution(); } }; From 18fe84dceb15d5dbccfb6e24f4dfbc6f0877f321 Mon Sep 17 00:00:00 2001 From: amory Date: Mon, 15 Jan 2024 16:54:47 +0800 Subject: [PATCH 003/200] [Improve](array) support array_enumerate_uniq and array_suffle for nereids (#29936) --- .../doris/catalog/BuiltinScalarFunctions.java | 4 + .../functions/scalar/ArrayEnumerateUniq.java | 73 ++++++++++++++++ .../functions/scalar/ArrayShuffle.java | 86 +++++++++++++++++++ .../visitor/ScalarFunctionVisitor.java | 10 +++ .../test_array_zip_array_enumerate_uniq.out | 12 +++ ...test_array_zip_array_enumerate_uniq.groovy | 32 ++++--- 6 files changed, 204 insertions(+), 13 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayEnumerateUniq.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayShuffle.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java index c689ab0d772834..c4c4043bf79301 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java @@ -37,6 +37,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayDifference; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayDistinct; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayEnumerate; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayEnumerateUniq; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayExcept; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayExists; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayFilter; @@ -59,6 +60,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayRemove; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayRepeat; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayReverseSort; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayShuffle; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArraySlice; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArraySort; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArraySortBy; @@ -445,6 +447,7 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(ArrayDifference.class, "array_difference"), scalar(ArrayDistinct.class, "array_distinct"), scalar(ArrayEnumerate.class, "array_enumerate"), + scalar(ArrayEnumerateUniq.class, "array_enumerate_uniq"), scalar(ArrayExcept.class, "array_except"), scalar(ArrayExists.class, "array_exists"), scalar(ArrayFilter.class, "array_filter"), @@ -470,6 +473,7 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(ArraySlice.class, "array_slice"), scalar(ArraySort.class, "array_sort"), scalar(ArraySortBy.class, "array_sortby"), + scalar(ArrayShuffle.class, "array_shuffle", "shuffle"), scalar(ArraySum.class, "array_sum"), scalar(ArrayUnion.class, "array_union"), scalar(ArrayWithConstant.class, "array_with_constant"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayEnumerateUniq.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayEnumerateUniq.java new file mode 100644 index 00000000000000..4743cf15516b5f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayEnumerateUniq.java @@ -0,0 +1,73 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; +import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.ArrayType; +import org.apache.doris.nereids.types.BigIntType; +import org.apache.doris.nereids.types.coercion.AnyDataType; +import org.apache.doris.nereids.util.ExpressionUtils; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * ScalarFunction 'array_enumerate_uniq'. + * more than 0 array as args + */ +public class ArrayEnumerateUniq extends ScalarFunction + implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(ArrayType.of(BigIntType.INSTANCE)).varArgs(ArrayType.of(new AnyDataType(0))) + ); + + /** + * constructor with more than 0 arguments. + */ + public ArrayEnumerateUniq(Expression arg, Expression ...varArgs) { + super("array_enumerate_uniq", ExpressionUtils.mergeArguments(arg, varArgs)); + } + + /** + * withChildren. + */ + @Override + public ArrayEnumerateUniq withChildren(List children) { + Preconditions.checkArgument(!children.isEmpty()); + return new ArrayEnumerateUniq(children.get(0), children.subList(1, children.size()).toArray(new Expression[0])); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitArrayEnumerateUniq(this, context); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayShuffle.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayShuffle.java new file mode 100644 index 00000000000000..6a8624f4166237 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayShuffle.java @@ -0,0 +1,86 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; +import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.ArrayType; +import org.apache.doris.nereids.types.BigIntType; +import org.apache.doris.nereids.types.coercion.AnyDataType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * ScalarFunction 'array_shuffle' + * with 1 or 2 arguments : array_shuffle(arr) or array_shuffle(arr, seed) + */ +public class ArrayShuffle extends ScalarFunction + implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.retArgType(0).args(ArrayType.of(new AnyDataType(0))), + FunctionSignature.retArgType(0) + .args(ArrayType.of(new AnyDataType(0)), BigIntType.INSTANCE) + ); + + /** + * constructor with 1 arguments. + */ + public ArrayShuffle(Expression arg) { + super("array_shuffle", arg); + } + + /** + * constructor with 2 arguments. + */ + public ArrayShuffle(Expression arg, Expression arg1) { + super("array_shuffle", arg, arg1); + } + + /** + * withChildren. + */ + @Override + public ArrayShuffle withChildren(List children) { + Preconditions.checkArgument(children.size() == 1 + || children.size() == 2); + if (children.size() == 1) { + return new ArrayShuffle(children.get(0)); + } else { + return new ArrayShuffle(children.get(0), children.get(1)); + } + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitArrayShuffle(this, context); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java index 0507e326256ab6..81ce549d582ead 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java @@ -39,6 +39,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayDifference; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayDistinct; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayEnumerate; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayEnumerateUniq; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayExcept; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayExists; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayFilter; @@ -59,6 +60,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayRemove; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayRepeat; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayReverseSort; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayShuffle; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArraySlice; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArraySort; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArraySortBy; @@ -484,6 +486,10 @@ default R visitArrayEnumerate(ArrayEnumerate arrayEnumerate, C context) { return visitScalarFunction(arrayEnumerate, context); } + default R visitArrayEnumerateUniq(ArrayEnumerateUniq arrayEnumerateUniq, C context) { + return visitScalarFunction(arrayEnumerateUniq, context); + } + default R visitArrayExcept(ArrayExcept arrayExcept, C context) { return visitScalarFunction(arrayExcept, context); } @@ -564,6 +570,10 @@ default R visitArraySortBy(ArraySortBy arraySortBy, C context) { return visitScalarFunction(arraySortBy, context); } + default R visitArrayShuffle(ArrayShuffle arrayShuffle, C context) { + return visitScalarFunction(arrayShuffle, context); + } + default R visitArrayMap(ArrayMap arraySort, C context) { return visitScalarFunction(arraySort, context); } diff --git a/regression-test/data/datatype_p0/nested_types/query/array_functions/test_array_zip_array_enumerate_uniq.out b/regression-test/data/datatype_p0/nested_types/query/array_functions/test_array_zip_array_enumerate_uniq.out index 9aadb17bd0b000..e29e7082664cb9 100644 --- a/regression-test/data/datatype_p0/nested_types/query/array_functions/test_array_zip_array_enumerate_uniq.out +++ b/regression-test/data/datatype_p0/nested_types/query/array_functions/test_array_zip_array_enumerate_uniq.out @@ -11,3 +11,15 @@ array_enumerate_uniq -- !old_sql -- [1] +-- !sql -- +array_enumerate_uniq + +-- !nereid_sql -- +[1, 1, 2] + +-- !nereid_sql -- +[1, 1, 1] + +-- !nereid_sql -- +[1] + diff --git a/regression-test/suites/datatype_p0/nested_types/query/array_functions/test_array_zip_array_enumerate_uniq.groovy b/regression-test/suites/datatype_p0/nested_types/query/array_functions/test_array_zip_array_enumerate_uniq.groovy index baff9ad73a420e..b8207c1ee76ab3 100644 --- a/regression-test/suites/datatype_p0/nested_types/query/array_functions/test_array_zip_array_enumerate_uniq.groovy +++ b/regression-test/suites/datatype_p0/nested_types/query/array_functions/test_array_zip_array_enumerate_uniq.groovy @@ -75,20 +75,26 @@ suite("test_array_zip_array_enumerate_uniq", "p0") { // nereid not support array_enumerate_uniq // ============= array_enumerate_uniq ========= -// qt_sql "SELECT 'array_enumerate_uniq';" -// order_qt_nereid_sql """ SELECT array_enumerate_uniq(array_enumerate_uniq(array(cast(10 as LargeInt), cast(100 as LargeInt), cast(2 as LargeInt))), array(cast(123 as LargeInt), cast(1023 as LargeInt), cast(123 as LargeInt))); """ -// -// order_qt_nereid_sql """SELECT array_enumerate_uniq( -// [111111, 222222, 333333], -// [444444, 555555, 666666], -// [111111, 222222, 333333], -// [444444, 555555, 666666], -// [111111, 222222, 333333], -// [444444, 555555, 666666], -// [111111, 222222, 333333], -// [444444, 555555, 666666]);""" -// order_qt_nereid_sql """SELECT array_enumerate_uniq(array(STDDEV_SAMP(910947.571364)), array(NULL)) from numbers;""" + qt_sql "SELECT 'array_enumerate_uniq';" + order_qt_nereid_sql """ SELECT array_enumerate_uniq(array_enumerate_uniq(array(cast(10 as LargeInt), cast(100 as LargeInt), cast(2 as LargeInt))), array(cast(123 as LargeInt), cast(1023 as LargeInt), cast(123 as LargeInt))); """ + + order_qt_nereid_sql """SELECT array_enumerate_uniq( + [111111, 222222, 333333], + [444444, 555555, 666666], + [111111, 222222, 333333], + [444444, 555555, 666666], + [111111, 222222, 333333], + [444444, 555555, 666666], + [111111, 222222, 333333], + [444444, 555555, 666666]);""" + order_qt_nereid_sql """SELECT array_enumerate_uniq(array(STDDEV_SAMP(910947.571364)), array(NULL)) from numbers;""" // //order_qt_sql """ SELECT max(array_join(arr)) FROM (SELECT array_enumerate_uniq(group_array(DIV(number, 54321)) AS nums, group_array(cast(DIV(number, 98765) as string))) AS arr FROM (SELECT number FROM numbers LIMIT 1000000) GROUP BY bitmap_hash(number) % 100000);""" + // array_shuffle + // do not check result, since shuffle result is random + sql "SELECT array_sum(array_shuffle([1, 2, 3, 3, null, null, 4, 4])), array_shuffle([1, 2, 3, 3, null, null, 4, 4], 0), shuffle([1, 2, 3, 3, null, null, 4, 4], 0)" + sql "SELECT array_sum(array_shuffle([1.111, 2.222, 3.333])), array_shuffle([1.111, 2.222, 3.333], 0), shuffle([1.111, 2.222, 3.333], 0)" + sql "SELECT array_size(array_shuffle(['aaa', null, 'bbb', 'fff'])), array_shuffle(['aaa', null, 'bbb', 'fff'], 0), shuffle(['aaa', null, 'bbb', 'fff'], 0)" + sql """select array_size(array("2020-01-02", "2022-01-03", "2021-01-01", "1996-04-17")), array_shuffle(array("2020-01-02", "2022-01-03", "2021-01-01", "1996-04-17"), 0), shuffle(array("2020-01-02", "2022-01-03", "2021-01-01", "1996-04-17"), 0)""" } From 1f6fbeea51d3d3e4611346a748b25ac9bf85f2d3 Mon Sep 17 00:00:00 2001 From: shuke <37901441+shuke987@users.noreply.github.com> Date: Mon, 15 Jan 2024 17:15:37 +0800 Subject: [PATCH 004/200] [test](regression-test) fix case bug, add 'order by' to make it stable (#29981) --- regression-test/suites/ddl_p0/test_alter_view.groovy | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/regression-test/suites/ddl_p0/test_alter_view.groovy b/regression-test/suites/ddl_p0/test_alter_view.groovy index c921a4162847c5..936dba881b0494 100644 --- a/regression-test/suites/ddl_p0/test_alter_view.groovy +++ b/regression-test/suites/ddl_p0/test_alter_view.groovy @@ -47,7 +47,7 @@ suite("test_alter_view") { (2, 10, 50) """ - qt_select " SELECT * FROM ${viewName} " + qt_select " SELECT * FROM ${viewName} order by k1, k2 " sql """ ALTER VIEW ${viewName} (k1, k2) @@ -55,7 +55,7 @@ suite("test_alter_view") { SELECT c1 as k1, sum(c3) as k2 FROM ${tableName} GROUP BY c1 """ - qt_select " SELECT * FROM ${viewName} " + qt_select " SELECT * FROM ${viewName} order by k1, k2 " sql "DROP VIEW ${viewName}" sql "DROP TABLE ${tableName}" From 0882223dcb6054975e02e76223db94cc533947df Mon Sep 17 00:00:00 2001 From: jakevin Date: Mon, 15 Jan 2024 17:34:08 +0800 Subject: [PATCH 005/200] [fix](Nereids): fix offset in PlanTranslator (#29789) Current BE operator don't support `offset`, we need add offset into `ExchangeNode` --- .../translator/PhysicalPlanTranslator.java | 23 +++++++---- .../post/AddOffsetIntoDistribute.java | 41 +++++++++++++++++++ .../processor/post/PlanPostProcessors.java | 1 + .../trees/plans/physical/PhysicalLimit.java | 5 +++ 4 files changed, 61 insertions(+), 9 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/AddOffsetIntoDistribute.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 6270c5795b3dd2..b20d0738140f59 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -268,13 +268,14 @@ public PlanFragment translatePlan(PhysicalPlan physicalPlan) { @Override public PlanFragment visitPhysicalDistribute(PhysicalDistribute distribute, PlanTranslatorContext context) { - PlanFragment inputFragment = distribute.child().accept(this, context); - List> distributeExprLists = getDistributeExprs(distribute.child()); + Plan child = distribute.child(); + PlanFragment inputFragment = child.accept(this, context); + List> distributeExprLists = getDistributeExprs(child); // TODO: why need set streaming here? should remove this. if (inputFragment.getPlanRoot() instanceof AggregationNode - && distribute.child() instanceof PhysicalHashAggregate - && context.getFirstAggregateInFragment(inputFragment) == distribute.child()) { - PhysicalHashAggregate hashAggregate = (PhysicalHashAggregate) distribute.child(); + && child instanceof PhysicalHashAggregate + && context.getFirstAggregateInFragment(inputFragment) == child) { + PhysicalHashAggregate hashAggregate = (PhysicalHashAggregate) child; if (hashAggregate.getAggPhase() == AggPhase.LOCAL && hashAggregate.getAggMode() == AggMode.INPUT_TO_BUFFER) { AggregationNode aggregationNode = (AggregationNode) inputFragment.getPlanRoot(); @@ -285,23 +286,26 @@ public PlanFragment visitPhysicalDistribute(PhysicalDistribute d ExchangeNode exchangeNode = new ExchangeNode(context.nextPlanNodeId(), inputFragment.getPlanRoot()); updateLegacyPlanIdToPhysicalPlan(exchangeNode, distribute); List validOutputIds = distribute.getOutputExprIds(); - if (distribute.child() instanceof PhysicalHashAggregate) { + if (child instanceof PhysicalHashAggregate) { // we must add group by keys to output list, // otherwise we could not process aggregate's output without group by keys - List keys = ((PhysicalHashAggregate) distribute.child()).getGroupByExpressions().stream() + List keys = ((PhysicalHashAggregate) child).getGroupByExpressions().stream() .filter(SlotReference.class::isInstance) .map(SlotReference.class::cast) .map(SlotReference::getExprId) .collect(Collectors.toList()); keys.addAll(validOutputIds); validOutputIds = keys; + } else if (child instanceof PhysicalLimit && ((PhysicalLimit) child).getPhase().isGlobal()) { + // because sort already contains Offset, we don't need to handle PhysicalTopN + exchangeNode.setOffset(((PhysicalLimit) child).getOffset()); } if (inputFragment instanceof MultiCastPlanFragment) { // TODO: remove this logic when we split to multi-window in logical window to physical window conversion MultiCastDataSink multiCastDataSink = (MultiCastDataSink) inputFragment.getSink(); DataStreamSink dataStreamSink = multiCastDataSink.getDataStreamSinks().get( multiCastDataSink.getDataStreamSinks().size() - 1); - if (!(distribute.child() instanceof PhysicalProject)) { + if (!(child instanceof PhysicalProject)) { List projectionExprs = new ArrayList<>(); PhysicalCTEConsumer consumer = getCTEConsumerChild(distribute); Preconditions.checkState(consumer != null, "consumer not found"); @@ -1591,7 +1595,8 @@ public PlanFragment visitPhysicalLimit(PhysicalLimit physicalLim PlanFragment inputFragment = physicalLimit.child(0).accept(this, context); PlanNode child = inputFragment.getPlanRoot(); child.setLimit(MergeLimits.mergeLimit(physicalLimit.getLimit(), physicalLimit.getOffset(), child.getLimit())); - child.setOffset(MergeLimits.mergeOffset(physicalLimit.getOffset(), child.getOffset())); + // TODO: plan node don't support limit + // child.setOffset(MergeLimits.mergeOffset(physicalLimit.getOffset(), child.getOffset())); updateLegacyPlanIdToPhysicalPlan(child, physicalLimit); return inputFragment; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/AddOffsetIntoDistribute.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/AddOffsetIntoDistribute.java new file mode 100644 index 00000000000000..deac398869833b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/AddOffsetIntoDistribute.java @@ -0,0 +1,41 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.nereids.processor.post; + +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.properties.DistributionSpecGather; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribute; +import org.apache.doris.nereids.trees.plans.physical.PhysicalLimit; + +/** + * Offset just can be in exchangeNode. + * So, `offset` action is after `limit` action. + * So, `limit` should update with `offset + limit` + */ +public class AddOffsetIntoDistribute extends PlanPostProcessor { + @Override + public Plan visitPhysicalLimit(PhysicalLimit limit, CascadesContext context) { + if (limit.getPhase().isLocal() || limit.getOffset() == 0) { + return limit; + } + + return new PhysicalDistribute<>(DistributionSpecGather.INSTANCE, + limit.withLimit(limit.getLimit() + limit.getOffset())); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/PlanPostProcessors.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/PlanPostProcessors.java index 17538d55d45c8c..6d85bebb2d9828 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/PlanPostProcessors.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/PlanPostProcessors.java @@ -61,6 +61,7 @@ public List getProcessors() { builder.add(new PushDownFilterThroughProject()); builder.add(new MergeProjectPostProcessor()); builder.add(new RecomputeLogicalPropertiesProcessor()); + builder.add(new AddOffsetIntoDistribute()); builder.add(new TopNScanOpt()); // after generate rf, DO NOT replace PLAN NODE builder.add(new FragmentProcessor()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalLimit.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalLimit.java index ab7f5f811eb8b9..bbc2143df61bfb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalLimit.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalLimit.java @@ -101,6 +101,11 @@ public boolean isGlobal() { return phase == LimitPhase.GLOBAL; } + public Plan withLimit(long limit) { + return new PhysicalLimit<>(limit, offset, phase, groupExpression, getLogicalProperties(), + physicalProperties, statistics, children.get(0)); + } + @Override public Plan withChildren(List children) { Preconditions.checkArgument(children.size() == 1); From 71c4a780457fc045aaa136997d07ea8ce0e655c0 Mon Sep 17 00:00:00 2001 From: yujun Date: Mon, 15 Jan 2024 17:35:22 +0800 Subject: [PATCH 006/200] (selectdb-cloud) Improve create dynamic partitions with auto bucket (#2936) (#29964) Cloud partition's get visible version is a rpc. When creating a new table, if create n dynamic partitions with autobucket, it need to invoke partition.getVisibleVersion n * (n -1) / 2 times, rather a lot. improve this. --- .../clone/DynamicPartitionScheduler.java | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java index 565f4c066f8291..d6b75d9c30e726 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java @@ -103,7 +103,7 @@ public DynamicPartitionScheduler(String name, long intervalMs) { public void executeDynamicPartitionFirstTime(Long dbId, Long tableId) { List> tempDynamicPartitionTableInfo = Lists.newArrayList(Pair.of(dbId, tableId)); - executeDynamicPartition(tempDynamicPartitionTableInfo); + executeDynamicPartition(tempDynamicPartitionTableInfo, true); } public void registerDynamicPartitionTable(Long dbId, Long tableId) { @@ -185,8 +185,9 @@ private static long getNextPartitionSize(ArrayList historyPartitionsSize) } } - private static int getBucketsNum(DynamicPartitionProperty property, OlapTable table) { - if (!table.isAutoBucket()) { + private static int getBucketsNum(DynamicPartitionProperty property, OlapTable table, boolean executeFirstTime) { + // if execute first time, all partitions no contain data + if (!table.isAutoBucket() || executeFirstTime) { return property.getBuckets(); } @@ -226,7 +227,7 @@ private static int getBucketsNum(DynamicPartitionProperty property, OlapTable ta } private ArrayList getAddPartitionClause(Database db, OlapTable olapTable, - Column partitionColumn, String partitionFormat) { + Column partitionColumn, String partitionFormat, boolean executeFirstTime) { ArrayList addPartitionClauses = new ArrayList<>(); DynamicPartitionProperty dynamicPartitionProperty = olapTable.getTableProperty().getDynamicPartitionProperty(); RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) olapTable.getPartitionInfo(); @@ -322,7 +323,7 @@ private ArrayList getAddPartitionClause(Database db, OlapTab DistributionDesc distributionDesc = null; DistributionInfo distributionInfo = olapTable.getDefaultDistributionInfo(); - int bucketsNum = getBucketsNum(dynamicPartitionProperty, olapTable); + int bucketsNum = getBucketsNum(dynamicPartitionProperty, olapTable, executeFirstTime); if (distributionInfo.getType() == DistributionInfo.DistributionInfoType.HASH) { HashDistributionInfo hashDistributionInfo = (HashDistributionInfo) distributionInfo; List distColumnNames = new ArrayList<>(); @@ -491,7 +492,8 @@ private ArrayList getDropPartitionClause(Database db, OlapT return dropPartitionClauses; } - private void executeDynamicPartition(Collection> dynamicPartitionTableInfoCol) { + private void executeDynamicPartition(Collection> dynamicPartitionTableInfoCol, + boolean executeFirstTime) { Iterator> iterator = dynamicPartitionTableInfoCol.iterator(); while (iterator.hasNext()) { Pair tableInfo = iterator.next(); @@ -550,7 +552,8 @@ private void executeDynamicPartition(Collection> dynamicPartiti } if (!skipAddPartition) { - addPartitionClauses = getAddPartitionClause(db, olapTable, partitionColumn, partitionFormat); + addPartitionClauses = getAddPartitionClause(db, olapTable, partitionColumn, partitionFormat, + executeFirstTime); } dropPartitionClauses = getDropPartitionClause(db, olapTable, partitionColumn, partitionFormat); tableName = olapTable.getName(); @@ -638,7 +641,7 @@ protected void runAfterCatalogReady() { } setInterval(Config.dynamic_partition_check_interval_seconds * 1000L); if (Config.dynamic_partition_enable) { - executeDynamicPartition(dynamicPartitionTableInfo); + executeDynamicPartition(dynamicPartitionTableInfo, false); } } } From 2e8f0204c3e49fd40386fa7e2b56d06f3a2d416a Mon Sep 17 00:00:00 2001 From: yiguolei <676222867@qq.com> Date: Mon, 15 Jan 2024 17:48:10 +0800 Subject: [PATCH 007/200] [bugfix](core) writer status is read and write concurrently and will core by use after free (#29982) Co-authored-by: yiguolei --- be/src/vec/sink/writer/async_result_writer.cpp | 6 +++++- be/src/vec/sink/writer/async_result_writer.h | 5 ++++- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/be/src/vec/sink/writer/async_result_writer.cpp b/be/src/vec/sink/writer/async_result_writer.cpp index 93b86ca843e410..0ce3196e3a3cad 100644 --- a/be/src/vec/sink/writer/async_result_writer.cpp +++ b/be/src/vec/sink/writer/async_result_writer.cpp @@ -51,13 +51,13 @@ Status AsyncResultWriter::sink(Block* block, bool eos) { add_block = _get_free_block(block, rows); } + std::lock_guard l(_m); // if io task failed, just return error status to // end the query if (!_writer_status.ok()) { return _writer_status; } - std::lock_guard l(_m); _eos = eos; if (_dependency && _is_finished()) { _dependency->set_ready(); @@ -128,6 +128,10 @@ void AsyncResultWriter::process_block(RuntimeState* state, RuntimeProfile* profi // if not in transaction or status is in error or force close we can do close in // async IO thread if (!_writer_status.ok() || !in_transaction()) { + std::lock_guard l(_m); + // Using lock to make sure the writer status is not modified + // There is a unique ptr err_msg in Status, if it is modified, the unique ptr + // maybe released. And it will core because use after free. _writer_status = close(_writer_status); _need_normal_close = false; } diff --git a/be/src/vec/sink/writer/async_result_writer.h b/be/src/vec/sink/writer/async_result_writer.h index d48e41daa624d6..ced123267c5292 100644 --- a/be/src/vec/sink/writer/async_result_writer.h +++ b/be/src/vec/sink/writer/async_result_writer.h @@ -84,7 +84,10 @@ class AsyncResultWriter : public ResultWriter { // Add the IO thread task process block() to thread pool to dispose the IO void start_writer(RuntimeState* state, RuntimeProfile* profile); - Status get_writer_status() { return _writer_status; } + Status get_writer_status() { + std::lock_guard l(_m); + return _writer_status; + } protected: Status _projection_block(Block& input_block, Block* output_block); From 45e69cd6dae76d065779f267633883b4a2813ed0 Mon Sep 17 00:00:00 2001 From: yangshijie Date: Mon, 15 Jan 2024 19:02:00 +0800 Subject: [PATCH 008/200] [feature](function) support ip function named ipv6_cidr_to_range(addr, cidr) (#29812) --- be/src/vec/common/ipv6_to_binary.h | 69 +++++++++ be/src/vec/data_types/data_type.h | 70 +++++---- be/src/vec/functions/function_ip.cpp | 1 + be/src/vec/functions/function_ip.h | 137 ++++++++++++++++++ .../doris/catalog/BuiltinScalarFunctions.java | 2 + .../functions/scalar/IPv6CIDRToRange.java | 79 ++++++++++ .../visitor/ScalarFunctionVisitor.java | 5 + gensrc/script/doris_builtins_functions.py | 2 + .../test_ipv6_cidr_to_range_function.out | 32 ++++ .../test_ipv6_cidr_to_range_function.groovy | 85 +++++++++++ 10 files changed, 446 insertions(+), 36 deletions(-) create mode 100644 be/src/vec/common/ipv6_to_binary.h create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IPv6CIDRToRange.java create mode 100644 regression-test/data/query_p0/sql_functions/ip_functions/test_ipv6_cidr_to_range_function.out create mode 100644 regression-test/suites/query_p0/sql_functions/ip_functions/test_ipv6_cidr_to_range_function.groovy diff --git a/be/src/vec/common/ipv6_to_binary.h b/be/src/vec/common/ipv6_to_binary.h new file mode 100644 index 00000000000000..0f22740fbcd57f --- /dev/null +++ b/be/src/vec/common/ipv6_to_binary.h @@ -0,0 +1,69 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/IPv6ToBinary.cpp +// and modified by Doris + +#pragma once + +#include "vec/common/format_ip.h" + +namespace doris::vectorized { + +/// Result array could be indexed with all possible uint8 values without extra check. +/// For values greater than 128 we will store same value as for 128 (all bits set). +constexpr size_t IPV6_MASKS_COUNT = 256; +using RawMaskArrayV6 = std::array; + +template +static constexpr RawMaskArrayT generate_bit_mask(size_t prefix) { + RawMaskArrayT arr {0}; + if (prefix >= arr.size() * 8) { + prefix = arr.size() * 8; + } + size_t i = 0; + for (; prefix >= 8; ++i, prefix -= 8) { + arr[i] = 0xff; + } + if (prefix > 0) { + arr[i++] = ~(0xff >> prefix); + } + while (i < arr.size()) { + arr[i++] = 0x00; + } + return arr; +} + +template +static constexpr std::array generate_bit_masks() { + std::array arr {}; + for (size_t i = 0; i < masksCount; ++i) { + arr[i] = generate_bit_mask(i); + } + return arr; +} + +/// Returns a reference to 16-byte array containing mask with first `prefix_len` bits set to `1` and `128 - prefix_len` to `0`. +/// The reference is valid during all program execution time. +/// Values of prefix_len greater than 128 interpreted as 128 exactly. +const std::array& get_cidr_mask_ipv6(uint8_t prefix_len) { + static constexpr auto IPV6_RAW_MASK_ARRAY = + generate_bit_masks(); + return IPV6_RAW_MASK_ARRAY[prefix_len]; +} + +} // namespace doris::vectorized diff --git a/be/src/vec/data_types/data_type.h b/be/src/vec/data_types/data_type.h index 8fa41f88886a18..bc53fac6a373b2 100644 --- a/be/src/vec/data_types/data_type.h +++ b/be/src/vec/data_types/data_type.h @@ -285,6 +285,10 @@ struct WhichDataType { bool is_date_or_datetime() const { return is_date() || is_date_time(); } bool is_date_v2_or_datetime_v2() const { return is_date_v2() || is_date_time_v2(); } + bool is_ipv4() const { return idx == TypeIndex::IPv4; } + bool is_ipv6() const { return idx == TypeIndex::IPv6; } + bool is_ip() const { return is_ipv4() || is_ipv6(); } + bool is_string() const { return idx == TypeIndex::String; } bool is_fixed_string() const { return idx == TypeIndex::FixedString; } bool is_string_or_fixed_string() const { return is_string() || is_fixed_string(); } @@ -307,42 +311,36 @@ struct WhichDataType { /// IDataType helpers (alternative for IDataType virtual methods with single point of truth) -inline bool is_date(const DataTypePtr& data_type) { - return WhichDataType(data_type).is_date(); -} -inline bool is_date_v2(const DataTypePtr& data_type) { - return WhichDataType(data_type).is_date_v2(); -} -inline bool is_date_time_v2(const DataTypePtr& data_type) { - return WhichDataType(data_type).is_date_time_v2(); -} -inline bool is_date_or_datetime(const DataTypePtr& data_type) { - return WhichDataType(data_type).is_date_or_datetime(); -} -inline bool is_date_v2_or_datetime_v2(const DataTypePtr& data_type) { - return WhichDataType(data_type).is_date_v2_or_datetime_v2(); -} -inline bool is_decimal(const DataTypePtr& data_type) { - return WhichDataType(data_type).is_decimal(); -} -inline bool is_decimal_v2(const DataTypePtr& data_type) { - return WhichDataType(data_type).is_decimal128v2(); -} -inline bool is_tuple(const DataTypePtr& data_type) { - return WhichDataType(data_type).is_tuple(); -} -inline bool is_array(const DataTypePtr& data_type) { - return WhichDataType(data_type).is_array(); -} -inline bool is_map(const DataTypePtr& data_type) { - return WhichDataType(data_type).is_map(); -} -inline bool is_struct(const DataTypePtr& data_type) { - return WhichDataType(data_type).is_struct(); -} -inline bool is_nothing(const DataTypePtr& data_type) { - return WhichDataType(data_type).is_nothing(); -} +#define IS_DATATYPE(name, method) \ + inline bool is_##name(const DataTypePtr& data_type) { \ + return WhichDataType(data_type).is_##method(); \ + } + +IS_DATATYPE(uint8, uint8) +IS_DATATYPE(uint16, uint16) +IS_DATATYPE(uint32, uint32) +IS_DATATYPE(uint64, uint64) +IS_DATATYPE(uint128, uint128) +IS_DATATYPE(int8, int8) +IS_DATATYPE(int16, int16) +IS_DATATYPE(int32, int32) +IS_DATATYPE(int64, int64) +IS_DATATYPE(int128, int128) +IS_DATATYPE(date, date) +IS_DATATYPE(date_v2, date_v2) +IS_DATATYPE(date_time_v2, date_time_v2) +IS_DATATYPE(date_or_datetime, date_or_datetime) +IS_DATATYPE(date_v2_or_datetime_v2, date_v2_or_datetime_v2) +IS_DATATYPE(decimal, decimal) +IS_DATATYPE(decimal_v2, decimal128v2) +IS_DATATYPE(tuple, tuple) +IS_DATATYPE(array, array) +IS_DATATYPE(map, map) +IS_DATATYPE(struct, struct) +IS_DATATYPE(ipv4, ipv4) +IS_DATATYPE(ipv6, ipv6) +IS_DATATYPE(ip, ip) +IS_DATATYPE(nothing, nothing) template bool is_uint8(const T& data_type) { diff --git a/be/src/vec/functions/function_ip.cpp b/be/src/vec/functions/function_ip.cpp index fb0c346e6a6592..dce648bfaaf518 100644 --- a/be/src/vec/functions/function_ip.cpp +++ b/be/src/vec/functions/function_ip.cpp @@ -37,5 +37,6 @@ void register_function_ip(SimpleFunctionFactory& factory) { factory.register_function(); factory.register_function(); factory.register_function(); + factory.register_function(); } } // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/functions/function_ip.h b/be/src/vec/functions/function_ip.h index 18c2033ff8dfd3..2997153049a6b0 100644 --- a/be/src/vec/functions/function_ip.h +++ b/be/src/vec/functions/function_ip.h @@ -23,12 +23,15 @@ #include "vec/columns/column.h" #include "vec/columns/column_string.h" +#include "vec/columns/column_struct.h" #include "vec/columns/column_vector.h" #include "vec/common/format_ip.h" +#include "vec/common/ipv6_to_binary.h" #include "vec/core/column_with_type_and_name.h" #include "vec/data_types/data_type_ipv6.h" #include "vec/data_types/data_type_number.h" #include "vec/data_types/data_type_string.h" +#include "vec/data_types/data_type_struct.h" #include "vec/functions/function.h" #include "vec/functions/function_helpers.h" #include "vec/functions/simple_function_factory.h" @@ -832,4 +835,138 @@ class FunctionIsIPAddressInRange : public IFunction { } }; +class FunctionIPv6CIDRToRange : public IFunction { +public: + static constexpr auto name = "ipv6_cidr_to_range"; + static FunctionPtr create() { return std::make_shared(); } + + String get_name() const override { return name; } + + size_t get_number_of_arguments() const override { return 2; } + + DataTypePtr get_return_type_impl(const DataTypes& arguments) const override { + const auto& ipv6_type = arguments[0]; + if (!is_string(remove_nullable(ipv6_type)) && !is_ipv6(remove_nullable(ipv6_type))) { + throw Exception( + ErrorCode::INVALID_ARGUMENT, + "Illegal type {} of first argument of function {}, expected String or IPv6", + ipv6_type->get_name(), get_name()); + } + const auto& cidr_type = arguments[1]; + if (!is_int16(remove_nullable(cidr_type))) { + throw Exception(ErrorCode::INVALID_ARGUMENT, + "Illegal type {} of second argument of function {}, expected Int16", + cidr_type->get_name(), get_name()); + } + DataTypePtr element = std::make_shared(); + return make_nullable(std::make_shared(DataTypes {element, element}, + Strings {"min", "max"})); + } + + bool use_default_implementation_for_nulls() const override { return true; } + + Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, + size_t result, size_t input_rows_count) const override { + const auto& addr_column_with_type_and_name = block.get_by_position(arguments[0]); + const auto& cidr_column_with_type_and_name = block.get_by_position(arguments[1]); + WhichDataType addr_type(addr_column_with_type_and_name.type); + WhichDataType cidr_type(cidr_column_with_type_and_name.type); + const ColumnPtr& addr_column = addr_column_with_type_and_name.column; + const ColumnPtr& cidr_column = cidr_column_with_type_and_name.column; + const ColumnInt16* cidr_col = nullptr; + const NullMap* cidr_null_map = nullptr; + ColumnPtr col_res = nullptr; + + if (cidr_type.is_nullable()) { + const auto* cidr_column_nullable = + assert_cast(cidr_column.get()); + cidr_col = assert_cast(&cidr_column_nullable->get_nested_column()); + cidr_null_map = &cidr_column_nullable->get_null_map_data(); + } else { + cidr_col = assert_cast(cidr_column.get()); + } + + if (addr_type.is_nullable()) { + const auto* addr_column_nullable = + assert_cast(addr_column.get()); + const NullMap* addr_null_map = &addr_column_nullable->get_null_map_data(); + WhichDataType sub_addr_type(remove_nullable(addr_column_with_type_and_name.type)); + + if (sub_addr_type.is_ipv6()) { + const auto* ipv6_addr_column = + check_and_get_column(addr_column_nullable->get_nested_column()); + col_res = execute_impl(*ipv6_addr_column, addr_null_map, *cidr_col, + cidr_null_map, input_rows_count); + } else if (sub_addr_type.is_string()) { + const auto* str_addr_column = check_and_get_column( + addr_column_nullable->get_nested_column()); + col_res = execute_impl(*str_addr_column, addr_null_map, *cidr_col, + cidr_null_map, input_rows_count); + } else { + return Status::RuntimeError( + "Illegal column {} of argument of function {}, Expected IPv6 or String", + addr_column->get_name(), get_name()); + } + } else { + if (addr_type.is_ipv6()) { + const auto* ipv6_addr_column = check_and_get_column(addr_column.get()); + col_res = execute_impl(*ipv6_addr_column, nullptr, *cidr_col, nullptr, + input_rows_count); + } else if (addr_type.is_string()) { + const auto* str_addr_column = check_and_get_column(addr_column.get()); + col_res = execute_impl(*str_addr_column, nullptr, *cidr_col, nullptr, + input_rows_count); + } else { + return Status::RuntimeError( + "Illegal column {} of argument of function {}, Expected IPv6 or String", + addr_column->get_name(), get_name()); + } + } + + block.replace_by_position(result, std::move(col_res)); + return Status::OK(); + } + + template + static ColumnPtr execute_impl(const FromColumn& from_column, const NullMap* from_null_map, + const ColumnInt16& cidr_column, const NullMap* cidr_null_map, + size_t input_rows_count) { + auto col_res_lower_range = ColumnIPv6::create(input_rows_count, 0); + auto col_res_upper_range = ColumnIPv6::create(input_rows_count, 0); + auto col_res_null_map = ColumnUInt8::create(input_rows_count, 0); + auto& vec_res_lower_range = col_res_lower_range->get_data(); + auto& vec_res_upper_range = col_res_upper_range->get_data(); + auto& vec_res_null_map = col_res_null_map->get_data(); + + static constexpr Int64 max_cidr_mask = IPV6_BINARY_LENGTH * 8; + + for (size_t i = 0; i < input_rows_count; ++i) { + if ((from_null_map && (*from_null_map)[i]) || (cidr_null_map && (*cidr_null_map)[i])) { + vec_res_null_map[i] = 1; + continue; + } + Int64 cidr = cidr_column.get_int(i); + cidr = std::min(cidr, max_cidr_mask); + apply_cidr_mask(from_column.get_data_at(i).data, + reinterpret_cast(&vec_res_lower_range[i]), + reinterpret_cast(&vec_res_upper_range[i]), cidr); + } + + auto col_res_struct = ColumnStruct::create( + Columns {std::move(col_res_lower_range), std::move(col_res_upper_range)}); + return ColumnNullable::create(std::move(col_res_struct), std::move(col_res_null_map)); + } + +private: + static void apply_cidr_mask(const char* __restrict src, char* __restrict dst_lower, + char* __restrict dst_upper, UInt8 bits_to_keep) { + const auto& mask = get_cidr_mask_ipv6(bits_to_keep); + + for (size_t i = 0; i < IPV6_BINARY_LENGTH; ++i) { + dst_lower[i] = src[i] & mask[i]; + dst_upper[i] = dst_lower[i] | ~mask[i]; + } + } +}; + } // namespace doris::vectorized \ No newline at end of file diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java index c4c4043bf79301..903a77b4494e7b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java @@ -188,6 +188,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursAdd; import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursDiff; import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursSub; +import org.apache.doris.nereids.trees.expressions.functions.scalar.IPv6CIDRToRange; import org.apache.doris.nereids.trees.expressions.functions.scalar.If; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ignore; import org.apache.doris.nereids.trees.expressions.functions.scalar.Initcap; @@ -611,6 +612,7 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(IsIpv4String.class, "is_ipv4_string"), scalar(IsIpv6String.class, "is_ipv6_string"), scalar(IsIpAddressInRange.class, "is_ip_address_in_range"), + scalar(IPv6CIDRToRange.class, "ipv6_cidr_to_range"), scalar(JsonArray.class, "json_array"), scalar(JsonObject.class, "json_object"), scalar(JsonQuote.class, "json_quote"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IPv6CIDRToRange.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IPv6CIDRToRange.java new file mode 100644 index 00000000000000..2ada5b0e095e1e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IPv6CIDRToRange.java @@ -0,0 +1,79 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.IPv6Type; +import org.apache.doris.nereids.types.SmallIntType; +import org.apache.doris.nereids.types.StringType; +import org.apache.doris.nereids.types.StructField; +import org.apache.doris.nereids.types.StructType; +import org.apache.doris.nereids.types.VarcharType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * scalar function `ipv6_cidr_to_range` + */ +public class IPv6CIDRToRange extends ScalarFunction + implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable { + + public static final List SIGNATURES; + + static { + ImmutableList.Builder structFields = ImmutableList.builder(); + structFields.add(new StructField("min", IPv6Type.INSTANCE, false, "")); + structFields.add(new StructField("max", IPv6Type.INSTANCE, false, "")); + StructType retType = new StructType(structFields.build()); + SIGNATURES = ImmutableList.of( + FunctionSignature.ret(retType).args(IPv6Type.INSTANCE, SmallIntType.INSTANCE), + FunctionSignature.ret(retType).args(VarcharType.SYSTEM_DEFAULT, SmallIntType.INSTANCE), + FunctionSignature.ret(retType).args(StringType.INSTANCE, SmallIntType.INSTANCE)); + } + + public IPv6CIDRToRange(Expression arg0, Expression arg1) { + super("ipv6_cidr_to_range", arg0, arg1); + } + + @Override + public IPv6CIDRToRange withChildren(List children) { + Preconditions.checkArgument(children.size() == 2, + "ipv6_cidr_to_range accept 2 args, but got %s (%s)", + children.size(), + children); + return new IPv6CIDRToRange(children.get(0), children.get(1)); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitIpv6CIDRToRange(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java index 81ce549d582ead..6f43a45dd7990f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java @@ -184,6 +184,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursAdd; import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursDiff; import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursSub; +import org.apache.doris.nereids.trees.expressions.functions.scalar.IPv6CIDRToRange; import org.apache.doris.nereids.trees.expressions.functions.scalar.If; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ignore; import org.apache.doris.nereids.trees.expressions.functions.scalar.Initcap; @@ -1166,6 +1167,10 @@ default R visitIsIPAddressInRange(IsIpAddressInRange isIpAddressInRange, C conte return visitScalarFunction(isIpAddressInRange, context); } + default R visitIpv6CIDRToRange(IPv6CIDRToRange ipv6CIDRToRange, C context) { + return visitScalarFunction(ipv6CIDRToRange, context); + } + default R visitJsonArray(JsonArray jsonArray, C context) { return visitScalarFunction(jsonArray, context); } diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py index 302cefd806a6fa..4f4c2f37b8ee42 100644 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -2020,6 +2020,8 @@ [['is_ipv6_string'], 'BOOLEAN', ['STRING'], 'ALWAYS_NULLABLE'], [['is_ip_address_in_range'], 'BOOLEAN', ['VARCHAR', 'VARCHAR'], 'ALWAYS_NOT_NULLABLE'], [['is_ip_address_in_range'], 'BOOLEAN', ['STRING', 'STRING'], 'ALWAYS_NOT_NULLABLE'], + [['ipv6_cidr_to_range'], 'STRUCT', ['VARCHAR', 'SMALLINT'], 'ALWAYS_NULLABLE'], + [['ipv6_cidr_to_range'], 'STRUCT', ['STRING', 'SMALLINT'], 'ALWAYS_NULLABLE'], ], "NonNullalbe": [ diff --git a/regression-test/data/query_p0/sql_functions/ip_functions/test_ipv6_cidr_to_range_function.out b/regression-test/data/query_p0/sql_functions/ip_functions/test_ipv6_cidr_to_range_function.out new file mode 100644 index 00000000000000..411010046cc4c4 --- /dev/null +++ b/regression-test/data/query_p0/sql_functions/ip_functions/test_ipv6_cidr_to_range_function.out @@ -0,0 +1,32 @@ +-- !sql -- +1 :: ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff +2 2001:db8:: 2001:db8:ffff:ffff:ffff:ffff:ffff:ffff +3 ffff:: ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff +4 2001:df8:: 2001:df8:ffff:ffff:ffff:ffff:ffff:ffff +5 2001:: 2001:ffff:ffff:ffff:ffff:ffff:ffff:ffff +6 :: ff:ffff:ffff:ffff:ffff:ffff:ffff:ffff +7 f000:: ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff + +-- !sql -- +1 :: ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff +2 2001:db8:: 2001:db8:ffff:ffff:ffff:ffff:ffff:ffff +3 ffff:: ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff +4 2001:df8:: 2001:df8:ffff:ffff:ffff:ffff:ffff:ffff +5 2001:: 2001:ffff:ffff:ffff:ffff:ffff:ffff:ffff +6 :: ff:ffff:ffff:ffff:ffff:ffff:ffff:ffff +7 f000:: ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff + +-- !sql -- +{"min": "::", "max": "ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff"} + +-- !sql -- +{"min": "2001:db8:0:85a3::ac1f:8001", "max": "2001:db8:0:85a3::ac1f:8001"} + +-- !sql -- +{"min": "ffff:ffff:ffff:ffff::", "max": "ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff"} + +-- !sql -- +{"min": "::", "max": "ff:ffff:ffff:ffff:ffff:ffff:ffff:ffff"} + +-- !sql -- +{"min": "f000::", "max": "ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff"} diff --git a/regression-test/suites/query_p0/sql_functions/ip_functions/test_ipv6_cidr_to_range_function.groovy b/regression-test/suites/query_p0/sql_functions/ip_functions/test_ipv6_cidr_to_range_function.groovy new file mode 100644 index 00000000000000..f606460377a463 --- /dev/null +++ b/regression-test/suites/query_p0/sql_functions/ip_functions/test_ipv6_cidr_to_range_function.groovy @@ -0,0 +1,85 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +suite("test_ipv6_cidr_to_range_function") { + sql """ DROP TABLE IF EXISTS test_ipv6_cidr_to_range_function """ + + sql """ SET enable_nereids_planner=true """ + sql """ SET enable_fallback_to_original_planner=false """ + + sql """ + CREATE TABLE test_ipv6_cidr_to_range_function ( + `id` int, + `addr` ipv6, + `cidr` int + ) ENGINE=OLAP + UNIQUE KEY (`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql """ + insert into test_ipv6_cidr_to_range_function values + (1, '2001:0db8:0000:85a3:0000:0000:ac1f:8001', 0), + (2, '2001:0db8:0000:85a3:ffff:ffff:ffff:ffff', 32), + (3, 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff', 16), + (4, '2001:df8:0:85a3::ac1f:8001', 32), + (5, '2001:0db8:85a3:85a3:0000:0000:ac1f:8001', 16), + (6, '0000:0000:0000:0000:0000:0000:0000:0000', 8), + (7, 'ffff:0000:0000:0000:0000:0000:0000:0000', 4) + """ + + qt_sql "select id, struct_element(ipv6_cidr_to_range(addr, cidr), 'min') as min_range, struct_element(ipv6_cidr_to_range(addr, cidr), 'max') as max_range from test_ipv6_cidr_to_range_function order by id" + + sql """ DROP TABLE IF EXISTS test_ipv6_cidr_to_range_function """ + + sql """ + CREATE TABLE test_str_cidr_to_range_function ( + `id` int, + `addr` string, + `cidr` int + ) ENGINE=OLAP + UNIQUE KEY (`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql """ + insert into test_str_cidr_to_range_function values + (1, '2001:0db8:0000:85a3:0000:0000:ac1f:8001', 0), + (2, '2001:0db8:0000:85a3:ffff:ffff:ffff:ffff', 32), + (3, 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff', 16), + (4, '2001:df8:0:85a3::ac1f:8001', 32), + (5, '2001:0db8:85a3:85a3:0000:0000:ac1f:8001', 16), + (6, '0000:0000:0000:0000:0000:0000:0000:0000', 8), + (7, 'ffff:0000:0000:0000:0000:0000:0000:0000', 4) + """ + + qt_sql "select id, struct_element(ipv6_cidr_to_range(ipv6_string_to_num(addr), cidr), 'min') as min_range, struct_element(ipv6_cidr_to_range(ipv6_string_to_num(addr), cidr), 'max') as max_range from test_str_cidr_to_range_function order by id" + + sql """ DROP TABLE IF EXISTS test_str_cidr_to_range_function """ + + qt_sql "select ipv6_cidr_to_range(ipv6_string_to_num('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 0)" + qt_sql "select ipv6_cidr_to_range(ipv6_string_to_num('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 128)" + qt_sql "select ipv6_cidr_to_range(ipv6_string_to_num('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'), 64)" + qt_sql "select ipv6_cidr_to_range(ipv6_string_to_num('0000:0000:0000:0000:0000:0000:0000:0000'), 8)" + qt_sql "select ipv6_cidr_to_range(ipv6_string_to_num('ffff:0000:0000:0000:0000:0000:0000:0000'), 4)" +} \ No newline at end of file From a4c8601cf06fa57d02f58b2f26fa141e2e72b3fb Mon Sep 17 00:00:00 2001 From: Dongyang Li Date: Mon, 15 Jan 2024 19:54:17 +0800 Subject: [PATCH 009/200] [chore](ci) replace clickbench pipeline to performance pipeline (#29929) --- .asf.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.asf.yaml b/.asf.yaml index 3c109b1729b16b..f85a3ec6d73eca 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -60,7 +60,7 @@ github: - Build Broker - Build Documents - ShellCheck - - clickbench-new (clickbench) + - performance (Performance New) - Build Third Party Libraries (Linux) - Build Third Party Libraries (macOS) - COMPILE (DORIS_COMPILE) @@ -96,7 +96,7 @@ github: - Build Broker - ShellCheck - Build Documents - - clickbench-new (clickbench) + - performance (Performance New) - Build Third Party Libraries (Linux) - Build Third Party Libraries (macOS) - COMPILE (DORIS_COMPILE) From d9803415721c67acf67b555d82d6078e6e578aa0 Mon Sep 17 00:00:00 2001 From: Dongyang Li Date: Mon, 15 Jan 2024 21:45:53 +0800 Subject: [PATCH 010/200] [fix](ci) Update .asf.yaml (#30004) --- .asf.yaml | 2 -- 1 file changed, 2 deletions(-) diff --git a/.asf.yaml b/.asf.yaml index f85a3ec6d73eca..cc73010384354f 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -60,7 +60,6 @@ github: - Build Broker - Build Documents - ShellCheck - - performance (Performance New) - Build Third Party Libraries (Linux) - Build Third Party Libraries (macOS) - COMPILE (DORIS_COMPILE) @@ -96,7 +95,6 @@ github: - Build Broker - ShellCheck - Build Documents - - performance (Performance New) - Build Third Party Libraries (Linux) - Build Third Party Libraries (macOS) - COMPILE (DORIS_COMPILE) From 30b41528955035102ab4a6b88597287700c54a66 Mon Sep 17 00:00:00 2001 From: Gabriel Date: Mon, 15 Jan 2024 21:46:31 +0800 Subject: [PATCH 011/200] [pipelineX](fix) Fix input data distribution for distinct streaming agg (#29980) --- ...inct_streaming_aggregation_sink_operator.h | 5 ++++ .../local_exchange_sink_operator.h | 24 ++++++++++-------- .../local_exchange/local_exchanger.cpp | 25 ++++++++++--------- be/src/pipeline/pipeline_x/operator.h | 3 ++- .../pipeline_x_fragment_context.cpp | 5 ++-- .../pipeline_x/pipeline_x_fragment_context.h | 1 + 6 files changed, 37 insertions(+), 26 deletions(-) diff --git a/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.h b/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.h index 6607516d6cb2a8..900aa7866348d4 100644 --- a/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.h +++ b/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.h @@ -112,6 +112,11 @@ class DistinctStreamingAggSinkOperatorX final SourceState source_state) override; DataDistribution required_data_distribution() const override { + if (_needs_finalize) { + return _is_colocate + ? DataDistribution(ExchangeType::BUCKET_HASH_SHUFFLE, _partition_exprs) + : DataDistribution(ExchangeType::HASH_SHUFFLE, _partition_exprs); + } return DataSinkOperatorX::required_data_distribution(); } }; diff --git a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h index 7275e5452056d4..7ed7d7d0e90357 100644 --- a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h +++ b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h @@ -86,13 +86,11 @@ class LocalExchangeSinkOperatorX final : public DataSinkOperatorX; LocalExchangeSinkOperatorX(int sink_id, int dest_id, int num_partitions, const std::vector& texprs, - const std::map& bucket_seq_to_instance_idx, - const std::map& shuffle_idx_to_instance_idx) + const std::map& bucket_seq_to_instance_idx) : Base(sink_id, dest_id, dest_id), _num_partitions(num_partitions), _texprs(texprs), - _bucket_seq_to_instance_idx(bucket_seq_to_instance_idx), - _shuffle_idx_to_instance_idx(shuffle_idx_to_instance_idx) {} + _bucket_seq_to_instance_idx(bucket_seq_to_instance_idx) {} Status init(const TPlanNode& tnode, RuntimeState* state) override { return Status::InternalError("{} should not init with TPlanNode", Base::_name); @@ -102,8 +100,8 @@ class LocalExchangeSinkOperatorX final : public DataSinkOperatorX& shuffle_idx_to_instance_idx) override { _name = "LOCAL_EXCHANGE_SINK_OPERATOR (" + get_exchange_type_name(type) + ")"; _type = type; if (_type == ExchangeType::HASH_SHUFFLE) { @@ -111,10 +109,16 @@ class LocalExchangeSinkOperatorX final : public DataSinkOperatorX& _texprs; std::unique_ptr _partitioner; const std::map _bucket_seq_to_instance_idx; - std::map _shuffle_idx_to_instance_idx; + std::vector> _shuffle_idx_to_instance_idx; }; } // namespace doris::pipeline diff --git a/be/src/pipeline/pipeline_x/local_exchange/local_exchanger.cpp b/be/src/pipeline/pipeline_x/local_exchange/local_exchanger.cpp index 900e31e6631903..1f680fdd1d910f 100644 --- a/be/src/pipeline/pipeline_x/local_exchange/local_exchanger.cpp +++ b/be/src/pipeline/pipeline_x/local_exchange/local_exchanger.cpp @@ -116,26 +116,26 @@ Status ShuffleExchanger::_split_rows(RuntimeState* state, const uint32_t* __rest return Status::OK(); } local_state._shared_state->add_total_mem_usage(new_block_wrapper->data_block.allocated_bytes()); - new_block_wrapper->ref(_num_partitions); if (get_type() == ExchangeType::HASH_SHUFFLE) { - auto map = local_state._parent->cast() - ._shuffle_idx_to_instance_idx; - for (size_t i = 0; i < _num_partitions; i++) { - DCHECK(map.contains(i)) << " i: " << i << " _num_partitions: " << _num_partitions - << " map.size(): " << map.size(); - DCHECK(map[i] >= 0 && map[i] < _num_partitions) << map[i] << " " << _num_partitions; - size_t start = local_state._partition_rows_histogram[i]; - size_t size = local_state._partition_rows_histogram[i + 1] - start; + const auto& map = local_state._parent->cast() + ._shuffle_idx_to_instance_idx; + new_block_wrapper->ref(map.size()); + for (const auto& it : map) { + DCHECK(it.second >= 0 && it.second < _num_partitions) + << it.first << " : " << it.second << " " << _num_partitions; + size_t start = local_state._partition_rows_histogram[it.first]; + size_t size = local_state._partition_rows_histogram[it.first + 1] - start; if (size > 0) { local_state._shared_state->add_mem_usage( - map[i], new_block_wrapper->data_block.allocated_bytes(), false); - data_queue[map[i]].enqueue({new_block_wrapper, {row_idx, start, size}}); - local_state._shared_state->set_ready_to_read(map[i]); + it.second, new_block_wrapper->data_block.allocated_bytes(), false); + data_queue[it.second].enqueue({new_block_wrapper, {row_idx, start, size}}); + local_state._shared_state->set_ready_to_read(it.second); } else { new_block_wrapper->unref(local_state._shared_state); } } } else if (_num_senders != _num_sources || _ignore_source_data_distribution) { + new_block_wrapper->ref(_num_partitions); for (size_t i = 0; i < _num_partitions; i++) { size_t start = local_state._partition_rows_histogram[i]; size_t size = local_state._partition_rows_histogram[i + 1] - start; @@ -149,6 +149,7 @@ Status ShuffleExchanger::_split_rows(RuntimeState* state, const uint32_t* __rest } } } else { + new_block_wrapper->ref(_num_partitions); auto map = local_state._parent->cast()._bucket_seq_to_instance_idx; for (size_t i = 0; i < _num_partitions; i++) { diff --git a/be/src/pipeline/pipeline_x/operator.h b/be/src/pipeline/pipeline_x/operator.h index 6792ce35f36878..d46dc859b0c162 100644 --- a/be/src/pipeline/pipeline_x/operator.h +++ b/be/src/pipeline/pipeline_x/operator.h @@ -470,7 +470,8 @@ class DataSinkOperatorXBase : public OperatorBase { Status init(const TDataSink& tsink) override; [[nodiscard]] virtual Status init(ExchangeType type, const int num_buckets, - const bool is_shuffled_hash_join) { + const bool is_shuffled_hash_join, + const std::map& shuffle_idx_to_instance_idx) { return Status::InternalError("init() is only implemented in local exchange!"); } diff --git a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp index e2f1d9742b49b0..a44db667450de9 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp +++ b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp @@ -733,11 +733,10 @@ Status PipelineXFragmentContext::_add_local_exchange_impl( : cur_pipe->sink_x()->is_shuffled_hash_join(); sink.reset(new LocalExchangeSinkOperatorX( sink_id, local_exchange_id, is_shuffled_hash_join ? _total_instances : _num_instances, - data_distribution.partition_exprs, bucket_seq_to_instance_idx, - shuffle_idx_to_instance_idx)); + data_distribution.partition_exprs, bucket_seq_to_instance_idx)); RETURN_IF_ERROR(new_pip->set_sink(sink)); RETURN_IF_ERROR(new_pip->sink_x()->init(data_distribution.distribution_type, num_buckets, - is_shuffled_hash_join)); + is_shuffled_hash_join, shuffle_idx_to_instance_idx)); // 2. Create and initialize LocalExchangeSharedState. auto shared_state = LocalExchangeSharedState::create_shared(); diff --git a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.h b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.h index 92178d359d9118..439b0072d72137 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.h +++ b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.h @@ -244,6 +244,7 @@ class PipelineXFragmentContext : public PipelineFragmentContext { std::vector> _runtime_filter_states; + // Total instance num running on all BEs int _total_instances = -1; }; From 3c403032b808af1d3ca3dc1ab3afd517d40ff9a3 Mon Sep 17 00:00:00 2001 From: Lei Zhang <27994433+SWJTU-ZhangLei@users.noreply.github.com> Date: Mon, 15 Jan 2024 21:48:32 +0800 Subject: [PATCH 012/200] [test](regression) Add debug level log of editlog for p0 p1 (#29992) --- .../src/main/java/org/apache/doris/persist/EditLog.java | 4 ++-- regression-test/pipeline/p0/conf/fe.conf | 2 ++ regression-test/pipeline/p1/conf/fe.conf | 2 ++ 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index de2d5e92ae316d..6eae5a688647ba 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -1230,8 +1230,8 @@ private synchronized long logEdit(short op, Writable writable) { } if (LOG.isDebugEnabled()) { - LOG.debug("nextId = {}, numTransactions = {}, totalTimeTransactions = {}, op = {}", txId, numTransactions, - totalTimeTransactions, op); + LOG.debug("nextId = {}, numTransactions = {}, totalTimeTransactions = {}, op = {} delta = {}", + txId, numTransactions, totalTimeTransactions, op, end - start); } if (txId >= Config.edit_log_roll_num) { diff --git a/regression-test/pipeline/p0/conf/fe.conf b/regression-test/pipeline/p0/conf/fe.conf index 991340a431e52b..0dea0f0286db40 100644 --- a/regression-test/pipeline/p0/conf/fe.conf +++ b/regression-test/pipeline/p0/conf/fe.conf @@ -107,3 +107,5 @@ enable_job_schedule_second_for_test = true enable_workload_group = true +sys_log_verbose_modules = org.apache.doris.persist.EditLog,org.apache.doris.journal.bdbje.BDBJEJournal + diff --git a/regression-test/pipeline/p1/conf/fe.conf b/regression-test/pipeline/p1/conf/fe.conf index 021760a3c6dd37..fe80f75954d34d 100644 --- a/regression-test/pipeline/p1/conf/fe.conf +++ b/regression-test/pipeline/p1/conf/fe.conf @@ -87,3 +87,5 @@ dynamic_partition_check_interval_seconds=3 enable_feature_binlog=true auth_token = 5ff161c3-2c08-4079-b108-26c8850b6598 + +sys_log_verbose_modules = org.apache.doris.persist.EditLog,org.apache.doris.journal.bdbje.BDBJEJournal From 333352a40511a4ff0427563b390fdc498e55a281 Mon Sep 17 00:00:00 2001 From: walter Date: Mon, 15 Jan 2024 21:54:12 +0800 Subject: [PATCH 013/200] [feature](merge-cloud) Add cloud meta manager rpc implementation (#29965) Co-authored-by: plat1ko Co-authored-by: Gavin Chou Co-authored-by: Xin Liao Co-authored-by: Xiaocc <598887962@qq.com> Co-authored-by: deardeng <565620795@qq.com> Co-authored-by: Lei Zhang <27994433+SWJTU-ZhangLei@users.noreply.github.com> Co-authored-by: Lightman <31928846+Lchangliang@users.noreply.github.com> Co-authored-by: Luwei <814383175@qq.com> Co-authored-by: Yongqiang YANG Co-authored-by: YueW <45946325+Tanya-W@users.noreply.github.com> Co-authored-by: bobhan1 --- be/src/cloud/cloud_meta_mgr.cpp | 308 ++++++++++++++++++++++++++------ be/src/cloud/cloud_meta_mgr.h | 67 ++++--- be/src/cloud/meta_mgr.h | 86 --------- be/src/common/status.h | 3 +- be/src/util/s3_util.h | 5 + 5 files changed, 304 insertions(+), 165 deletions(-) delete mode 100644 be/src/cloud/meta_mgr.h diff --git a/be/src/cloud/cloud_meta_mgr.cpp b/be/src/cloud/cloud_meta_mgr.cpp index e6915eae914b96..cbe6ab8ae24112 100644 --- a/be/src/cloud/cloud_meta_mgr.cpp +++ b/be/src/cloud/cloud_meta_mgr.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include "cloud/config.h" @@ -46,7 +47,9 @@ namespace doris::cloud { using namespace ErrorCode; -bvar::LatencyRecorder g_get_rowset_latency("doris_CloudMetaMgr", "get_rowset"); +static bvar::LatencyRecorder g_get_rowset_latency("doris_CloudMetaMgr", "get_rowset"); + +static constexpr int BRPC_RETRY_TIMES = 3; class MetaServiceProxy { public: @@ -166,53 +169,105 @@ class MetaServiceProxy { std::shared_ptr _stub; }; -Status CloudMetaMgr::get_tablet_meta(int64_t tablet_id, TabletMetaSharedPtr* tablet_meta) { - VLOG_DEBUG << "send GetTabletRequest, tablet_id: " << tablet_id; - TEST_SYNC_POINT_RETURN_WITH_VALUE("CloudMetaMgr::get_tablet_meta", Status::OK(), tablet_id, - tablet_meta); +template +struct is_any : std::disjunction...> {}; + +template +constexpr bool is_any_v = is_any::value; + +template +static std::string debug_info(const Request& req) { + if constexpr (is_any_v) { + return fmt::format(" txn_id={}", req.txn_id()); + } else if constexpr (is_any_v) { + return fmt::format(" tablet_id={}", req.job().idx().tablet_id()); + } else if constexpr (is_any_v) { + return fmt::format(" tablet_id={}, lock_id={}", req.tablet_id(), req.lock_id()); + } else if constexpr (is_any_v) { + return fmt::format(" table_id={}, lock_id={}", req.table_id(), req.lock_id()); + } else if constexpr (is_any_v) { + return fmt::format(" tablet_id={}", req.tablet_id()); + } else if constexpr (is_any_v) { + return ""; + } else { + static_assert(!sizeof(Request)); + } +} + +static inline std::default_random_engine make_random_engine() { + return std::default_random_engine( + static_cast(std::chrono::steady_clock::now().time_since_epoch().count())); +} +template +using MetaServiceMethod = void (MetaService_Stub::*)(::google::protobuf::RpcController*, + const Request*, Response*, + ::google::protobuf::Closure*); + +template +static Status retry_rpc(std::string_view op_name, const Request& req, Response* res, + MetaServiceMethod method) { + static_assert(std::is_base_of_v<::google::protobuf::Message, Request>); + static_assert(std::is_base_of_v<::google::protobuf::Message, Response>); + + int retry_times = 0; + uint32_t duration_ms = 0; + std::string error_msg; + std::default_random_engine rng = make_random_engine(); + std::uniform_int_distribution u(20, 200); + std::uniform_int_distribution u2(500, 1000); std::shared_ptr stub; RETURN_IF_ERROR(MetaServiceProxy::get_client(&stub)); - - int tried = 0; while (true) { brpc::Controller cntl; cntl.set_timeout_ms(config::meta_service_brpc_timeout_ms); - GetTabletRequest req; - GetTabletResponse resp; - req.set_cloud_unique_id(config::cloud_unique_id); - req.set_tablet_id(tablet_id); - stub->get_tablet(&cntl, &req, &resp, nullptr); - int retry_times = config::meta_service_rpc_retry_times; - if (cntl.Failed()) { - if (tried++ < retry_times) { - auto rng = std::default_random_engine(static_cast( - std::chrono::steady_clock::now().time_since_epoch().count())); - std::uniform_int_distribution u(20, 200); - std::uniform_int_distribution u1(500, 1000); - uint32_t duration_ms = tried >= 100 ? u(rng) : u1(rng); - std::this_thread::sleep_for(std::chrono::milliseconds(duration_ms)); - LOG_INFO("failed to get tablet meta") - .tag("reason", cntl.ErrorText()) - .tag("tablet_id", tablet_id) - .tag("tried", tried) - .tag("sleep", duration_ms); - continue; - } - return Status::RpcError("failed to get tablet meta: {}", cntl.ErrorText()); + cntl.set_max_retry(BRPC_RETRY_TIMES); + res->Clear(); + (stub.get()->*method)(&cntl, &req, res, nullptr); + if (UNLIKELY(cntl.Failed())) { + error_msg = cntl.ErrorText(); + } else if (res->status().code() == MetaServiceCode::OK) { + return Status::OK(); + } else if (res->status().code() != MetaServiceCode::KV_TXN_CONFLICT) { + return Status::Error("failed to {}: {}", op_name, + res->status().msg()); + } else { + error_msg = res->status().msg(); } + + if (++retry_times > config::meta_service_rpc_retry_times) { + break; + } + + duration_ms = retry_times <= 100 ? u(rng) : u2(rng); + LOG(WARNING) << "failed to " << op_name << debug_info(req) << " retry_times=" << retry_times + << " sleep=" << duration_ms << "ms : " << cntl.ErrorText(); + bthread_usleep(duration_ms * 1000); + } + return Status::RpcError("failed to {}: rpc timeout, last msg={}", op_name, error_msg); +} + +Status CloudMetaMgr::get_tablet_meta(int64_t tablet_id, TabletMetaSharedPtr* tablet_meta) { + VLOG_DEBUG << "send GetTabletRequest, tablet_id: " << tablet_id; + TEST_SYNC_POINT_RETURN_WITH_VALUE("CloudMetaMgr::get_tablet_meta", Status::OK(), tablet_id, + tablet_meta); + GetTabletRequest req; + GetTabletResponse resp; + req.set_cloud_unique_id(config::cloud_unique_id); + req.set_tablet_id(tablet_id); + Status st = retry_rpc("get tablet meta", req, &resp, &MetaService_Stub::get_tablet); + if (!st.ok()) { if (resp.status().code() == MetaServiceCode::TABLET_NOT_FOUND) { return Status::NotFound("failed to get tablet meta: {}", resp.status().msg()); } - if (resp.status().code() != MetaServiceCode::OK) { - return Status::InternalError("failed to get tablet meta: {}", resp.status().msg()); - } - *tablet_meta = std::make_shared(); - (*tablet_meta) - ->init_from_pb(cloud_tablet_meta_to_doris(std::move(*resp.mutable_tablet_meta()))); - VLOG_DEBUG << "get tablet meta, tablet_id: " << (*tablet_meta)->tablet_id(); - return Status::OK(); + return st; } + + *tablet_meta = std::make_shared(); + (*tablet_meta) + ->init_from_pb(cloud_tablet_meta_to_doris(std::move(*resp.mutable_tablet_meta()))); + VLOG_DEBUG << "get tablet meta, tablet_id: " << (*tablet_meta)->tablet_id(); + return Status::OK(); } Status CloudMetaMgr::sync_tablet_rowsets(Tablet* tablet, bool warmup_delta_data) { @@ -226,12 +281,12 @@ Status CloudMetaMgr::sync_tablet_delete_bitmap( return Status::NotSupported("CloudMetaMgr::sync_tablet_delete_bitmap is not implemented"); } -Status CloudMetaMgr::prepare_rowset(const RowsetMeta* rs_meta, bool is_tmp, +Status CloudMetaMgr::prepare_rowset(const RowsetMeta& rs_meta, bool is_tmp, RowsetMetaSharedPtr* existed_rs_meta) { return Status::NotSupported("CloudMetaMgr::prepare_rowset is not implemented"); } -Status CloudMetaMgr::commit_rowset(const RowsetMeta* rs_meta, bool is_tmp, +Status CloudMetaMgr::commit_rowset(const RowsetMeta& rs_meta, bool is_tmp, RowsetMetaSharedPtr* existed_rs_meta) { return Status::NotSupported("CloudMetaMgr::commit_rowset is not implemented"); } @@ -240,50 +295,195 @@ Status CloudMetaMgr::update_tmp_rowset(const RowsetMeta& rs_meta) { return Status::NotSupported("CloudMetaMgr::update_tmp_rowset is not implemented"); } -Status CloudMetaMgr::commit_txn(StreamLoadContext* ctx, bool is_2pc) { - return Status::NotSupported("CloudMetaMgr::commit_txn is not implemented"); +Status CloudMetaMgr::commit_txn(const StreamLoadContext& ctx, bool is_2pc) { + VLOG_DEBUG << "commit txn, db_id: " << ctx.db_id << ", txn_id: " << ctx.txn_id + << ", label: " << ctx.label << ", is_2pc: " << is_2pc; + CommitTxnRequest req; + CommitTxnResponse res; + req.set_cloud_unique_id(config::cloud_unique_id); + req.set_db_id(ctx.db_id); + req.set_txn_id(ctx.txn_id); + req.set_is_2pc(is_2pc); + return retry_rpc("commit txn", req, &res, &MetaService_Stub::commit_txn); } -Status CloudMetaMgr::abort_txn(StreamLoadContext* ctx) { - return Status::NotSupported("CloudMetaMgr::abort_txn is not implemented"); +Status CloudMetaMgr::abort_txn(const StreamLoadContext& ctx) { + VLOG_DEBUG << "abort txn, db_id: " << ctx.db_id << ", txn_id: " << ctx.txn_id + << ", label: " << ctx.label; + AbortTxnRequest req; + AbortTxnResponse res; + req.set_cloud_unique_id(config::cloud_unique_id); + if (ctx.db_id > 0 && !ctx.label.empty()) { + req.set_db_id(ctx.db_id); + req.set_label(ctx.label); + } else { + req.set_txn_id(ctx.txn_id); + } + return retry_rpc("abort txn", req, &res, &MetaService_Stub::abort_txn); } -Status CloudMetaMgr::precommit_txn(StreamLoadContext* ctx) { - return Status::NotSupported("CloudMetaMgr::precommit_txn is not implemented"); +Status CloudMetaMgr::precommit_txn(const StreamLoadContext& ctx) { + VLOG_DEBUG << "precommit txn, db_id: " << ctx.db_id << ", txn_id: " << ctx.txn_id + << ", label: " << ctx.label; + PrecommitTxnRequest req; + PrecommitTxnResponse res; + req.set_cloud_unique_id(config::cloud_unique_id); + req.set_db_id(ctx.db_id); + req.set_txn_id(ctx.txn_id); + return retry_rpc("precommit txn", req, &res, &MetaService_Stub::precommit_txn); } Status CloudMetaMgr::get_s3_info(std::vector>* s3_infos) { - return Status::NotSupported("CloudMetaMgr::get_s3_info is not implemented"); + GetObjStoreInfoRequest req; + GetObjStoreInfoResponse resp; + req.set_cloud_unique_id(config::cloud_unique_id); + Status s = retry_rpc("get s3 info", req, &resp, &MetaService_Stub::get_obj_store_info); + if (!s.ok()) { + return s; + } + + for (const auto& obj_store : resp.obj_info()) { + S3Conf s3_conf; + s3_conf.ak = obj_store.ak(); + s3_conf.sk = obj_store.sk(); + s3_conf.endpoint = obj_store.endpoint(); + s3_conf.region = obj_store.region(); + s3_conf.bucket = obj_store.bucket(); + s3_conf.prefix = obj_store.prefix(); + s3_conf.sse_enabled = obj_store.sse_enabled(); + s3_conf.provider = obj_store.provider(); + s3_infos->emplace_back(obj_store.id(), std::move(s3_conf)); + } + return Status::OK(); } Status CloudMetaMgr::prepare_tablet_job(const TabletJobInfoPB& job, StartTabletJobResponse* res) { - return Status::NotSupported("CloudMetaMgr::prepare_tablet_job is not implemented"); + VLOG_DEBUG << "prepare_tablet_job: " << job.ShortDebugString(); + TEST_SYNC_POINT_RETURN_WITH_VALUE("CloudMetaMgr::prepare_tablet_job", Status::OK(), job, res); + + StartTabletJobRequest req; + req.mutable_job()->CopyFrom(job); + req.set_cloud_unique_id(config::cloud_unique_id); + return retry_rpc("start tablet job", req, res, &MetaService_Stub::start_tablet_job); } Status CloudMetaMgr::commit_tablet_job(const TabletJobInfoPB& job, FinishTabletJobResponse* res) { - return Status::NotSupported("CloudMetaMgr::commit_tablet_job is not implemented"); + VLOG_DEBUG << "commit_tablet_job: " << job.ShortDebugString(); + TEST_SYNC_POINT_RETURN_WITH_VALUE("CloudMetaMgr::commit_tablet_job", Status::OK(), job, res); + + FinishTabletJobRequest req; + req.mutable_job()->CopyFrom(job); + req.set_action(FinishTabletJobRequest::COMMIT); + req.set_cloud_unique_id(config::cloud_unique_id); + return retry_rpc("commit tablet job", req, res, &MetaService_Stub::finish_tablet_job); } Status CloudMetaMgr::abort_tablet_job(const TabletJobInfoPB& job) { - return Status::NotSupported("CloudMetaMgr::alter_tablet_job is not implemented"); + VLOG_DEBUG << "abort_tablet_job: " << job.ShortDebugString(); + FinishTabletJobRequest req; + FinishTabletJobResponse res; + req.mutable_job()->CopyFrom(job); + req.set_action(FinishTabletJobRequest::ABORT); + req.set_cloud_unique_id(config::cloud_unique_id); + return retry_rpc("abort tablet job", req, &res, &MetaService_Stub::finish_tablet_job); } Status CloudMetaMgr::lease_tablet_job(const TabletJobInfoPB& job) { - return Status::NotSupported("CloudMetaMgr::lease_tablet_job is not implemented"); + VLOG_DEBUG << "lease_tablet_job: " << job.ShortDebugString(); + FinishTabletJobRequest req; + FinishTabletJobResponse res; + req.mutable_job()->CopyFrom(job); + req.set_action(FinishTabletJobRequest::LEASE); + req.set_cloud_unique_id(config::cloud_unique_id); + return retry_rpc("lease tablet job", req, &res, &MetaService_Stub::finish_tablet_job); } -Status CloudMetaMgr::update_tablet_schema(int64_t tablet_id, const TabletSchema* tablet_schema) { - return Status::NotSupported("CloudMetaMgr::update_tablet_schema is not implemented"); +Status CloudMetaMgr::update_tablet_schema(int64_t tablet_id, const TabletSchema& tablet_schema) { + VLOG_DEBUG << "send UpdateTabletSchemaRequest, tablet_id: " << tablet_id; + + std::shared_ptr stub; + RETURN_IF_ERROR(MetaServiceProxy::get_client(&stub)); + + brpc::Controller cntl; + cntl.set_timeout_ms(config::meta_service_brpc_timeout_ms); + UpdateTabletSchemaRequest req; + UpdateTabletSchemaResponse resp; + req.set_cloud_unique_id(config::cloud_unique_id); + req.set_tablet_id(tablet_id); + + TabletSchemaPB tablet_schema_pb; + tablet_schema.to_schema_pb(&tablet_schema_pb); + doris_tablet_schema_to_cloud(req.mutable_tablet_schema(), std::move(tablet_schema_pb)); + stub->update_tablet_schema(&cntl, &req, &resp, nullptr); + if (cntl.Failed()) { + return Status::RpcError("failed to update tablet schema: {}", cntl.ErrorText()); + } + if (resp.status().code() != MetaServiceCode::OK) { + return Status::InternalError("failed to update tablet schema: {}", resp.status().msg()); + } + VLOG_DEBUG << "succeed to update tablet schema, tablet_id: " << tablet_id; + return Status::OK(); } Status CloudMetaMgr::update_delete_bitmap(const Tablet* tablet, int64_t lock_id, int64_t initiator, DeleteBitmap* delete_bitmap) { - return Status::NotSupported("CloudMetaMgr::update_delete_bitmap is not implemented"); + VLOG_DEBUG << "update_delete_bitmap , tablet_id: " << tablet->tablet_id(); + UpdateDeleteBitmapRequest req; + UpdateDeleteBitmapResponse res; + req.set_cloud_unique_id(config::cloud_unique_id); + req.set_table_id(tablet->table_id()); + req.set_partition_id(tablet->partition_id()); + req.set_tablet_id(tablet->tablet_id()); + req.set_lock_id(lock_id); + req.set_initiator(initiator); + for (auto iter = delete_bitmap->delete_bitmap.begin(); + iter != delete_bitmap->delete_bitmap.end(); ++iter) { + req.add_rowset_ids(std::get<0>(iter->first).to_string()); + req.add_segment_ids(std::get<1>(iter->first)); + req.add_versions(std::get<2>(iter->first)); + // To save space, convert array and bitmap containers to run containers + iter->second.runOptimize(); + std::string bitmap_data(iter->second.getSizeInBytes(), '\0'); + iter->second.write(bitmap_data.data()); + *(req.add_segment_delete_bitmaps()) = std::move(bitmap_data); + } + auto st = retry_rpc("update delete bitmap", req, &res, &MetaService_Stub::update_delete_bitmap); + if (res.status().code() == MetaServiceCode::LOCK_EXPIRED) { + return Status::Error( + "lock expired when update delete bitmap, tablet_id: {}, lock_id: {}", + tablet->tablet_id(), lock_id); + } + return st; } Status CloudMetaMgr::get_delete_bitmap_update_lock(const Tablet* tablet, int64_t lock_id, int64_t initiator) { - return Status::NotSupported("CloudMetaMgr::get_delete_bitmap_update_lock is not implemented"); + VLOG_DEBUG << "get_delete_bitmap_update_lock , tablet_id: " << tablet->tablet_id(); + GetDeleteBitmapUpdateLockRequest req; + GetDeleteBitmapUpdateLockResponse res; + req.set_cloud_unique_id(config::cloud_unique_id); + req.set_table_id(tablet->table_id()); + req.set_lock_id(lock_id); + req.set_initiator(initiator); + req.set_expiration(10); // 10s expiration time for compaction and schema_change + int retry_times = 0; + Status st; + std::default_random_engine rng = make_random_engine(); + std::uniform_int_distribution u(500, 2000); + do { + st = retry_rpc("get delete bitmap update lock", req, &res, + &MetaService_Stub::get_delete_bitmap_update_lock); + if (res.status().code() != MetaServiceCode::LOCK_CONFLICT) { + break; + } + + uint32_t duration_ms = u(rng); + LOG(WARNING) << "get delete bitmap lock conflict. " << debug_info(req) + << " retry_times=" << retry_times << " sleep=" << duration_ms + << "ms : " << res.status().msg(); + bthread_usleep(duration_ms * 1000); + } while (++retry_times <= 100); + return st; } } // namespace doris::cloud diff --git a/be/src/cloud/cloud_meta_mgr.h b/be/src/cloud/cloud_meta_mgr.h index fe65e0441ff64b..6557a6eab8a1db 100644 --- a/be/src/cloud/cloud_meta_mgr.h +++ b/be/src/cloud/cloud_meta_mgr.h @@ -16,55 +16,73 @@ // under the License. #pragma once -#include "cloud/meta_mgr.h" +#include +#include +#include +#include + +#include "common/status.h" #include "olap/rowset/rowset_meta.h" +#include "util/s3_util.h" + +namespace doris { + +class DeleteBitmap; +class StreamLoadContext; +class Tablet; +class TabletMeta; +class TabletSchema; +class RowsetMeta; + +namespace cloud { -namespace doris::cloud { +class FinishTabletJobResponse; +class StartTabletJobResponse; +class TabletJobInfoPB; class TabletStatsPB; class TabletIndexPB; -class CloudMetaMgr final : public MetaMgr { +class CloudMetaMgr { public: CloudMetaMgr() = default; - ~CloudMetaMgr() override = default; + ~CloudMetaMgr() = default; CloudMetaMgr(const CloudMetaMgr&) = delete; CloudMetaMgr& operator=(const CloudMetaMgr&) = delete; - Status get_tablet_meta(int64_t tablet_id, std::shared_ptr* tablet_meta) override; + Status get_tablet_meta(int64_t tablet_id, std::shared_ptr* tablet_meta); - Status sync_tablet_rowsets(Tablet* tablet, bool warmup_delta_data = false) override; + Status sync_tablet_rowsets(Tablet* tablet, bool warmup_delta_data = false); - Status prepare_rowset(const RowsetMeta* rs_meta, bool is_tmp, - std::shared_ptr* existed_rs_meta = nullptr) override; + Status prepare_rowset(const RowsetMeta& rs_meta, bool is_tmp, + std::shared_ptr* existed_rs_meta = nullptr); - Status commit_rowset(const RowsetMeta* rs_meta, bool is_tmp, - std::shared_ptr* existed_rs_meta = nullptr) override; + Status commit_rowset(const RowsetMeta& rs_meta, bool is_tmp, + std::shared_ptr* existed_rs_meta = nullptr); - Status update_tmp_rowset(const RowsetMeta& rs_meta) override; + Status update_tmp_rowset(const RowsetMeta& rs_meta); - Status commit_txn(StreamLoadContext* ctx, bool is_2pc) override; + Status commit_txn(const StreamLoadContext& ctx, bool is_2pc); - Status abort_txn(StreamLoadContext* ctx) override; + Status abort_txn(const StreamLoadContext& ctx); - Status precommit_txn(StreamLoadContext* ctx) override; + Status precommit_txn(const StreamLoadContext& ctx); - Status get_s3_info(std::vector>* s3_infos) override; + Status get_s3_info(std::vector>* s3_infos); - Status prepare_tablet_job(const TabletJobInfoPB& job, StartTabletJobResponse* res) override; + Status prepare_tablet_job(const TabletJobInfoPB& job, StartTabletJobResponse* res); - Status commit_tablet_job(const TabletJobInfoPB& job, FinishTabletJobResponse* res) override; + Status commit_tablet_job(const TabletJobInfoPB& job, FinishTabletJobResponse* res); - Status abort_tablet_job(const TabletJobInfoPB& job) override; + Status abort_tablet_job(const TabletJobInfoPB& job); - Status lease_tablet_job(const TabletJobInfoPB& job) override; + Status lease_tablet_job(const TabletJobInfoPB& job); - Status update_tablet_schema(int64_t tablet_id, const TabletSchema* tablet_schema) override; + Status update_tablet_schema(int64_t tablet_id, const TabletSchema& tablet_schema); Status update_delete_bitmap(const Tablet* tablet, int64_t lock_id, int64_t initiator, - DeleteBitmap* delete_bitmap) override; + DeleteBitmap* delete_bitmap); - Status get_delete_bitmap_update_lock(const Tablet* tablet, int64_t lock_id, - int64_t initiator) override; + Status get_delete_bitmap_update_lock(const Tablet* tablet, int64_t lock_id, int64_t initiator); private: Status sync_tablet_delete_bitmap( @@ -73,4 +91,5 @@ class CloudMetaMgr final : public MetaMgr { const TabletStatsPB& stas, const TabletIndexPB& idx, DeleteBitmap* delete_bitmap); }; -} // namespace doris::cloud +} // namespace cloud +} // namespace doris diff --git a/be/src/cloud/meta_mgr.h b/be/src/cloud/meta_mgr.h deleted file mode 100644 index c573d43ff7685b..00000000000000 --- a/be/src/cloud/meta_mgr.h +++ /dev/null @@ -1,86 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you 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. -#pragma once - -#include -#include -#include -#include - -#include "common/status.h" -#include "util/s3_util.h" - -namespace doris { -class StreamLoadContext; -class Tablet; -class TabletMeta; -class RowsetMeta; -class TabletSchema; -class DeleteBitmap; - -namespace cloud { - -class TabletJobInfoPB; -class StartTabletJobResponse; -class FinishTabletJobResponse; - -class MetaMgr { -public: - virtual ~MetaMgr() = default; - - virtual Status open() { return Status::OK(); } - - virtual Status get_tablet_meta(int64_t tablet_id, std::shared_ptr* tablet_meta) = 0; - - // If `warmup_delta_data` is true, download the new version rowset data in background - virtual Status sync_tablet_rowsets(Tablet* tablet, bool warmup_delta_data = false) = 0; - - virtual Status prepare_rowset(const RowsetMeta* rs_meta, bool is_tmp, - std::shared_ptr* existed_rs_meta = nullptr) = 0; - - virtual Status commit_rowset(const RowsetMeta* rs_meta, bool is_tmp, - std::shared_ptr* existed_rs_meta = nullptr) = 0; - - virtual Status update_tmp_rowset(const RowsetMeta& rs_meta) = 0; - - virtual Status commit_txn(StreamLoadContext* ctx, bool is_2pc) = 0; - - virtual Status abort_txn(StreamLoadContext* ctx) = 0; - - virtual Status precommit_txn(StreamLoadContext* ctx) = 0; - - virtual Status get_s3_info(std::vector>* s3_infos) = 0; - - virtual Status prepare_tablet_job(const TabletJobInfoPB& job, StartTabletJobResponse* res) = 0; - - virtual Status commit_tablet_job(const TabletJobInfoPB& job, FinishTabletJobResponse* res) = 0; - - virtual Status abort_tablet_job(const TabletJobInfoPB& job) = 0; - - virtual Status lease_tablet_job(const TabletJobInfoPB& job) = 0; - - virtual Status update_delete_bitmap(const Tablet* tablet, int64_t lock_id, int64_t initiator, - DeleteBitmap* delete_bitmap) = 0; - - virtual Status get_delete_bitmap_update_lock(const Tablet* tablet, int64_t lock_id, - int64_t initiator) = 0; - - virtual Status update_tablet_schema(int64_t tablet_id, const TabletSchema* tablet_schema) = 0; -}; - -} // namespace cloud -} // namespace doris diff --git a/be/src/common/status.h b/be/src/common/status.h index bea168299e118c..5199e86d120da2 100644 --- a/be/src/common/status.h +++ b/be/src/common/status.h @@ -54,7 +54,8 @@ namespace ErrorCode { TStatusError(DATA_QUALITY_ERROR, false); \ TStatusError(LABEL_ALREADY_EXISTS, true); \ TStatusError(NOT_AUTHORIZED, true); \ - TStatusError(HTTP_ERROR, true); + TStatusError(HTTP_ERROR, true); \ + TStatusError(DELETE_BITMAP_LOCK_ERROR, false); // E error_name, error_code, print_stacktrace #define APPLY_FOR_OLAP_ERROR_CODES(E) \ E(OK, 0, false); \ diff --git a/be/src/util/s3_util.h b/be/src/util/s3_util.h index 82f5cff8ff9c55..873f6b06f973d1 100644 --- a/be/src/util/s3_util.h +++ b/be/src/util/s3_util.h @@ -20,6 +20,7 @@ #include #include #include +#include #include #include @@ -77,6 +78,10 @@ struct S3Conf { int max_connections = -1; int request_timeout_ms = -1; int connect_timeout_ms = -1; + + bool sse_enabled = false; + cloud::ObjectStoreInfoPB::Provider provider; + bool use_virtual_addressing = true; std::string to_string() const { From 6e04724f557f3f69cab85b09b44f858c3f8dcd5c Mon Sep 17 00:00:00 2001 From: walter Date: Mon, 15 Jan 2024 21:54:34 +0800 Subject: [PATCH 014/200] [future](merge-cloud) Add rpc fields for cloud (#29995) --- be/src/service/backend_service.cpp | 26 ++++++ be/src/service/backend_service.h | 16 ++++ .../apache/doris/common/GenericPoolTest.java | 36 ++++++++ .../doris/utframe/MockedBackendFactory.java | 35 +++++++ gensrc/proto/internal_service.proto | 46 ++++++++++ gensrc/thrift/AgentService.thrift | 22 +++++ gensrc/thrift/BackendService.thrift | 91 +++++++++++++++++++ gensrc/thrift/FrontendService.thrift | 8 ++ gensrc/thrift/HeartbeatService.thrift | 4 + gensrc/thrift/MasterService.thrift | 3 + gensrc/thrift/PaloInternalService.thrift | 3 + 11 files changed, 290 insertions(+) diff --git a/be/src/service/backend_service.cpp b/be/src/service/backend_service.cpp index fb98d60dce4493..306402eca0d514 100644 --- a/be/src/service/backend_service.cpp +++ b/be/src/service/backend_service.cpp @@ -885,4 +885,30 @@ void BackendService::query_ingest_binlog(TQueryIngestBinlogResult& result, break; } } + +void BackendService::pre_cache_async(TPreCacheAsyncResponse& response, + const TPreCacheAsyncRequest& request) { + LOG(FATAL) << "BackendService is not implemented"; +} + +void BackendService::check_pre_cache(TCheckPreCacheResponse& response, + const TCheckPreCacheRequest& request) { + LOG(FATAL) << "BackendService is not implemented"; +} + +void BackendService::sync_load_for_tablets(TSyncLoadForTabletsResponse& response, + const TSyncLoadForTabletsRequest& request) { + LOG(FATAL) << "BackendService is not implemented"; +} + +void BackendService::get_top_n_hot_partitions(TGetTopNHotPartitionsResponse& response, + const TGetTopNHotPartitionsRequest& request) { + LOG(FATAL) << "BackendService is not implemented"; +} + +void BackendService::warm_up_tablets(TWarmUpTabletsResponse& response, + const TWarmUpTabletsRequest& request) { + LOG(FATAL) << "BackendService is not implemented"; +} + } // namespace doris diff --git a/be/src/service/backend_service.h b/be/src/service/backend_service.h index 4ee200796a6a21..fed63454b7bfaa 100644 --- a/be/src/service/backend_service.h +++ b/be/src/service/backend_service.h @@ -142,6 +142,22 @@ class BackendService : public BackendServiceIf { void query_ingest_binlog(TQueryIngestBinlogResult& result, const TQueryIngestBinlogRequest& request) override; + void pre_cache_async(TPreCacheAsyncResponse& response, + const TPreCacheAsyncRequest& request) override; + + void check_pre_cache(TCheckPreCacheResponse& response, + const TCheckPreCacheRequest& request) override; + + // If another cluster load, FE need to notify the cluster to sync the load data + void sync_load_for_tablets(TSyncLoadForTabletsResponse& response, + const TSyncLoadForTabletsRequest& request) override; + + void get_top_n_hot_partitions(TGetTopNHotPartitionsResponse& response, + const TGetTopNHotPartitionsRequest& request) override; + + void warm_up_tablets(TWarmUpTabletsResponse& response, + const TWarmUpTabletsRequest& request) override; + private: Status start_plan_fragment_execution(const TExecPlanFragmentParams& exec_params); ExecEnv* _exec_env = nullptr; diff --git a/fe/fe-core/src/test/java/org/apache/doris/common/GenericPoolTest.java b/fe/fe-core/src/test/java/org/apache/doris/common/GenericPoolTest.java index ba66d07ec6b4ba..eb9ac858b3e661 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/common/GenericPoolTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/common/GenericPoolTest.java @@ -23,15 +23,21 @@ import org.apache.doris.thrift.TAgentTaskRequest; import org.apache.doris.thrift.TCancelPlanFragmentParams; import org.apache.doris.thrift.TCancelPlanFragmentResult; +import org.apache.doris.thrift.TCheckPreCacheRequest; +import org.apache.doris.thrift.TCheckPreCacheResponse; import org.apache.doris.thrift.TCheckStorageFormatResult; import org.apache.doris.thrift.TDiskTrashInfo; import org.apache.doris.thrift.TExecPlanFragmentParams; import org.apache.doris.thrift.TExecPlanFragmentResult; import org.apache.doris.thrift.TExportStatusResult; import org.apache.doris.thrift.TExportTaskRequest; +import org.apache.doris.thrift.TGetTopNHotPartitionsRequest; +import org.apache.doris.thrift.TGetTopNHotPartitionsResponse; import org.apache.doris.thrift.TIngestBinlogRequest; import org.apache.doris.thrift.TIngestBinlogResult; import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TPreCacheAsyncRequest; +import org.apache.doris.thrift.TPreCacheAsyncResponse; import org.apache.doris.thrift.TPublishTopicRequest; import org.apache.doris.thrift.TPublishTopicResult; import org.apache.doris.thrift.TQueryIngestBinlogRequest; @@ -46,10 +52,14 @@ import org.apache.doris.thrift.TSnapshotRequest; import org.apache.doris.thrift.TStatus; import org.apache.doris.thrift.TStreamLoadRecordResult; +import org.apache.doris.thrift.TSyncLoadForTabletsRequest; +import org.apache.doris.thrift.TSyncLoadForTabletsResponse; import org.apache.doris.thrift.TTabletStatResult; import org.apache.doris.thrift.TTransmitDataParams; import org.apache.doris.thrift.TTransmitDataResult; import org.apache.doris.thrift.TUniqueId; +import org.apache.doris.thrift.TWarmUpTabletsRequest; +import org.apache.doris.thrift.TWarmUpTabletsResponse; import org.apache.doris.utframe.UtFrameUtils; import org.apache.commons.pool2.impl.GenericKeyedObjectPoolConfig; @@ -237,6 +247,32 @@ public TQueryIngestBinlogResult queryIngestBinlog(TQueryIngestBinlogRequest quer throws TException { return null; } + + @Override + public TPreCacheAsyncResponse preCacheAsync(TPreCacheAsyncRequest request) throws TException { + return null; + } + + @Override + public TCheckPreCacheResponse checkPreCache(TCheckPreCacheRequest request) throws TException { + return null; + } + + @Override + public TSyncLoadForTabletsResponse syncLoadForTablets(TSyncLoadForTabletsRequest request) throws TException { + return null; + } + + @Override + public TGetTopNHotPartitionsResponse getTopNHotPartitions(TGetTopNHotPartitionsRequest request) + throws TException { + return null; + } + + @Override + public TWarmUpTabletsResponse warmUpTablets(TWarmUpTabletsRequest request) throws TException { + return null; + } } @Test diff --git a/fe/fe-core/src/test/java/org/apache/doris/utframe/MockedBackendFactory.java b/fe/fe-core/src/test/java/org/apache/doris/utframe/MockedBackendFactory.java index bd1ae4c69e1cf1..afbffaa26c9d78 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/utframe/MockedBackendFactory.java +++ b/fe/fe-core/src/test/java/org/apache/doris/utframe/MockedBackendFactory.java @@ -36,6 +36,8 @@ import org.apache.doris.thrift.TBackendInfo; import org.apache.doris.thrift.TCancelPlanFragmentParams; import org.apache.doris.thrift.TCancelPlanFragmentResult; +import org.apache.doris.thrift.TCheckPreCacheRequest; +import org.apache.doris.thrift.TCheckPreCacheResponse; import org.apache.doris.thrift.TCheckStorageFormatResult; import org.apache.doris.thrift.TCloneReq; import org.apache.doris.thrift.TDiskTrashInfo; @@ -46,11 +48,15 @@ import org.apache.doris.thrift.TExportStatusResult; import org.apache.doris.thrift.TExportTaskRequest; import org.apache.doris.thrift.TFinishTaskRequest; +import org.apache.doris.thrift.TGetTopNHotPartitionsRequest; +import org.apache.doris.thrift.TGetTopNHotPartitionsResponse; import org.apache.doris.thrift.THeartbeatResult; import org.apache.doris.thrift.TIngestBinlogRequest; import org.apache.doris.thrift.TIngestBinlogResult; import org.apache.doris.thrift.TMasterInfo; import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TPreCacheAsyncRequest; +import org.apache.doris.thrift.TPreCacheAsyncResponse; import org.apache.doris.thrift.TPublishTopicRequest; import org.apache.doris.thrift.TPublishTopicResult; import org.apache.doris.thrift.TQueryIngestBinlogRequest; @@ -67,12 +73,16 @@ import org.apache.doris.thrift.TStatusCode; import org.apache.doris.thrift.TStorageMediumMigrateReq; import org.apache.doris.thrift.TStreamLoadRecordResult; +import org.apache.doris.thrift.TSyncLoadForTabletsRequest; +import org.apache.doris.thrift.TSyncLoadForTabletsResponse; import org.apache.doris.thrift.TTabletInfo; import org.apache.doris.thrift.TTabletStatResult; import org.apache.doris.thrift.TTaskType; import org.apache.doris.thrift.TTransmitDataParams; import org.apache.doris.thrift.TTransmitDataResult; import org.apache.doris.thrift.TUniqueId; +import org.apache.doris.thrift.TWarmUpTabletsRequest; +import org.apache.doris.thrift.TWarmUpTabletsResponse; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -410,11 +420,36 @@ public TCheckStorageFormatResult checkStorageFormat() throws TException { return new TCheckStorageFormatResult(); } + @Override + public TPreCacheAsyncResponse preCacheAsync(TPreCacheAsyncRequest request) throws TException { + return new TPreCacheAsyncResponse(); + } + + @Override + public TCheckPreCacheResponse checkPreCache(TCheckPreCacheRequest request) throws TException { + return new TCheckPreCacheResponse(); + } + @Override public TIngestBinlogResult ingestBinlog(TIngestBinlogRequest ingestBinlogRequest) throws TException { return null; } + @Override + public TSyncLoadForTabletsResponse syncLoadForTablets(TSyncLoadForTabletsRequest request) throws TException { + return new TSyncLoadForTabletsResponse(); + } + + @Override + public TGetTopNHotPartitionsResponse getTopNHotPartitions(TGetTopNHotPartitionsRequest request) throws TException { + return new TGetTopNHotPartitionsResponse(); + } + + @Override + public TWarmUpTabletsResponse warmUpTablets(TWarmUpTabletsRequest request) throws TException { + return new TWarmUpTabletsResponse(); + } + @Override public TQueryIngestBinlogResult queryIngestBinlog(TQueryIngestBinlogRequest queryIngestBinlogRequest) throws TException { diff --git a/gensrc/proto/internal_service.proto b/gensrc/proto/internal_service.proto index f197cd162dcfc8..433144b304b975 100644 --- a/gensrc/proto/internal_service.proto +++ b/gensrc/proto/internal_service.proto @@ -62,6 +62,11 @@ message PTabletWithPartition { required int64 tablet_id = 2; } +message PTabletLoadRowsetInfo { + required int32 current_rowset_nums = 1; + required int32 max_config_rowset_nums = 2; +} + message PTabletID { optional int64 partition_id = 1; optional int64 index_id = 2; @@ -102,6 +107,7 @@ message PTabletWriterOpenRequest { message PTabletWriterOpenResult { required PStatus status = 1; + repeated PTabletLoadRowsetInfo tablet_load_rowset_num_infos = 2; }; // add batch to tablet writer @@ -183,6 +189,11 @@ message PTabletWriterAddBatchResult { optional int64 wait_execution_time_us = 5; repeated PTabletError tablet_errors = 6; map success_slave_tablet_node_ids = 7; + + // For cloud + optional int64 build_rowset_latency_ms = 1000; + optional int64 commit_rowset_latency_ms = 1001; + repeated PTabletLoadRowsetInfo tablet_load_rowset_num_infos = 1002; }; message PTabletWriterAddBlockResult { @@ -194,6 +205,11 @@ message PTabletWriterAddBlockResult { repeated PTabletError tablet_errors = 6; map success_slave_tablet_node_ids = 7; optional bytes load_channel_profile = 8; + + // For cloud + optional int64 build_rowset_latency_ms = 1000; + optional int64 commit_rowset_latency_ms = 1001; + repeated PTabletLoadRowsetInfo tablet_load_rowset_num_infos = 1002; }; // tablet writer cancel @@ -287,6 +303,10 @@ message PTabletKeyLookupRequest { optional bytes output_expr = 5; // return binary mysql row format if true optional bool is_binary_row = 6; + + // For cloud + // version to read + optional int64 version = 7; } message PTabletKeyLookupResponse { @@ -698,6 +718,31 @@ message PGetTabletVersionsResponse { repeated PVersion versions = 2; }; +message PGetFileCacheMetaRequest { + repeated int64 tablet_ids = 1; +} + +enum FileCacheType { + TTL = 0; + INDEX = 1; + NORMAL = 2; +} + +message FileCacheSegmentMeta { + required int64 tablet_id = 1; + required string rowset_id = 2; + required int64 segment_id = 3; + required string file_name = 4; + required int64 offset = 5; + required int64 size = 6; + required FileCacheType cache_type = 7; + required int64 expiration_time = 8; +} + +message PGetFileCacheMetaResponse { + repeated FileCacheSegmentMeta file_cache_segment_metas = 1; +} + message PReportStreamLoadStatusRequest { optional PUniqueId load_id = 1; optional PStatus status = 2; @@ -864,6 +909,7 @@ service PBackendService { rpc response_slave_tablet_pull_rowset(PTabletWriteSlaveDoneRequest) returns (PTabletWriteSlaveDoneResult); rpc fetch_table_schema(PFetchTableSchemaRequest) returns (PFetchTableSchemaResult); rpc multiget_data(PMultiGetRequest) returns (PMultiGetResponse); + rpc get_file_cache_meta_by_tablet_id(PGetFileCacheMetaRequest) returns (PGetFileCacheMetaResponse); rpc tablet_fetch_data(PTabletKeyLookupRequest) returns (PTabletKeyLookupResponse); rpc get_column_ids_by_tablet_ids(PFetchColIdsRequest) returns (PFetchColIdsResponse); rpc get_tablet_rowset_versions(PGetTabletVersionsRequest) returns (PGetTabletVersionsResponse); diff --git a/gensrc/thrift/AgentService.thrift b/gensrc/thrift/AgentService.thrift index a1c4020b1486e0..b9b50663c3ee6b 100644 --- a/gensrc/thrift/AgentService.thrift +++ b/gensrc/thrift/AgentService.thrift @@ -151,6 +151,10 @@ struct TCreateTabletReq { 24: optional i64 time_series_compaction_file_count_threshold = 2000 25: optional i64 time_series_compaction_time_threshold_seconds = 3600 26: optional i64 time_series_compaction_empty_rowsets_threshold = 5 + + // For cloud + 1000: optional bool is_in_memory = false + 1001: optional bool is_persistent = false } struct TDropTabletReq { @@ -193,6 +197,10 @@ struct TAlterTabletReqV2 { 9: optional Descriptors.TDescriptorTable desc_tbl 10: optional list columns 11: optional i32 be_exec_version = 0 + + // For cloud + 1000: optional i64 job_id + 1001: optional i64 expiration } struct TAlterInvertedIndexReq { @@ -380,6 +388,17 @@ struct TPublishVersionRequest { 3: optional bool strict_mode = false } +struct TCalcDeleteBitmapPartitionInfo { + 1: required Types.TPartitionId partition_id + 2: required Types.TVersion version + 3: required list tablet_ids +} + +struct TCalcDeleteBitmapRequest { + 1: required Types.TTransactionId transaction_id + 2: required list partitions; +} + struct TClearAlterTaskRequest { 1: required Types.TTabletId tablet_id 2: required Types.TSchemaHash schema_hash @@ -479,6 +498,9 @@ struct TAgentTaskRequest { 31: optional TPushStoragePolicyReq push_storage_policy_req 32: optional TAlterInvertedIndexReq alter_inverted_index_req 33: optional TGcBinlogReq gc_binlog_req + + // For cloud + 1000: optional TCalcDeleteBitmapRequest calc_delete_bitmap_req } struct TAgentResult { diff --git a/gensrc/thrift/BackendService.thrift b/gensrc/thrift/BackendService.thrift index dab0b8606771d7..8559698ffd7368 100644 --- a/gensrc/thrift/BackendService.thrift +++ b/gensrc/thrift/BackendService.thrift @@ -124,6 +124,87 @@ struct TCheckStorageFormatResult { 2: optional list v2_tablets; } +struct TPreCacheAsyncRequest { + 1: required string host + 2: required i32 brpc_port + 3: required list tablet_ids +} + +struct TPreCacheAsyncResponse { + 1: required Status.TStatus status +} + +struct TCheckPreCacheRequest { + 1: optional list tablets +} + +struct TCheckPreCacheResponse { + 1: required Status.TStatus status + 2: optional map task_done; +} + +struct TSyncLoadForTabletsRequest { + 1: required list tablet_ids +} + +struct TSyncLoadForTabletsResponse { +} + +struct THotPartition { + 1: required i64 partition_id + 2: required i64 last_access_time + 3: optional i64 query_per_day + 4: optional i64 query_per_week +} + +struct THotTableMessage { + 1: required i64 table_id + 2: required i64 index_id + 3: optional list hot_partitions +} + +struct TGetTopNHotPartitionsRequest { +} + +struct TGetTopNHotPartitionsResponse { + 1: required i64 file_cache_size + 2: optional list hot_tables +} + +enum TDownloadType { + BE = 0, + S3 = 1, +} + +enum TWarmUpTabletsRequestType { + SET_JOB = 0, + SET_BATCH = 1, + GET_CURRENT_JOB_STATE_AND_LEASE = 2, + CLEAR_JOB = 3, +} + +struct TJobMeta { + 1: required TDownloadType download_type + 2: optional string be_ip + 3: optional i32 brpc_port + 4: optional list tablet_ids +} + +struct TWarmUpTabletsRequest { + 1: required i64 job_id + 2: required i64 batch_id + 3: optional list job_metas + 4: required TWarmUpTabletsRequestType type +} + +struct TWarmUpTabletsResponse { + 1: required Status.TStatus status; + 2: optional i64 job_id + 3: optional i64 batch_id + 4: optional i64 pending_job_size + 5: optional i64 finish_job_size +} + struct TIngestBinlogRequest { 1: optional i64 txn_id; 2: optional i64 remote_tablet_id; @@ -291,6 +372,16 @@ service BackendService { // check tablet rowset type TCheckStorageFormatResult check_storage_format(); + TPreCacheAsyncResponse pre_cache_async(1: TPreCacheAsyncRequest request); + + TCheckPreCacheResponse check_pre_cache(1: TCheckPreCacheRequest request); + + TSyncLoadForTabletsResponse sync_load_for_tablets(1: TSyncLoadForTabletsRequest request); + + TGetTopNHotPartitionsResponse get_top_n_hot_partitions(1: TGetTopNHotPartitionsRequest request); + + TWarmUpTabletsResponse warm_up_tablets(1: TWarmUpTabletsRequest request); + TIngestBinlogResult ingest_binlog(1: TIngestBinlogRequest ingest_binlog_request); TQueryIngestBinlogResult query_ingest_binlog(1: TQueryIngestBinlogRequest query_ingest_binlog_request); diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index 02d3efe50edb81..e19c7e400694b2 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -517,6 +517,10 @@ struct TMasterOpRequest { 24: optional bool syncJournalOnly // if set to true, this request means to do nothing but just sync max journal id of master 25: optional string defaultCatalog 26: optional string defaultDatabase + + // selectdb cloud + 1000: optional string cloud_cluster + 1001: optional bool noAuth; } struct TColumnDefinition { @@ -673,6 +677,7 @@ struct TStreamLoadPutRequest { // For cloud 1000: optional string cloud_cluster + 1001: optional i64 table_id } struct TStreamLoadPutResult { @@ -785,6 +790,9 @@ struct TLoadTxn2PCRequest { 9: optional string token 10: optional i64 thrift_rpc_timeout_ms 11: optional string label + + // For cloud + 1000: optional string auth_code_uuid } struct TLoadTxn2PCResult { diff --git a/gensrc/thrift/HeartbeatService.thrift b/gensrc/thrift/HeartbeatService.thrift index 5a7e47d982b8fb..459bc2f8f31d0f 100644 --- a/gensrc/thrift/HeartbeatService.thrift +++ b/gensrc/thrift/HeartbeatService.thrift @@ -51,6 +51,10 @@ struct TBackendInfo { 7: optional string be_node_role 8: optional bool is_shutdown 9: optional Types.TPort arrow_flight_sql_port + + // For cloud + 1000: optional i64 fragment_executing_count + 1001: optional i64 fragment_last_active_time } struct THeartbeatResult { diff --git a/gensrc/thrift/MasterService.thrift b/gensrc/thrift/MasterService.thrift index 9acd3f85f7b043..fb5dd416bd1ae5 100644 --- a/gensrc/thrift/MasterService.thrift +++ b/gensrc/thrift/MasterService.thrift @@ -46,6 +46,9 @@ struct TTabletInfo { // 18: optional bool is_cooldown 19: optional i64 cooldown_term 20: optional Types.TUniqueId cooldown_meta_id + + // For cloud + 1000: optional bool is_persistent } struct TFinishTaskRequest { diff --git a/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift index 7559451e373867..fb9a1888f6ce7a 100644 --- a/gensrc/thrift/PaloInternalService.thrift +++ b/gensrc/thrift/PaloInternalService.thrift @@ -649,6 +649,9 @@ struct TCondition { // using unique id to distinguish them 4: optional i32 column_unique_id 5: optional bool marked_by_runtime_filter = false + + // For cloud + 1000: optional TCompoundType compound_type = TCompoundType.UNKNOWN } struct TExportStatusResult { From 3f22c7962b8e8b1328eb10a62b0f8084232ab444 Mon Sep 17 00:00:00 2001 From: HHoflittlefish777 <77738092+HHoflittlefish777@users.noreply.github.com> Date: Mon, 15 Jan 2024 21:55:53 +0800 Subject: [PATCH 015/200] [fix](regression-test) fix err log limit test global impact for setting param (#29993) --- .../load_p0/stream_load/test_stream_load_err_log_limit.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/regression-test/suites/load_p0/stream_load/test_stream_load_err_log_limit.groovy b/regression-test/suites/load_p0/stream_load/test_stream_load_err_log_limit.groovy index 261e0e1d49d2dd..e0d8f347b2ff6c 100644 --- a/regression-test/suites/load_p0/stream_load/test_stream_load_err_log_limit.groovy +++ b/regression-test/suites/load_p0/stream_load/test_stream_load_err_log_limit.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_stream_load_err_log_limit", "p0") { +suite("test_stream_load_err_log_limit", "p0, nonConcurrent") { sql "show tables" def tableName = "test_stream_load_err_log_limit_table" From 2555e78076e8665593ae0efe22237656ad9cc46d Mon Sep 17 00:00:00 2001 From: jakevin Date: Tue, 16 Jan 2024 10:09:25 +0800 Subject: [PATCH 016/200] [fix](Nereids): fix NPE InferPredicates (#29978) PredicatePropagation shouldn't add null into List. --- .../apache/doris/nereids/rules/rewrite/InferPredicates.java | 4 ++-- .../doris/nereids/rules/rewrite/PredicatePropagation.java | 5 ++++- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InferPredicates.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InferPredicates.java index 36236c3db8dcc7..bafbc45cae39de 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InferPredicates.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InferPredicates.java @@ -122,8 +122,8 @@ private Set pullUpPredicates(Plan plan) { private Plan inferNewPredicate(Plan plan, Set expressions) { Set predicates = expressions.stream() - .filter(c -> !c.getInputSlots().isEmpty() && plan.getOutputSet().containsAll( - c.getInputSlots())).collect(Collectors.toSet()); + .filter(c -> !c.getInputSlots().isEmpty() && plan.getOutputSet().containsAll(c.getInputSlots())) + .collect(Collectors.toSet()); predicates.removeAll(plan.accept(pollUpPredicates, null)); return PlanUtils.filterOrSelf(predicates, plan); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PredicatePropagation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PredicatePropagation.java index 5d11a1fa542c77..d1eba6cce36157 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PredicatePropagation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PredicatePropagation.java @@ -95,7 +95,10 @@ public static Set infer(Set predicates) { slotPredicates.forEach((left, exprs) -> { for (Slot right : equalSet.calEqualSet(left)) { for (Expression expr : exprs) { - inferred.add(doInferPredicate(left, right, expr)); + Expression inferPredicate = doInferPredicate(left, right, expr); + if (inferPredicate != null) { + inferred.add(inferPredicate); + } } } }); From 29b8c847cfc66386dc9c9325011c23371ff53180 Mon Sep 17 00:00:00 2001 From: Jerry Hu Date: Tue, 16 Jan 2024 11:36:16 +0800 Subject: [PATCH 017/200] [fix](scan) crashing caused by unlocked reading of tablet (#30005) --- be/src/olap/parallel_scanner_builder.cpp | 5 ++++- be/src/vec/exec/scan/pip_scanner_context.h | 3 +++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/be/src/olap/parallel_scanner_builder.cpp b/be/src/olap/parallel_scanner_builder.cpp index 4e64551dc3b669..d25bcdc7a2f7f5 100644 --- a/be/src/olap/parallel_scanner_builder.cpp +++ b/be/src/olap/parallel_scanner_builder.cpp @@ -168,7 +168,10 @@ Status ParallelScannerBuilder::_load() { for (auto&& [tablet, version] : _tablets) { const auto tablet_id = tablet->tablet_id(); auto& rowsets = _all_rowsets[tablet_id]; - RETURN_IF_ERROR(tablet->capture_consistent_rowsets({0, version}, &rowsets)); + { + std::shared_lock read_lock(tablet->get_header_lock()); + RETURN_IF_ERROR(tablet->capture_consistent_rowsets({0, version}, &rowsets)); + } for (auto& rowset : rowsets) { RETURN_IF_ERROR(rowset->load()); diff --git a/be/src/vec/exec/scan/pip_scanner_context.h b/be/src/vec/exec/scan/pip_scanner_context.h index 77237bb71af5fe..1afe3b599e5980 100644 --- a/be/src/vec/exec/scan/pip_scanner_context.h +++ b/be/src/vec/exec/scan/pip_scanner_context.h @@ -310,6 +310,9 @@ class PipXScannerContext final : public vectorized::ScannerContext { _blocks_queue_buffered = std::move(_blocks_queue); } + + // `get_block_from_queue` should not be called concurrently from multiple threads, + // so here no need to lock. *block = std::move(_blocks_queue_buffered.front()); _blocks_queue_buffered.pop_front(); From 6d8567c32206cce7a9938a7788cb71db346bcb5b Mon Sep 17 00:00:00 2001 From: starocean999 <40539150+starocean999@users.noreply.github.com> Date: Tue, 16 Jan 2024 11:37:30 +0800 Subject: [PATCH 018/200] [fix](nereids)need validate auto partition columns in DDL (#29985) --- .../plans/commands/info/CreateTableInfo.java | 25 ++++++------------- 1 file changed, 7 insertions(+), 18 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java index f47a693d25b2e5..a1a5dd9836cd70 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java @@ -139,7 +139,6 @@ public CreateTableInfo(boolean ifNotExists, boolean isExternal, String ctlName, this.autoPartitionExprs = autoPartitionExprs; this.partitionType = partitionType; this.partitionColumns = partitionColumns; - appendColumnFromExprs(); this.partitions = partitions; this.distribution = distribution; this.rollups = Utils.copyRequiredList(rollups); @@ -175,7 +174,6 @@ public CreateTableInfo(boolean ifNotExists, boolean isExternal, String ctlName, this.autoPartitionExprs = autoPartitionExprs; this.partitionType = partitionType; this.partitionColumns = partitionColumns; - appendColumnFromExprs(); this.partitions = partitions; this.distribution = distribution; this.rollups = Utils.copyRequiredList(rollups); @@ -459,6 +457,12 @@ public void validate(ConnectContext ctx) { } }); + if (isAutoPartition) { + partitionColumns = ExpressionUtils + .collectAll(autoPartitionExprs, UnboundSlot.class::isInstance).stream() + .map(slot -> ((UnboundSlot) slot).getName()).collect(Collectors.toList()); + } + if (partitionColumns != null) { partitionColumns.forEach(p -> { if (!columnMap.containsKey(p)) { @@ -636,7 +640,7 @@ private void checkEngineName() { throw new AnalysisException("odbc, mysql and broker table is no longer supported." + " For odbc and mysql external table, use jdbc table or jdbc catalog instead." + " For broker table, use table valued function instead." - + ". Or you can temporarily set 'disable_odbc_mysql_broker_table=false'" + + ". Or you can temporarily set 'enable_odbc_mysql_broker_table=true'" + " in fe.conf to reopen this feature."); } } @@ -796,11 +800,6 @@ private void validateKeyColumns() { * translate to catalog create table stmt */ public CreateTableStmt translateToLegacyStmt() { - if (isAutoPartition) { - partitionColumns = ExpressionUtils - .collectAll(autoPartitionExprs, UnboundSlot.class::isInstance).stream() - .map(slot -> ((UnboundSlot) slot).getName()).collect(Collectors.toList()); - } PartitionDesc partitionDesc = null; if (partitionColumns != null || isAutoPartition) { List partitionDescs = @@ -894,14 +893,4 @@ private static List convertToLegacyArguments(List children) { } }).collect(Collectors.toList()); } - - private void appendColumnFromExprs() { - for (Expression autoExpr : autoPartitionExprs) { - for (Expression child : autoExpr.children()) { - if (child instanceof UnboundSlot) { - partitionColumns.add(((UnboundSlot) child).getName()); - } - } - } - } } From 04701b605542349f658adaf03cd61215dd14505e Mon Sep 17 00:00:00 2001 From: zclllyybb Date: Tue, 16 Jan 2024 13:01:28 +0800 Subject: [PATCH 019/200] [Enhance](tools) change SSB perf tests to standardize behavior (#30002) change SSB perf tests to standardize behavior --- tools/ssb-tools/bin/run-ssb-flat-queries.sh | 15 +++++++++------ tools/ssb-tools/bin/run-ssb-queries.sh | 13 +++++++++---- 2 files changed, 18 insertions(+), 10 deletions(-) diff --git a/tools/ssb-tools/bin/run-ssb-flat-queries.sh b/tools/ssb-tools/bin/run-ssb-flat-queries.sh index 0cdb4c68d9ccaa..5b82c56ab85f2c 100755 --- a/tools/ssb-tools/bin/run-ssb-flat-queries.sh +++ b/tools/ssb-tools/bin/run-ssb-flat-queries.sh @@ -132,12 +132,15 @@ echo '============================================' sum=0 for i in '1.1' '1.2' '1.3' '2.1' '2.2' '2.3' '3.1' '3.2' '3.3' '3.4' '4.1' '4.2' '4.3'; do - # First run to prevent the affect of cold start - mysql -h"${FE_HOST}" -P"${FE_QUERY_PORT}" -u"${USER}" -D "${DB}" <"${QUERIES_DIR}/q${i}.sql" >/dev/null 2>&1 - # Then run 3 times and takes the average time - res=$(mysqlslap -h"${FE_HOST}" -P"${FE_QUERY_PORT}" -u"${USER}" --create-schema="${DB}" --query="${QUERIES_DIR}/q${i}.sql" -F '\r' -i 3 | sed -n '2p' | cut -d ' ' -f 9,10) - echo "q${i}: ${res}" - cost=$(echo "${res}" | cut -d' ' -f1) + # Each query is executed 3 times and takes the min time + res1=$(mysql -vvv -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" -e "$(cat "${QUERIES_DIR}"/q"${i}".sql)" | perl -nle 'print $1 if /\((\d+\.\d+)+ sec\)/' || :) + res2=$(mysql -vvv -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" -e "$(cat "${QUERIES_DIR}"/q"${i}".sql)" | perl -nle 'print $1 if /\((\d+\.\d+)+ sec\)/' || :) + res3=$(mysql -vvv -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" -e "$(cat "${QUERIES_DIR}"/q"${i}".sql)" | perl -nle 'print $1 if /\((\d+\.\d+)+ sec\)/' || :) + + min_value=$(echo "${res1} ${res2} ${res3}" | tr ' ' '\n' | sort -n | head -n 1) + echo -e "q${i}:\t${res1}\t${res2}\t${res3}\tfast:${min_value}" + + cost=$(echo "${min_value}" | cut -d' ' -f1) sum=$(echo "${sum} + ${cost}" | bc) done echo "total time: ${sum} seconds" diff --git a/tools/ssb-tools/bin/run-ssb-queries.sh b/tools/ssb-tools/bin/run-ssb-queries.sh index e2329dba728861..1394d8e3ec4879 100755 --- a/tools/ssb-tools/bin/run-ssb-queries.sh +++ b/tools/ssb-tools/bin/run-ssb-queries.sh @@ -145,10 +145,15 @@ echo '============================================' sum=0 for i in '1.1' '1.2' '1.3' '2.1' '2.2' '2.3' '3.1' '3.2' '3.3' '3.4' '4.1' '4.2' '4.3'; do - # Each query is executed 3 times and takes the average time - res=$(mysqlslap -h"${FE_HOST}" -P"${FE_QUERY_PORT}" -u"${USER}" --create-schema="${DB}" --query="${QUERIES_DIR}/q${i}.sql" -F '\r' -i 3 | sed -n '2p' | cut -d ' ' -f 9,10) - echo "q${i}: ${res}" - cost=$(echo "${res}" | cut -d' ' -f1) + # Each query is executed 3 times and takes the min time + res1=$(mysql -vvv -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" -e "$(cat "${QUERIES_DIR}"/q"${i}".sql)" | perl -nle 'print $1 if /\((\d+\.\d+)+ sec\)/' || :) + res2=$(mysql -vvv -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" -e "$(cat "${QUERIES_DIR}"/q"${i}".sql)" | perl -nle 'print $1 if /\((\d+\.\d+)+ sec\)/' || :) + res3=$(mysql -vvv -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" -e "$(cat "${QUERIES_DIR}"/q"${i}".sql)" | perl -nle 'print $1 if /\((\d+\.\d+)+ sec\)/' || :) + + min_value=$(echo "${res1} ${res2} ${res3}" | tr ' ' '\n' | sort -n | head -n 1) + echo -e "q${i}:\t${res1}\t${res2}\t${res3}\tfast:${min_value}" + + cost=$(echo "${min_value}" | cut -d' ' -f1) sum=$(echo "${sum} + ${cost}" | bc) done echo "total time: ${sum} seconds" From df78d64477ed2ba9d727fecbd3062b1d0f499aa0 Mon Sep 17 00:00:00 2001 From: Jibing-Li <64681310+Jibing-Li@users.noreply.github.com> Date: Tue, 16 Jan 2024 14:18:22 +0800 Subject: [PATCH 020/200] [doc](statistics)Update docs for statistics (#29926) Update docs for statistics --- docs/en/docs/query-acceleration/statistics.md | 10 ++++++++-- docs/zh-CN/docs/query-acceleration/statistics.md | 10 ++++++++-- 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/docs/en/docs/query-acceleration/statistics.md b/docs/en/docs/query-acceleration/statistics.md index 667db64e3af0ee..10486c080597d7 100644 --- a/docs/en/docs/query-acceleration/statistics.md +++ b/docs/en/docs/query-acceleration/statistics.md @@ -64,7 +64,13 @@ Where: - `sync`: Collect statistics synchronously. Returns after collection. If not specified, it executes asynchronously and returns a JOB ID. - `sample percent | rows`: Collect statistics with sampling. You can specify a sampling percentage or a number of sampling rows. -Here are some examples: +By default (WITH SAMPLE is not specified), a table will be fully analyzed. For relatively large tables (e.g. above 5GiB), from the perspective of system resources, we recommend sampling analyze, and the number of sampled rows is recommended to be no less than 4 million rows. Here are some examples: + +Collect statistics for a table with full analyze: + +```sql +ANALYZE TABLE lineitem; +``` Collect statistics for a table with a 10% sampling rate: @@ -84,7 +90,7 @@ ANALYZE TABLE lineitem WITH SAMPLE ROWS 100000; This feature has been officially supported since 2.0.3 and is enabled by default. The basic operation logic is described below. After each import transaction commit, Doris records the number of rows updated by the import transaction to estimate the health of the existing table's statistics data (for tables that have not collected statistics, their health is 0). When the health of a table is below 60 (adjustable through the `table_stats_health_threshold` parameter), Doris considers the statistics for that table outdated and triggers statistics collection jobs for that table in subsequent operations. For tables with a health value above 60, no repeated collection is performed. -The collection jobs for statistics themselves consume a certain amount of system resources. To minimize the overhead, for tables with a large amount of data (default 5 GiB, adjustable with the FE parameter `huge_table_lower_bound_size_in_bytes`), Doris automatically uses sampling to collect statistics. Automatic sampling defaults to sampling 4,194,304 (2^22) rows to reduce the system's burden and complete the collection job as quickly as possible. If you want to sample more rows to obtain a more accurate data distribution, you can increase the sampling row count by adjusting the `huge_table_default_sample_rows` parameter. In addition, for tables with data larger than `huge_table_lower_bound_size_in_bytes` * 5, Doris ensures that the collection time interval is not less than 12 hours (which can be controlled by adjusting the `huge_table_auto_analyze_interval_in_millis` parameter). +The collection jobs for statistics themselves consume a certain amount of system resources. To minimize the overhead, Doris automatically uses sampling to collect statistics. Automatic sampling defaults to sample 4,194,304 (2^22) rows to reduce the system's burden and complete the collection job as quickly as possible. If you want to sample more rows to obtain a more accurate data distribution, you can increase the sampling row count by adjusting the `huge_table_default_sample_rows` parameter. You can also control the full collection of small tables and the collection interval of large tables through session variables. For detailed configuration, please refer to [3.1](statistics.md#31-session-variables). If you are concerned about automatic collection jobs interfering with your business, you can specify a time frame for the automatic collection jobs to run during low business loads by setting the `auto_analyze_start_time` and `auto_analyze_end_time` parameters according to your needs. You can also completely disable this feature by setting the `enable_auto_analyze` parameter to `false`. diff --git a/docs/zh-CN/docs/query-acceleration/statistics.md b/docs/zh-CN/docs/query-acceleration/statistics.md index b54009a7c2ac68..446aa5605a8aa0 100644 --- a/docs/zh-CN/docs/query-acceleration/statistics.md +++ b/docs/zh-CN/docs/query-acceleration/statistics.md @@ -66,8 +66,14 @@ ANALYZE < TABLE table_name | DATABASE db_name > - sync:同步收集统计信息。收集完后返回。若不指定则异步执行并返回JOB ID。 - sample percent | rows:抽样收集统计信息。可以指定抽样比例或者抽样行数。 +默认情况下(不指定WITH SAMPLE),会对一张表全量采样。 对于比较大的表(5GiB以上),从集群资源的角度出发,一般情况下我们建议采样收集,采样的行数建议不低于400万行。下面是一些例子 + +对一张表全量收集统计信息: + +```sql +ANALYZE TABLE lineitem; +``` -下面是一些例子 对一张表按照10%的比例采样收集统计数据: @@ -87,7 +93,7 @@ ANALYZE TABLE lineitem WITH SAMPLE ROWS 100000; 此功能从2.0.3开始正式支持,默认为全天开启状态。下面对其基本运行逻辑进行阐述,在每次导入事务提交后,Doris将记录本次导入事务更新的表行数用以估算当前已有表的统计数据的健康度(对于没有收集过统计数据的表,其健康度为0)。当表的健康度低于60(可通过参数`table_stats_health_threshold`调节)时,Doris会认为该表的统计信息已经过时,并在之后触发对该表的统计信息收集作业。而对于统计信息健康度高于60的表,则不会重复进行收集。 -统计信息的收集作业本身需要占用一定的系统资源,为了尽可能降低开销,对于数据量较大(默认为5GiB,可通过设置FE参数`huge_table_lower_bound_size_in_bytes`来调节此行为)的表,Doris会自动采取采样的方式去收集,自动采样默认采样4194304(2^22)行,以尽可能降低对系统造成的负担并尽快完成收集作业。如果希望采样更多的行以获得更准确的数据分布信息,可通过调整参数`huge_table_default_sample_rows`增大采样行数。另外对于数据量大于`huge_table_lower_bound_size_in_bytes` * 5 的表,Doris保证其收集时间间隔不小于12小时(该时间可通过调整参数`huge_table_auto_analyze_interval_in_millis`控制)。 +统计信息的收集作业本身需要占用一定的系统资源,为了尽可能降低开销,Doris会使用采样的方式去收集,自动采样默认采样4194304(2^22)行,以尽可能降低对系统造成的负担并尽快完成收集作业。如果希望采样更多的行以获得更准确的数据分布信息,可通过调整参数`huge_table_default_sample_rows`增大采样行数。用户还可通过参数控制小表全量收集,大表收集时间间隔等行为。详细配置请参考详[3.1](statistics.md#31-会话变量)。 如果担心自动收集作业对业务造成干扰,可结合自身需求通过设置参数`auto_analyze_start_time`和参数`auto_analyze_end_time`指定自动收集作业在业务负载较低的时间段执行。也可以通过设置参数`enable_auto_analyze` 为`false`来彻底关闭本功能。 From 59ae5b395783150feed8540b753f7bb6d73a9e81 Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Tue, 16 Jan 2024 14:31:04 +0800 Subject: [PATCH 021/200] [fix](Nereids) should not fold constant when do ordinal group by (#29976) --- .../analysis/ResolveOrdinalInOrderByAndGroupBy.java | 12 +++--------- .../data/nereids_syntax_p0/group_by_constant.out | 3 +++ .../nereids_syntax_p0/group_by_constant.groovy | 2 ++ 3 files changed, 8 insertions(+), 9 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ResolveOrdinalInOrderByAndGroupBy.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ResolveOrdinalInOrderByAndGroupBy.java index 7e6cecd6f3a44e..1cefd203ff74aa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ResolveOrdinalInOrderByAndGroupBy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ResolveOrdinalInOrderByAndGroupBy.java @@ -20,8 +20,6 @@ import org.apache.doris.nereids.properties.OrderKey; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; -import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; -import org.apache.doris.nereids.rules.expression.rules.FoldConstantRule; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; @@ -51,10 +49,8 @@ public List buildRules() { List childOutput = sort.child().getOutput(); List orderKeys = sort.getOrderKeys(); List orderKeysWithoutOrd = new ArrayList<>(); - ExpressionRewriteContext context = new ExpressionRewriteContext(ctx.cascadesContext); for (OrderKey k : orderKeys) { Expression expression = k.getExpr(); - expression = FoldConstantRule.INSTANCE.rewrite(expression, context); if (expression instanceof IntegerLikeLiteral) { IntegerLikeLiteral i = (IntegerLikeLiteral) expression; int ord = i.getIntValue(); @@ -69,14 +65,12 @@ public List buildRules() { }) )) .add(RuleType.RESOLVE_ORDINAL_IN_GROUP_BY.build( - logicalAggregate().whenNot(agg -> agg.isOrdinalIsResolved()).thenApply(ctx -> { + logicalAggregate().whenNot(LogicalAggregate::isOrdinalIsResolved).thenApply(ctx -> { LogicalAggregate agg = ctx.root; List aggOutput = agg.getOutputExpressions(); List groupByWithoutOrd = new ArrayList<>(); - ExpressionRewriteContext context = new ExpressionRewriteContext(ctx.cascadesContext); boolean ordExists = false; for (Expression groupByExpr : agg.getGroupByExpressions()) { - groupByExpr = FoldConstantRule.INSTANCE.rewrite(groupByExpr, context); if (groupByExpr instanceof IntegerLikeLiteral) { IntegerLikeLiteral i = (IntegerLikeLiteral) groupByExpr; int ord = i.getIntValue(); @@ -92,8 +86,8 @@ public List buildRules() { } } if (ordExists) { - return new LogicalAggregate(groupByWithoutOrd, agg.getOutputExpressions(), true, - agg.child()); + return new LogicalAggregate<>(groupByWithoutOrd, agg.getOutputExpressions(), + true, agg.child()); } else { return agg; } diff --git a/regression-test/data/nereids_syntax_p0/group_by_constant.out b/regression-test/data/nereids_syntax_p0/group_by_constant.out index d41d1aa34077c6..803d8e53f56d24 100644 --- a/regression-test/data/nereids_syntax_p0/group_by_constant.out +++ b/regression-test/data/nereids_syntax_p0/group_by_constant.out @@ -43,3 +43,6 @@ str 24 1310179 10 1 -- !sql -- 2 +-- !sql -- +1 + diff --git a/regression-test/suites/nereids_syntax_p0/group_by_constant.groovy b/regression-test/suites/nereids_syntax_p0/group_by_constant.groovy index 3cdce346431ddf..30a24e5fdb8de3 100644 --- a/regression-test/suites/nereids_syntax_p0/group_by_constant.groovy +++ b/regression-test/suites/nereids_syntax_p0/group_by_constant.groovy @@ -55,4 +55,6 @@ suite("group_by_constant") { qt_sql """select SUM(if(lo_tax=1,lo_tax,0)) FROM lineorder where false;""" qt_sql """select 2 FROM lineorder group by 1;""" + + qt_sql """select 1 from lineorder group by 1 + 1""" } From ee83a20a33abc27ab1e5e0b2d5c83c34380d18b9 Mon Sep 17 00:00:00 2001 From: zzzxl <33418555+zzzxl1993@users.noreply.github.com> Date: Tue, 16 Jan 2024 15:03:49 +0800 Subject: [PATCH 022/200] [opt](invert index) optimize the code structure of inverted index queries (#29771) --- .../query/conjunction_query.cpp | 15 +-- .../inverted_index/query/conjunction_query.h | 32 +++--- .../query/disjunction_query.cpp | 10 +- .../inverted_index/query/disjunction_query.h | 27 +++-- .../query/phrase_prefix_query.cpp | 27 +++-- .../query/phrase_prefix_query.h | 31 +++--- .../inverted_index/query/phrase_query.cpp | 4 + .../inverted_index/query/phrase_query.h | 23 ++-- .../inverted_index/query/prefix_query.cpp | 4 +- .../inverted_index/query/prefix_query.h | 6 +- .../segment_v2/inverted_index/query/query.h | 51 +++++++++ .../inverted_index/query/query_factory.h | 53 ++++++++++ .../inverted_index/query/regexp_query.cpp | 19 +++- .../inverted_index/query/regexp_query.h | 18 ++-- .../segment_v2/inverted_index_reader.cpp | 100 ++++-------------- .../rowset/segment_v2/inverted_index_reader.h | 27 +---- .../docs/data-table/index/inverted-index.md | 12 +++ .../docs/data-table/index/inverted-index.md | 14 ++- 18 files changed, 256 insertions(+), 217 deletions(-) create mode 100644 be/src/olap/rowset/segment_v2/inverted_index/query/query.h create mode 100644 be/src/olap/rowset/segment_v2/inverted_index/query/query_factory.h diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/conjunction_query.cpp b/be/src/olap/rowset/segment_v2/inverted_index/query/conjunction_query.cpp index 56cccdf3e3fa58..fb2479517166fc 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index/query/conjunction_query.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/conjunction_query.cpp @@ -17,12 +17,13 @@ #include "conjunction_query.h" -#include +namespace doris::segment_v2 { -namespace doris { - -ConjunctionQuery::ConjunctionQuery(IndexReader* reader) - : _reader(reader), _index_version(reader->getIndexVersion()) {} +ConjunctionQuery::ConjunctionQuery(const std::shared_ptr& searcher, + const TQueryOptions& query_options) + : _searcher(searcher), + _index_version(_searcher->getReader()->getIndexVersion()), + _conjunction_ratio(query_options.inverted_index_conjunction_opt_threshold) {} ConjunctionQuery::~ConjunctionQuery() { for (auto& term_doc : _term_docs) { @@ -47,7 +48,7 @@ void ConjunctionQuery::add(const std::wstring& field_name, const std::vectortermDocs(t); + TermDocs* term_doc = _searcher->getReader()->termDocs(t); _term_docs.push_back(term_doc); iterators.emplace_back(term_doc); } @@ -165,4 +166,4 @@ int32_t ConjunctionQuery::do_next(int32_t doc) { } } -} // namespace doris \ No newline at end of file +} // namespace doris::segment_v2 \ No newline at end of file diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/conjunction_query.h b/be/src/olap/rowset/segment_v2/inverted_index/query/conjunction_query.h index 36d9478c20dde3..2571392d5294e9 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index/query/conjunction_query.h +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/conjunction_query.h @@ -17,29 +17,21 @@ #pragma once -#include -#include -#include -#include -#include - -#include "roaring/roaring.hh" +#include "olap/rowset/segment_v2/inverted_index/query/query.h" CL_NS_USE(index) +CL_NS_USE(search) -namespace doris { +namespace doris::segment_v2 { -class ConjunctionQuery { +class ConjunctionQuery : public Query { public: - ConjunctionQuery(IndexReader* reader); - ~ConjunctionQuery(); - - void set_conjunction_ratio(int32_t conjunction_ratio) { - _conjunction_ratio = conjunction_ratio; - } + ConjunctionQuery(const std::shared_ptr& searcher, + const TQueryOptions& query_options); + ~ConjunctionQuery() override; - void add(const std::wstring& field_name, const std::vector& terms); - void search(roaring::Roaring& roaring); + void add(const std::wstring& field_name, const std::vector& terms) override; + void search(roaring::Roaring& roaring) override; private: void search_by_bitmap(roaring::Roaring& roaring); @@ -47,7 +39,9 @@ class ConjunctionQuery { int32_t do_next(int32_t doc); - IndexReader* _reader = nullptr; +public: + std::shared_ptr _searcher; + IndexVersion _index_version = IndexVersion::kV0; int32_t _conjunction_ratio = 1000; bool _use_skip = false; @@ -60,4 +54,4 @@ class ConjunctionQuery { std::vector _term_docs; }; -} // namespace doris \ No newline at end of file +} // namespace doris::segment_v2 \ No newline at end of file diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/disjunction_query.cpp b/be/src/olap/rowset/segment_v2/inverted_index/query/disjunction_query.cpp index 7b797d7b54a91e..0514e1a372a4bb 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index/query/disjunction_query.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/disjunction_query.cpp @@ -17,9 +17,11 @@ #include "disjunction_query.h" -namespace doris { +namespace doris::segment_v2 { -DisjunctionQuery::DisjunctionQuery(IndexReader* reader) : _reader(reader) {} +DisjunctionQuery::DisjunctionQuery(const std::shared_ptr& searcher, + const TQueryOptions& query_options) + : _searcher(searcher) {} DisjunctionQuery::~DisjunctionQuery() { for (auto& term_doc : _term_docs) { @@ -43,7 +45,7 @@ void DisjunctionQuery::add(const std::wstring& field_name, const std::vectortermDocs(t); + TermDocs* term_doc = _searcher->getReader()->termDocs(t); _term_docs.push_back(term_doc); _term_iterators.emplace_back(term_doc); } @@ -77,4 +79,4 @@ void DisjunctionQuery::search(roaring::Roaring& roaring) { } } -} // namespace doris \ No newline at end of file +} // namespace doris::segment_v2 \ No newline at end of file diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/disjunction_query.h b/be/src/olap/rowset/segment_v2/inverted_index/query/disjunction_query.h index bb0a837f42a313..9a1e5df759cb22 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index/query/disjunction_query.h +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/disjunction_query.h @@ -17,31 +17,28 @@ #pragma once -#include -#include -#include -#include -#include - -#include "roaring/roaring.hh" +#include "olap/rowset/segment_v2/inverted_index/query/query.h" CL_NS_USE(index) +CL_NS_USE(search) -namespace doris { +namespace doris::segment_v2 { -class DisjunctionQuery { +class DisjunctionQuery : public Query { public: - DisjunctionQuery(IndexReader* reader); - ~DisjunctionQuery(); + DisjunctionQuery(const std::shared_ptr& searcher, + const TQueryOptions& query_options); + ~DisjunctionQuery() override; - void add(const std::wstring& field_name, const std::vector& terms); - void search(roaring::Roaring& roaring); + void add(const std::wstring& field_name, const std::vector& terms) override; + void search(roaring::Roaring& roaring) override; private: - IndexReader* _reader = nullptr; + std::shared_ptr _searcher; + std::vector _terms; std::vector _term_docs; std::vector _term_iterators; }; -} // namespace doris \ No newline at end of file +} // namespace doris::segment_v2 \ No newline at end of file diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.cpp b/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.cpp index 4b0340cda4a011..7920336c752765 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.cpp @@ -19,24 +19,25 @@ #include "olap/rowset//segment_v2/inverted_index/query/prefix_query.h" -namespace doris { +namespace doris::segment_v2 { -namespace segment_v2 { - -PhrasePrefixQuery::PhrasePrefixQuery(const std::shared_ptr& searcher) - : _searcher(searcher) {} +PhrasePrefixQuery::PhrasePrefixQuery(const std::shared_ptr& searcher, + const TQueryOptions& query_options) + : _searcher(searcher), + _query(std::make_unique()), + _max_expansions(query_options.inverted_index_max_expansions) {} void PhrasePrefixQuery::add(const std::wstring& field_name, const std::vector& terms) { if (terms.empty()) { - return; + _CLTHROWA(CL_ERR_IllegalArgument, "PhrasePrefixQuery::add: terms empty"); } for (size_t i = 0; i < terms.size(); i++) { if (i < terms.size() - 1) { std::wstring ws = StringUtil::string_to_wstring(terms[i]); Term* t = _CLNEW Term(field_name.c_str(), ws.c_str()); - _query.add(t); - _CLDECDELETE(t); + _query->add(t); + _CLLDECDELETE(t); } else { std::vector prefix_terms; PrefixQuery::get_prefix_terms(_searcher->getReader(), field_name, terms[i], @@ -44,20 +45,18 @@ void PhrasePrefixQuery::add(const std::wstring& field_name, const std::vectoradd(prefix_terms); for (auto& t : prefix_terms) { - _CLDECDELETE(t); + _CLLDECDELETE(t); } } } } void PhrasePrefixQuery::search(roaring::Roaring& roaring) { - _searcher->_search(&_query, [&roaring](const int32_t docid, const float_t /*score*/) { + _searcher->_search(_query.get(), [&roaring](const int32_t docid, const float_t /*score*/) { roaring.add(docid); }); } -} // namespace segment_v2 - -} // namespace doris \ No newline at end of file +} // namespace doris::segment_v2 \ No newline at end of file diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.h b/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.h index 28007620ce581e..e565c0409cf4cd 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.h +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.h @@ -17,38 +17,31 @@ #pragma once -#include -#include - #include +// clang-format off +#include "olap/rowset/segment_v2/inverted_index/query/query.h" #include "CLucene/search/MultiPhraseQuery.h" -#include "roaring/roaring.hh" +// clang-format on -CL_NS_USE(index) CL_NS_USE(search) -namespace doris { - -namespace segment_v2 { +namespace doris::segment_v2 { -class PhrasePrefixQuery { +class PhrasePrefixQuery : public Query { public: - PhrasePrefixQuery(const std::shared_ptr& searcher); - ~PhrasePrefixQuery() = default; + PhrasePrefixQuery(const std::shared_ptr& searcher, + const TQueryOptions& query_options); + ~PhrasePrefixQuery() override = default; - void set_max_expansions(int32_t max_expansions) { _max_expansions = max_expansions; } - - void add(const std::wstring& field_name, const std::vector& terms); - void search(roaring::Roaring& roaring); + void add(const std::wstring& field_name, const std::vector& terms) override; + void search(roaring::Roaring& roaring) override; private: std::shared_ptr _searcher; - MultiPhraseQuery _query; + std::unique_ptr _query; int32_t _max_expansions = 50; }; -} // namespace segment_v2 - -} // namespace doris \ No newline at end of file +} // namespace doris::segment_v2 \ No newline at end of file diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_query.cpp b/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_query.cpp index 527e89a8878127..a4b7f7502d158f 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_query.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_query.cpp @@ -19,6 +19,10 @@ namespace doris::segment_v2 { +PhraseQuery::PhraseQuery(const std::shared_ptr& searcher, + const TQueryOptions& query_options) + : _searcher(searcher) {} + PhraseQuery::~PhraseQuery() { for (auto& term_doc : _term_docs) { if (term_doc) { diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_query.h b/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_query.h index f4b464ce3586f7..8f62989d86b619 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_query.h +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_query.h @@ -17,28 +17,21 @@ #pragma once -#include -#include -#include -#include -#include - -#include - -#include "roaring/roaring.hh" +#include "olap/rowset/segment_v2/inverted_index/query/query.h" CL_NS_USE(index) +CL_NS_USE(search) namespace doris::segment_v2 { -class PhraseQuery { +class PhraseQuery : public Query { public: - PhraseQuery(const std::shared_ptr& searcher) - : _searcher(searcher) {} - ~PhraseQuery(); + PhraseQuery(const std::shared_ptr& searcher, + const TQueryOptions& query_options); + ~PhraseQuery() override; - void add(const std::wstring& field_name, const std::vector& terms); - void search(roaring::Roaring& roaring); + void add(const std::wstring& field_name, const std::vector& terms) override; + void search(roaring::Roaring& roaring) override; private: class PostingsAndPosition { diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/prefix_query.cpp b/be/src/olap/rowset/segment_v2/inverted_index/query/prefix_query.cpp index 7d23d6eb60f348..1400622735214f 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index/query/prefix_query.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/prefix_query.cpp @@ -17,7 +17,7 @@ #include "prefix_query.h" -namespace doris { +namespace doris::segment_v2 { void PrefixQuery::get_prefix_terms(IndexReader* reader, const std::wstring& field_name, const std::string& prefix, @@ -77,4 +77,4 @@ void PrefixQuery::get_prefix_terms(IndexReader* reader, const std::wstring& fiel }); } -} // namespace doris \ No newline at end of file +} // namespace doris::segment_v2 \ No newline at end of file diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/prefix_query.h b/be/src/olap/rowset/segment_v2/inverted_index/query/prefix_query.h index 5deb0c1a3628ad..9a33b13dd8da76 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index/query/prefix_query.h +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/prefix_query.h @@ -24,12 +24,12 @@ CL_NS_USE(index) -namespace doris { +namespace doris::segment_v2 { class PrefixQuery { public: PrefixQuery() = default; - ~PrefixQuery() = default; + virtual ~PrefixQuery() = default; static void get_prefix_terms(IndexReader* reader, const std::wstring& field_name, const std::string& prefix, @@ -37,4 +37,4 @@ class PrefixQuery { int32_t max_expansions = 50); }; -} // namespace doris \ No newline at end of file +} // namespace doris::segment_v2 \ No newline at end of file diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/query.h b/be/src/olap/rowset/segment_v2/inverted_index/query/query.h new file mode 100644 index 00000000000000..091ba7d395804c --- /dev/null +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/query.h @@ -0,0 +1,51 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include // IWYU pragma: keep +#include +#include +#include +#include +#include + +#include + +#include "roaring/roaring.hh" + +CL_NS_USE(index) +CL_NS_USE(search) +CL_NS_USE(util) + +namespace doris::segment_v2 { + +class Query { +public: + virtual ~Query() = default; + + // a unified data preparation interface that provides the field names to be queried and the terms for the query. + // @param field_name The name of the field within the data source to search against. + // @param terms a vector of tokenized strings that represent the search terms. + virtual void add(const std::wstring& field_name, const std::vector& terms) = 0; + + // a unified query interface for retrieving the ids obtained from the search. + // @param roaring a Roaring bitmap to be populated with the search results, + virtual void search(roaring::Roaring& roaring) = 0; +}; + +} // namespace doris::segment_v2 \ No newline at end of file diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/query_factory.h b/be/src/olap/rowset/segment_v2/inverted_index/query/query_factory.h new file mode 100644 index 00000000000000..6de8a7e4c250a2 --- /dev/null +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/query_factory.h @@ -0,0 +1,53 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include "olap/rowset/segment_v2/inverted_index_query_type.h" + +namespace doris::segment_v2 { + +class Query; +class DisjunctionQuery; +class ConjunctionQuery; +class PhraseQuery; +class PhrasePrefixQuery; +class RegexpQuery; + +class QueryFactory { +public: + template + static std::unique_ptr create(InvertedIndexQueryType query_type, Args&&... args) { + switch (query_type) { + case InvertedIndexQueryType::MATCH_ANY_QUERY: + case InvertedIndexQueryType::EQUAL_QUERY: + return std::make_unique(std::forward(args)...); + case InvertedIndexQueryType::MATCH_ALL_QUERY: + return std::make_unique(std::forward(args)...); + case InvertedIndexQueryType::MATCH_PHRASE_QUERY: + return std::make_unique(std::forward(args)...); + case InvertedIndexQueryType::MATCH_PHRASE_PREFIX_QUERY: + return std::make_unique(std::forward(args)...); + case InvertedIndexQueryType::MATCH_REGEXP_QUERY: + return std::make_unique(std::forward(args)...); + default: + return nullptr; + } + } +}; + +} // namespace doris::segment_v2 \ No newline at end of file diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/regexp_query.cpp b/be/src/olap/rowset/segment_v2/inverted_index/query/regexp_query.cpp index ee959e1da7367e..007da8289dcdb0 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index/query/regexp_query.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/regexp_query.cpp @@ -24,10 +24,19 @@ namespace doris::segment_v2 { -RegexpQuery::RegexpQuery(const std::shared_ptr& searcher) - : _searcher(searcher), query(searcher->getReader()) {} +RegexpQuery::RegexpQuery(const std::shared_ptr& searcher, + const TQueryOptions& query_options) + : _searcher(searcher), + _max_expansions(query_options.inverted_index_max_expansions), + _query(searcher, query_options) {} + +void RegexpQuery::add(const std::wstring& field_name, const std::vector& patterns) { + if (patterns.size() != 1) { + _CLTHROWA(CL_ERR_IllegalArgument, "RegexpQuery::add: terms size != 1"); + } + + const std::string& pattern = patterns[0]; -void RegexpQuery::add(const std::wstring& field_name, const std::string& pattern) { hs_database_t* database = nullptr; hs_compile_error_t* compile_err = nullptr; hs_scratch_t* scratch = nullptr; @@ -94,11 +103,11 @@ void RegexpQuery::add(const std::wstring& field_name, const std::string& pattern return; } - query.add(field_name, terms); + _query.add(field_name, terms); } void RegexpQuery::search(roaring::Roaring& roaring) { - query.search(roaring); + _query.search(roaring); } } // namespace doris::segment_v2 diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/regexp_query.h b/be/src/olap/rowset/segment_v2/inverted_index/query/regexp_query.h index 3791ad50d8f78f..336b2d0b6a671d 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index/query/regexp_query.h +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/regexp_query.h @@ -17,30 +17,28 @@ #pragma once -#include - #include "olap/rowset/segment_v2/inverted_index/query/disjunction_query.h" +#include "olap/rowset/segment_v2/inverted_index/query/query.h" CL_NS_USE(index) CL_NS_USE(search) namespace doris::segment_v2 { -class RegexpQuery { +class RegexpQuery : public Query { public: - RegexpQuery(const std::shared_ptr& searcher); - ~RegexpQuery() = default; - - void set_max_expansions(int32_t max_expansions) { _max_expansions = max_expansions; } + RegexpQuery(const std::shared_ptr& searcher, + const TQueryOptions& query_options); + ~RegexpQuery() override = default; - void add(const std::wstring& field_name, const std::string& pattern); - void search(roaring::Roaring& roaring); + void add(const std::wstring& field_name, const std::vector& patterns) override; + void search(roaring::Roaring& roaring) override; private: std::shared_ptr _searcher; int32_t _max_expansions = 50; - DisjunctionQuery query; + DisjunctionQuery _query; }; } // namespace doris::segment_v2 diff --git a/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp b/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp index 5eaacb4640f7a5..172679f1fa6f3d 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp @@ -61,6 +61,7 @@ #include "olap/rowset/segment_v2/inverted_index/query/conjunction_query.h" #include "olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.h" #include "olap/rowset/segment_v2/inverted_index/query/phrase_query.h" +#include "olap/rowset/segment_v2/inverted_index/query/query_factory.h" #include "olap/rowset/segment_v2/inverted_index/query/regexp_query.h" #include "olap/rowset/segment_v2/inverted_index_cache.h" #include "olap/rowset/segment_v2/inverted_index_compound_directory.h" @@ -328,20 +329,10 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, RuntimeState* run std::get_if(&searcher_variant)) { term_match_bitmap = std::make_shared(); - Status res = Status::OK(); - if (query_type == InvertedIndexQueryType::MATCH_PHRASE_QUERY) { - res = match_phrase_index_search(stats, runtime_state, field_ws, - analyse_result, *searcher_ptr, - term_match_bitmap); - } else if (query_type == InvertedIndexQueryType::MATCH_PHRASE_PREFIX_QUERY) { - res = match_phrase_prefix_index_search(stats, runtime_state, field_ws, - analyse_result, *searcher_ptr, - term_match_bitmap); - } else { - res = match_all_index_search(stats, runtime_state, field_ws, analyse_result, - *searcher_ptr, term_match_bitmap); - } - if (!res.ok()) { + Status res = + match_index_search(stats, runtime_state, query_type, field_ws, + analyse_result, *searcher_ptr, term_match_bitmap); + if (!res) { return res; } @@ -352,8 +343,6 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, RuntimeState* run } query_match_bitmap = *term_match_bitmap; } else if (query_type == InvertedIndexQueryType::MATCH_REGEXP_QUERY) { - const std::string& pattern = analyse_result[0]; - std::shared_ptr term_match_bitmap = nullptr; auto* cache = InvertedIndexQueryCache::instance(); @@ -361,7 +350,7 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, RuntimeState* run cache_key.index_path = index_file_path; cache_key.column_name = column_name; cache_key.query_type = query_type; - cache_key.value = pattern; + cache_key.value = analyse_result[0]; InvertedIndexQueryCacheHandle cache_handle; if (cache->lookup(cache_key, &cache_handle)) { stats->inverted_index_query_cache_hit++; @@ -377,8 +366,9 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, RuntimeState* run std::get_if(&searcher_variant)) { term_match_bitmap = std::make_shared(); - Status res = match_regexp_index_search(stats, runtime_state, field_ws, pattern, - *searcher_ptr, term_match_bitmap); + Status res = + match_index_search(stats, runtime_state, query_type, field_ws, + analyse_result, *searcher_ptr, term_match_bitmap); if (!res.ok()) { return res; } @@ -499,73 +489,21 @@ Status FullTextIndexReader::normal_index_search( return Status::OK(); } -Status FullTextIndexReader::match_all_index_search( - OlapReaderStatistics* stats, RuntimeState* runtime_state, const std::wstring& field_ws, - const std::vector& analyse_result, - const FulltextIndexSearcherPtr& index_searcher, - const std::shared_ptr& term_match_bitmap) { - TQueryOptions queryOptions = runtime_state->query_options(); - try { - SCOPED_RAW_TIMER(&stats->inverted_index_searcher_search_timer); - ConjunctionQuery query(index_searcher->getReader()); - query.set_conjunction_ratio(queryOptions.inverted_index_conjunction_opt_threshold); - query.add(field_ws, analyse_result); - query.search(*term_match_bitmap); - } catch (const CLuceneError& e) { - return Status::Error("CLuceneError occured: {}", - e.what()); - } - return Status::OK(); -} - -Status FullTextIndexReader::match_phrase_index_search( - OlapReaderStatistics* stats, RuntimeState* runtime_state, const std::wstring& field_ws, - const std::vector& analyse_result, - const FulltextIndexSearcherPtr& index_searcher, - const std::shared_ptr& term_match_bitmap) { - TQueryOptions queryOptions = runtime_state->query_options(); - try { - SCOPED_RAW_TIMER(&stats->inverted_index_searcher_search_timer); - PhraseQuery query(index_searcher); - query.add(field_ws, analyse_result); - query.search(*term_match_bitmap); - } catch (const CLuceneError& e) { - return Status::Error("CLuceneError occured: {}", - e.what()); - } - return Status::OK(); -} - -Status FullTextIndexReader::match_phrase_prefix_index_search( - OlapReaderStatistics* stats, RuntimeState* runtime_state, const std::wstring& field_ws, - const std::vector& analyse_result, +Status FullTextIndexReader::match_index_search( + OlapReaderStatistics* stats, RuntimeState* runtime_state, InvertedIndexQueryType query_type, + const std::wstring& field_ws, const std::vector& analyse_result, const FulltextIndexSearcherPtr& index_searcher, const std::shared_ptr& term_match_bitmap) { TQueryOptions queryOptions = runtime_state->query_options(); try { SCOPED_RAW_TIMER(&stats->inverted_index_searcher_search_timer); - PhrasePrefixQuery query(index_searcher); - query.set_max_expansions(queryOptions.inverted_index_max_expansions); - query.add(field_ws, analyse_result); - query.search(*term_match_bitmap); - } catch (const CLuceneError& e) { - return Status::Error("CLuceneError occured: {}", - e.what()); - } - return Status::OK(); -} - -Status FullTextIndexReader::match_regexp_index_search( - OlapReaderStatistics* stats, RuntimeState* runtime_state, const std::wstring& field_ws, - const std::string& pattern, const FulltextIndexSearcherPtr& index_searcher, - const std::shared_ptr& term_match_bitmap) { - TQueryOptions queryOptions = runtime_state->query_options(); - try { - SCOPED_RAW_TIMER(&stats->inverted_index_searcher_search_timer); - RegexpQuery query(index_searcher); - query.set_max_expansions(queryOptions.inverted_index_max_expansions); - query.add(field_ws, pattern); - query.search(*term_match_bitmap); + auto query = QueryFactory::create(query_type, index_searcher, queryOptions); + if (!query) { + return Status::Error( + "query type " + query_type_to_string(query_type) + ", query is nullptr"); + } + query->add(field_ws, analyse_result); + query->search(*term_match_bitmap); } catch (const CLuceneError& e) { return Status::Error("CLuceneError occured: {}", e.what()); diff --git a/be/src/olap/rowset/segment_v2/inverted_index_reader.h b/be/src/olap/rowset/segment_v2/inverted_index_reader.h index 0d81022d5ba1d9..430ec9173292a2 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_reader.h +++ b/be/src/olap/rowset/segment_v2/inverted_index_reader.h @@ -161,28 +161,11 @@ class FullTextIndexReader : public InvertedIndexReader { const std::unique_ptr& query, const std::shared_ptr& term_match_bitmap); - Status match_all_index_search(OlapReaderStatistics* stats, RuntimeState* runtime_state, - const std::wstring& field_ws, - const std::vector& analyse_result, - const FulltextIndexSearcherPtr& index_searcher, - const std::shared_ptr& term_match_bitmap); - - Status match_phrase_index_search(OlapReaderStatistics* stats, RuntimeState* runtime_state, - const std::wstring& field_ws, - const std::vector& analyse_result, - const FulltextIndexSearcherPtr& index_searcher, - const std::shared_ptr& term_match_bitmap); - - Status match_phrase_prefix_index_search( - OlapReaderStatistics* stats, RuntimeState* runtime_state, const std::wstring& field_ws, - const std::vector& analyse_result, - const FulltextIndexSearcherPtr& index_searcher, - const std::shared_ptr& term_match_bitmap); - - Status match_regexp_index_search(OlapReaderStatistics* stats, RuntimeState* runtime_state, - const std::wstring& field_ws, const std::string& pattern, - const FulltextIndexSearcherPtr& index_searcher, - const std::shared_ptr& term_match_bitmap); + Status match_index_search(OlapReaderStatistics* stats, RuntimeState* runtime_state, + InvertedIndexQueryType query_type, const std::wstring& field_ws, + const std::vector& analyse_result, + const FulltextIndexSearcherPtr& index_searcher, + const std::shared_ptr& term_match_bitmap); void check_null_bitmap(const FulltextIndexSearcherPtr& index_searcher, bool& null_bitmap_already_read); diff --git a/docs/en/docs/data-table/index/inverted-index.md b/docs/en/docs/data-table/index/inverted-index.md index 75a8f6a3b1bc69..2f511f312f4a6e 100644 --- a/docs/en/docs/data-table/index/inverted-index.md +++ b/docs/en/docs/data-table/index/inverted-index.md @@ -51,6 +51,9 @@ The features for inverted index is as follows: - add fulltext search on text(string, varchar, char) field - MATCH_ALL matches all keywords, MATCH_ANY matches any keywords + - support phrase query MATCH_PHRASE + - support phrase + prefix query MATCH_PHRASE_PREFIX + - support regexp query MATCH_REGEXP - support fulltext on array of text field - support english, chinese and mixed unicode word parser - accelerate normal equal, range query, replacing bitmap index in the future @@ -181,6 +184,15 @@ SELECT * FROM table_name WHERE logmsg MATCH_ALL 'keyword1 keyword2'; -- 1.4 find rows that logmsg contains both keyword1 and keyword2, and in the order of keyword1 appearing first and keyword2 appearing later. SELECT * FROM table_name WHERE logmsg MATCH_PHRASE 'keyword1 keyword2'; +-- 1.5 perform prefix matching on the last word "keyword2" while maintaining the order of words, defaulting to finding 50 prefix words (controlled by the session variable inverted_index_max_expansions) +SELECT * FROM table_name WHERE logmsg MATCH_PHRASE_PREFIX 'keyword1 keyword2'; + +-- 1.6 If only one word is entered, it degrades to a prefix query, defaulting to finding 50 prefix words (controlled by the session variable inverted_index_max_expansions) +SELECT * FROM table_name WHERE logmsg MATCH_PHRASE_PREFIX 'keyword1'; + +-- 1.7 perform regex matching on the tokenized words, defaulting to matching 50 tokens (controlled by the session variable inverted_index_max_expansions) +SELECT * FROM table_name WHERE logmsg MATCH_REGEXP 'key*'; + -- 2. normal equal, range query SELECT * FROM table_name WHERE id = 123; SELECT * FROM table_name WHERE ts > '2023-01-01 00:00:00'; diff --git a/docs/zh-CN/docs/data-table/index/inverted-index.md b/docs/zh-CN/docs/data-table/index/inverted-index.md index 2f4c3f85bfee19..ff29c75d6db70e 100644 --- a/docs/zh-CN/docs/data-table/index/inverted-index.md +++ b/docs/zh-CN/docs/data-table/index/inverted-index.md @@ -50,7 +50,10 @@ under the License. Doris倒排索引的功能简要介绍如下: - 增加了字符串类型的全文检索 - - 支持字符串全文检索,包括同时匹配多个关键字MATCH_ALL、匹配任意一个关键字MATCH_ANY、匹配短语词组MATCH_PHRASE + - 支持字符串全文检索,包括同时匹配多个关键字MATCH_ALL、匹配任意一个关键字MATCH_ANY + - 支持短语查询 MATCH_PHRASE + - 支持短语+前缀 MATCH_PHRASE_PREFIX + - 支持正则查询 MATCH_REGEXP - 支持字符串数组类型的全文检索 - 支持英文、中文以及Unicode多语言分词 - 加速普通等值、范围查询,覆盖bitmap索引的功能,未来会代替bitmap索引 @@ -179,6 +182,15 @@ SELECT * FROM table_name WHERE logmsg MATCH_ALL 'keyword1 keyword2'; -- 1.4 logmsg中同时包含keyword1和keyword2的行,并且按照keyword1在前,keyword2在后的顺序 SELECT * FROM table_name WHERE logmsg MATCH_PHRASE 'keyword1 keyword2'; +-- 1.5 在保持词顺序的前提下,对最后一个词keyword2做前缀匹配,默认找50个前缀词(session变量inverted_index_max_expansions控制) +SELECT * FROM table_name WHERE logmsg MATCH_PHRASE_PREFIX 'keyword1 keyword2'; + +-- 1.6 如果只填一个词会退化为前缀查询,默认找50个前缀词(session变量inverted_index_max_expansions控制) +SELECT * FROM table_name WHERE logmsg MATCH_PHRASE_PREFIX 'keyword1'; + +-- 1.7 对分词后的词进行正则匹配,默认匹配50个(session变量inverted_index_max_expansions控制) +SELECT * FROM table_name WHERE logmsg MATCH_REGEXP 'key*'; + -- 2. 普通等值、范围、IN、NOT IN,正常的SQL语句即可,例如 SELECT * FROM table_name WHERE id = 123; From 7d280b397d1f9f690bac968c060c923033a56a9a Mon Sep 17 00:00:00 2001 From: meiyi Date: Tue, 16 Jan 2024 15:08:41 +0800 Subject: [PATCH 023/200] (fix)[group-commit] Fix some group commit case (#30008) --- be/src/runtime/group_commit_mgr.cpp | 3 +- .../apache/doris/analysis/ArrayLiteral.java | 16 ++++++++ .../java/org/apache/doris/analysis/Expr.java | 4 ++ .../apache/doris/analysis/LiteralExpr.java | 12 ++++++ .../doris/analysis/NativeInsertStmt.java | 39 +++++++++++-------- .../apache/doris/analysis/StructLiteral.java | 7 ++++ .../commands/InsertIntoTableCommand.java | 9 +++-- .../org/apache/doris/qe/StmtExecutor.java | 23 ++++++----- 8 files changed, 84 insertions(+), 29 deletions(-) diff --git a/be/src/runtime/group_commit_mgr.cpp b/be/src/runtime/group_commit_mgr.cpp index 692f7c6846b467..3931306cd6fd1c 100644 --- a/be/src/runtime/group_commit_mgr.cpp +++ b/be/src/runtime/group_commit_mgr.cpp @@ -226,7 +226,8 @@ Status GroupCommitTable::get_first_block_load_queue( } } } - return Status::InternalError("can not get a block queue"); + return Status::InternalError("can not get a block queue for table_id: " + + std::to_string(_table_id)); } Status GroupCommitTable::_create_group_commit_load( diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ArrayLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ArrayLiteral.java index 540a0747b49452..5b4b5a9109878f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ArrayLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ArrayLiteral.java @@ -148,6 +148,22 @@ public String getStringValueInFe() { return "[" + StringUtils.join(list, ", ") + "]"; } + @Override + public String getStringValueForStreamLoad() { + List list = new ArrayList<>(children.size()); + children.forEach(v -> { + String stringLiteral; + if (v instanceof NullLiteral) { + stringLiteral = "null"; + } else { + stringLiteral = getStringLiteralForStreamLoad(v); + } + // we should use type to decide we output array is suitable for json format + list.add(stringLiteral); + }); + return "[" + StringUtils.join(list, ", ") + "]"; + } + @Override protected void toThrift(TExprNode msg) { msg.node_type = TExprNodeType.ARRAY_LITERAL; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java index 869ace0ef14c08..adb7621a3466d2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java @@ -2243,6 +2243,10 @@ public String getStringValueInFe() { return getStringValue(); } + public String getStringValueForStreamLoad() { + return getStringValue(); + } + // A special method only for array literal, all primitive type in array // will be wrapped by double quote. eg: // ["1", "2", "3"] diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/LiteralExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/LiteralExpr.java index c222070e284c7e..ce89b2fc3c9149 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/LiteralExpr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/LiteralExpr.java @@ -123,6 +123,18 @@ public static String getStringLiteralForComplexType(Expr v) { } } + public static String getStringLiteralForStreamLoad(Expr v) { + if (!(v instanceof NullLiteral) && v.getType().isScalarType() + && (Type.getNumericTypes().contains((ScalarType) v.getActualScalarType(v.getType())) + || v.getType() == Type.BOOLEAN)) { + return v.getStringValueInFe(); + } else if (v.getType().isComplexType()) { + // these type should also call getStringValueInFe which should handle special case for itself + return v.getStringValueForStreamLoad(); + } else { + return v.getStringValueForArray(); + } + } /** * Init LiteralExpr's Type information diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java index 41e20df9adcf34..17d3a9ffd0094d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java @@ -39,6 +39,7 @@ import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.FeConstants; import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.util.DebugUtil; @@ -1142,6 +1143,7 @@ public void analyzeGroupCommit(Analyzer analyzer) throws AnalysisException { && ConnectContext.get().getSessionVariable().getSqlMode() != SqlModeHelper.MODE_NO_BACKSLASH_ESCAPES && targetTable instanceof OlapTable && ((OlapTable) targetTable).getTableProperty().getUseSchemaLightChange() + && !targetTable.getQualifiedDbName().equalsIgnoreCase(FeConstants.INTERNAL_DB_NAME) && !ConnectContext.get().isTxnModel() && getQueryStmt() instanceof SelectStmt && ((SelectStmt) getQueryStmt()).getTableRefs().isEmpty() && targetPartitionNames == null @@ -1187,24 +1189,29 @@ public boolean isReuseGroupCommitPlan() { public GroupCommitPlanner planForGroupCommit(TUniqueId queryId) throws UserException, TException { OlapTable olapTable = (OlapTable) getTargetTable(); - if (groupCommitPlanner != null && olapTable.getBaseSchemaVersion() == baseSchemaVersion) { - LOG.debug("reuse group commit plan, table={}", olapTable); - reuseGroupCommitPlan = true; + olapTable.readLock(); + try { + if (groupCommitPlanner != null && olapTable.getBaseSchemaVersion() == baseSchemaVersion) { + LOG.debug("reuse group commit plan, table={}", olapTable); + reuseGroupCommitPlan = true; + return groupCommitPlanner; + } + reuseGroupCommitPlan = false; + if (!targetColumns.isEmpty()) { + Analyzer analyzerTmp = analyzer; + reset(); + this.analyzer = analyzerTmp; + } + analyzeSubquery(analyzer, true); + groupCommitPlanner = new GroupCommitPlanner((Database) db, olapTable, targetColumnNames, queryId, + ConnectContext.get().getSessionVariable().getGroupCommit()); + // save plan message to be reused for prepare stmt + loadId = queryId; + baseSchemaVersion = olapTable.getBaseSchemaVersion(); return groupCommitPlanner; + } finally { + olapTable.readUnlock(); } - reuseGroupCommitPlan = false; - if (!targetColumns.isEmpty()) { - Analyzer analyzerTmp = analyzer; - reset(); - this.analyzer = analyzerTmp; - } - analyzeSubquery(analyzer, true); - groupCommitPlanner = new GroupCommitPlanner((Database) db, olapTable, targetColumnNames, queryId, - ConnectContext.get().getSessionVariable().getGroupCommit()); - // save plan message to be reused for prepare stmt - loadId = queryId; - baseSchemaVersion = olapTable.getBaseSchemaVersion(); - return groupCommitPlanner; } public TUniqueId getLoadId() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/StructLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/StructLiteral.java index ac67e0c16a75d9..c96797b5fef463 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/StructLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/StructLiteral.java @@ -125,6 +125,13 @@ public String getStringValueInFe() { return "{" + StringUtils.join(list, ", ") + "}"; } + @Override + public String getStringValueForStreamLoad() { + List list = new ArrayList<>(children.size()); + children.forEach(v -> list.add(getStringLiteralForComplexType(v))); + return "{" + StringUtils.join(list, ", ") + "}"; + } + @Override protected void toThrift(TExprNode msg) { msg.node_type = TExprNodeType.STRUCT_LITERAL; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java index c1ec90fff73bdf..d5101aabcdac23 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java @@ -25,6 +25,7 @@ import org.apache.doris.catalog.TableIf; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.FeConstants; import org.apache.doris.common.UserException; import org.apache.doris.common.util.ProfileManager.ProfileType; import org.apache.doris.load.loadv2.LoadStatistic; @@ -251,10 +252,12 @@ private boolean analyzeGroupCommit(ConnectContext ctx, DataSink sink, || ctx.getSessionVariable().isEnableUniqueKeyPartialUpdate()) { return false; } + OlapTable targetTable = physicalOlapTableSink.getTargetTable(); return ctx.getSessionVariable().getSqlMode() != SqlModeHelper.MODE_NO_BACKSLASH_ESCAPES - && physicalOlapTableSink.getTargetTable() instanceof OlapTable && !ctx.isTxnModel() - && sink.getFragment().getPlanRoot() instanceof UnionNode && physicalOlapTableSink.getPartitionIds() - .isEmpty() && physicalOlapTableSink.getTargetTable().getTableProperty().getUseSchemaLightChange(); + && !ctx.isTxnModel() && sink.getFragment().getPlanRoot() instanceof UnionNode + && physicalOlapTableSink.getPartitionIds().isEmpty() && targetTable.getTableProperty() + .getUseSchemaLightChange() && !targetTable.getQualifiedDbName() + .equalsIgnoreCase(FeConstants.INTERNAL_DB_NAME); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java index d97bf54c5021b0..9c234698865f74 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -295,9 +295,9 @@ public static InternalService.PDataRow getRowStringValue(List cols) throws if (expr instanceof NullLiteral) { row.addColBuilder().setValue(NULL_VALUE_FOR_LOAD); } else if (expr instanceof ArrayLiteral) { - row.addColBuilder().setValue(String.format("\"%s\"", expr.getStringValueInFe())); + row.addColBuilder().setValue(String.format("\"%s\"", expr.getStringValueForStreamLoad())); } else { - String stringValue = expr.getStringValueInFe(); + String stringValue = expr.getStringValueForStreamLoad(); if (stringValue.equals(NULL_VALUE_FOR_LOAD) || stringValue.startsWith("\"") || stringValue.endsWith( "\"")) { row.addColBuilder().setValue(String.format("\"%s\"", stringValue)); @@ -1944,6 +1944,8 @@ private void handleInsertStmt() throws Exception { } else if (insertStmt instanceof NativeInsertStmt && ((NativeInsertStmt) insertStmt).isGroupCommit()) { isGroupCommit = true; NativeInsertStmt nativeInsertStmt = (NativeInsertStmt) insertStmt; + long dbId = nativeInsertStmt.getTargetTable().getDatabase().getId(); + long tableId = nativeInsertStmt.getTargetTable().getId(); int maxRetry = 3; for (int i = 0; i < maxRetry; i++) { GroupCommitPlanner groupCommitPlanner = nativeInsertStmt.planForGroupCommit(context.queryId); @@ -1954,10 +1956,11 @@ private void handleInsertStmt() throws Exception { ProtocolStringList errorMsgsList = response.getStatus().getErrorMsgsList(); if (code == TStatusCode.DATA_QUALITY_ERROR && !errorMsgsList.isEmpty() && errorMsgsList.get(0) .contains("schema version not match")) { - LOG.info("group commit insert failed. stmt: {}, backend id: {}, status: {}, " - + "schema version: {}, retry: {}", insertStmt.getOrigStmt().originStmt, - groupCommitPlanner.getBackend().getId(), - response.getStatus(), nativeInsertStmt.getBaseSchemaVersion(), i); + LOG.info("group commit insert failed. stmt: {}, query_id: {}, db_id: {}, table_id: {}" + + ", schema version: {}, backend_id: {}, status: {}, retry: {}", + insertStmt.getOrigStmt().originStmt, DebugUtil.printId(context.queryId()), dbId, tableId, + nativeInsertStmt.getBaseSchemaVersion(), groupCommitPlanner.getBackend().getId(), + response.getStatus(), i); if (i < maxRetry) { List tables = Lists.newArrayList(insertStmt.getTargetTable()); MetaLockUtils.readLockTables(tables); @@ -1970,12 +1973,14 @@ private void handleInsertStmt() throws Exception { } continue; } else { - errMsg = "group commit insert failed. backend id: " + errMsg = "group commit insert failed. db_id: " + dbId + ", table_id: " + tableId + + ", query_id: " + DebugUtil.printId(context.queryId()) + ", backend_id: " + groupCommitPlanner.getBackend().getId() + ", status: " + response.getStatus(); } } else if (code != TStatusCode.OK) { - errMsg = "group commit insert failed. backend id: " + groupCommitPlanner.getBackend().getId() - + ", status: " + response.getStatus(); + errMsg = "group commit insert failed. db_id: " + dbId + ", table_id: " + tableId + ", query_id: " + + DebugUtil.printId(context.queryId()) + ", backend_id: " + groupCommitPlanner.getBackend() + .getId() + ", status: " + response.getStatus(); ErrorReport.reportDdlException(errMsg, ErrorCode.ERR_FAILED_WHEN_INSERT); } label = response.getLabel(); From 6e023e801d1ea752f3e5e7c48de696b780a08edd Mon Sep 17 00:00:00 2001 From: minghong Date: Tue, 16 Jan 2024 15:22:36 +0800 Subject: [PATCH 024/200] [opt](nereids) if column stats are unknown, 10-20 table-join optimization use cascading instead of dphyp (#29902) * if column stats are unknown, do not use dphyp tpcds query64 is optimized in case of no stats sf500, query64 improved from 15sec to 7sec on hdfs, and from 4sec to 3.85sec on olaptable --- .../doris/nereids/StatementContext.java | 15 ++ .../nereids/jobs/executor/Optimizer.java | 14 +- .../doris/nereids/stats/StatsCalculator.java | 7 + .../noStatsRfPrune/query64.out | 165 +++++++++--------- .../no_stats_shape/query64.out | 165 +++++++++--------- 5 files changed, 195 insertions(+), 171 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java index f2e5370d952a25..c4a6e37bbeca98 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java @@ -74,6 +74,13 @@ public class StatementContext { private boolean isDpHyp = false; private boolean isOtherJoinReorder = false; + // hasUnknownColStats true if any column stats in the tables used by this sql is unknown + // the algorithm to derive plan when column stats are unknown is implemented in cascading framework, not in dphyper. + // And hence, when column stats are unknown, even if the tables used by a sql is more than + // MAX_TABLE_COUNT_USE_CASCADES_JOIN_REORDER, join reorder should choose cascading framework. + // Thus hasUnknownColStats has higher priority than isDpHyp + private boolean hasUnknownColStats = false; + private final IdGenerator exprIdGenerator = ExprId.createGenerator(); private final IdGenerator objectIdGenerator = ObjectId.createGenerator(); private final IdGenerator relationIdGenerator = RelationId.createGenerator(); @@ -261,4 +268,12 @@ public List getJoinFilters() { public void addJoinFilters(Collection newJoinFilters) { this.joinFilters.addAll(newJoinFilters); } + + public boolean isHasUnknownColStats() { + return hasUnknownColStats; + } + + public void setHasUnknownColStats(boolean hasUnknownColStats) { + this.hasUnknownColStats = hasUnknownColStats; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Optimizer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Optimizer.java index 19dd3b00bba8bb..4f042c527cb9e0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Optimizer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Optimizer.java @@ -22,6 +22,7 @@ import org.apache.doris.nereids.jobs.cascades.OptimizeGroupJob; import org.apache.doris.nereids.jobs.joinorder.JoinOrderJob; import org.apache.doris.nereids.memo.Group; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SessionVariable; import java.util.Objects; @@ -49,11 +50,22 @@ public void execute() { cascadesContext.pushJob(new DeriveStatsJob(cascadesContext.getMemo().getRoot().getLogicalExpression(), cascadesContext.getCurrentJobContext())); cascadesContext.getJobScheduler().executeJobPool(cascadesContext); + boolean optimizeWithUnknownColStats = false; + if (ConnectContext.get() != null && ConnectContext.get().getStatementContext() != null) { + if (ConnectContext.get().getStatementContext().isHasUnknownColStats()) { + optimizeWithUnknownColStats = true; + } + } // DPHyp optimize + int maxTableCount = getSessionVariable().getMaxTableCountUseCascadesJoinReorder(); + if (optimizeWithUnknownColStats) { + // if column stats are unknown, 10~20 table-join is optimized by cascading framework + maxTableCount = 2 * maxTableCount; + } int maxJoinCount = cascadesContext.getMemo().countMaxContinuousJoin(); cascadesContext.getStatementContext().setMaxContinuousJoin(maxJoinCount); boolean isDpHyp = getSessionVariable().enableDPHypOptimizer - || maxJoinCount > getSessionVariable().getMaxTableCountUseCascadesJoinReorder(); + || maxJoinCount > maxTableCount; cascadesContext.getStatementContext().setDpHyp(isDpHyp); cascadesContext.getStatementContext().setOtherJoinReorder(false); if (!getSessionVariable().isDisableJoinReorder() && isDpHyp diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java index 9812dc0f0f6042..14ead9990aa4d7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java @@ -623,6 +623,7 @@ private Statistics computeCatalogRelation(CatalogRelation catalogRelation) { Map columnStatisticMap = new HashMap<>(); TableIf table = catalogRelation.getTable(); double rowCount = catalogRelation.getTable().estimatedRowCount(); + boolean hasUnknownCol = false; for (SlotReference slotReference : slotSet) { String colName = slotReference.getName(); boolean shouldIgnoreThisCol = StatisticConstants.shouldIgnoreCol(table, slotReference.getColumn().get()); @@ -644,13 +645,19 @@ private Statistics computeCatalogRelation(CatalogRelation catalogRelation) { } if (!cache.isUnKnown) { rowCount = Math.max(rowCount, cache.count); + } else { + hasUnknownCol = true; } if (ConnectContext.get() != null && ConnectContext.get().getSessionVariable().enableStats) { columnStatisticMap.put(slotReference, cache); } else { columnStatisticMap.put(slotReference, ColumnStatistic.UNKNOWN); + hasUnknownCol = true; } } + if (hasUnknownCol && ConnectContext.get() != null && ConnectContext.get().getStatementContext() != null) { + ConnectContext.get().getStatementContext().setHasUnknownColStats(true); + } Statistics stats = new Statistics(rowCount, columnStatisticMap); stats = normalizeCatalogRelationColumnStatsRowCount(stats); return stats; diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query64.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query64.out index 1494aa9d6839dd..eb79094ac608fa 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query64.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query64.out @@ -5,105 +5,100 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ----PhysicalProject ------hashAgg[LOCAL] --------PhysicalProject -----------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = cs_ui.cs_item_sk)) otherCondition=() build RFs:RF19 cs_item_sk->[ss_item_sk,i_item_sk,sr_item_sk] -------------PhysicalProject ---------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() -----------------PhysicalProject -------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF17 d_date_sk->[ss_sold_date_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((hd2.hd_income_band_sk = ib2.ib_income_band_sk)) otherCondition=() -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN] hashCondition=((hd1.hd_income_band_sk = ib1.ib_income_band_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = hd1.hd_demo_sk)) otherCondition=() +----------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF19 i_item_sk->[ss_item_sk,sr_item_sk,cs_item_sk] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((hd2.hd_income_band_sk = ib2.ib_income_band_sk)) otherCondition=() +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN] hashCondition=((hd1.hd_income_band_sk = ib1.ib_income_band_sk)) otherCondition=() +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_hdemo_sk = hd2.hd_demo_sk)) otherCondition=() +------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = ad2.ca_address_sk)) otherCondition=() --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_hdemo_sk = hd2.hd_demo_sk)) otherCondition=() -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() ---------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() -------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF8 i_item_sk->[ss_item_sk] +----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() +------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = hd1.hd_demo_sk)) otherCondition=() +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_shipto_date_sk = d3.d_date_sk)) otherCondition=() +------------------------------------------PhysicalProject +--------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() +----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() +------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) --------------------------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------------------------PhysicalProject -------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_shipto_date_sk = d3.d_date_sk)) otherCondition=() ---------------------------------------------------------PhysicalProject -----------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() -------------------------------------------------------------PhysicalProject ---------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = ad2.ca_address_sk)) otherCondition=() +------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() +----------------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) ---------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() ---------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF8 RF17 RF19 ---------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() ---------------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------------------PhysicalOlapScan[customer] ---------------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------------------PhysicalOlapScan[customer_demographics] ---------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = cs_ui.cs_item_sk)) otherCondition=() build RFs:RF4 cs_item_sk->[ss_item_sk,sr_item_sk] +--------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() +------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF7 RF19 +------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF4 RF19 +--------------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------PhysicalOlapScan[customer_demographics] -----------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------PhysicalOlapScan[customer_address] -------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------------------------------filter((sale > (2 * refund))) +--------------------------------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() +------------------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF19 +------------------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[customer] +----------------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------PhysicalOlapScan[customer_demographics] --------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------------------------------------PhysicalProject -------------------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------------------filter(d_year IN (2001, 2002)) +--------------------------------------------------------------PhysicalOlapScan[date_dim] --------------------------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------------------------PhysicalProject -------------------------------------------------------filter((item.i_current_price <= 33.00) and (item.i_current_price >= 24.00) and i_color IN ('blanched', 'brown', 'burlywood', 'chocolate', 'drab', 'medium')) ---------------------------------------------------------PhysicalOlapScan[item] apply RFs: RF9 RF19 -------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF19 ---------------------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store] -----------------------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[household_demographics] +----------------------------------------------------PhysicalOlapScan[customer_address] +----------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[household_demographics] ------------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_address] +----------------------------------------PhysicalOlapScan[promotion] --------------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[household_demographics] -----------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[income_band] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[income_band] ---------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------PhysicalProject -------------------------filter(d_year IN (2001, 2002)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecReplicated] -------------------PhysicalProject ---------------------PhysicalOlapScan[promotion] -------------PhysicalProject ---------------filter((sale > (2 * refund))) -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_returns] +------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[household_demographics] +--------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store] +----------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[income_band] +------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------PhysicalOlapScan[income_band] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------filter((item.i_current_price <= 33.00) and (item.i_current_price >= 24.00) and i_color IN ('blanched', 'brown', 'burlywood', 'chocolate', 'drab', 'medium')) +------------------PhysicalOlapScan[item] --PhysicalResultSink ----PhysicalQuickSort[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query64.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query64.out index 2caec5e3f93e19..e3b2f84df37126 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query64.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query64.out @@ -5,105 +5,100 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ----PhysicalProject ------hashAgg[LOCAL] --------PhysicalProject -----------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = cs_ui.cs_item_sk)) otherCondition=() build RFs:RF19 cs_item_sk->[ss_item_sk,i_item_sk,sr_item_sk] -------------PhysicalProject ---------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF18 p_promo_sk->[ss_promo_sk] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF17 d_date_sk->[ss_sold_date_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((hd2.hd_income_band_sk = ib2.ib_income_band_sk)) otherCondition=() build RFs:RF16 ib_income_band_sk->[hd_income_band_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN] hashCondition=((hd1.hd_income_band_sk = ib1.ib_income_band_sk)) otherCondition=() build RFs:RF15 ib_income_band_sk->[hd_income_band_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = hd1.hd_demo_sk)) otherCondition=() build RFs:RF14 hd_demo_sk->[ss_hdemo_sk] +----------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF19 i_item_sk->[ss_item_sk,sr_item_sk,cs_item_sk] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((hd2.hd_income_band_sk = ib2.ib_income_band_sk)) otherCondition=() build RFs:RF18 ib_income_band_sk->[hd_income_band_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN] hashCondition=((hd1.hd_income_band_sk = ib1.ib_income_band_sk)) otherCondition=() build RFs:RF17 ib_income_band_sk->[hd_income_band_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF16 s_store_sk->[ss_store_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_hdemo_sk = hd2.hd_demo_sk)) otherCondition=() build RFs:RF15 hd_demo_sk->[c_current_hdemo_sk] +------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = ad2.ca_address_sk)) otherCondition=() build RFs:RF14 ca_address_sk->[c_current_addr_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() build RFs:RF13 ca_address_sk->[ss_addr_sk] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_hdemo_sk = hd2.hd_demo_sk)) otherCondition=() build RFs:RF12 hd_demo_sk->[c_current_hdemo_sk] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF11 s_store_sk->[ss_store_sk] ---------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF9 sr_item_sk->[ss_item_sk,i_item_sk];RF10 sr_ticket_number->[ss_ticket_number] -------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF8 i_item_sk->[ss_item_sk] +----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF13 p_promo_sk->[ss_promo_sk] +------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = hd1.hd_demo_sk)) otherCondition=() build RFs:RF12 hd_demo_sk->[ss_hdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_shipto_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF11 d_date_sk->[c_first_shipto_date_sk] +------------------------------------------PhysicalProject +--------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[c_first_sales_date_sk] +----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() build RFs:RF9 ca_address_sk->[ss_addr_sk] +------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) build RFs:RF8 cd_demo_sk->[c_current_cdemo_sk] --------------------------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------------------------PhysicalProject -------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_shipto_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[c_first_shipto_date_sk] ---------------------------------------------------------PhysicalProject -----------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[c_first_sales_date_sk] -------------------------------------------------------------PhysicalProject ---------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = ad2.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] +------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF6 cd_demo_sk->[ss_cdemo_sk] +----------------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] +--------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) build RFs:RF4 cd_demo_sk->[ss_cdemo_sk] ---------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] ---------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 RF8 RF9 RF10 RF11 RF13 RF14 RF17 RF18 RF19 ---------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[c_current_cdemo_sk] ---------------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF5 RF6 RF7 RF12 ---------------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------------------PhysicalOlapScan[customer_demographics] ---------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = cs_ui.cs_item_sk)) otherCondition=() build RFs:RF4 cs_item_sk->[ss_item_sk,sr_item_sk] +--------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_item_sk->[ss_item_sk];RF3 sr_ticket_number->[ss_ticket_number] +------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF4 RF5 RF6 RF7 RF9 RF12 RF13 RF16 RF19 +------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF4 RF19 +--------------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------PhysicalOlapScan[customer_demographics] -----------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------PhysicalOlapScan[customer_address] -------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------------------------------filter((sale > (2 * refund))) +--------------------------------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF0 cr_order_number->[cs_order_number];RF1 cr_item_sk->[cs_item_sk] +------------------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF19 +------------------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[customer] apply RFs: RF8 RF10 RF11 RF14 RF15 +----------------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------PhysicalOlapScan[customer_demographics] --------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------------------------------------PhysicalProject -------------------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------------------filter(d_year IN (2001, 2002)) +--------------------------------------------------------------PhysicalOlapScan[date_dim] --------------------------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------------------------PhysicalProject -------------------------------------------------------filter((item.i_current_price <= 33.00) and (item.i_current_price >= 24.00) and i_color IN ('blanched', 'brown', 'burlywood', 'chocolate', 'drab', 'medium')) ---------------------------------------------------------PhysicalOlapScan[item] apply RFs: RF9 RF19 -------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF19 ---------------------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store] -----------------------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF16 +----------------------------------------------------PhysicalOlapScan[customer_address] +----------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF17 ------------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_address] +----------------------------------------PhysicalOlapScan[promotion] --------------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF15 -----------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[income_band] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[income_band] ---------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------PhysicalProject -------------------------filter(d_year IN (2001, 2002)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecReplicated] -------------------PhysicalProject ---------------------PhysicalOlapScan[promotion] -------------PhysicalProject ---------------filter((sale > (2 * refund))) -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF0 cr_order_number->[cs_order_number];RF1 cr_item_sk->[cs_item_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_returns] +------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF18 +--------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store] +----------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[income_band] +------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------PhysicalOlapScan[income_band] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------filter((item.i_current_price <= 33.00) and (item.i_current_price >= 24.00) and i_color IN ('blanched', 'brown', 'burlywood', 'chocolate', 'drab', 'medium')) +------------------PhysicalOlapScan[item] --PhysicalResultSink ----PhysicalQuickSort[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] From df007b95dd1585b4248150502fb662485e1ed2b8 Mon Sep 17 00:00:00 2001 From: Dongyang Li Date: Tue, 16 Jan 2024 15:22:47 +0800 Subject: [PATCH 025/200] [fix](case) add sync after streamload (#30009) Co-authored-by: stephen --- regression-test/suites/nereids_syntax_p0/set_operation.groovy | 1 + 1 file changed, 1 insertion(+) diff --git a/regression-test/suites/nereids_syntax_p0/set_operation.groovy b/regression-test/suites/nereids_syntax_p0/set_operation.groovy index ee2bf1d14f06c4..b23811727bfd00 100644 --- a/regression-test/suites/nereids_syntax_p0/set_operation.groovy +++ b/regression-test/suites/nereids_syntax_p0/set_operation.groovy @@ -323,6 +323,7 @@ suite("test_nereids_set_operation") { } } + sql "sync" order_qt_check_child_col_order """ select avg(tap), potno from dwd_daytable where potno=3601 and ddate >= '2023-08-01' group by potno limit 10 union From b6aa60c8b0de90e668f442af53665d2128ab74d5 Mon Sep 17 00:00:00 2001 From: Xin Liao Date: Tue, 16 Jan 2024 15:29:56 +0800 Subject: [PATCH 026/200] [fix](regression) spare .testfile to make disk checker happy when injecting fault (#30012) --- be/src/io/fs/local_file_writer.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/be/src/io/fs/local_file_writer.cpp b/be/src/io/fs/local_file_writer.cpp index 68012f563479cd..4c35101584c560 100644 --- a/be/src/io/fs/local_file_writer.cpp +++ b/be/src/io/fs/local_file_writer.cpp @@ -41,6 +41,7 @@ #include "io/fs/file_writer.h" #include "io/fs/local_file_system.h" #include "io/fs/path.h" +#include "olap/data_dir.h" #include "util/doris_metrics.h" namespace doris { @@ -201,7 +202,10 @@ Status LocalFileWriter::_close(bool sync) { } DBUG_EXECUTE_IF("LocalFileWriter.close.failed", { - return Status::IOError("cannot close {}: {}", _path.native(), std::strerror(errno)); + // spare '.testfile' to make bad disk checker happy + if (_path.filename().compare(kTestFilePath)) { + return Status::IOError("cannot close {}: {}", _path.native(), std::strerror(errno)); + } }); _closed = true; From d11e66efb6ed962f019079edf91c7b4c0c751031 Mon Sep 17 00:00:00 2001 From: zy-kkk Date: Tue, 16 Jan 2024 15:54:52 +0800 Subject: [PATCH 027/200] [improvement](catalog) fix jdbc mysql catalog to_date fun pushdown (#29900) --- .../planner/external/jdbc/JdbcFunctionPushDownRule.java | 1 + .../data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out | 6 ++++++ .../external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy | 6 ++++++ 3 files changed, 13 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/jdbc/JdbcFunctionPushDownRule.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/jdbc/JdbcFunctionPushDownRule.java index 27fd693ca4dcb3..4fb8788bb0b477 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/jdbc/JdbcFunctionPushDownRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/jdbc/JdbcFunctionPushDownRule.java @@ -62,6 +62,7 @@ private static boolean isClickHouseFunctionUnsupported(String functionName) { static { REPLACE_MYSQL_FUNCTIONS.put("nvl", "ifnull"); + REPLACE_MYSQL_FUNCTIONS.put("to_date", "date"); } private static boolean isReplaceMysqlFunctions(String functionName) { diff --git a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out index 4a7c9a9035db9b..96710f09e36aa7 100644 --- a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out +++ b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out @@ -185,6 +185,12 @@ bca 2022-11-02 2022-11-02 8012 vivo -- !ex_tb21_6 -- 1 1 +-- !ex_tb21_7 -- +2 1 + +-- !ex_tb21_8 -- +2 2 + -- !information_schema -- character_sets collations diff --git a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy index ec09d2a319d690..fdc1f84e382eb2 100644 --- a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy +++ b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy @@ -172,6 +172,8 @@ suite("test_mysql_jdbc_catalog", "p0,external,mysql,external_docker,external_doc order_qt_ex_tb21_4 """ select `key`, `id` from ${ex_tb21} where abs(`key`) = 2 order by id;""" order_qt_ex_tb21_5 """ select `key`, `id` from ${ex_tb21} where `key` between 1 and 2 order by id;""" order_qt_ex_tb21_6 """ select `key`, `id` from ${ex_tb21} where `key` = case when id = 1 then 1 else 0 end order by id;""" + order_qt_ex_tb21_7 """ select (`key` +1) as k, `id` from ${ex_tb21} having abs(k) = 2 order by id;""" + order_qt_ex_tb21_8 """ select `key` as k, `id` from ${ex_tb21} having abs(k) = 2 order by id;""" order_qt_information_schema """ show tables from information_schema; """ order_qt_auto_default_t """insert into ${auto_default_t}(name) values('a'); """ order_qt_dt """select * from ${dt}; """ @@ -179,6 +181,10 @@ suite("test_mysql_jdbc_catalog", "p0,external,mysql,external_docker,external_doc order_qt_test_dz """select * from ${test_zd} order by 1; """ order_qt_test_filter_not """select * from ${ex_tb13} where name not like '%张三0%' order by 1; """ order_qt_test_filter_not_old_plan """select /*+ SET_VAR(enable_nereids_planner=false) */ * from ${ex_tb13} where name not like '%张三0%' order by 1; """ + explain { + sql("select `datetime` from all_types where to_date(`datetime`) = '2012-10-25';") + contains """ SELECT `datetime` FROM `doris_test`.`all_types` WHERE (date(`datetime`) = '2012-10-25')""" + } // test insert String uuid1 = UUID.randomUUID().toString(); From f4d05e03adfafd5e6f8165dda9bd8c3895a880a0 Mon Sep 17 00:00:00 2001 From: zy-kkk Date: Tue, 16 Jan 2024 15:55:43 +0800 Subject: [PATCH 028/200] [enhancement](jdbc catalog) Enhance function pushdown of Jdbc Oracle Catalog (#29972) --- docs/en/docs/lakehouse/multi-catalog/jdbc.md | 6 ++++- .../docs/lakehouse/multi-catalog/jdbc.md | 6 ++++- .../jdbc/JdbcFunctionPushDownRule.java | 25 +++++++++++++++++++ .../planner/external/jdbc/JdbcScanNode.java | 3 ++- .../jdbc/test_oracle_jdbc_catalog.groovy | 6 +++++ 5 files changed, 43 insertions(+), 3 deletions(-) diff --git a/docs/en/docs/lakehouse/multi-catalog/jdbc.md b/docs/en/docs/lakehouse/multi-catalog/jdbc.md index 11a02b049a600b..662fcc81118063 100644 --- a/docs/en/docs/lakehouse/multi-catalog/jdbc.md +++ b/docs/en/docs/lakehouse/multi-catalog/jdbc.md @@ -114,7 +114,7 @@ In some cases, the keywords in the database might be used as the field names. Fo 1. When executing a query like `where dt = '2022-01-01'`, Doris can push down these filtering conditions to the external data source, thereby directly excluding data that does not meet the conditions at the data source level, reducing the number of unqualified Necessary data acquisition and transfer. This greatly improves query performance while also reducing the load on external data sources. -2. When `enable_func_pushdown` is set to true, the function conditions after where will also be pushed down to the external data source. Currently, only MySQL and ClickHouse are supported. If you encounter a function that is not supported by MySQL or ClickHouse, you can set this parameter to false. , currently Doris will automatically identify some functions not supported by MySQL and functions supported by CLickHouse for push-down condition filtering, which can be viewed through explain sql. +2. When `enable_func_pushdown` is set to true, the function conditions after where will also be pushed down to the external data source. Currently, only MySQL, ClickHouse, and Oracle are supported. If you encounter functions that are not supported by MySQL, ClickHouse, and Oracle, you can use this The parameter is set to false. At present, Doris will automatically identify some functions that are not supported by MySQL and functions supported by CLickHouse and Oracle for push-down condition filtering. You can view them through explain sql. Functions that are currently not pushed down include: @@ -130,6 +130,10 @@ Functions that are currently pushed down include: | FROM_UNIXTIME | | UNIX_TIMESTAMP | +| Oracle | +|:------:| +| NVL | + ### Line Limit If there is a limit keyword in the query, Doris will translate it into semantics suitable for different data sources. diff --git a/docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md b/docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md index e57acbe688f80c..e3eb90d7f40472 100644 --- a/docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md +++ b/docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md @@ -114,7 +114,7 @@ select * from mysql_catalog.mysql_database.mysql_table where k1 > 1000 and k3 =' 1. 当执行类似于 `where dt = '2022-01-01'` 这样的查询时,Doris 能够将这些过滤条件下推到外部数据源,从而直接在数据源层面排除不符合条件的数据,减少了不必要的数据获取和传输。这大大提高了查询性能,同时也降低了对外部数据源的负载。 -2. 当 `enable_func_pushdown` 设置为true,会将 where 之后的函数条件也下推到外部数据源,目前仅支持 MySQL 以及 ClickHouse,如遇到 MySQL 或 ClickHouse 不支持的函数,可以将此参数设置为 false,目前 Doris 会自动识别部分 MySQL 不支持的函数以及 CLickHouse 支持的函数进行下推条件过滤,可通过 explain sql 查看。 +2. 当 `enable_func_pushdown` 设置为true,会将 where 之后的函数条件也下推到外部数据源,目前仅支持 MySQL、ClickHouse、Oracle,如遇到 MySQL、ClickHouse、Oracle 不支持的函数,可以将此参数设置为 false,目前 Doris 会自动识别部分 MySQL 不支持的函数以及 CLickHouse、Oracle 支持的函数进行下推条件过滤,可通过 explain sql 查看。 目前不会下推的函数有: @@ -130,6 +130,10 @@ select * from mysql_catalog.mysql_database.mysql_table where k1 > 1000 and k3 =' | FROM_UNIXTIME | | UNIX_TIMESTAMP | +| Oracle | +|:------:| +| NVL | + ### 行数限制 如果在查询中带有 limit 关键字,Doris 会将其转译成适合不同数据源的语义。 diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/jdbc/JdbcFunctionPushDownRule.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/jdbc/JdbcFunctionPushDownRule.java index 4fb8788bb0b477..d328952593fb46 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/jdbc/JdbcFunctionPushDownRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/jdbc/JdbcFunctionPushDownRule.java @@ -49,6 +49,13 @@ public class JdbcFunctionPushDownRule { CLICKHOUSE_SUPPORTED_FUNCTIONS.add("unix_timestamp"); } + private static final TreeSet ORACLE_SUPPORTED_FUNCTIONS = new TreeSet<>(String.CASE_INSENSITIVE_ORDER); + + static { + ORACLE_SUPPORTED_FUNCTIONS.add("nvl"); + ORACLE_SUPPORTED_FUNCTIONS.add("ifnull"); + } + private static boolean isMySQLFunctionUnsupported(String functionName) { return MYSQL_UNSUPPORTED_FUNCTIONS.contains(functionName.toLowerCase()); } @@ -57,6 +64,9 @@ private static boolean isClickHouseFunctionUnsupported(String functionName) { return !CLICKHOUSE_SUPPORTED_FUNCTIONS.contains(functionName.toLowerCase()); } + private static boolean isOracleFunctionUnsupported(String functionName) { + return !ORACLE_SUPPORTED_FUNCTIONS.contains(functionName.toLowerCase()); + } private static final Map REPLACE_MYSQL_FUNCTIONS = Maps.newHashMap(); @@ -80,6 +90,16 @@ private static boolean isReplaceClickHouseFunctions(String functionName) { return REPLACE_CLICKHOUSE_FUNCTIONS.containsKey(functionName.toLowerCase()); } + private static final Map REPLACE_ORACLE_FUNCTIONS = Maps.newHashMap(); + + static { + REPLACE_ORACLE_FUNCTIONS.put("ifnull", "nvl"); + } + + private static boolean isReplaceOracleFunctions(String functionName) { + return REPLACE_ORACLE_FUNCTIONS.containsKey(functionName.toLowerCase()); + } + public static Expr processFunctions(TOdbcTableType tableType, Expr expr, List errors) { if (tableType == null || expr == null) { return expr; @@ -94,6 +114,9 @@ public static Expr processFunctions(TOdbcTableType tableType, Expr expr, List '2022-01-20 00:00:00') and (T1 < '2022-01-23 00:00:00' or T1 > '2022-01-19 00:00:00'); """ order_qt_date7 """select * from TEST_TIMESTAMP where T2 < str_to_date('2020-12-21 12:34:56', '%Y-%m-%d %H:%i:%s');""" + // test nvl + explain { + sql("SELECT * FROM STUDENT WHERE nvl(score, 0) < 95;") + contains """SELECT "ID", "NAME", "AGE", "SCORE" FROM "DORIS_TEST"."STUDENT" WHERE (nvl("SCORE", 0.0) < 95.0)""" + } + // for old planner order_qt_filter4_old_plan """ select /*+ SET_VAR(enable_nereids_planner=false) */ * from STUDENT where NAME NOT like '%bob%' order by ID; """ order_qt_filter5_old_plan """ select /*+ SET_VAR(enable_nereids_planner=false) */ * from STUDENT where NAME NOT like '%bob%' or NAME NOT LIKE '%jerry%' order by ID; """ From 5bb5dc7388aec140f5e8f6e65b6d68cd617fa7a0 Mon Sep 17 00:00:00 2001 From: slothever <18522955+wsjz@users.noreply.github.com> Date: Tue, 16 Jan 2024 17:02:00 +0800 Subject: [PATCH 029/200] [fix](jobs)fix create task int job (#30011) --- .../org/apache/doris/job/extensions/insert/InsertJob.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java index 44cc63ee9992d8..ce918c426f8411 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java @@ -244,9 +244,11 @@ public InsertJob(ConnectContext ctx, @Override public List createTasks(TaskType taskType, Map taskContext) { + List newTasks = new ArrayList<>(); if (plans.isEmpty()) { InsertTask task = new InsertTask(labelName, getCurrentDbName(), getExecuteSql(), getCreateUser()); idToTasks.put(task.getTaskId(), task); + newTasks.add(task); recordTask(task.getTaskId()); } else { // use for load stmt @@ -256,11 +258,12 @@ public List createTasks(TaskType taskType, Map taskC } InsertTask task = new InsertTask(logicalPlan, ctx, stmtExecutor, loadStatistic); idToTasks.put(task.getTaskId(), task); + newTasks.add(task); recordTask(task.getTaskId()); } } - initTasks(idToTasks.values(), taskType); - return new ArrayList<>(idToTasks.values()); + initTasks(newTasks, taskType); + return new ArrayList<>(newTasks); } public void recordTask(long id) { From d9b3b5918146a877879008c98d8b1d64092d0c65 Mon Sep 17 00:00:00 2001 From: yiguolei <676222867@qq.com> Date: Tue, 16 Jan 2024 17:06:59 +0800 Subject: [PATCH 030/200] [chore](removelogs) remove debug query timeout logs (#30006) --------- Co-authored-by: yiguolei --- be/src/vec/exec/scan/pip_scanner_context.h | 4 ---- be/src/vec/exec/scan/scanner_context.cpp | 3 --- be/src/vec/exec/scan/scanner_scheduler.cpp | 6 ------ be/src/vec/exec/scan/vscan_node.cpp | 5 ----- regression-test/pipeline/p0/conf/be.conf | 2 +- 5 files changed, 1 insertion(+), 19 deletions(-) diff --git a/be/src/vec/exec/scan/pip_scanner_context.h b/be/src/vec/exec/scan/pip_scanner_context.h index 1afe3b599e5980..c2490a5e0d0149 100644 --- a/be/src/vec/exec/scan/pip_scanner_context.h +++ b/be/src/vec/exec/scan/pip_scanner_context.h @@ -48,7 +48,6 @@ class PipScannerContext final : public vectorized::ScannerContext { } if (!status().ok()) { - LOG(INFO) << "yyyy status not ok " << debug_string(); return _process_status; } } @@ -61,12 +60,10 @@ class PipScannerContext final : public vectorized::ScannerContext { // if done, then eos is returned to indicate that the scan operator finished. if (_blocks_queues[id].empty()) { *eos = done(); - LOG(INFO) << "yyyy queue is empty and ctx finished " << debug_string(); return Status::OK(); } if (_process_status.is()) { *eos = true; - LOG(INFO) << "yyyy process status is cancelled " << debug_string(); return Status::OK(); } *block = std::move(_blocks_queues[id].front()); @@ -85,7 +82,6 @@ class PipScannerContext final : public vectorized::ScannerContext { } if (_blocks_queues[id].empty()) { - LOG(INFO) << "yyyy block queue is empty, try to resched ctx " << debug_string(); this->reschedule_scanner_ctx(); } } diff --git a/be/src/vec/exec/scan/scanner_context.cpp b/be/src/vec/exec/scan/scanner_context.cpp index 3b1cecd7ac2425..0b19f38992029a 100644 --- a/be/src/vec/exec/scan/scanner_context.cpp +++ b/be/src/vec/exec/scan/scanner_context.cpp @@ -482,7 +482,6 @@ void ScannerContext::push_back_scanner_and_reschedule(std::shared_ptr_scanner->need_to_close()) { @@ -504,8 +503,6 @@ void ScannerContext::push_back_scanner_and_reschedule(std::shared_ptr ctx) { - LOG(INFO) << "yyyy submit scanner ctx " << ctx->debug_string(); if (ctx->done()) { - LOG(INFO) << "yyyy ctx is done, not submit" << ctx->debug_string(); return Status::EndOfFile("ScannerContext is done"); } ctx->queue_idx = (_queue_idx++ % QUEUE_NUM); if (!_pending_queues[ctx->queue_idx]->blocking_put(ctx)) { - LOG(INFO) << "yyyy put to queue failed, not submit" << ctx->debug_string(); return Status::InternalError("failed to submit scanner context to scheduler"); } return Status::OK(); @@ -181,21 +178,18 @@ void ScannerScheduler::_schedule_scanners(std::shared_ptr ctx) { << " maybe finished"; return; } - LOG(INFO) << "yyyy scheduled, query " << ctx->debug_string(); MonotonicStopWatch watch; watch.reset(); watch.start(); ctx->incr_num_ctx_scheduling(1); if (ctx->done()) { - LOG(INFO) << "yyyy ctx done, " << ctx->debug_string(); return; } std::list> this_run; ctx->get_next_batch_of_scanners(&this_run); if (this_run.empty()) { - LOG(INFO) << "yyyy run is empty, skip, " << ctx->debug_string(); // There will be 2 cases when this_run is empty: // 1. The blocks queue reaches limit. // The consumer will continue scheduling the ctx. diff --git a/be/src/vec/exec/scan/vscan_node.cpp b/be/src/vec/exec/scan/vscan_node.cpp index cf52341b34f2dc..78a15bbe40414b 100644 --- a/be/src/vec/exec/scan/vscan_node.cpp +++ b/be/src/vec/exec/scan/vscan_node.cpp @@ -197,8 +197,6 @@ Status VScanNode::alloc_resource(RuntimeState* state) { if (_scanner_ctx) { DCHECK(!_eos && _num_scanners->value() > 0); RETURN_IF_ERROR(_scanner_ctx->init()); - LOG(INFO) << "yyyy instance " << print_id(state->fragment_instance_id()) - << " submit scanner ctx " << _scanner_ctx->debug_string(); RETURN_IF_ERROR(_state->exec_env()->scanner_scheduler()->submit(_scanner_ctx)); } if (_shared_scan_opt) { @@ -221,9 +219,6 @@ Status VScanNode::alloc_resource(RuntimeState* state) { : Status::OK()); if (_scanner_ctx) { RETURN_IF_ERROR(_scanner_ctx->init()); - - LOG(INFO) << "yyyy instance " << print_id(state->fragment_instance_id()) - << " submit scanner ctx " << _scanner_ctx->debug_string(); RETURN_IF_ERROR(_state->exec_env()->scanner_scheduler()->submit(_scanner_ctx)); } } diff --git a/regression-test/pipeline/p0/conf/be.conf b/regression-test/pipeline/p0/conf/be.conf index 59060adb285a26..c2c09ec89dd868 100644 --- a/regression-test/pipeline/p0/conf/be.conf +++ b/regression-test/pipeline/p0/conf/be.conf @@ -78,4 +78,4 @@ max_sys_mem_available_low_water_mark_bytes=69206016 user_files_secure_path=/ enable_debug_points=true # debug scanner context dead loop -enable_debug_log_timeout_secs=300 +enable_debug_log_timeout_secs=0 From ff8e756fbbe7e9c63a905e6060497a381d2fc119 Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Tue, 16 Jan 2024 17:10:54 +0800 Subject: [PATCH 031/200] [fix](Nereids) struct type coercion call wrong api (#30018) --- .../java/org/apache/doris/nereids/util/TypeCoercionUtils.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/TypeCoercionUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/TypeCoercionUtils.java index 63a6a552f6d2b4..185bcc055b392f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/TypeCoercionUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/TypeCoercionUtils.java @@ -1120,7 +1120,7 @@ private static Optional findCommonComplexTypeForComparison( } List newFields = Lists.newArrayList(); for (int i = 0; i < leftFields.size(); i++) { - Optional newDataType = findCommonComplexTypeForComparison(leftFields.get(i).getDataType(), + Optional newDataType = findWiderTypeForTwoForComparison(leftFields.get(i).getDataType(), rightFields.get(i).getDataType(), intStringToString); if (newDataType.isPresent()) { newFields.add(leftFields.get(i).withDataType(newDataType.get())); @@ -1348,7 +1348,7 @@ private static Optional findCommonComplexTypeForCaseWhen(DataType left } List newFields = Lists.newArrayList(); for (int i = 0; i < leftFields.size(); i++) { - Optional newDataType = findCommonComplexTypeForCaseWhen(leftFields.get(i).getDataType(), + Optional newDataType = findWiderTypeForTwoForCaseWhen(leftFields.get(i).getDataType(), rightFields.get(i).getDataType()); if (newDataType.isPresent()) { newFields.add(leftFields.get(i).withDataType(newDataType.get())); From 0e140a63de66d1dff8f9f2fec2d32f2c98d3582f Mon Sep 17 00:00:00 2001 From: TengJianPing <18241664+jacktengg@users.noreply.github.com> Date: Tue, 16 Jan 2024 17:16:20 +0800 Subject: [PATCH 032/200] [fix](decimal) fix wrong decimal overflow caused by uninitialized nested column of null value (#29960) --- be/src/vec/functions/function_cast.h | 100 ++++++++++++++++++++++++++- 1 file changed, 97 insertions(+), 3 deletions(-) diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h index 82e4b3d972bfa4..953d3054c6bbe3 100644 --- a/be/src/vec/functions/function_cast.h +++ b/be/src/vec/functions/function_cast.h @@ -2268,6 +2268,94 @@ class FunctionCast final : public IFunctionBase { return wrapper; } + static bool need_replace_null_data_to_default(FunctionContext* context, + const DataTypePtr& from_type, + const DataTypePtr& to_type) { + if (from_type->equals(*to_type)) { + return false; + } + + auto make_default_wrapper = [&](const auto& types) -> bool { + using Types = std::decay_t; + using ToDataType = typename Types::LeftType; + + if constexpr (!(IsDataTypeDecimalOrNumber || IsTimeType || + IsTimeV2Type || + std::is_same_v || + std::is_same_v)) { + return false; + } + return call_on_index_and_data_type< + ToDataType>(from_type->get_type_id(), [&](const auto& types2) -> bool { + using Types2 = std::decay_t; + using FromDataType = typename Types2::LeftType; + if constexpr (!(IsDataTypeDecimalOrNumber || + IsTimeType || IsTimeV2Type || + std::is_same_v || + std::is_same_v)) { + return false; + } + if constexpr (IsDataTypeDecimal || IsDataTypeDecimal) { + using FromFieldType = typename FromDataType::FieldType; + using ToFieldType = typename ToDataType::FieldType; + UInt32 from_precision = NumberTraits::max_ascii_len(); + UInt32 from_scale = 0; + + if constexpr (IsDataTypeDecimal) { + const auto* from_decimal_type = + check_and_get_data_type(from_type.get()); + from_precision = + NumberTraits::max_ascii_len(); + from_scale = from_decimal_type->get_scale(); + } + + UInt32 to_max_digits = 0; + UInt32 to_precision = 0; + UInt32 to_scale = 0; + + ToFieldType max_result {0}; + ToFieldType min_result {0}; + if constexpr (IsDataTypeDecimal) { + to_max_digits = + NumberTraits::max_ascii_len(); + + const auto* to_decimal_type = + check_and_get_data_type(to_type.get()); + to_precision = to_decimal_type->get_precision(); + ToDataType::check_type_precision(to_precision); + + to_scale = to_decimal_type->get_scale(); + ToDataType::check_type_scale(to_scale); + + max_result = ToDataType::get_max_digits_number(to_precision); + min_result = -max_result; + } + if constexpr (std::is_integral_v || + std::is_floating_point_v) { + max_result = type_limit::max(); + min_result = type_limit::min(); + to_max_digits = NumberTraits::max_ascii_len(); + to_precision = to_max_digits; + } + + bool narrow_integral = + context->check_overflow_for_decimal() && + (to_precision - to_scale) <= (from_precision - from_scale); + + bool multiply_may_overflow = context->check_overflow_for_decimal(); + if (to_scale > from_scale) { + multiply_may_overflow &= + (from_precision + to_scale - from_scale) >= to_max_digits; + } + return narrow_integral || multiply_may_overflow; + } + return false; + }); + }; + + return call_on_index_and_data_type(to_type->get_type_id(), make_default_wrapper); + } + WrapperType prepare_remove_nullable(FunctionContext* context, const DataTypePtr& from_type, const DataTypePtr& to_type, bool skip_not_null_check) const { @@ -2278,13 +2366,19 @@ class FunctionCast final : public IFunctionBase { return [this, from_type, to_type](FunctionContext* context, Block& block, const ColumnNumbers& arguments, const size_t result, size_t input_rows_count) { + auto from_type_not_nullable = remove_nullable(from_type); + auto to_type_not_nullable = remove_nullable(to_type); + + bool replace_null_data_to_default = need_replace_null_data_to_default( + context, from_type_not_nullable, to_type_not_nullable); + auto nested_result_index = block.columns(); block.insert(block.get_by_position(result).get_nested()); auto nested_source_index = block.columns(); - block.insert(block.get_by_position(arguments[0]).get_nested()); + block.insert(block.get_by_position(arguments[0]) + .get_nested(replace_null_data_to_default)); - RETURN_IF_ERROR(prepare_impl(context, remove_nullable(from_type), - remove_nullable(to_type), + RETURN_IF_ERROR(prepare_impl(context, from_type_not_nullable, to_type_not_nullable, true)(context, block, {nested_source_index}, nested_result_index, input_rows_count)); From 3e9be17f63b10c8ea09eb0d53532d7e4052eebe4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=A2=E5=81=A5?= Date: Tue, 16 Jan 2024 17:16:41 +0800 Subject: [PATCH 033/200] [feature](Nereids): optimize logical group expression in dphyp (#30000) --- .../doris/nereids/StatementContext.java | 9 - .../cascades/OptimizeGroupExpressionJob.java | 7 +- .../nereids/jobs/executor/Optimizer.java | 2 - .../hypergraph/receiver/PlanReceiver.java | 144 +----- .../apache/doris/nereids/rules/RuleSet.java | 9 +- .../trees/plans/logical/LogicalJoin.java | 7 + .../mv/join/dphyp_inner/inner_join_dphyp.out | 335 +++++++++++++ .../mv/join/dphyp_outer/outer_join_dphyp.out | 297 +++++++++++ .../shape/query64.out | 128 ++--- .../rf_prune/query64.out | 92 ++-- .../shape/query64.out | 100 ++-- .../join/dphyp_inner/inner_join_dphyp.groovy | 464 +++++++++++++++++ .../join/dphyp_outer/outer_join_dphyp.groovy | 469 ++++++++++++++++++ 13 files changed, 1760 insertions(+), 303 deletions(-) create mode 100644 regression-test/data/nereids_rules_p0/mv/join/dphyp_inner/inner_join_dphyp.out create mode 100644 regression-test/data/nereids_rules_p0/mv/join/dphyp_outer/outer_join_dphyp.out create mode 100644 regression-test/suites/nereids_rules_p0/mv/join/dphyp_inner/inner_join_dphyp.groovy create mode 100644 regression-test/suites/nereids_rules_p0/mv/join/dphyp_outer/outer_join_dphyp.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java index c4a6e37bbeca98..4a19ab925451be 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java @@ -72,7 +72,6 @@ public class StatementContext { private int maxNAryInnerJoin = 0; private boolean isDpHyp = false; - private boolean isOtherJoinReorder = false; // hasUnknownColStats true if any column stats in the tables used by this sql is unknown // the algorithm to derive plan when column stats are unknown is implemented in cascading framework, not in dphyper. @@ -158,14 +157,6 @@ public void setDpHyp(boolean dpHyp) { isDpHyp = dpHyp; } - public boolean isOtherJoinReorder() { - return isOtherJoinReorder; - } - - public void setOtherJoinReorder(boolean otherJoinReorder) { - isOtherJoinReorder = otherJoinReorder; - } - public ExprId getNextExprId() { return exprIdGenerator.getNextId(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/OptimizeGroupExpressionJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/OptimizeGroupExpressionJob.java index 72426f0fa15dc5..16cf90b786eee6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/OptimizeGroupExpressionJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/OptimizeGroupExpressionJob.java @@ -75,7 +75,6 @@ private List getExplorationRules() { || context.getCascadesContext().getMemo().getGroupExpressionsSize() > context.getCascadesContext() .getConnectContext().getSessionVariable().memoMaxGroupExpressionSize; boolean isDpHyp = context.getCascadesContext().getStatementContext().isDpHyp(); - boolean isOtherJoinReorder = context.getCascadesContext().getStatementContext().isOtherJoinReorder(); boolean isEnableBushyTree = context.getCascadesContext().getConnectContext().getSessionVariable() .isEnableBushyTree(); boolean isLeftZigZagTree = context.getCascadesContext().getConnectContext() @@ -86,11 +85,7 @@ private List getExplorationRules() { if (isDisableJoinReorder) { return Collections.emptyList(); } else if (isDpHyp) { - if (isOtherJoinReorder) { - return getRuleSet().getDPHypReorderRules(); - } else { - return Collections.emptyList(); - } + return getRuleSet().getDPHypReorderRules(); } else if (isLeftZigZagTree) { return getRuleSet().getLeftZigZagTreeJoinReorder(); } else if (isEnableBushyTree) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Optimizer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Optimizer.java index 4f042c527cb9e0..5f945ab8383f9c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Optimizer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Optimizer.java @@ -67,7 +67,6 @@ public void execute() { boolean isDpHyp = getSessionVariable().enableDPHypOptimizer || maxJoinCount > maxTableCount; cascadesContext.getStatementContext().setDpHyp(isDpHyp); - cascadesContext.getStatementContext().setOtherJoinReorder(false); if (!getSessionVariable().isDisableJoinReorder() && isDpHyp && maxJoinCount <= getSessionVariable().getMaxJoinNumberOfReorder()) { //RightNow, dphyper can only order 64 join operators @@ -85,7 +84,6 @@ private void dpHypOptimize() { // Due to EnsureProjectOnTopJoin, root group can't be Join Group, so DPHyp doesn't change the root group cascadesContext.pushJob(new JoinOrderJob(root, cascadesContext.getCurrentJobContext())); cascadesContext.getJobScheduler().executeJobPool(cascadesContext); - cascadesContext.getStatementContext().setOtherJoinReorder(true); } private SessionVariable getSessionVariable() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/receiver/PlanReceiver.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/receiver/PlanReceiver.java index eece2d8c3d6fe6..8af8a517099839 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/receiver/PlanReceiver.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/receiver/PlanReceiver.java @@ -17,10 +17,9 @@ package org.apache.doris.nereids.jobs.joinorder.hypergraph.receiver; -import org.apache.doris.nereids.hint.DistributeHint; import org.apache.doris.nereids.jobs.JobContext; -import org.apache.doris.nereids.jobs.cascades.CostAndEnforcerJob; import org.apache.doris.nereids.jobs.cascades.DeriveStatsJob; +import org.apache.doris.nereids.jobs.cascades.OptimizeGroupExpressionJob; import org.apache.doris.nereids.jobs.joinorder.hypergraph.HyperGraph; import org.apache.doris.nereids.jobs.joinorder.hypergraph.bitmap.LongBitmap; import org.apache.doris.nereids.jobs.joinorder.hypergraph.edge.Edge; @@ -29,40 +28,28 @@ import org.apache.doris.nereids.memo.Group; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.memo.Memo; -import org.apache.doris.nereids.properties.FunctionalDependencies; -import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; -import org.apache.doris.nereids.trees.plans.DistributeType; import org.apache.doris.nereids.trees.plans.GroupPlan; import org.apache.doris.nereids.trees.plans.JoinType; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; -import org.apache.doris.nereids.trees.plans.physical.AbstractPhysicalJoin; -import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribute; -import org.apache.doris.nereids.trees.plans.physical.PhysicalHashJoin; -import org.apache.doris.nereids.trees.plans.physical.PhysicalNestedLoopJoin; -import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; import org.apache.doris.nereids.util.ExpressionUtils; -import org.apache.doris.nereids.util.JoinUtils; import org.apache.doris.nereids.util.PlanUtils; import org.apache.doris.qe.ConnectContext; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; import java.util.ArrayList; import java.util.BitSet; import java.util.HashMap; import java.util.HashSet; import java.util.List; -import java.util.Optional; import java.util.Set; -import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -91,7 +78,6 @@ public PlanReceiver(JobContext jobContext, int limit, HyperGraph hyperGraph, Set this.finalOutputs = outputs; } - /** * Emit a new plan from bottom to top *

@@ -130,21 +116,18 @@ public boolean emitCsgCmp(long left, long right, List edges) { } long fullKey = LongBitmap.newBitmapUnion(left, right); - List physicalJoins = proposeAllPhysicalJoins(joinType, leftPlan, rightPlan, hashConjuncts, + LogicalPlan logicalPlan = proposeJoin(joinType, leftPlan, rightPlan, hashConjuncts, otherConjuncts); - List physicalPlans = proposeProject(physicalJoins, edges, left, right); + logicalPlan = proposeProject(logicalPlan, edges, left, right); // Second, we copy all physical plan to Group and generate properties and calculate cost if (!planTable.containsKey(fullKey)) { - planTable.put(fullKey, memo.newGroup(physicalPlans.get(0).getLogicalProperties())); + planTable.put(fullKey, memo.newGroup(logicalPlan.getLogicalProperties())); } Group group = planTable.get(fullKey); - for (Plan plan : physicalPlans) { - CopyInResult copyInResult = memo.copyIn(plan, group, false, planTable); - GroupExpression physicalExpression = copyInResult.correspondingExpression; - proposeAllDistributedPlans(physicalExpression); - } + CopyInResult copyInResult = memo.copyIn(logicalPlan, group, false, planTable); + proposeAllDistributedPlans(copyInResult.correspondingExpression); return true; } @@ -204,7 +187,7 @@ private void processMissedEdges(long left, long right, List edges) { } private void proposeAllDistributedPlans(GroupExpression groupExpression) { - jobContext.getCascadesContext().pushJob(new CostAndEnforcerJob(groupExpression, + jobContext.getCascadesContext().pushJob(new OptimizeGroupExpressionJob(groupExpression, new JobContext(jobContext.getCascadesContext(), PhysicalProperties.ANY, Double.MAX_VALUE))); if (!groupExpression.isStatDerived()) { jobContext.getCascadesContext().pushJob(new DeriveStatsJob(groupExpression, @@ -213,42 +196,16 @@ private void proposeAllDistributedPlans(GroupExpression groupExpression) { jobContext.getCascadesContext().getJobScheduler().executeJobPool(jobContext.getCascadesContext()); } - private List proposeAllPhysicalJoins(JoinType joinType, Plan left, Plan right, List hashConjuncts, + private LogicalPlan proposeJoin(JoinType joinType, Plan left, Plan right, List hashConjuncts, List otherConjuncts) { - // Check whether only NSL can be performed - LogicalProperties joinProperties = new LogicalProperties( - () -> JoinUtils.getJoinOutput(joinType, left, right), () -> FunctionalDependencies.EMPTY_FUNC_DEPS); - List plans = Lists.newArrayList(); - if (JoinUtils.shouldNestedLoopJoin(joinType, hashConjuncts)) { - plans.add(new PhysicalNestedLoopJoin<>(joinType, hashConjuncts, otherConjuncts, - Optional.empty(), joinProperties, - left, right)); - if (joinType.isSwapJoinType()) { - plans.add(new PhysicalNestedLoopJoin<>(joinType.swap(), hashConjuncts, otherConjuncts, Optional.empty(), - joinProperties, - right, left)); - } - } else { - plans.add(new PhysicalHashJoin<>(joinType, hashConjuncts, otherConjuncts, - new DistributeHint(DistributeType.NONE), Optional.empty(), - joinProperties, - left, right)); - if (joinType.isSwapJoinType()) { - plans.add(new PhysicalHashJoin<>(joinType.swap(), hashConjuncts, otherConjuncts, - new DistributeHint(DistributeType.NONE), - Optional.empty(), - joinProperties, - right, left)); - } - } - return plans; + return new LogicalJoin<>(joinType, hashConjuncts, otherConjuncts, left, right); } @Override public void addGroup(long bitmap, Group group) { Preconditions.checkArgument(LongBitmap.getCardinality(bitmap) == 1); usdEdges.put(bitmap, new BitSet()); - Plan plan = proposeProject(Lists.newArrayList(new GroupPlan(group)), new ArrayList<>(), bitmap, bitmap).get(0); + Plan plan = proposeProject(new GroupPlan(group), new ArrayList<>(), bitmap, bitmap); if (!(plan instanceof GroupPlan)) { CopyInResult copyInResult = jobContext.getCascadesContext().getMemo().copyIn(plan, null, false, planTable); group = copyInResult.correspondingExpression.getOwnerGroup(); @@ -274,59 +231,13 @@ public void reset() { @Override public Group getBestPlan(long bitmap) { - // If there are some rules relied on the logical join, we need to make logical Expression - // However, it cost 15% of total optimized time. - makeLogicalExpression(() -> planTable.get(bitmap)); return planTable.get(bitmap); } - private void makeLogicalExpression(Supplier root) { - if (!root.get().getLogicalExpressions().isEmpty()) { - return; - } - - // only makeLogicalExpression for those winners - Set hasGenerated = new HashSet<>(); - for (PhysicalProperties physicalProperties : root.get().getAllProperties()) { - GroupExpression groupExpression = root.get().getBestPlan(physicalProperties); - if (hasGenerated.contains(groupExpression) || groupExpression.getPlan() instanceof PhysicalDistribute) { - continue; - } - hasGenerated.add(groupExpression); - - // process child first, plan's child may be changed due to mergeGroup - // due to mergeGroup, the children Group of groupExpression may be replaced, so we need to use lambda to - // get the child to make we can get child at the time we use child. - // If we use for child: groupExpression.children(), it means that we take it in advance. It may cause NPE, - // work flow: get children() to get left, right -> copyIn left() -> mergeGroup -> right is merged -> NPE - Plan physicalPlan = groupExpression.getPlan(); - for (int i = 0; i < groupExpression.children().size(); i++) { - int childIdx = i; - makeLogicalExpression(() -> groupExpression.child(childIdx)); - } - - Plan logicalPlan; - if (physicalPlan instanceof PhysicalProject) { - PhysicalProject physicalProject = (PhysicalProject) physicalPlan; - logicalPlan = new LogicalProject<>(physicalProject.getProjects(), - new GroupPlan(groupExpression.child(0))); - } else if (physicalPlan instanceof AbstractPhysicalJoin) { - AbstractPhysicalJoin physicalJoin = (AbstractPhysicalJoin) physicalPlan; - logicalPlan = new LogicalJoin<>(physicalJoin.getJoinType(), physicalJoin.getHashJoinConjuncts(), - physicalJoin.getOtherJoinConjuncts(), - new DistributeHint(DistributeType.NONE), physicalJoin.getMarkJoinSlotReference(), - groupExpression.children().stream().map(g -> new GroupPlan(g)).collect(Collectors.toList())); - } else { - throw new RuntimeException("DPhyp can only handle join and project operator"); - } - jobContext.getCascadesContext().getMemo().copyIn(logicalPlan, root.get(), false, planTable); - } - } - - private List proposeProject(List allChild, List edges, long left, long right) { + private LogicalPlan proposeProject(LogicalPlan join, List edges, long left, long right) { long fullKey = LongBitmap.newBitmapUnion(left, right); - List outputs = allChild.get(0).getOutput(); - Set outputSet = allChild.get(0).getOutputSet(); + List outputs = join.getOutput(); + Set outputSet = join.getOutputSet(); List complexProjects = new ArrayList<>(); // Calculate complex expression should be done by current(fullKey) node @@ -354,40 +265,29 @@ private List proposeProject(List allChild, List edges, lon // calculate required columns by all parents Set requireSlots = calculateRequiredSlots(left, right, edges); List allProjects = Stream.concat( - outputs.stream().filter(e -> requireSlots.contains(e)), + outputs.stream().filter(requireSlots::contains), complexProjects.stream().filter(e -> requireSlots.contains(e.toSlot())) ).collect(Collectors.toList()); - // propose physical project + // propose logical project if (allProjects.isEmpty()) { allProjects.add(ExpressionUtils.selectMinimumColumn(outputs)); } if (outputSet.equals(new HashSet<>(allProjects))) { - return allChild; + return join; } - Set childOutputSet = allChild.get(0).getOutputSet(); + Set childOutputSet = join.getOutputSet(); List projects = allProjects.stream() .filter(expr -> childOutputSet.containsAll(expr.getInputSlots())) .collect(Collectors.toList()); + LogicalPlan project = join; if (!outputSet.equals(new HashSet<>(projects))) { - LogicalProperties projectProperties = new LogicalProperties( - () -> projects.stream() - .map(NamedExpression::toSlot) - .collect(ImmutableList.toImmutableList()), () -> FunctionalDependencies.EMPTY_FUNC_DEPS); - allChild = allChild.stream() - .map(c -> new PhysicalProject<>(projects, projectProperties, c)) - .collect(Collectors.toList()); - } - if (!(!projects.isEmpty() && projects.size() == allProjects.size())) { - Set s1 = projects.stream().collect(Collectors.toSet()); - List s2 = allProjects.stream().filter(e -> !s1.contains(e)).collect(Collectors.toList()); - System.out.println(s2); + project = new LogicalProject<>(projects, join); } Preconditions.checkState(!projects.isEmpty() && projects.size() == allProjects.size(), - " there are some projects left " + projects + allProjects); - - return allChild; + " there are some projects left %s %s", projects, allProjects); + return project; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java index 92a5bb9ddb15a6..bb5b2f3dcf813a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java @@ -226,10 +226,6 @@ public class RuleSet { .addAll(OTHER_REORDER_RULES) .build(); - public static final List DPHYP_REORDER_RULES = ImmutableList.builder() - .add(JoinCommute.BUSHY.build()) - .build(); - public static final List MATERIALIZED_VIEW_RULES = planRuleFactories() .add(MaterializedViewOnlyJoinRule.INSTANCE) .add(MaterializedViewProjectJoinRule.INSTANCE) @@ -243,6 +239,11 @@ public class RuleSet { .add(MaterializedViewFilterProjectAggregateRule.INSTANCE) .build(); + public static final List DPHYP_REORDER_RULES = ImmutableList.builder() + .addAll(MATERIALIZED_VIEW_RULES) + .add(JoinCommute.BUSHY.build()) + .build(); + public List getDPHypReorderRules() { return DPHYP_REORDER_RULES; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java index 6c78193abf1dba..3d98abd3af6af9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java @@ -91,6 +91,13 @@ public LogicalJoin(JoinType joinType, List hashJoinConjuncts, LEFT_C Optional.empty(), Optional.empty(), leftChild, rightChild); } + public LogicalJoin(JoinType joinType, List hashJoinConjuncts, List otherJoinConjuncts, + LEFT_CHILD_TYPE leftChild, RIGHT_CHILD_TYPE rightChild) { + this(joinType, hashJoinConjuncts, otherJoinConjuncts, + new DistributeHint(DistributeType.NONE), Optional.empty(), + Optional.empty(), Optional.empty(), leftChild, rightChild); + } + public LogicalJoin(JoinType joinType, List hashJoinConjuncts, List otherJoinConjuncts, DistributeHint hint, LEFT_CHILD_TYPE leftChild, RIGHT_CHILD_TYPE rightChild) { this(joinType, hashJoinConjuncts, otherJoinConjuncts, hint, Optional.empty(), Optional.empty(), diff --git a/regression-test/data/nereids_rules_p0/mv/join/dphyp_inner/inner_join_dphyp.out b/regression-test/data/nereids_rules_p0/mv/join/dphyp_inner/inner_join_dphyp.out new file mode 100644 index 00000000000000..1cf464023ee3bf --- /dev/null +++ b/regression-test/data/nereids_rules_p0/mv/join/dphyp_inner/inner_join_dphyp.out @@ -0,0 +1,335 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !query1_0_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query1_0_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query1_1_before -- +4 +4 +4 +4 +6 +6 +6 +6 + +-- !query1_1_after -- +4 +4 +4 +4 +6 +6 +6 +6 + +-- !query1_2_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query1_2_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query1_3_before -- +1 1 +1 1 +1 1 +1 1 +1 1 +2 2 +2 2 +2 2 + +-- !query1_3_after -- +1 1 +1 1 +1 1 +1 1 +1 1 +2 2 +2 2 +2 2 + +-- !query1_4_before -- +4 +4 +4 +4 +6 +6 +6 +6 + +-- !query1_4_after -- +4 +4 +4 +4 +6 +6 +6 +6 + +-- !query1_5_before -- +6 +6 + +-- !query1_5_after -- +6 +6 + +-- !query2_0_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query2_0_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query2_1_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query2_1_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query2_2_before -- +4 +4 +4 +4 +6 +6 + +-- !query2_2_after -- +4 +4 +4 +4 +6 +6 + +-- !query2_3_before -- +4 +4 +4 +4 +6 +6 +6 +6 + +-- !query2_3_after -- +4 +4 +4 +4 +6 +6 +6 +6 + +-- !query3_0_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query3_0_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query3_1_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query3_1_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query3_2_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query3_2_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query3_3_before -- +4 +4 +4 +4 +6 +6 +6 +6 + +-- !query3_3_after -- +4 +4 +4 +4 +6 +6 +6 +6 + +-- !query3_4_before -- +1 1 +1 1 +1 1 +1 1 +1 1 + +-- !query3_4_after -- +1 1 +1 1 +1 1 +1 1 +1 1 + +-- !query4_0_before -- +4 +4 +4 +4 +4 +4 + +-- !query4_0_after -- +4 +4 +4 +4 +4 +4 + +-- !query5_0_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query5_0_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query6_0_before -- +2 3 2023-12-08 +2 3 2023-12-08 + +-- !query6_0_after -- +2 3 2023-12-08 +2 3 2023-12-08 + +-- !query7_0_before -- +2 3 2023-12-08 +2 3 2023-12-08 + +-- !query7_0_after -- +2 3 2023-12-08 +2 3 2023-12-08 + +-- !query10_0_before -- + +-- !query10_0_after -- + diff --git a/regression-test/data/nereids_rules_p0/mv/join/dphyp_outer/outer_join_dphyp.out b/regression-test/data/nereids_rules_p0/mv/join/dphyp_outer/outer_join_dphyp.out new file mode 100644 index 00000000000000..845ef3933dc8ee --- /dev/null +++ b/regression-test/data/nereids_rules_p0/mv/join/dphyp_outer/outer_join_dphyp.out @@ -0,0 +1,297 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !query1_0_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query1_0_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query1_1_before -- +4 +4 +4 +4 +4 +4 +4 +4 +6 +6 +6 +6 + +-- !query1_1_after -- +4 +4 +4 +4 +4 +4 +4 +4 +6 +6 +6 +6 + +-- !query1_2_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query1_2_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query1_3_before -- +1 1 +1 1 +1 1 +1 1 +1 1 +2 2 +2 2 +2 2 + +-- !query1_3_after -- +1 1 +1 1 +1 1 +1 1 +1 1 +2 2 +2 2 +2 2 + +-- !query2_0_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query2_0_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query2_1_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query2_1_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query2_2_before -- +4 +4 +4 +4 +6 +6 + +-- !query2_2_after -- +4 +4 +4 +4 +6 +6 + +-- !query2_3_before -- +4 +4 +4 +4 +6 +6 + +-- !query2_3_after -- +4 +4 +4 +4 +6 +6 + +-- !query3_0_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query3_0_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query3_1_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query3_1_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query3_2_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query3_2_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query4_0_before -- +4 +4 + +-- !query4_0_after -- +4 +4 + +-- !query5_0_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query5_0_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query5_1_before -- +2023-12-08 2023-12-08 2 3 +2023-12-08 2023-12-08 2 3 + +-- !query5_1_after -- +2023-12-08 2023-12-08 2 3 +2023-12-08 2023-12-08 2 3 + +-- !query6_0_before -- +2 3 2023-12-08 +2 3 2023-12-08 +2 3 2023-12-12 +2 4 2023-12-10 +3 3 2023-12-11 +4 3 2023-12-09 + +-- !query6_0_after -- +2 3 2023-12-08 +2 3 2023-12-08 +2 3 2023-12-12 +2 4 2023-12-10 +3 3 2023-12-11 +4 3 2023-12-09 + +-- !query7_0_before -- +3 3 2023-12-11 + +-- !query7_0_after -- +3 3 2023-12-11 + +-- !query7_1_before -- + +-- !query7_1_after -- + +-- !query8_0_before -- +1 0 8 0 10.0000 10.50 9.50 +2 0 2 0 11.5000 11.50 11.50 +3 0 0 0 23.0000 33.50 12.50 +4 0 0 0 43.2000 43.20 43.20 +5 0 0 0 28.7000 56.20 1.20 + +-- !query8_0_after -- +1 0 8 0 10.0000 10.50 9.50 +2 0 2 0 11.5000 11.50 11.50 +3 0 0 0 23.0000 33.50 12.50 +4 0 0 0 43.2000 43.20 43.20 +5 0 0 0 28.7000 56.20 1.20 + diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query64.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query64.out index fed32dc285e8d8..7c84cef7588bbc 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query64.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query64.out @@ -9,72 +9,42 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ------------PhysicalProject --------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_shipto_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF19 d_date_sk->[c_first_shipto_date_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) build RFs:RF18 ss_customer_sk->[c_customer_sk] +------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) build RFs:RF18 c_customer_sk->[ss_customer_sk] --------------------PhysicalDistribute[DistributionSpecHash] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = ad2.ca_address_sk)) otherCondition=() build RFs:RF17 ca_address_sk->[c_current_addr_sk] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF16 cd_demo_sk->[c_current_cdemo_sk] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[c_first_sales_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_hdemo_sk = hd2.hd_demo_sk)) otherCondition=() build RFs:RF14 hd_demo_sk->[c_current_hdemo_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer] apply RFs: RF14 RF15 RF16 RF17 RF18 RF19 -------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((hd2.hd_income_band_sk = ib2.ib_income_band_sk)) otherCondition=() build RFs:RF13 ib_income_band_sk->[hd_income_band_sk] -------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF13 -------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[income_band] ---------------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF11 ss_item_sk->[sr_item_sk];RF12 ss_ticket_number->[sr_ticket_number] +------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF17 p_promo_sk->[ss_promo_sk] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_returns] apply RFs: RF11 RF12 ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = cs_ui.cs_item_sk)) otherCondition=() build RFs:RF10 cs_item_sk->[ss_item_sk,i_item_sk] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF9 p_promo_sk->[ss_promo_sk] +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() build RFs:RF16 ca_address_sk->[ss_addr_sk] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF14 ss_item_sk->[sr_item_sk];RF15 ss_ticket_number->[sr_ticket_number] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF14 RF15 +------------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF8 cd_demo_sk->[ss_cdemo_sk] +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = cs_ui.cs_item_sk)) otherCondition=() build RFs:RF13 cs_item_sk->[ss_item_sk,i_item_sk] ------------------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() build RFs:RF7 ca_address_sk->[ss_addr_sk] +----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF12 cd_demo_sk->[ss_cdemo_sk] ------------------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF6 i_item_sk->[ss_item_sk] +--------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk] ----------------------------------------------------PhysicalProject -------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] +------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF10 s_store_sk->[ss_store_sk] --------------------------------------------------------PhysicalProject -----------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((hd1.hd_income_band_sk = ib1.ib_income_band_sk)) otherCondition=() build RFs:RF4 ib_income_band_sk->[hd_income_band_sk] +----------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((hd1.hd_income_band_sk = ib1.ib_income_band_sk)) otherCondition=() build RFs:RF9 ib_income_band_sk->[hd_income_band_sk] ------------------------------------------------------------PhysicalProject ---------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = hd1.hd_demo_sk)) otherCondition=() build RFs:RF3 hd_demo_sk->[ss_hdemo_sk] +--------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = hd1.hd_demo_sk)) otherCondition=() build RFs:RF8 hd_demo_sk->[ss_hdemo_sk] ----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[ss_sold_date_sk] --------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF5 RF6 RF7 RF8 RF9 RF10 +----------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF7 RF8 RF10 RF11 RF12 RF13 RF16 RF17 RF18 --------------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------------------------------------------------PhysicalProject ------------------------------------------------------------------------filter(d_year IN (1999, 2000)) --------------------------------------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF4 +--------------------------------------------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF9 ------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------------------------------------PhysicalProject ----------------------------------------------------------------PhysicalOlapScan[income_band] @@ -84,27 +54,57 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ----------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------------------------------PhysicalProject --------------------------------------------------------filter((item.i_current_price <= 58.00) and (item.i_current_price >= 49.00) and i_color IN ('blush', 'lace', 'lawn', 'misty', 'orange', 'pink')) -----------------------------------------------------------PhysicalOlapScan[item] apply RFs: RF10 +----------------------------------------------------------PhysicalOlapScan[item] apply RFs: RF13 ------------------------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[customer_address] -------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------------PhysicalProject +--------------------------------------------filter((sale > (2 * refund))) +----------------------------------------------hashAgg[GLOBAL] +------------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------------hashAgg[LOCAL] +----------------------------------------------------PhysicalProject +------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF5 cr_order_number->[cs_order_number];RF6 cr_item_sk->[cs_item_sk] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 RF6 +--------------------------------------------------------PhysicalProject +----------------------------------------------------------PhysicalOlapScan[catalog_returns] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_address] +--------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[promotion] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = ad2.ca_address_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF3 cd_demo_sk->[c_current_cdemo_sk] +--------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[c_first_sales_date_sk] +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_hdemo_sk = hd2.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[c_current_hdemo_sk] +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 RF2 RF3 RF4 RF19 +------------------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((hd2.hd_income_band_sk = ib2.ib_income_band_sk)) otherCondition=() build RFs:RF0 ib_income_band_sk->[hd_income_band_sk] +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF0 +------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[income_band] --------------------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[promotion] ---------------------------------PhysicalProject -----------------------------------filter((sale > (2 * refund))) -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF0 cr_order_number->[cs_order_number];RF1 cr_item_sk->[cs_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_returns] +------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[customer_demographics] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer_address] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query64.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query64.out index 5443c04d5c4b81..e5e915d4422886 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query64.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query64.out @@ -11,52 +11,27 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ----------------PhysicalProject ------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) build RFs:RF18 ss_customer_sk->[c_customer_sk] +----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = ad2.ca_address_sk)) otherCondition=() -------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_hdemo_sk = hd2.hd_demo_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer] apply RFs: RF18 -------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((hd2.hd_income_band_sk = ib2.ib_income_band_sk)) otherCondition=() -------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[household_demographics] -------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[income_band] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_demographics] -------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer_address] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF12 ss_item_sk->[sr_item_sk];RF13 ss_ticket_number->[sr_ticket_number] +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF12 RF13 -------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() +--------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() build RFs:RF16 ss_addr_sk->[ca_address_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF16 +----------------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF10 ss_cdemo_sk->[cd_demo_sk] -----------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF10 +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF14 ss_item_sk->[sr_item_sk];RF15 ss_ticket_number->[sr_ticket_number] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF14 RF15 ----------------------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() build RFs:RF9 ss_addr_sk->[ca_address_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF9 +--------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF13 ss_cdemo_sk->[cd_demo_sk] +----------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF13 ----------------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF7 i_item_sk->[ss_item_sk];RF8 i_item_sk->[cs_item_sk] +------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk];RF12 i_item_sk->[cs_item_sk] --------------------------------------------------PhysicalProject ----------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() ------------------------------------------------------PhysicalProject @@ -64,11 +39,11 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ----------------------------------------------------------PhysicalProject ------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = hd1.hd_demo_sk)) otherCondition=() --------------------------------------------------------------PhysicalProject -----------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +----------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[ss_sold_date_sk] ------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = cs_ui.cs_item_sk)) otherCondition=() build RFs:RF2 cs_item_sk->[ss_item_sk] +--------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = cs_ui.cs_item_sk)) otherCondition=() build RFs:RF6 cs_item_sk->[ss_item_sk] ----------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF7 +------------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF11 ----------------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------------------------------------------------PhysicalProject --------------------------------------------------------------------------filter((sale > (2 * refund))) @@ -76,9 +51,9 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ------------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------------------------------------------------------hashAgg[LOCAL] ----------------------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF0 cr_order_number->[cs_order_number];RF1 cr_item_sk->[cs_item_sk] +------------------------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF4 cr_order_number->[cs_order_number];RF5 cr_item_sk->[cs_item_sk] --------------------------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF8 +----------------------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF12 --------------------------------------------------------------------------------------PhysicalProject ----------------------------------------------------------------------------------------PhysicalOlapScan[catalog_returns] ------------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] @@ -98,9 +73,34 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ----------------------------------------------------PhysicalProject ------------------------------------------------------filter((item.i_current_price <= 33.00) and (item.i_current_price >= 24.00) and i_color IN ('blanched', 'brown', 'burlywood', 'chocolate', 'drab', 'medium')) --------------------------------------------------------PhysicalOlapScan[item] -------------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[promotion] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = ad2.ca_address_sk)) otherCondition=() +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() +------------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[promotion] +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_hdemo_sk = hd2.hd_demo_sk)) otherCondition=() +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[customer] +------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------------------PhysicalProject +----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((hd2.hd_income_band_sk = ib2.ib_income_band_sk)) otherCondition=() +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[household_demographics] +------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[income_band] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_address] --------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------PhysicalProject ------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query64.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query64.out index 55860e7b0b6239..43bdb50fcce033 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query64.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query64.out @@ -11,64 +11,39 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ----------------PhysicalProject ------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF19 d_date_sk->[c_first_sales_date_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) build RFs:RF18 ss_customer_sk->[c_customer_sk] +----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) build RFs:RF18 c_customer_sk->[ss_customer_sk] ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = ad2.ca_address_sk)) otherCondition=() build RFs:RF17 ca_address_sk->[c_current_addr_sk] -------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF16 cd_demo_sk->[c_current_cdemo_sk] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_hdemo_sk = hd2.hd_demo_sk)) otherCondition=() build RFs:RF15 hd_demo_sk->[c_current_hdemo_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer] apply RFs: RF15 RF16 RF17 RF18 RF19 RF20 -------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((hd2.hd_income_band_sk = ib2.ib_income_band_sk)) otherCondition=() build RFs:RF14 ib_income_band_sk->[hd_income_band_sk] -------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF14 -------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[income_band] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_demographics] -------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer_address] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF12 ss_item_sk->[sr_item_sk];RF13 ss_ticket_number->[sr_ticket_number] +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF17 p_promo_sk->[ss_promo_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF12 RF13 -------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF11 p_promo_sk->[ss_promo_sk] +--------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() build RFs:RF16 ss_addr_sk->[ca_address_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF16 +----------------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF10 ss_cdemo_sk->[cd_demo_sk] -----------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF10 +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF14 ss_item_sk->[sr_item_sk];RF15 ss_ticket_number->[sr_ticket_number] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF14 RF15 ----------------------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() build RFs:RF9 ss_addr_sk->[ca_address_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF9 +--------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF13 ss_cdemo_sk->[cd_demo_sk] +----------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF13 ----------------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF7 i_item_sk->[ss_item_sk];RF8 i_item_sk->[cs_item_sk] +------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk];RF12 i_item_sk->[cs_item_sk] --------------------------------------------------PhysicalProject -----------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF6 s_store_sk->[ss_store_sk] +----------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF10 s_store_sk->[ss_store_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((hd1.hd_income_band_sk = ib1.ib_income_band_sk)) otherCondition=() build RFs:RF5 ib_income_band_sk->[hd_income_band_sk] +--------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((hd1.hd_income_band_sk = ib1.ib_income_band_sk)) otherCondition=() build RFs:RF9 ib_income_band_sk->[hd_income_band_sk] ----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = hd1.hd_demo_sk)) otherCondition=() build RFs:RF4 hd_demo_sk->[ss_hdemo_sk] +------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = hd1.hd_demo_sk)) otherCondition=() build RFs:RF8 hd_demo_sk->[ss_hdemo_sk] --------------------------------------------------------------PhysicalProject -----------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +----------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[ss_sold_date_sk] ------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = cs_ui.cs_item_sk)) otherCondition=() build RFs:RF2 cs_item_sk->[ss_item_sk] +--------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = cs_ui.cs_item_sk)) otherCondition=() build RFs:RF6 cs_item_sk->[ss_item_sk] ----------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF4 RF6 RF7 RF11 +------------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 RF10 RF11 RF17 RF18 ----------------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------------------------------------------------PhysicalProject --------------------------------------------------------------------------filter((sale > (2 * refund))) @@ -76,9 +51,9 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ------------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------------------------------------------------------hashAgg[LOCAL] ----------------------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF0 cr_order_number->[cs_order_number];RF1 cr_item_sk->[cs_item_sk] +------------------------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF4 cr_order_number->[cs_order_number];RF5 cr_item_sk->[cs_item_sk] --------------------------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF8 +----------------------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF12 --------------------------------------------------------------------------------------PhysicalProject ----------------------------------------------------------------------------------------PhysicalOlapScan[catalog_returns] ------------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] @@ -87,7 +62,7 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ------------------------------------------------------------------------PhysicalOlapScan[date_dim] --------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF5 +------------------------------------------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF9 ----------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------------------------------------PhysicalProject --------------------------------------------------------------PhysicalOlapScan[income_band] @@ -98,9 +73,34 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ----------------------------------------------------PhysicalProject ------------------------------------------------------filter((item.i_current_price <= 33.00) and (item.i_current_price >= 24.00) and i_color IN ('blanched', 'brown', 'burlywood', 'chocolate', 'drab', 'medium')) --------------------------------------------------------PhysicalOlapScan[item] -------------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[promotion] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = ad2.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[c_current_cdemo_sk] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_hdemo_sk = hd2.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[c_current_hdemo_sk] +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 RF2 RF3 RF19 RF20 +------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------------------PhysicalProject +----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((hd2.hd_income_band_sk = ib2.ib_income_band_sk)) otherCondition=() build RFs:RF0 ib_income_band_sk->[hd_income_band_sk] +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF0 +------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[income_band] +------------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[promotion] +----------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_address] --------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------PhysicalProject ------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/suites/nereids_rules_p0/mv/join/dphyp_inner/inner_join_dphyp.groovy b/regression-test/suites/nereids_rules_p0/mv/join/dphyp_inner/inner_join_dphyp.groovy new file mode 100644 index 00000000000000..5fd8db4bcdcf6e --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/join/dphyp_inner/inner_join_dphyp.groovy @@ -0,0 +1,464 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +suite("inner_join_dphyp") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "SET enable_nereids_planner=true" + sql "set runtime_filter_mode=OFF" + sql "SET enable_fallback_to_original_planner=false" + sql "SET enable_materialized_view_rewrite=true" + sql "SET enable_nereids_timeout = false" + sql "SET enable_dphyp_optimizer = true" + + sql """ + drop table if exists orders + """ + + sql """ + CREATE TABLE IF NOT EXISTS orders ( + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, + o_comment VARCHAR(79) NOT NULL + ) + DUPLICATE KEY(o_orderkey, o_custkey) + PARTITION BY RANGE(o_orderdate) (PARTITION `day_2` VALUES LESS THAN ('2023-12-30')) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + sql """ + drop table if exists lineitem + """ + + sql""" + CREATE TABLE IF NOT EXISTS lineitem ( + l_orderkey INTEGER NOT NULL, + l_partkey INTEGER NOT NULL, + l_suppkey INTEGER NOT NULL, + l_linenumber INTEGER NOT NULL, + l_quantity DECIMALV3(15,2) NOT NULL, + l_extendedprice DECIMALV3(15,2) NOT NULL, + l_discount DECIMALV3(15,2) NOT NULL, + l_tax DECIMALV3(15,2) NOT NULL, + l_returnflag CHAR(1) NOT NULL, + l_linestatus CHAR(1) NOT NULL, + l_shipdate DATE NOT NULL, + l_commitdate DATE NOT NULL, + l_receiptdate DATE NOT NULL, + l_shipinstruct CHAR(25) NOT NULL, + l_shipmode CHAR(10) NOT NULL, + l_comment VARCHAR(44) NOT NULL + ) + DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber) + PARTITION BY RANGE(l_shipdate) (PARTITION `day_1` VALUES LESS THAN ('2023-12-30')) + DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + sql """ + drop table if exists partsupp + """ + + sql """ + CREATE TABLE IF NOT EXISTS partsupp ( + ps_partkey INTEGER NOT NULL, + ps_suppkey INTEGER NOT NULL, + ps_availqty INTEGER NOT NULL, + ps_supplycost DECIMALV3(15,2) NOT NULL, + ps_comment VARCHAR(199) NOT NULL + ) + DUPLICATE KEY(ps_partkey, ps_suppkey) + DISTRIBUTED BY HASH(ps_partkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + sql """ insert into lineitem values + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx'); + """ + + sql """ + insert into orders values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); + """ + + sql """ + insert into partsupp values + (2, 3, 9, 10.01, 'supply1'), + (2, 3, 10, 11.01, 'supply2'); + """ + + def check_rewrite = { mv_sql, query_sql, mv_name -> + + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS ${mv_sql} + """ + + def job_name = getJobName(db, mv_name); + waitingMTMVTaskFinished(job_name) + explain { + sql("${query_sql}") + contains("${mv_name}(${mv_name})") + } + } + + def check_not_match = { mv_sql, query_sql, mv_name -> + + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS ${mv_sql} + """ + + def job_name = getJobName(db, mv_name); + waitingMTMVTaskFinished(job_name) + explain { + sql("${query_sql}") + notContains("${mv_name}(${mv_name})") + } + } + + // without filter + def mv1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + def query1_0 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + order_qt_query1_0_before "${query1_0}" + check_rewrite(mv1_0, query1_0, "mv1_0") + order_qt_query1_0_after "${query1_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_0""" + + + def mv1_1 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, partsupp.PS_AVAILQTY " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " + + "and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY" + def query1_1 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " + + "and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY" + order_qt_query1_1_before "${query1_1}" + check_rewrite(mv1_1, query1_1, "mv1_1") + order_qt_query1_1_after "${query1_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_1""" + + def mv1_2 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + + "from orders " + + "inner join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + def query1_2 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + order_qt_query1_2_before "${query1_2}" + // join direction is not same, should not match + check_rewrite(mv1_2, query1_2, "mv1_2") + order_qt_query1_2_after "${query1_2}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_2""" + + // select with complex expression + def mv1_3 = "select l_linenumber, o_custkey " + + "from orders " + + "inner join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + def query1_3 = "select IFNULL(orders.O_CUSTKEY, 0) as custkey_not_null, " + + "case when l_linenumber in (1,2,3) then l_linenumber else o_custkey end as case_when " + + "from orders " + + "inner join lineitem on orders.O_ORDERKEY = lineitem.L_ORDERKEY" + order_qt_query1_3_before "${query1_3}" + check_rewrite(mv1_3, query1_3, "mv1_3") + order_qt_query1_3_after "${query1_3}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_3""" + + def mv1_4 = """ + select lineitem.L_LINENUMBER, orders.O_CUSTKEY, partsupp.PS_AVAILQTY + from lineitem + inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY + inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY + and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY; + """ + def query1_4 = """ + select lineitem.L_LINENUMBER + from lineitem + inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY + inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY + and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY; + """ + order_qt_query1_4_before "${query1_4}" + check_rewrite(mv1_4, query1_4, "mv1_4") + order_qt_query1_4_after "${query1_4}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_4""" + + def mv1_5 = """ + select lineitem.L_LINENUMBER, orders.O_CUSTKEY, l_partkey, o_shippriority + from lineitem + inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY; + """ + def query1_5 = """ + select lineitem.L_LINENUMBER + from lineitem + inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY + and o_shippriority = l_partkey; + """ + order_qt_query1_5_before "${query1_5}" + check_rewrite(mv1_5, query1_5, "mv1_5") + order_qt_query1_5_after "${query1_5}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_5""" + + // filter outside + left + def mv2_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + + "from orders " + + "inner join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + def query2_0 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where lineitem.L_LINENUMBER > 0" + order_qt_query2_0_before "${query2_0}" + check_rewrite(mv2_0, query2_0, "mv2_0") + order_qt_query2_0_after "${query2_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_0""" + + + def mv2_1 = "select t1.L_LINENUMBER, orders.O_CUSTKEY " + + "from (select * from lineitem where L_LINENUMBER > 1) t1 " + + "inner join orders on t1.L_ORDERKEY = orders.O_ORDERKEY " + def query2_1 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where lineitem.L_LINENUMBER > 1" + order_qt_query2_1_before "${query2_1}" + check_rewrite(mv2_1, query2_1, "mv2_1") + order_qt_query2_1_after "${query2_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_1""" + + + def mv2_2 = "select t1.L_LINENUMBER, orders.O_CUSTKEY, l_suppkey " + + "from (select * from lineitem where L_LINENUMBER > 1) t1 " + + "inner join orders on t1.L_ORDERKEY = orders.O_ORDERKEY " + def query2_2 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where lineitem.L_LINENUMBER > 1 and l_suppkey = 3" + order_qt_query2_2_before "${query2_2}" + check_rewrite(mv2_2, query2_2, "mv2_2") + order_qt_query2_2_after "${query2_2}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_2""" + + + def mv2_3 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, partsupp.PS_AVAILQTY, l_suppkey " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " + + "and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY" + def query2_3= "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " + + "and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY " + + "where lineitem.L_LINENUMBER > 1 and l_suppkey = 3 " + order_qt_query2_3_before "${query2_3}" + check_rewrite(mv2_3, query2_3, "mv2_3") + order_qt_query2_3_after "${query2_3}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_3""" + + + // filter outside + right + def mv3_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + def query3_0 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where orders.O_ORDERSTATUS = 'o'" + order_qt_query3_0_before "${query3_0}" + // use a filed not from mv, should not success + check_not_match(mv3_0, query3_0, "mv3_0") + order_qt_query3_0_after "${query3_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_0""" + + + def mv3_1 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, orders.O_ORDERSTATUS " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + def query3_1 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where orders.O_ORDERSTATUS = 'o'" + order_qt_query3_1_before "${query3_1}" + check_rewrite(mv3_1, query3_1, "mv3_1") + order_qt_query3_1_after "${query3_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_1""" + + + def mv3_2 = "select lineitem.L_LINENUMBER, t2.O_CUSTKEY, t2.O_ORDERSTATUS " + + "from lineitem " + + "inner join " + + "(select * from orders where O_ORDERSTATUS = 'o') t2 " + + "on lineitem.L_ORDERKEY = t2.O_ORDERKEY " + def query3_2 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where orders.O_ORDERSTATUS = 'o'" + order_qt_query3_2_before "${query3_2}" + check_rewrite(mv3_2, query3_2, "mv3_2") + order_qt_query3_2_after "${query3_2}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_2""" + + + def mv3_3 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, partsupp.PS_AVAILQTY " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " + + "and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY" + def query3_3= "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " + + "and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY " + + "where o_custkey in (1, 2, 3, 4) " + order_qt_query3_3_before "${query3_3}" + check_rewrite(mv3_3, query3_3, "mv3_3") + order_qt_query3_3_after "${query3_3}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_3""" + + // join derive, the mv is outer join with filter and query is inner join + // the predicate should be ComparisonPredicate + def mv3_4 = """ + select l_linenumber, o_custkey + from orders + left join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY + where o_custkey = 1; + """ + def query3_4 = """ + select IFNULL(orders.O_CUSTKEY, 0) as custkey_not_null, + case when l_linenumber in (1,2,3) then l_linenumber else o_custkey end as case_when + from orders + inner join lineitem on orders.O_ORDERKEY = lineitem.L_ORDERKEY + where o_custkey = 1 and l_linenumber > 0; + """ + order_qt_query3_4_before "${query3_4}" + check_rewrite(mv3_4, query3_4, "mv3_4") + order_qt_query3_4_after "${query3_4}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_4""" + + + // filter outside + left + right + def mv4_0 = "select l_linenumber, o_custkey, o_orderkey, o_orderstatus " + + "from lineitem " + + "inner join orders on lineitem.l_orderkey = orders.o_orderkey " + def query4_0 = "select lineitem.l_linenumber " + + "from lineitem " + + "inner join orders on lineitem.l_orderkey = orders.o_orderkey " + + "where o_orderstatus = 'o' AND l_linenumber in (1, 2, 3, 4, 5) " + order_qt_query4_0_before "${query4_0}" + check_rewrite(mv4_0, query4_0, "mv4_0") + order_qt_query4_0_after "${query4_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv4_0""" + + + // filter inside + left + def mv5_0 = "select lineitem.l_linenumber, orders.o_custkey " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where lineitem.L_LINENUMBER > 1" + def query5_0 = "select t1.L_LINENUMBER " + + "from (select * from lineitem where l_linenumber > 1) t1 " + + "inner join orders on t1.l_orderkey = orders.O_ORDERKEY " + order_qt_query5_0_before "${query5_0}" + check_rewrite(mv5_0, query5_0, "mv5_0") + order_qt_query5_0_after "${query5_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv5_0""" + + + // filter inside + right + def mv6_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey " + + "from lineitem " + + "inner join (select * from orders where o_orderdate = '2023-12-08') t2 " + + "on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate " + def query6_0 = "select l_partkey, l_suppkey, l_shipdate " + + "from lineitem t1 " + + "inner join (select * from orders where o_orderdate = '2023-12-08') t2 " + + "on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " + order_qt_query6_0_before "${query6_0}" + check_rewrite(mv6_0, query6_0, "mv6_0") + order_qt_query6_0_after "${query6_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv6_0""" + + + // filter inside + left + right + def mv7_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey " + + "from lineitem " + + "inner join (select * from orders where o_orderdate = '2023-12-08') t2 " + + "on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate " + def query7_0 = "select l_partkey, l_suppkey, l_shipdate " + + "from (select l_shipdate, l_orderkey, l_partkey, l_suppkey " + + "from lineitem where l_partkey in (2, 3, 4)) t1 " + + "inner join (select * from orders where o_orderdate = '2023-12-08') t2 " + + "on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " + + "where l_partkey = 2" + order_qt_query7_0_before "${query7_0}" + check_rewrite(mv7_0, query7_0, "mv7_0") + order_qt_query7_0_after "${query7_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv7_0""" + + + // check not match, because use a filed orders.O_SHIPPRIORITY which not in mv + def mv10_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY" + def query10_0 = "select orders.O_CUSTKEY " + + "from orders " + + "inner join lineitem on orders.O_ORDERKEY = lineitem.L_ORDERKEY " + + "WHERE lineitem.L_LINENUMBER > 0 AND orders.O_CUSTKEY = 1 AND " + + "orders.O_SHIPPRIORITY = 2" + order_qt_query10_0_before "${query10_0}" + check_not_match(mv10_0, query10_0, "mv10_0") + order_qt_query10_0_after "${query10_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv10_0""" +} diff --git a/regression-test/suites/nereids_rules_p0/mv/join/dphyp_outer/outer_join_dphyp.groovy b/regression-test/suites/nereids_rules_p0/mv/join/dphyp_outer/outer_join_dphyp.groovy new file mode 100644 index 00000000000000..de253de7823c25 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/join/dphyp_outer/outer_join_dphyp.groovy @@ -0,0 +1,469 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +suite("outer_join_dphyp") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "SET enable_nereids_planner=true" + sql "set runtime_filter_mode=OFF"; + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + sql "SET enable_fallback_to_original_planner=false" + sql "SET enable_materialized_view_rewrite=true" + sql "SET enable_nereids_timeout = false" + sql "SET enable_dphyp_optimizer = true" + sql """ + drop table if exists orders + """ + + sql """ + CREATE TABLE IF NOT EXISTS orders ( + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, + O_COMMENT VARCHAR(79) NOT NULL + ) + DUPLICATE KEY(o_orderkey, o_custkey) + PARTITION BY RANGE(o_orderdate) (PARTITION `day_2` VALUES LESS THAN ('2023-12-30')) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + drop table if exists lineitem + """ + + sql""" + CREATE TABLE IF NOT EXISTS lineitem ( + l_orderkey INTEGER NOT NULL, + l_partkey INTEGER NOT NULL, + l_suppkey INTEGER NOT NULL, + l_linenumber INTEGER NOT NULL, + l_quantity DECIMALV3(15,2) NOT NULL, + l_extendedprice DECIMALV3(15,2) NOT NULL, + l_discount DECIMALV3(15,2) NOT NULL, + l_tax DECIMALV3(15,2) NOT NULL, + l_returnflag CHAR(1) NOT NULL, + l_linestatus CHAR(1) NOT NULL, + l_shipdate DATE NOT NULL, + l_commitdate DATE NOT NULL, + l_receiptdate DATE NOT NULL, + l_shipinstruct CHAR(25) NOT NULL, + l_shipmode CHAR(10) NOT NULL, + l_comment VARCHAR(44) NOT NULL + ) + DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber) + PARTITION BY RANGE(l_shipdate) (PARTITION `day_1` VALUES LESS THAN ('2023-12-30')) + DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + drop table if exists partsupp + """ + + sql """ + CREATE TABLE IF NOT EXISTS partsupp ( + ps_partkey INTEGER NOT NULL, + ps_suppkey INTEGER NOT NULL, + ps_availqty INTEGER NOT NULL, + ps_supplycost DECIMALV3(15,2) NOT NULL, + ps_comment VARCHAR(199) NOT NULL + ) + DUPLICATE KEY(ps_partkey, ps_suppkey) + DISTRIBUTED BY HASH(ps_partkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ insert into lineitem values + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx'); + """ + + sql """ + insert into orders values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); + """ + + sql """ + insert into partsupp values + (2, 3, 9, 10.01, 'supply1'), + (2, 3, 10, 11.01, 'supply2'); + """ + + def check_rewrite = { mv_sql, query_sql, mv_name -> + + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS ${mv_sql} + """ + + def job_name = getJobName(db, mv_name); + waitingMTMVTaskFinished(job_name) + explain { + sql("${query_sql}") + contains("${mv_name}(${mv_name})") + } + } + + def check_not_match = { mv_sql, query_sql, mv_name -> + + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS ${mv_sql} + """ + + def job_name = getJobName(db, mv_name); + waitingMTMVTaskFinished(job_name) + explain { + sql("${query_sql}") + notContains("${mv_name}(${mv_name})") + } + } + + // without filter + def mv1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + + "from lineitem " + + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + def query1_0 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + order_qt_query1_0_before "${query1_0}" + check_rewrite(mv1_0, query1_0, "mv1_0") + order_qt_query1_0_after "${query1_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_0""" + + + def mv1_1 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, partsupp.PS_AVAILQTY " + + "from lineitem " + + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "left join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " + + "and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY" + def query1_1 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "left join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " + + "and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY" + order_qt_query1_1_before "${query1_1}" + check_rewrite(mv1_1, query1_1, "mv1_1") + order_qt_query1_1_after "${query1_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_1""" + + def mv1_2 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + + "from orders " + + "left join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + def query1_2 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + order_qt_query1_2_before "${query1_2}" + // join direction is not same, should not match + check_not_match(mv1_2, query1_2, "mv1_2") + order_qt_query1_2_after "${query1_2}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_2""" + + // select with complex expression + def mv1_3 = "select l_linenumber, o_custkey " + + "from orders " + + "left join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + def query1_3 = "select IFNULL(orders.O_CUSTKEY, 0) as custkey_not_null, " + + "case when l_linenumber in (1,2,3) then l_linenumber else o_custkey end as case_when " + + "from orders " + + "left join lineitem on orders.O_ORDERKEY = lineitem.L_ORDERKEY" + order_qt_query1_3_before "${query1_3}" + check_rewrite(mv1_3, query1_3, "mv1_3") + order_qt_query1_3_after "${query1_3}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_3""" + + + // filter outside + left + def mv2_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + + "from orders " + + "left join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + def query2_0 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where lineitem.L_LINENUMBER > 0" + order_qt_query2_0_before "${query2_0}" + check_not_match(mv2_0, query2_0, "mv2_0") + order_qt_query2_0_after "${query2_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_0""" + + + def mv2_1 = "select t1.L_LINENUMBER, orders.O_CUSTKEY " + + "from (select * from lineitem where L_LINENUMBER > 1) t1 " + + "left join orders on t1.L_ORDERKEY = orders.O_ORDERKEY " + def query2_1 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where lineitem.L_LINENUMBER > 1" + order_qt_query2_1_before "${query2_1}" + check_rewrite(mv2_1, query2_1, "mv2_1") + order_qt_query2_1_after "${query2_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_1""" + + + def mv2_2 =""" + select t1.L_LINENUMBER, orders.O_CUSTKEY + from (select * from lineitem where L_LINENUMBER > 1) t1 + left join orders on t1.L_ORDERKEY = orders.O_ORDERKEY; + """ + def query2_2 = """ + select lineitem.L_LINENUMBER + from lineitem + left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY + where lineitem.L_LINENUMBER > 1 and l_suppkey = 3; + """ + order_qt_query2_2_before "${query2_2}" + check_not_match(mv2_2, query2_2, "mv2_2") + order_qt_query2_2_after "${query2_2}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_2""" + + + def mv2_3 =""" + select t1.L_LINENUMBER, orders.O_CUSTKEY, l_suppkey + from (select * from lineitem where L_LINENUMBER > 1) t1 + left join orders on t1.L_ORDERKEY = orders.O_ORDERKEY; + """ + def query2_3 = """ + select lineitem.L_LINENUMBER + from lineitem + left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY + where lineitem.L_LINENUMBER > 1 and l_suppkey = 3; + """ + order_qt_query2_3_before "${query2_3}" + check_rewrite(mv2_3, query2_3, "mv2_3") + order_qt_query2_3_after "${query2_3}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_3""" + + + // filter outside + right + def mv3_0 = """ + select lineitem.L_LINENUMBER, orders.O_CUSTKEY + from lineitem + left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY; + """ + def query3_0 = """ + select lineitem.L_LINENUMBER + from lineitem + left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY + where orders.O_ORDERSTATUS = 'o'; + """ + order_qt_query3_0_before "${query3_0}" + // use a filed not from mv, should not success + check_not_match(mv3_0, query3_0, "mv3_0") + order_qt_query3_0_after "${query3_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_0""" + + + def mv3_1 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, orders.O_ORDERSTATUS " + + "from lineitem " + + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + def query3_1 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where orders.O_ORDERSTATUS = 'o'" + order_qt_query3_1_before "${query3_1}" + check_rewrite(mv3_1, query3_1, "mv3_1") + order_qt_query3_1_after "${query3_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_1""" + + + def mv3_2 = "select lineitem.L_LINENUMBER, t2.O_CUSTKEY, t2.O_ORDERSTATUS " + + "from lineitem " + + "left join " + + "(select * from orders where O_ORDERSTATUS = 'o') t2 " + + "on lineitem.L_ORDERKEY = t2.O_ORDERKEY " + def query3_2 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where orders.O_ORDERSTATUS = 'o'" + order_qt_query3_2_before "${query3_2}" + // should not success, as mv filter is under left outer input + check_not_match(mv3_2, query3_2, "mv3_2") + order_qt_query3_2_after "${query3_2}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_2""" + + + // filter outside + left + right + def mv4_0 = """ + select l_linenumber, o_custkey, o_orderkey, o_orderstatus + from lineitem + left join orders on lineitem.l_orderkey = orders.o_orderkey; + """ + def query4_0 = """ + select lineitem.l_linenumber + from lineitem + left join orders on lineitem.l_orderkey = orders.o_orderkey + where o_orderstatus = 'o' AND o_orderkey = 1; + """ + order_qt_query4_0_before "${query4_0}" + check_rewrite(mv4_0, query4_0, "mv4_0") + order_qt_query4_0_after "${query4_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv4_0""" + + + // filter inside + left + def mv5_0 = "select lineitem.l_linenumber, orders.o_custkey " + + "from lineitem " + + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where lineitem.L_LINENUMBER > 1" + def query5_0 = "select t1.L_LINENUMBER " + + "from (select * from lineitem where l_linenumber > 1) t1 " + + "left join orders on t1.l_orderkey = orders.O_ORDERKEY " + order_qt_query5_0_before "${query5_0}" + check_rewrite(mv5_0, query5_0, "mv5_0") + order_qt_query5_0_after "${query5_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv5_0""" + + + def mv5_1 = """ + select l_shipdate, o_orderdate, l_partkey, l_suppkey + from (select * from lineitem where l_shipdate = '2023-12-08' ) t1 + left join orders + on t1.l_orderkey = orders.o_orderkey + """ + def query5_1 = """ + select l_shipdate, o_orderdate, l_partkey, l_suppkey + from lineitem + left join orders + on lineitem.l_orderkey = orders.o_orderkey + where o_orderdate = '2023-12-08' + """ + order_qt_query5_1_before "${query5_1}" + check_not_match(mv5_1, query5_1, "mv5_1") + order_qt_query5_1_after "${query5_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv5_1""" + + + // filter inside + right + def mv6_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey " + + "from lineitem " + + "left join (select * from orders where o_orderdate = '2023-12-08') t2 " + + "on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate " + def query6_0 = "select l_partkey, l_suppkey, l_shipdate " + + "from lineitem t1 " + + "left join (select * from orders where o_orderdate = '2023-12-08') t2 " + + "on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " + order_qt_query6_0_before "${query6_0}" + check_rewrite(mv6_0, query6_0, "mv6_0") + order_qt_query6_0_after "${query6_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv6_0""" + + + // filter inside + left + right + def mv7_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey " + + "from lineitem " + + "left join (select * from orders where o_orderdate = '2023-12-08') t2 " + + "on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate " + def query7_0 = "select l_partkey, l_suppkey, l_shipdate " + + "from (select l_shipdate, l_orderkey, l_partkey, l_suppkey " + + "from lineitem where l_partkey in (3, 4)) t1 " + + "left join (select * from orders where o_orderdate = '2023-12-08') t2 " + + "on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " + + "where l_partkey = 3" + order_qt_query7_0_before "${query7_0}" + check_rewrite(mv7_0, query7_0, "mv7_0") + order_qt_query7_0_after "${query7_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv7_0""" + + + def mv7_1 = """ + select l_shipdate, o_orderdate, l_partkey, l_suppkey + from lineitem + left join orders + on lineitem.l_orderkey = orders.o_orderkey + where l_shipdate = '2023-12-08' and o_orderdate = '2023-12-08'; + """ + def query7_1 = """ + select l_shipdate, o_orderdate, l_partkey, l_suppkey + from (select * from lineitem where l_shipdate = '2023-10-17' ) t1 + left join orders + on t1.l_orderkey = orders.o_orderkey; + """ + order_qt_query7_1_before "${query7_1}" + check_not_match(mv7_1, query7_1, "mv7_1") + order_qt_query7_1_after "${query7_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv7_1""" + + + // self join test + def mv8_0 = """ + select + a.o_orderkey, + count(distinct a.o_orderstatus) num1, + SUM(CASE WHEN a.o_orderstatus = 'o' AND a.o_shippriority = 1 AND a.o_orderdate = '2023-12-08' AND b.o_orderdate = '2023-12-09' THEN a.o_shippriority+b.o_custkey ELSE 0 END) num2, + SUM(CASE WHEN a.o_orderstatus = 'o' AND a.o_shippriority = 1 AND a.o_orderdate >= '2023-12-01' AND a.o_orderdate <= '2023-12-09' THEN a.o_shippriority+b.o_custkey ELSE 0 END) num3, + SUM(CASE WHEN a.o_orderstatus = 'o' AND a.o_shippriority in (1,2) AND a.o_orderdate >= '2023-12-08' AND b.o_orderdate <= '2023-12-09' THEN a.o_shippriority-b.o_custkey ELSE 0 END) num4, + AVG(a.o_totalprice) num5, + MAX(b.o_totalprice) num6, + MIN(a.o_totalprice) num7 + from + orders a + left outer join orders b + on a.o_orderkey = b.o_orderkey + and a.o_custkey = b.o_custkey + group by a.o_orderkey; + """ + def query8_0 = """ + select + a.o_orderkey, + SUM(CASE WHEN a.o_orderstatus = 'o' AND a.o_shippriority = 1 AND a.o_orderdate = '2023-12-08' AND b.o_orderdate = '2023-12-09' THEN a.o_shippriority+b.o_custkey ELSE 0 END) num2, + SUM(CASE WHEN a.o_orderstatus = 'o' AND a.o_shippriority = 1 AND a.o_orderdate >= '2023-12-01' AND a.o_orderdate <= '2023-12-09' THEN a.o_shippriority+b.o_custkey ELSE 0 END) num3, + SUM(CASE WHEN a.o_orderstatus = 'o' AND a.o_shippriority in (1,2) AND a.o_orderdate >= '2023-12-08' AND b.o_orderdate <= '2023-12-09' THEN a.o_shippriority-b.o_custkey ELSE 0 END) num4, + AVG(a.o_totalprice) num5, + MAX(b.o_totalprice) num6, + MIN(a.o_totalprice) num7 + from + orders a + left outer join orders b + on a.o_orderkey = b.o_orderkey + and a.o_custkey = b.o_custkey + group by a.o_orderkey; + """ + order_qt_query8_0_before "${query8_0}" + check_rewrite(mv8_0, query8_0, "mv8_0") + order_qt_query8_0_after "${query8_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv8_0""" +} From bc7e1e123b45e66b8c12541ad77450e3733ecc5e Mon Sep 17 00:00:00 2001 From: wuwenchi Date: Tue, 16 Jan 2024 17:18:10 +0800 Subject: [PATCH 034/200] [bugfix](paimon)support native and jni to read paimon for minio/cos #29933 --- .../en/docs/lakehouse/multi-catalog/paimon.md | 32 +++++++-- .../docs/lakehouse/multi-catalog/paimon.md | 32 +++++++-- .../paimon/PaimonFileExternalCatalog.java | 15 +++++ .../property/constants/PaimonProperties.java | 1 + .../external/paimon/PaimonScanNode.java | 17 ++++- .../org/apache/doris/qe/SessionVariable.java | 14 ++++ .../paimon/paimon_base_filesystem.out | 17 +++++ .../paimon/paimon_base_filesystem.groovy | 66 +++++++++++++++++++ 8 files changed, 180 insertions(+), 14 deletions(-) create mode 100644 regression-test/data/external_table_p2/paimon/paimon_base_filesystem.out create mode 100644 regression-test/suites/external_table_p2/paimon/paimon_base_filesystem.groovy diff --git a/docs/en/docs/lakehouse/multi-catalog/paimon.md b/docs/en/docs/lakehouse/multi-catalog/paimon.md index 7ff14528b957b7..c4c7bb64860542 100644 --- a/docs/en/docs/lakehouse/multi-catalog/paimon.md +++ b/docs/en/docs/lakehouse/multi-catalog/paimon.md @@ -33,7 +33,7 @@ under the License. ## Instructions for use 1. When data in hdfs,need to put core-site.xml, hdfs-site.xml and hive-site.xml in the conf directory of FE and BE. First read the hadoop configuration file in the conf directory, and then read the related to the environment variable `HADOOP_CONF_DIR` configuration file. -2. The currently adapted version of the payment is 0.5.0 +2. The currently adapted version of the payment is 0.6.0 ## Create Catalog @@ -73,11 +73,11 @@ CREATE CATALOG `paimon_kerberos` PROPERTIES ( ); ``` -#### S3 +#### MINIO > Note that. > -> user need download [paimon-s3-0.5.0-incubating.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-s3/0.5.0-incubating/paimon-s3-0.5.0-incubating.jar) +> user need download [paimon-s3-0.6.0-incubating.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-s3/0.6.0-incubating/paimon-s3-0.6.0-incubating.jar) > > Place it in directory `${DORIS_HOME}/be/lib/java_extensions/preload-extensions` and restart be > @@ -86,18 +86,38 @@ CREATE CATALOG `paimon_kerberos` PROPERTIES ( ```sql CREATE CATALOG `paimon_s3` PROPERTIES ( "type" = "paimon", - "warehouse" = "s3://paimon-1308700295.cos.ap-beijing.myqcloud.com/paimoncos", - "s3.endpoint" = "cos.ap-beijing.myqcloud.com", + "warehouse" = "s3://bucket_name/paimons3", + "s3.endpoint" = "http://:", "s3.access_key" = "ak", "s3.secret_key" = "sk" ); ``` +#### COS + +> Note that. +> +> user need download [paimon-s3-0.6.0-incubating.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-s3/0.6.0-incubating/paimon-s3-0.6.0-incubating.jar) +> +> Place it in directory `${DORIS_HOME}/be/lib/java_extensions/preload-extensions` and restart be +> +> Starting from version 2.0.2, this file can be placed in BE's `custom_lib/` directory (if it does not exist, just create it manually) to prevent the file from being lost due to the replacement of the lib directory when upgrading the cluster. + +```sql +CREATE CATALOG `paimon_cos` PROPERTIES ( + "type" = "paimon", + "warehouse" = "cosn://paimon-1308700295/paimoncos", + "cos.endpoint" = "cos.ap-beijing.myqcloud.com", + "cos.access_key" = "ak", + "cos.secret_key" = "sk" +); +``` + #### OSS >Note that. > -> user need download [paimon-oss-0.5.0-incubating.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-oss/0.5.0-incubating/paimon-oss-0.5.0-incubating.jar) +> user need download [paimon-oss-0.6.0-incubating.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-oss/0.6.0-incubating/paimon-oss-0.6.0-incubating.jar) > > Place it in directory `${DORIS_HOME}/be/lib/java_extensions/preload-extensions` and restart be diff --git a/docs/zh-CN/docs/lakehouse/multi-catalog/paimon.md b/docs/zh-CN/docs/lakehouse/multi-catalog/paimon.md index 8c79943e087d2f..79632be67725f4 100644 --- a/docs/zh-CN/docs/lakehouse/multi-catalog/paimon.md +++ b/docs/zh-CN/docs/lakehouse/multi-catalog/paimon.md @@ -33,7 +33,7 @@ under the License. ## 使用须知 1. 数据放在hdfs时,需要将 core-site.xml,hdfs-site.xml 和 hive-site.xml 放到 FE 和 BE 的 conf 目录下。优先读取 conf 目录下的 hadoop 配置文件,再读取环境变量 `HADOOP_CONF_DIR` 的相关配置文件。 -2. 当前适配的paimon版本为0.5.0 +2. 当前适配的paimon版本为0.6.0 ## 创建 Catalog @@ -73,11 +73,11 @@ CREATE CATALOG `paimon_kerberos` PROPERTIES ( ); ``` -#### S3 +#### MINIO > 注意: > -> 用户需要手动下载[paimon-s3-0.5.0-incubating.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-s3/0.5.0-incubating/paimon-s3-0.5.0-incubating.jar) +> 用户需要手动下载[paimon-s3-0.6.0-incubating.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-s3/0.6.0-incubating/paimon-s3-0.6.0-incubating.jar) > 放在 `${DORIS_HOME}/be/lib/java_extensions/preload-extensions` 目录下并重启be。 > @@ -86,18 +86,38 @@ CREATE CATALOG `paimon_kerberos` PROPERTIES ( ```sql CREATE CATALOG `paimon_s3` PROPERTIES ( "type" = "paimon", - "warehouse" = "s3://paimon-1308700295.cos.ap-beijing.myqcloud.com/paimoncos", - "s3.endpoint" = "cos.ap-beijing.myqcloud.com", + "warehouse" = "s3://bucket_name/paimons3", + "s3.endpoint" = "http://:", "s3.access_key" = "ak", "s3.secret_key" = "sk" ); ``` +#### COS + +> 注意: +> +> 用户需要手动下载[paimon-s3-0.6.0-incubating.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-s3/0.6.0-incubating/paimon-s3-0.6.0-incubating.jar) + +> 放在 `${DORIS_HOME}/be/lib/java_extensions/preload-extensions` 目录下并重启be。 +> +> 从 2.0.2 版本起,可以将这个文件放置在BE的 `custom_lib/` 目录下(如不存在,手动创建即可),以防止升级集群时因为 lib 目录被替换而导致文件丢失。 + +```sql +CREATE CATALOG `paimon_s3` PROPERTIES ( + "type" = "paimon", + "warehouse" = "cosn://paimon-1308700295/paimoncos", + "cos.endpoint" = "cos.ap-beijing.myqcloud.com", + "cos.access_key" = "ak", + "cos.secret_key" = "sk" +); +``` + #### OSS >注意: > -> 用户需要手动下载[paimon-oss-0.5.0-incubating.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-oss/0.5.0-incubating/paimon-oss-0.5.0-incubating.jar) +> 用户需要手动下载[paimon-oss-0.6.0-incubating.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-oss/0.6.0-incubating/paimon-oss-0.6.0-incubating.jar) > 放在 `${DORIS_HOME}/be/lib/java_extensions/preload-extensions` 目录下并重启be ```sql diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonFileExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonFileExternalCatalog.java index e317de5efeff7a..9e8fa2dc455e75 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonFileExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonFileExternalCatalog.java @@ -18,6 +18,8 @@ package org.apache.doris.datasource.paimon; import org.apache.doris.datasource.CatalogProperty; +import org.apache.doris.datasource.property.PropertyConverter; +import org.apache.doris.datasource.property.constants.CosProperties; import org.apache.doris.datasource.property.constants.PaimonProperties; import org.apache.logging.log4j.LogManager; @@ -57,6 +59,19 @@ protected void setPaimonCatalogOptions(Map properties, Map predicates; @@ -110,6 +115,7 @@ public void setPaimonParams(TFileRangeDesc rangeDesc, PaimonSplit paimonSplit) { TPaimonFileDesc fileDesc = new TPaimonFileDesc(); org.apache.paimon.table.source.Split split = paimonSplit.getSplit(); if (split != null) { + // use jni reader fileDesc.setPaimonSplit(encodeObjectToString(split)); } fileDesc.setFileFormat(source.getFileFormat()); @@ -129,6 +135,7 @@ public void setPaimonParams(TFileRangeDesc rangeDesc, PaimonSplit paimonSplit) { @Override public List getSplits() throws UserException { + boolean forceJniScanner = ConnectContext.get().getSessionVariable().isForceJniScanner(); List splits = new ArrayList<>(); int[] projected = desc.getSlots().stream().mapToInt( slot -> (source.getPaimonTable().rowType().getFieldNames().indexOf(slot.getColumn().getName()))) @@ -139,7 +146,7 @@ public List getSplits() throws UserException { .newScan().plan().splits(); boolean supportNative = supportNativeReader(); for (org.apache.paimon.table.source.Split split : paimonSplits) { - if (supportNative && split instanceof DataSplit) { + if (!forceJniScanner && supportNative && split instanceof DataSplit) { DataSplit dataSplit = (DataSplit) split; Optional> optRowFiles = dataSplit.convertToRawFiles(); if (optRowFiles.isPresent()) { @@ -234,7 +241,13 @@ public TableIf getTargetTable() { @Override public Map getLocationProperties() throws MetaNotFoundException, DdlException { - return source.getCatalog().getCatalogProperty().getHadoopProperties(); + HashMap map = new HashMap<>(source.getCatalog().getProperties()); + source.getCatalog().getCatalogProperty().getHadoopProperties().forEach((k, v) -> { + if (!map.containsKey(k)) { + map.put(k, v); + } + }); + return map; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 8f51cf20992d40..5c1a4120dc43d6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -492,6 +492,8 @@ public class SessionVariable implements Serializable, Writable { public static final String DESCRIBE_EXTEND_VARIANT_COLUMN = "describe_extend_variant_column"; + public static final String FORCE_JNI_SCANNER = "force_jni_scanner"; + public static final List DEBUG_VARIABLES = ImmutableList.of( SKIP_DELETE_PREDICATE, SKIP_DELETE_BITMAP, @@ -1510,6 +1512,10 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { + "when using rewriting based on struct info"}) public boolean materializedViewRewriteEnableContainExternalTable = false; + @VariableMgr.VarAttr(name = FORCE_JNI_SCANNER, + description = {"强制使用jni方式读取外表", "Force the use of jni mode to read external table"}) + private boolean forceJniScanner = false; + public static final String IGNORE_RUNTIME_FILTER_IDS = "ignore_runtime_filter_ids"; public Set getIgnoredRuntimeFilterIds() { @@ -3293,4 +3299,12 @@ public void setDisableEmptyPartitionPrune(boolean val) { disableEmptyPartitionPrune = val; } // CLOUD_VARIABLES_END + + public boolean isForceJniScanner() { + return forceJniScanner; + } + + public void setForceJniScanner(boolean force) { + forceJniScanner = force; + } } diff --git a/regression-test/data/external_table_p2/paimon/paimon_base_filesystem.out b/regression-test/data/external_table_p2/paimon/paimon_base_filesystem.out new file mode 100644 index 00000000000000..d24ebd07ef6b16 --- /dev/null +++ b/regression-test/data/external_table_p2/paimon/paimon_base_filesystem.out @@ -0,0 +1,17 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !c1 -- +1 2 3 4 5 6 7 8 9.1 10.1 11.10 2020-02-02 13str 14varchar a true 2023-08-10T14:35:38.768 +1 2 3 4 5 6 7 8 9.1 10.1 11.10 2020-02-02 13str 14varchar a true 2023-08-10T15:58:54.364 + +-- !c2 -- +7 Hugo +8 Stop + +-- !c3 -- +1 2 3 4 5 6 7 8 9.1 10.1 11.10 2020-02-02 13str 14varchar a true 2023-08-10T14:35:38.768 +1 2 3 4 5 6 7 8 9.1 10.1 11.10 2020-02-02 13str 14varchar a true 2023-08-10T15:58:54.364 + +-- !c4 -- +7 Hugo +8 Stop + diff --git a/regression-test/suites/external_table_p2/paimon/paimon_base_filesystem.groovy b/regression-test/suites/external_table_p2/paimon/paimon_base_filesystem.groovy new file mode 100644 index 00000000000000..a091e3615fa904 --- /dev/null +++ b/regression-test/suites/external_table_p2/paimon/paimon_base_filesystem.groovy @@ -0,0 +1,66 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +suite("paimon_base_filesystem", "p2,external,paimon,external_remote,external_remote_paimon") { + String enabled = context.config.otherConfigs.get("enableExternalPaimonTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String catalog_cos = "paimon_cos" + String catalog_oss = "paimon_oss" + String ak = context.config.otherConfigs.get("aliYunAk") + String sk = context.config.otherConfigs.get("aliYunSk") + + String s3ak = getS3AK() + String s3sk = getS3SK() + + def cos = """select c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c18 from ${catalog_cos}.zd.all_table order by c18""" + def oss = """select * from ${catalog_oss}.paimonossdb1.test_tableoss order by a""" + + sql """drop catalog if exists ${catalog_cos};""" + sql """drop catalog if exists ${catalog_oss};""" + sql """ + create catalog if not exists ${catalog_cos} properties ( + "type" = "paimon", + "warehouse" = "cosn://paimon-1308700295/paimoncos", + "cos.access_key" = "${s3ak}", + "cos.secret_key" = "${s3sk}", + "cos.endpoint" = "cos.ap-beijing.myqcloud.com" + ); + """ + sql """ + create catalog if not exists ${catalog_oss} properties ( + "type" = "paimon", + "warehouse" = "oss://paimon-zd/paimonoss", + "oss.endpoint"="oss-cn-beijing.aliyuncs.com", + "oss.access_key"="${ak}", + "oss.secret_key"="${sk}" + ); + """ + logger.info("catalog " + catalog_cos + " created") + logger.info("catalog " + catalog_oss + " created") + + sql """set force_jni_scanner=false""" + qt_c1 cos + qt_c2 oss + + sql """set force_jni_scanner=true""" + qt_c3 cos + qt_c4 oss + + sql """set force_jni_scanner=false""" + } +} + From b9db7c417cd991893f2143e9041f2f672fc38413 Mon Sep 17 00:00:00 2001 From: deardeng <565620795@qq.com> Date: Tue, 16 Jan 2024 17:31:44 +0800 Subject: [PATCH 035/200] [fix](disk balance) Change disk rebalance unpick time to configurable (#28949) --- .../src/main/java/org/apache/doris/common/Config.java | 6 ++++++ .../src/main/java/org/apache/doris/clone/Rebalancer.java | 3 ++- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index 4d38f041fe5017..1b1d8906d96b9e 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -1020,6 +1020,12 @@ public class Config extends ConfigBase { @ConfField(mutable = true, masterOnly = true) public static boolean disable_balance = false; + /** + * when be rebalancer idle, then disk balance will occurs. + */ + @ConfField(mutable = true, masterOnly = true) + public static int be_rebalancer_idle_seconds = 60; + /** * if set to true, TabletScheduler will not do disk balance. */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/Rebalancer.java b/fe/fe-core/src/main/java/org/apache/doris/clone/Rebalancer.java index 8f6b1d229f0d0b..def9e18a7e3a29 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/clone/Rebalancer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/clone/Rebalancer.java @@ -19,6 +19,7 @@ import org.apache.doris.catalog.TabletInvertedIndex; import org.apache.doris.clone.TabletScheduler.PathSlot; +import org.apache.doris.common.Config; import org.apache.doris.resource.Tag; import org.apache.doris.system.Backend; import org.apache.doris.system.SystemInfoService; @@ -100,7 +101,7 @@ protected boolean unPickOverLongTime(Tag tag, TStorageMedium medium) { Long lastPickTime = lastPickTimeTable.get(tag, medium); Long now = System.currentTimeMillis(); LOG.debug("tag={}, medium={}, lastPickTime={}, now={}", tag, medium, lastPickTime, now); - return lastPickTime == null || now - lastPickTime >= 5 * 60 * 1000L; + return lastPickTime == null || now - lastPickTime >= Config.be_rebalancer_idle_seconds * 1000L; } public AgentTask createBalanceTask(TabletSchedCtx tabletCtx) From 0ae9fa1c33708139ed4efab980cd2d064ffab906 Mon Sep 17 00:00:00 2001 From: zclllyybb Date: Tue, 16 Jan 2024 17:42:20 +0800 Subject: [PATCH 036/200] [Chore](arm) Support auto-SIMD on arm and refactor compilation docs (#29956) --- be/CMakeLists.txt | 10 + docs/en/docs/advanced/time-zone.md | 7 +- .../install/source-install/compilation-arm.md | 659 ++++++++--------- docs/zh-CN/docs/advanced/time-zone.md | 7 +- .../install/source-install/compilation-arm.md | 670 +++++++++--------- 5 files changed, 697 insertions(+), 656 deletions(-) diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt index 70330ec9e3869d..bba09ebc2a63a4 100644 --- a/be/CMakeLists.txt +++ b/be/CMakeLists.txt @@ -106,6 +106,10 @@ execute_process(COMMAND bash "-c" "uname -m" OUTPUT_STRIP_TRAILING_WHITESPACE) message(STATUS "Build target arch is ${CMAKE_BUILD_TARGET_ARCH}") +if ("${CMAKE_BUILD_TARGET_ARCH}" STREQUAL "aarch64" OR "${CMAKE_BUILD_TARGET_ARCH}" STREQUAL "arm64") + set (ARCH_ARM 1) +endif() + # Set dirs set(BASE_DIR "${CMAKE_CURRENT_SOURCE_DIR}") set(ENV{DORIS_HOME} "${BASE_DIR}/..") @@ -302,6 +306,7 @@ if (RECORD_COMPILER_SWITCHES) add_compile_options(-frecord-gcc-switches) endif() +# simd for architectures if ("${CMAKE_BUILD_TARGET_ARCH}" STREQUAL "x86" OR "${CMAKE_BUILD_TARGET_ARCH}" STREQUAL "x86_64") add_compile_options(-msse4.2) if (USE_AVX2) @@ -309,6 +314,11 @@ if ("${CMAKE_BUILD_TARGET_ARCH}" STREQUAL "x86" OR "${CMAKE_BUILD_TARGET_ARCH}" endif() endif() +if (ARCH_ARM) + add_compile_options(-march=armv8-a+crc) +endif() +# + if (WITH_MYSQL) add_compile_options(-DDORIS_WITH_MYSQL) endif() diff --git a/docs/en/docs/advanced/time-zone.md b/docs/en/docs/advanced/time-zone.md index e18d06c632a915..7fa30c855db247 100644 --- a/docs/en/docs/advanced/time-zone.md +++ b/docs/en/docs/advanced/time-zone.md @@ -49,11 +49,10 @@ The following two time zone related parameters exist within Doris: The time zone data contains the name of the time zone, the corresponding time offset, and the change of daylight saving time. On the machine where the BE is located, the sources of the data are as follows: -1. the directory returned by the `TZDIR` command -2. the `/usr/share/zoneinfo` directory -3. the `zoneinfo` directory generated under the doris BE deployment directory. The `resource/zoneinfo.tar.gz` directory from the doris repository. +1. the directory returned by command `TZDIR`. If was not supported, the directory `/usr/share/zoneinfo`. +2. the `zoneinfo` directory generated under the doris BE deployment directory. The `resource/zoneinfo.tar.gz` directory from the doris repository. -Look up the above data sources in order and use the current item if found. If none of the three are found, the doris BE will fail to start, please rebuild the BE correctly or get the distribution. +Look up the above data sources in order and use the current item if found. If neither is found, the doris BE will fail to start, please rebuild the BE correctly or get the distribution. ## Impact of time zone diff --git a/docs/en/docs/install/source-install/compilation-arm.md b/docs/en/docs/install/source-install/compilation-arm.md index adc9027822fcfe..6a828752192fe6 100644 --- a/docs/en/docs/install/source-install/compilation-arm.md +++ b/docs/en/docs/install/source-install/compilation-arm.md @@ -28,15 +28,15 @@ under the License. This topic is about how to compile Doris on the ARM64 platform. -Note that this is for reference only. Other errors may occur when compiling in different environments. +Note that this document is intended as a guide only. Other errors may occur when compiling in different environments. Feel free to [issue](https://github.com/apache/doris/issues) Doris with problems or solutions. -## KylinOS +## Hardware/OS environment -### Software and Hardware Environment +### KylinOS 1. KylinOS Version: - ``` +```shell $> cat /etc/.kyinfo name=Kylin-Server milestone=10-SP1-Release-Build04-20200711 @@ -44,31 +44,16 @@ Note that this is for reference only. Other errors may occur when compiling in d beta=False time=2020-07-11 17:16:54 dist_id=Kylin-Server-10-SP1-Release-Build04-20200711-arm64-2020-07-11 17:16:54 - ``` +``` 2. CPU Model: - ``` +```shell $> cat /proc/cpuinfo model name : Phytium,FT-2000+/64 - ``` - -### Compile with ldb-Toolchain - -This method works for Doris versions after [commit 7f3564](https://github.com/apache/doris/commit/7f3564cca62de49c9f2ea67fcf735921dbebb4d1). - -Download [ldb\_toolchain\_gen.aarch64.sh](https://github.com/amosbird/ldb_toolchain_gen/releases/download/v0.9.1/ldb_toolchain_gen.aarch64.sh) - -For detailed instructions, please refer to [Compile with ldb-toolchain](./compilation-with-ldb-toolchain.md) - -Note that you need to download the corresponding aarch64 versions of jdk and nodejs: - -1. [Java8-aarch64](https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/jdk-8u291-linux-aarch64.tar.gz) -2. [Node v16.3.0-aarch64](https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/node-v16.3.0-linux-arm64.tar.xz) - -## CentOS & Ubuntu +``` -### Hardware Environment +### CentOS & Ubuntu 1. System Version: CentOS 8.4, Ubuntu 20.04 2. System Architecture: ARM X64 @@ -76,9 +61,9 @@ Note that you need to download the corresponding aarch64 versions of jdk and nod 4. Memory: 16 GB 5. Hard Disk: 40GB (SSD), 100GB (SSD) -### Software Environment +## Software Environment -#### Software Environment List +### Software Environment List | Component Name | Component Version | | ------------------------------------------------------------ | ------------------------------ | @@ -91,271 +76,291 @@ Note that you need to download the corresponding aarch64 versions of jdk and nod | autoconf | 2.69 | | bison | 3.0.4 | -#### Software Environment Installation Command +### Software Environment Installation Command -##### CentOS 8.4 + + +

+ 1. Create root directories for pacakges -- Create root directories +```shell + # Create root directory for software download and installation packages + mkdir /opt/tools + # Create root directory for software installation + mkdir /opt/software +``` +

+

+ 2. Installing dependencies - ```shell - # Create root directory for software download and installation packages - mkdir /opt/tools - # Create root directory for software installation - mkdir /opt/software - ```` + - Git -- Git +```shell + # yum install (save the trouble of compilation) + yum install -y git +``` - ```shell - # yum install (save the trouble of compilation) - yum install -y git - ```` + - JDK8 (2 methods) -- JDK8 (2 methods) +```shell + # 1. yum install, which can avoid additional download and configuration. Installing the devel package is to get tools such as the jps command. + yum install -y java-1.8.0-openjdk java-1.8.0-openjdk-devel + + # 2. Download the installation package of the arm64 architecture, decompress it, and configure the environment variables. + cd /opt/tools + wget https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/jdk-8u291-linux-aarch64.tar.gz && \ + tar -zxvf jdk-8u291-linux-aarch64.tar.gz && \ + mv jdk1.8.0_291 /opt/software/jdk8 +``` - ```shell - # 1. yum install, which can avoid additional download and configuration. Installing the devel package is to get tools such as the jps command. - yum install -y java-1.8.0-openjdk java-1.8.0-openjdk-devel - - # 2. Download the installation package of the arm64 architecture, decompress it, and configure the environment variables. - cd /opt/tools - wget https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/jdk-8u291-linux-aarch64.tar.gz && \ - tar -zxvf jdk-8u291-linux-aarch64.tar.gz && \ - mv jdk1.8.0_291 /opt/software/jdk8 - ```` - -- Maven - - ```shell - cd /opt/tools - # Download the wget tool, decompress it, and configure the environment variables. - wget https://dlcdn.apache.org/maven/maven-3/3.6.3/binaries/apache-maven-3.6.3-bin.tar.gz && \ - tar -zxvf apache-maven-3.6.3-bin.tar.gz && \ - mv apache-maven-3.6.3 /opt/software/maven - ```` - -- NodeJS - - ```shell - cd /opt/tools - # Download the installation package of the arm64 architecture - wget https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/node-v16.3.0-linux-arm64.tar.xz && \ - tar -xvf node-v16.3.0-linux-arm64.tar.xz && \ - mv node-v16.3.0-linux-arm64 /opt/software/nodejs - ```` - -- ldb-toolchain - - ```shell - cd /opt/tools - # Download ldb-toolchain ARM version - wget https://github.com/amosbird/ldb_toolchain_gen/releases/download/v0.9.1/ldb_toolchain_gen.aarch64.sh && \ - sh ldb_toolchain_gen.aarch64.sh /opt/software/ldb_toolchain/ - ```` - -- Configure environment variables - - ```shell - # Configure environment variables - vim /etc/profile.d/doris.sh - export JAVA_HOME=/opt/software/jdk8 - export MAVEN_HOME=/opt/software/maven - export NODE_JS_HOME=/opt/software/nodejs - export LDB_HOME=/opt/software/ldb_toolchain - export PATH=$JAVA_HOME/bin:$MAVEN_HOME/bin:$NODE_JS_HOME/bin:$LDB_HOME/bin:$PATH - - # Save, exit, and refresh environment variables - source /etc/profile.d/doris.sh - - # Test - java -version - > java version "1.8.0_291" - mvn -version - > Apache Maven 3.6.3 - node --version - > v16.3.0 - gcc --version - > gcc-11 - ```` - -- Install other environments and components - - ```shell - # Install required system packages - sudo yum install -y byacc patch automake libtool make which file ncurses-devel gettext-devel unzip bzip2 bison zip util-linux wget git python2 - - # Install autoconf-2.69 - cd /opt/tools - wget http://ftp.gnu.org/gnu/autoconf/autoconf-2.69.tar.gz && \ - tar zxf autoconf-2.69.tar.gz && \ - mv autoconf-2.69 /opt/software/autoconf && \ - cd /opt/software/autoconf && \ - ./configure && \ - make && \ - make install - ```` - -##### Ubuntu 20.04 - -- Update apt-get repository - - ```shell - apt-get update - ```` - -- Check the shell command set - - The Ubuntu shell installs dash instead of bash by default. It needs to be switched to bash for proper execution. Run the following command to view the details of sh and confirm which program corresponds to the shell: - - ```shell - ls -al /bin/sh - ```` - - The shell can be switched back to bash by: - - ```shell - sudo dpkg-reconfigure dash - ```` - - Then select no to confirm. - - After these steps, dash will no longer be the default shell tool. - -- Create root directories - - ```shell - # Create root directory for software download and installation packages - mkdir /opt/tools - # Create root directory for software installation - mkdir /opt/software - ```` - -- Git - - ```shell - # apt-get install, which can save the trouble of compilation - apt-get -y install git - ```` - -- JDK8 - - ```shell - # Download the installation package of the ARM64 architecture, decompress it, and configure environment variables. - cd /opt/tools - wget https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/jdk-8u291-linux-aarch64.tar.gz && \ - tar -zxvf jdk-8u291-linux-aarch64.tar.gz && \ - mv jdk1.8.0_291 /opt/software/jdk8 - ```` - -- Maven - - ```shell - cd /opt/tools - # Download the wget tool, decompress it, and configure the environment variables. - wget https://dlcdn.apache.org/maven/maven-3/3.6.3/binaries/apache-maven-3.6.3-bin.tar.gz && \ - tar -zxvf apache-maven-3.6.3-bin.tar.gz && \ - mv apache-maven-3.6.3 /opt/software/maven - ```` - -- NodeJS - - ```shell - cd /opt/tools - # Download the installation package of ARM64 architecture. - wget https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/node-v16.3.0-linux-arm64.tar.xz && \ - tar -xvf node-v16.3.0-linux-arm64.tar.xz && \ - mv node-v16.3.0-linux-arm64 /opt/software/nodejs - ```` - -- ldb-toolchain - - ```shell - cd /opt/tools - # Download ldb-toolchain ARM version - wget https://github.com/amosbird/ldb_toolchain_gen/releases/download/v0.9.1/ldb_toolchain_gen.aarch64.sh && \ - sh ldb_toolchain_gen.aarch64.sh /opt/software/ldb_toolchain/ - ```` - -- Configure environment variables - - ```shell - # Configure environment variables - vim /etc/profile.d/doris.sh - export JAVA_HOME=/opt/software/jdk8 - export MAVEN_HOME=/opt/software/maven - export NODE_JS_HOME=/opt/software/nodejs - export LDB_HOME=/opt/software/ldb_toolchain - export PATH=$JAVA_HOME/bin:$MAVEN_HOME/bin:$NODE_JS_HOME/bin:$LDB_HOME/bin:$PATH - - # Save, exit, and refresh environment variables - source /etc/profile.d/doris.sh - - # Test - java -version - > java version "1.8.0_291" - mvn -version - > Apache Maven 3.6.3 - node --version - > v16.3.0 - gcc --version - > gcc-11 - ```` - -- Install other environments and components - - ```shell - # Install required system packages - sudo apt install -y build-essential cmake flex automake bison binutils-dev libiberty-dev zip libncurses5-dev curl ninja-build - sudo apt-get install -y make - sudo apt-get install -y unzip - sudo apt-get install -y python2 - sudo apt-get install -y byacc - sudo apt-get install -y automake - sudo apt-get install -y libtool - sudo apt-get install -y bzip2 - sudo add-apt-repository ppa:ubuntu-toolchain-r/ppa - sudo apt update - sudo apt install gcc-11 g++-11 - sudo apt-get -y install autoconf autopoint - - # Install autoconf-2.69 - cd /opt/tools - wget http://ftp.gnu.org/gnu/autoconf/autoconf-2.69.tar.gz && \ - tar zxf autoconf-2.69.tar.gz && \ - mv autoconf-2.69 /opt/software/autoconf && \ - cd /opt/software/autoconf && \ - ./configure && \ - make && \ - make install - ```` - -#### Download Source Code + - Maven ```shell -cd /opt -git clone https://github.com/apache/doris.git + cd /opt/tools + # Download the wget tool, decompress it, and configure the environment variables. + wget https://dlcdn.apache.org/maven/maven-3/3.6.3/binaries/apache-maven-3.6.3-bin.tar.gz && \ + tar -zxvf apache-maven-3.6.3-bin.tar.gz && \ + mv apache-maven-3.6.3 /opt/software/maven ``` -#### Install and Deploy + - NodeJS -##### Check if the AVX2 instruction set is supported +```shell + cd /opt/tools + # Download the installation package of the arm64 architecture + wget https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/node-v16.3.0-linux-arm64.tar.xz && \ + tar -xvf node-v16.3.0-linux-arm64.tar.xz && \ + mv node-v16.3.0-linux-arm64 /opt/software/nodejs +``` -If there is data returned, that means AVX2 is supported; otherwise, AVX2 is not supported. + - ldb-toolchain ```shell -cat /proc/cpuinfo | grep avx2 -```` + cd /opt/tools + # Download ldb-toolchain ARM version + wget https://github.com/amosbird/ldb_toolchain_gen/releases/download/v0.9.1/ldb_toolchain_gen.aarch64.sh && \ + sh ldb_toolchain_gen.aarch64.sh /opt/software/ldb_toolchain/ +``` +

+

+ 3. Configure environment variables + +```shell + # Configure environment variables + vim /etc/profile.d/doris.sh + export JAVA_HOME=/opt/software/jdk8 + export MAVEN_HOME=/opt/software/maven + export NODE_JS_HOME=/opt/software/nodejs + export LDB_HOME=/opt/software/ldb_toolchain + export PATH=$JAVA_HOME/bin:$MAVEN_HOME/bin:$NODE_JS_HOME/bin:$LDB_HOME/bin:$PATH + + # Save, exit, and refresh environment variables + source /etc/profile.d/doris.sh + + # Test + java -version + > java version "1.8.0_291" + mvn -version + > Apache Maven 3.6.3 + node --version + > v16.3.0 + gcc --version + > gcc-11 +``` +

+

+ 4. Install other environments and components -##### Execute compilation +```shell + # Install required system packages + sudo yum install -y byacc patch automake libtool make which file ncurses-devel gettext-devel unzip bzip2 bison zip util-linux wget git python2 + + # Install autoconf-2.69 + cd /opt/tools + wget http://ftp.gnu.org/gnu/autoconf/autoconf-2.69.tar.gz && \ + tar zxf autoconf-2.69.tar.gz && \ + mv autoconf-2.69 /opt/software/autoconf && \ + cd /opt/software/autoconf && \ + ./configure && \ + make && \ + make install +``` +

+ + +

+ 1. Update apt-get repository ```shell -# For machines that support the AVX2 instruction set, compile directly -sh build.sh -# For machines that do not support the AVX2 instruction set, use the following command to compile -USE_AVX2=OFF sh build.sh -```` + apt-get update +``` +

+

+ 2. Check the shell command set + + The Ubuntu shell installs dash instead of bash by default. It needs to be switched to bash for proper execution. Run the following command to view the details of sh and confirm which program corresponds to the shell: + +```shell + ls -al /bin/sh +``` + + The shell can be switched back to bash by: + +```shell + sudo dpkg-reconfigure dash +``` + + Then select no to confirm. + + After these steps, dash will no longer be the default shell tool. +

+

+ 3. Create root directories for packages + +```shell + # Create root directory for software download and installation packages + mkdir /opt/tools + # Create root directory for software installation + mkdir /opt/software +``` +

+

+ 4. Installing dependencies + - Git + +```shell + # apt-get install, which can save the trouble of compilation + apt-get -y install git +``` -### FAQ + - JDK8 + +```shell + # Download the installation package of the ARM64 architecture, decompress it, and configure environment variables. + cd /opt/tools + wget https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/jdk-8u291-linux-aarch64.tar.gz && \ + tar -zxvf jdk-8u291-linux-aarch64.tar.gz && \ + mv jdk1.8.0_291 /opt/software/jdk8 +``` + + - Maven + +```shell + cd /opt/tools + # Download the wget tool, decompress it, and configure the environment variables. + wget https://dlcdn.apache.org/maven/maven-3/3.6.3/binaries/apache-maven-3.6.3-bin.tar.gz && \ + tar -zxvf apache-maven-3.6.3-bin.tar.gz && \ + mv apache-maven-3.6.3 /opt/software/maven +``` + + - NodeJS + +```shell + cd /opt/tools + # Download the installation package of ARM64 architecture. + wget https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/node-v16.3.0-linux-arm64.tar.xz && \ + tar -xvf node-v16.3.0-linux-arm64.tar.xz && \ + mv node-v16.3.0-linux-arm64 /opt/software/nodejs +``` + + - ldb-toolchain + +```shell + cd /opt/tools + # Download ldb-toolchain ARM version + wget https://github.com/amosbird/ldb_toolchain_gen/releases/download/v0.9.1/ldb_toolchain_gen.aarch64.sh && \ + sh ldb_toolchain_gen.aarch64.sh /opt/software/ldb_toolchain/ +``` +

+

+ 5. Configure environment variables + +```shell + # Configure environment variables + vim /etc/profile.d/doris.sh + export JAVA_HOME=/opt/software/jdk8 + export MAVEN_HOME=/opt/software/maven + export NODE_JS_HOME=/opt/software/nodejs + export LDB_HOME=/opt/software/ldb_toolchain + export PATH=$JAVA_HOME/bin:$MAVEN_HOME/bin:$NODE_JS_HOME/bin:$LDB_HOME/bin:$PATH + + # Save, exit, and refresh environment variables + source /etc/profile.d/doris.sh + + # Test + java -version + > java version "1.8.0_291" + mvn -version + > Apache Maven 3.6.3 + node --version + > v16.3.0 + gcc --version + > gcc-11 +``` +

+

+ 6. Install other environments and components + +```shell + # Install required system packages + sudo apt install -y build-essential cmake flex automake bison binutils-dev libiberty-dev zip libncurses5-dev curl ninja-build + sudo apt-get install -y make + sudo apt-get install -y unzip + sudo apt-get install -y python2 + sudo apt-get install -y byacc + sudo apt-get install -y automake + sudo apt-get install -y libtool + sudo apt-get install -y bzip2 + sudo add-apt-repository ppa:ubuntu-toolchain-r/ppa + sudo apt update + sudo apt install gcc-11 g++-11 + sudo apt-get -y install autoconf autopoint + + # Install autoconf-2.69 + cd /opt/tools + wget http://ftp.gnu.org/gnu/autoconf/autoconf-2.69.tar.gz && \ + tar zxf autoconf-2.69.tar.gz && \ + mv autoconf-2.69 /opt/software/autoconf && \ + cd /opt/software/autoconf && \ + ./configure && \ + make && \ + make install +``` +

+
+ + +## Compile + +:::tip +Currently the only recommended method for ARM environments is to compile with ldb-toolchain, which works for [commit 7f3564](https://github.com/apache/doris/commit/7f3564cca62de49c9f2ea67fcf735921dbebb4d1) for Doris releases after that. +::: + +Download [ldb\_toolchain\_gen.aarch64.sh](https://github.com/amosbird/ldb_toolchain_gen/releases/download/v0.9.1/ldb_toolchain_gen.aarch64.sh) + +See [Compilation with LDB toolchain](./compilation-with-ldb-toolchain) for the steps then. + +The jdk and nodejs should be replaced with the corresponding aarch64 versions: + +1. [Java8-aarch64](https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/jdk-8u291-linux-aarch64.tar.gz) +2. [Node v16.3.0-aarch64](https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/node-v16.3.0-linux-arm64.tar.xz) + +If you have problems compiling with the downloaded precompiled third-party libraries, please use `tools/build_thirdparty.sh` to compile them by yourself. When compiling the thirdparty library, use gcc: +```shell +export DORIS_TOOLCHAIN=gcc +``` + +When compiling Doris on ARM platforms, **please disable AVX2 and LIBUNWIND tripartite libraries**: +```shell +export USE_AVX2=OFF +export USE_UNWIND=OFF +``` + +If you still encounter problems when compiling or starting, please consult the [FAQ](#FAQ). If there is no relevant solution, feel free to [raise an issue](https://github.com/apache/doris/issues). + +## FAQ + +### Problems about Compiling 1. File not found when compiling the third-party library libhdfs3.a. @@ -373,10 +378,10 @@ USE_AVX2=OFF sh build.sh - Use a third-party download repository - ```shell +```shell export REPOSITORY_URL=https://doris-thirdparty-repo.bj.bcebos.com/thirdparty sh /opt/doris/thirdparty/build-thirdparty.sh - ```` +```` REPOSITORY_URL contains all third-party library source packages and their historical versions. @@ -398,12 +403,12 @@ USE_AVX2=OFF sh build.sh Establish a soft link to the `python` command in `\usr\bin` - ```shell +```shell # View python installation directory whereis python # Establish soft connection sudo ln -s /usr/bin/python2.7 /usr/bin/python - ```` +```` 3. There is no output directory after compilation @@ -417,9 +422,9 @@ USE_AVX2=OFF sh build.sh - Solution - ```shell +```shell sh build.sh --clean - ```` +```` 4. spark-dpp compilation fails @@ -457,43 +462,7 @@ USE_AVX2=OFF sh build.sh Expand the free space on the device by deleting files you don't need, etc. -6. Failed to start FE, transaction error -20 - - - Problem Description - - When starting FE, a transaction error 20 is reported with UNKNOWN status. - - > [BDBEnvironment.setup():198] error to open replicated environment. will exit. - > com.sleepycat.je.rep.ReplicaWriteException: (JE 18.3.12) Problem closing transaction 20. The current state is:UNKNOWN. The node transitioned to this state at:Fri Apr 22 12:48:08 CST 2022 - - - Cause - - Insufficient hard disk space - - - Solution - - Free up hard disk space or mount a new hard disk - -7. Abnormal BDB environment setting, disk search error - - - Problem Description - - An exception is reported when starting FE after migrating the drive letter where FE is located - - > 2022-04-22 16:21:44,092 ERROR (MASTER 172.28.7.231_9010_1650606822109(-1)|1) [BDBJEJournal.open():306] catch an exception when setup bdb environment. will exit. - > com.sleepycat.je.DiskLimitException: (JE 18.3.12) Disk usage is not within je.maxDisk or je.freeDisk limits and write operations are prohibited: maxDiskLimit=0 freeDiskLimit=5,368,709,120 adjustedMaxDiskLimit=0 maxDiskOverage=0 freeDiskShortage=1,536,552,960 diskFreeSpace =3,832,156,160 availableLogSize=-1,536,552,960 totalLogSize=4,665 activeLogSize=4,665 reservedLogSize=0 protectedLogSize=0 protectedLogSizeMap={} - - - Cause - - FE has been migrated to another location, which doesn't match the hard disk information stored in the metadata; or the hard disk is damaged or not mounted - - - Solution - - - Check if the hard disk is normal, initialized and mounted correctly - - Fix FE metadata - - If it is a test machine, you can delete the metadata directory and restart - -8. Could not find pkg.m4 file in pkg.config +6. Could not find pkg.m4 file in pkg.config - Problem Description @@ -514,12 +483,12 @@ USE_AVX2=OFF sh build.sh Copy the `pkg.m4` file in the ldb/aclocal directory into the libxml2/m4 directory, and recompile the third-party library. - ```shell +```shell cp /opt/software/ldb_toolchain/share/aclocal/pkg.m4 /opt/incubator-doris/thirdparty/src/libxml2-v2.9.10/m4 sh /opt/incubator-doris/thirdparty/build-thirdparty.sh - ```` +```` -9. Failed to execute test CURL_HAS_TLS_PROXY +7. Failed to execute test CURL_HAS_TLS_PROXY - Problem Description @@ -545,7 +514,7 @@ USE_AVX2=OFF sh build.sh Configure ldb environment variables - ```shell +```shell # Configure environment variables vim /etc/profile.d/ldb.sh export LDB_HOME=/opt/software/ldb_toolchain @@ -555,10 +524,60 @@ USE_AVX2=OFF sh build.sh # Test gcc --version > gcc-11 - ```` +```` + +### Problems about Starting + +1. Failed to start FE, transaction error -20 + + - Problem Description + + When starting FE, a transaction error 20 is reported with UNKNOWN status. + + > [BDBEnvironment.setup():198] error to open replicated environment. will exit. + > com.sleepycat.je.rep.ReplicaWriteException: (JE 18.3.12) Problem closing transaction 20. The current state is:UNKNOWN. The node transitioned to this state at:Fri Apr 22 12:48:08 CST 2022 + + - Cause + + Insufficient hard disk space + + - Solution + + Free up hard disk space or mount a new hard disk + +2. Abnormal BDB environment setting, disk search error + + - Problem Description + + An exception is reported when starting FE after migrating the drive letter where FE is located + + > 2022-04-22 16:21:44,092 ERROR (MASTER 172.28.7.231_9010_1650606822109(-1)|1) [BDBJEJournal.open():306] catch an exception when setup bdb environment. will exit. + > com.sleepycat.je.DiskLimitException: (JE 18.3.12) Disk usage is not within je.maxDisk or je.freeDisk limits and write operations are prohibited: maxDiskLimit=0 freeDiskLimit=5,368,709,120 adjustedMaxDiskLimit=0 maxDiskOverage=0 freeDiskShortage=1,536,552,960 diskFreeSpace =3,832,156,160 availableLogSize=-1,536,552,960 totalLogSize=4,665 activeLogSize=4,665 reservedLogSize=0 protectedLogSize=0 protectedLogSizeMap={} + + - Cause + + FE has been migrated to another location, which doesn't match the hard disk information stored in the metadata; or the hard disk is damaged or not mounted + + - Solution + + - Check if the hard disk is normal, initialized and mounted correctly + - Fix FE metadata + - If it is a test machine, you can delete the metadata directory and restart + +3. BE coredumps at startup and the word `libc.so` or `glibc-compatibility/` is visible in the core stack. + + - Problem cause + The current machine glibc version is too low. You can be confirmed with `ldd --version`. This may occur if the version is less than 2.27. + + - Solution + Recompile the BE, with environment variables added: +```shell + export GLIBC_COMPATIBILITY=OFF +``` -10. Other problems +### Other Component Issues +1. - Problem Description The follow error prompts are all due to one root cause. diff --git a/docs/zh-CN/docs/advanced/time-zone.md b/docs/zh-CN/docs/advanced/time-zone.md index 847572fe610357..d0e65fd4c3dd78 100644 --- a/docs/zh-CN/docs/advanced/time-zone.md +++ b/docs/zh-CN/docs/advanced/time-zone.md @@ -49,11 +49,10 @@ Doris 内部存在以下两个时区相关参数: 时区数据包含时区名、对应时间偏移量、夏令时变化情况等。在 BE 所在机器上,其数据来源依次为: -1. `TZDIR` 命令返回的目录 -2. `/usr/share/zoneinfo` 目录 -3. doris BE 部署目录下生成的 `zoneinfo` 目录。来自 doris repository 下的 `resource/zoneinfo.tar.gz` +1. `TZDIR` 命令返回的目录,如不支持该命令,则为 `/usr/share/zoneinfo` 目录 +2. doris BE 部署目录下生成的 `zoneinfo` 目录。来自 doris repository 下的 `resource/zoneinfo.tar.gz` -按顺序查找以上数据源,如果找到则使用当前项。三项均未找到,则 doris BE 将启动失败,请重新正确构建 BE 或获取发行版。 +按顺序查找以上数据源,如果找到则使用当前项。两项均未找到,则 doris BE 将启动失败,请重新正确构建 BE 或获取发行版。 ## 时区的影响 diff --git a/docs/zh-CN/docs/install/source-install/compilation-arm.md b/docs/zh-CN/docs/install/source-install/compilation-arm.md index c08821deb6e219..1aab7134484786 100644 --- a/docs/zh-CN/docs/install/source-install/compilation-arm.md +++ b/docs/zh-CN/docs/install/source-install/compilation-arm.md @@ -28,15 +28,15 @@ under the License. 本文档介绍如何在 ARM64 平台上编译 Doris。 -注意,该文档仅作为指导性文档。在不同环境中编译可能出现其他错误。 +注意,该文档仅作为指导性文档。在不同环境中编译可能出现其他错误。如遇问题,欢迎向 Doris [提出issue](https://github.com/apache/doris/issues) 或解决方案。 -## KylinOS +## 硬件/操作系统环境 -### 软硬件环境 +### KylinOS 1. KylinOS 版本: - ``` +```shell $> cat /etc/.kyinfo name=Kylin-Server milestone=10-SP1-Release-Build04-20200711 @@ -44,31 +44,16 @@ under the License. beta=False time=2020-07-11 17:16:54 dist_id=Kylin-Server-10-SP1-Release-Build04-20200711-arm64-2020-07-11 17:16:54 - ``` +``` 2. CPU型号: - ``` +```shell $> cat /proc/cpuinfo model name : Phytium,FT-2000+/64 - ``` - -### 使用 ldb-toolchain 编译 - -该方法适用于 [commit 7f3564](https://github.com/apache/doris/commit/7f3564cca62de49c9f2ea67fcf735921dbebb4d1) 之后的 Doris 版本。 - -下载 [ldb\_toolchain\_gen.aarch64.sh](https://github.com/amosbird/ldb_toolchain_gen/releases/download/v0.9.1/ldb_toolchain_gen.aarch64.sh) - -之后的编译方式参阅 [使用 LDB toolchain 编译](./compilation-with-ldb-toolchain.md) - -注意其中 jdk 和 nodejs 都需要下载对应的 aarch64 版本: - -1. [Java8-aarch64](https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/jdk-8u291-linux-aarch64.tar.gz) -2. [Node v16.3.0-aarch64](https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/node-v16.3.0-linux-arm64.tar.xz) - -## CentOS & Ubuntu +``` -### 硬件环境 +### CentOS & Ubuntu 1. 系统版本:CentOS 8.4、Ubuntu 20.04 2. 系统架构:ARM X64 @@ -76,9 +61,9 @@ under the License. 4. 内存:16 GB 5. 硬盘:40GB(SSD)、100GB(SSD) -### 软件环境 +## 软件环境配置 -#### 软件环境对照表 +### 软件环境对照表 | 组件名称 | 组件版本 | | ------------------------------------------------------------ | -------------------- | @@ -91,271 +76,291 @@ under the License. | autoconf | 2.69 | | bison | 3.0.4 | -#### 软件环境安装命令 - -##### CentOS 8.4 - -- 创建软件下载安装包根目录和软件安装根目录 - - ```shell - # 创建软件下载安装包根目录 - mkdir /opt/tools - # 创建软件安装根目录 - mkdir /opt/software - ``` - -- Git - - ```shell - # 省去编译麻烦,直接使用 yum 安装 - yum install -y git - ``` - -- JDK8 - - ```shell - # 两种方式,第一种是省去额外下载和配置,直接使用 yum 安装,安装 devel 包是为了获取一些工具,如 jps 命令 - yum install -y java-1.8.0-openjdk java-1.8.0-openjdk-devel - - # 第二种是下载 arm64 架构的安装包,解压配置环境变量后使用 - cd /opt/tools - wget https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/jdk-8u291-linux-aarch64.tar.gz && \ - tar -zxvf jdk-8u291-linux-aarch64.tar.gz && \ - mv jdk1.8.0_291 /opt/software/jdk8 - ``` - -- Maven - - ```shell - cd /opt/tools - # wget 工具下载后,直接解压缩配置环境变量使用 - wget https://dlcdn.apache.org/maven/maven-3/3.6.3/binaries/apache-maven-3.6.3-bin.tar.gz && \ - tar -zxvf apache-maven-3.6.3-bin.tar.gz && \ - mv apache-maven-3.6.3 /opt/software/maven - ``` - -- NodeJS - - ```shell - cd /opt/tools - # 下载 arm64 架构的安装包 - wget https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/node-v16.3.0-linux-arm64.tar.xz && \ - tar -xvf node-v16.3.0-linux-arm64.tar.xz && \ - mv node-v16.3.0-linux-arm64 /opt/software/nodejs - ``` - -- LDB-Toolchain - - ```shell - cd /opt/tools - # 下载 LDB-Toolchain ARM 版本 - wget https://github.com/amosbird/ldb_toolchain_gen/releases/download/v0.9.1/ldb_toolchain_gen.aarch64.sh && \ - sh ldb_toolchain_gen.aarch64.sh /opt/software/ldb_toolchain/ - ``` - -- 配置环境变量 - - ```shell - # 配置环境变量 - vim /etc/profile.d/doris.sh - export JAVA_HOME=/opt/software/jdk8 - export MAVEN_HOME=/opt/software/maven - export NODE_JS_HOME=/opt/software/nodejs - export LDB_HOME=/opt/software/ldb_toolchain - export PATH=$JAVA_HOME/bin:$MAVEN_HOME/bin:$NODE_JS_HOME/bin:$LDB_HOME/bin:$PATH - - # 保存退出并刷新环境变量 - source /etc/profile.d/doris.sh - - # 测试是否成功 - java -version - > java version "1.8.0_291" - mvn -version - > Apache Maven 3.6.3 - node --version - > v16.3.0 - gcc --version - > gcc-11 - ``` - -- 安装其他额外环境和组件 - - ```shell - # install required system packages - sudo yum install -y byacc patch automake libtool make which file ncurses-devel gettext-devel unzip bzip2 bison zip util-linux wget git python2 - - # install autoconf-2.69 - cd /opt/tools - wget http://ftp.gnu.org/gnu/autoconf/autoconf-2.69.tar.gz && \ - tar zxf autoconf-2.69.tar.gz && \ - mv autoconf-2.69 /opt/software/autoconf && \ - cd /opt/software/autoconf && \ - ./configure && \ - make && \ - make install - ``` - -##### Ubuntu 20.04 - -- 更新 apt-get 软件库 - - ```shell - apt-get update - ``` - -- 检查 shell 命令集 - - ubuntu 的 shell 默认安装的是 dash,而不是 bash,要切换成 bash 才能执行,运行以下命令查看 sh 的详细信息,确认 shell 对应的程序是哪个: - - ```shell - ls -al /bin/sh - ``` - - 通过以下方式可以使 shell 切换回 bash: - - ```shell - sudo dpkg-reconfigure dash - ``` - - 然后选择 no 或者 否 ,并确认 - - 这样做将重新配置 dash,并使其不作为默认的 shell 工具 - -- 创建软件下载安装包根目录和软件安装根目录 - - ```shell - # 创建软件下载安装包根目录 - mkdir /opt/tools - # 创建软件安装根目录 - mkdir /opt/software - ``` - -- Git - - ```shell - # 省去编译麻烦,直接使用 apt-get 安装 - apt-get -y install git - ``` - -- JDK8 - - ```shell - # 下载 arm64 架构的安装包,解压配置环境变量后使用 - cd /opt/tools - wget https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/jdk-8u291-linux-aarch64.tar.gz && \ - tar -zxvf jdk-8u291-linux-aarch64.tar.gz && \ - mv jdk1.8.0_291 /opt/software/jdk8 - ``` - -- Maven - - ```shell - cd /opt/tools - # wget 工具下载后,直接解压缩配置环境变量使用 - wget https://dlcdn.apache.org/maven/maven-3/3.6.3/binaries/apache-maven-3.6.3-bin.tar.gz && \ - tar -zxvf apache-maven-3.6.3-bin.tar.gz && \ - mv apache-maven-3.6.3 /opt/software/maven - ``` - -- NodeJS - - ```shell - cd /opt/tools - # 下载 arm64 架构的安装包 - wget https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/node-v16.3.0-linux-arm64.tar.xz && \ - tar -xvf node-v16.3.0-linux-arm64.tar.xz && \ - mv node-v16.3.0-linux-arm64 /opt/software/nodejs - ``` - -- LDB-Toolchain - - ```shell - cd /opt/tools - # 下载 LDB-Toolchain ARM 版本 - wget https://github.com/amosbird/ldb_toolchain_gen/releases/download/v0.9.1/ldb_toolchain_gen.aarch64.sh && \ - sh ldb_toolchain_gen.aarch64.sh /opt/software/ldb_toolchain/ - ``` - -- 配置环境变量 - - ```shell - # 配置环境变量 - vim /etc/profile.d/doris.sh - export JAVA_HOME=/opt/software/jdk8 - export MAVEN_HOME=/opt/software/maven - export NODE_JS_HOME=/opt/software/nodejs - export LDB_HOME=/opt/software/ldb_toolchain - export PATH=$JAVA_HOME/bin:$MAVEN_HOME/bin:$NODE_JS_HOME/bin:$LDB_HOME/bin:$PATH - - # 保存退出并刷新环境变量 - source /etc/profile.d/doris.sh - - # 测试是否成功 - java -version - > java version "1.8.0_291" - mvn -version - > Apache Maven 3.6.3 - node --version - > v16.3.0 - gcc --version - > gcc-11 - ``` - -- 安装其他额外环境和组件 - - ```shell - # install required system packages - sudo apt install -y build-essential cmake flex automake bison binutils-dev libiberty-dev zip libncurses5-dev curl ninja-build - sudo apt-get install -y make - sudo apt-get install -y unzip - sudo apt-get install -y python2 - sudo apt-get install -y byacc - sudo apt-get install -y automake - sudo apt-get install -y libtool - sudo apt-get install -y bzip2 - sudo add-apt-repository ppa:ubuntu-toolchain-r/ppa - sudo apt update - sudo apt install gcc-11 g++-11 - sudo apt-get -y install autoconf autopoint - - # install autoconf-2.69 - cd /opt/tools - wget http://ftp.gnu.org/gnu/autoconf/autoconf-2.69.tar.gz && \ - tar zxf autoconf-2.69.tar.gz && \ - mv autoconf-2.69 /opt/software/autoconf && \ - cd /opt/software/autoconf && \ - ./configure && \ - make && \ - make install - ``` - -#### 下载源码 +### 软件环境安装命令 + + + +

+ 1. 创建软件下载安装包根目录和软件安装根目录 + +```shell + # 创建软件下载安装包根目录 + mkdir /opt/tools + # 创建软件安装根目录 + mkdir /opt/software +``` +

+

+ 2. 安装依赖项 + + - Git + +```shell + # 省去编译麻烦,直接使用 yum 安装 + yum install -y git +``` + + - JDK8 + +```shell + # 两种方式,第一种是省去额外下载和配置,直接使用 yum 安装,安装 devel 包是为了获取一些工具,如 jps 命令 + yum install -y java-1.8.0-openjdk java-1.8.0-openjdk-devel + + # 第二种是下载 arm64 架构的安装包,解压配置环境变量后使用 + cd /opt/tools + wget https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/jdk-8u291-linux-aarch64.tar.gz && \ + tar -zxvf jdk-8u291-linux-aarch64.tar.gz && \ + mv jdk1.8.0_291 /opt/software/jdk8 +``` + + - Maven + +```shell + cd /opt/tools + # wget 工具下载后,直接解压缩配置环境变量使用 + wget https://dlcdn.apache.org/maven/maven-3/3.6.3/binaries/apache-maven-3.6.3-bin.tar.gz && \ + tar -zxvf apache-maven-3.6.3-bin.tar.gz && \ + mv apache-maven-3.6.3 /opt/software/maven +``` + + - NodeJS + +```shell + cd /opt/tools + # 下载 arm64 架构的安装包 + wget https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/node-v16.3.0-linux-arm64.tar.xz && \ + tar -xvf node-v16.3.0-linux-arm64.tar.xz && \ + mv node-v16.3.0-linux-arm64 /opt/software/nodejs +``` + + - LDB-Toolchain + +```shell + cd /opt/tools + # 下载 LDB-Toolchain ARM 版本 + wget https://github.com/amosbird/ldb_toolchain_gen/releases/download/v0.9.1/ldb_toolchain_gen.aarch64.sh && \ + sh ldb_toolchain_gen.aarch64.sh /opt/software/ldb_toolchain/ +``` +

+

+ 3. 配置环境变量 + +```shell + # 配置环境变量 + vim /etc/profile.d/doris.sh + export JAVA_HOME=/opt/software/jdk8 + export MAVEN_HOME=/opt/software/maven + export NODE_JS_HOME=/opt/software/nodejs + export LDB_HOME=/opt/software/ldb_toolchain + export PATH=$JAVA_HOME/bin:$MAVEN_HOME/bin:$NODE_JS_HOME/bin:$LDB_HOME/bin:$PATH + + # 保存退出并刷新环境变量 + source /etc/profile.d/doris.sh + + # 测试是否成功 + java -version + > java version "1.8.0_291" + mvn -version + > Apache Maven 3.6.3 + node --version + > v16.3.0 + gcc --version + > gcc-11 +``` +

+

+ 4. 安装其他额外环境和组件 + +```shell + # install required system packages + sudo yum install -y byacc patch automake libtool make which file ncurses-devel gettext-devel unzip bzip2 bison zip util-linux wget git python2 + + # install autoconf-2.69 + cd /opt/tools + wget http://ftp.gnu.org/gnu/autoconf/autoconf-2.69.tar.gz && \ + tar zxf autoconf-2.69.tar.gz && \ + mv autoconf-2.69 /opt/software/autoconf && \ + cd /opt/software/autoconf && \ + ./configure && \ + make && \ + make install +``` +

+
+ +

+ 1. 更新 apt-get 软件库 + +```shell + apt-get update +``` +

+

+ 2. 检查 shell 命令集 + + ubuntu 的 shell 默认安装的是 dash,而不是 bash,要切换成 bash 才能执行,运行以下命令查看 sh 的详细信息,确认 shell 对应的程序是哪个: + +```shell + ls -al /bin/sh +``` + + 通过以下方式可以使 shell 切换回 bash: + +```shell + sudo dpkg-reconfigure dash +``` + + 然后选择 no 或者 否 ,并确认 + + 这样做将重新配置 dash,并使其不作为默认的 shell 工具 +

+

+ 3. 创建软件下载安装包根目录和软件安装根目录 + +```shell + # 创建软件下载安装包根目录 + mkdir /opt/tools + # 创建软件安装根目录 + mkdir /opt/software +``` +

+

+ 4. 安装依赖项 + - Git + +```shell + # 省去编译麻烦,直接使用 apt-get 安装 + apt-get -y install git +``` + + - JDK8 ```shell -cd /opt -git clone https://github.com/apache/doris.git + # 下载 arm64 架构的安装包,解压配置环境变量后使用 + cd /opt/tools + wget https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/jdk-8u291-linux-aarch64.tar.gz && \ + tar -zxvf jdk-8u291-linux-aarch64.tar.gz && \ + mv jdk1.8.0_291 /opt/software/jdk8 ``` -#### 安装部署 + - Maven -##### 查看是否支持 AVX2 指令集 +```shell + cd /opt/tools + # wget 工具下载后,直接解压缩配置环境变量使用 + wget https://dlcdn.apache.org/maven/maven-3/3.6.3/binaries/apache-maven-3.6.3-bin.tar.gz && \ + tar -zxvf apache-maven-3.6.3-bin.tar.gz && \ + mv apache-maven-3.6.3 /opt/software/maven +``` -若有数据返回,则代表支持,若无数据返回,则代表不支持 + - NodeJS ```shell -cat /proc/cpuinfo | grep avx2 + cd /opt/tools + # 下载 arm64 架构的安装包 + wget https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/node-v16.3.0-linux-arm64.tar.xz && \ + tar -xvf node-v16.3.0-linux-arm64.tar.xz && \ + mv node-v16.3.0-linux-arm64 /opt/software/nodejs ``` -##### 执行编译 + - LDB-Toolchain + +```shell + cd /opt/tools + # 下载 LDB-Toolchain ARM 版本 + wget https://github.com/amosbird/ldb_toolchain_gen/releases/download/v0.9.1/ldb_toolchain_gen.aarch64.sh && \ + sh ldb_toolchain_gen.aarch64.sh /opt/software/ldb_toolchain/ +``` +

+

+ 5. 配置环境变量 + +```shell + # 配置环境变量 + vim /etc/profile.d/doris.sh + export JAVA_HOME=/opt/software/jdk8 + export MAVEN_HOME=/opt/software/maven + export NODE_JS_HOME=/opt/software/nodejs + export LDB_HOME=/opt/software/ldb_toolchain + export PATH=$JAVA_HOME/bin:$MAVEN_HOME/bin:$NODE_JS_HOME/bin:$LDB_HOME/bin:$PATH + + # 保存退出并刷新环境变量 + source /etc/profile.d/doris.sh + + # 测试是否成功 + java -version + > java version "1.8.0_291" + mvn -version + > Apache Maven 3.6.3 + node --version + > v16.3.0 + gcc --version + > gcc-11 +``` +

+

+ 6. 安装其他额外环境和组件 + +```shell + # install required system packages + sudo apt install -y build-essential cmake flex automake bison binutils-dev libiberty-dev zip libncurses5-dev curl ninja-build + sudo apt-get install -y make + sudo apt-get install -y unzip + sudo apt-get install -y python2 + sudo apt-get install -y byacc + sudo apt-get install -y automake + sudo apt-get install -y libtool + sudo apt-get install -y bzip2 + sudo add-apt-repository ppa:ubuntu-toolchain-r/ppa + sudo apt update + sudo apt install gcc-11 g++-11 + sudo apt-get -y install autoconf autopoint + + # install autoconf-2.69 + cd /opt/tools + wget http://ftp.gnu.org/gnu/autoconf/autoconf-2.69.tar.gz && \ + tar zxf autoconf-2.69.tar.gz && \ + mv autoconf-2.69 /opt/software/autoconf && \ + cd /opt/software/autoconf && \ + ./configure && \ + make && \ + make install +``` +

+
+
+ +## 编译 + +:::tip +目前 ARM 环境仅推荐使用 ldb-toolchain 进行编译,该方法适用于 [commit 7f3564](https://github.com/apache/doris/commit/7f3564cca62de49c9f2ea67fcf735921dbebb4d1) 之后的 Doris 版本。 +::: + +下载 [ldb\_toolchain\_gen.aarch64.sh](https://github.com/amosbird/ldb_toolchain_gen/releases/download/v0.9.1/ldb_toolchain_gen.aarch64.sh) + +之后的编译方式参阅 [使用 LDB toolchain 编译](./compilation-with-ldb-toolchain) + +其中,jdk 和 nodejs 替换为对应的 aarch64 版本: + +1. [Java8-aarch64](https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/jdk-8u291-linux-aarch64.tar.gz) +2. [Node v16.3.0-aarch64](https://doris-thirdparty-repo.bj.bcebos.com/thirdparty/node-v16.3.0-linux-arm64.tar.xz) + +如下载预编译的三方库遇到问题,请使用`tools/build_thirdparty.sh`自行编译。在编译三方库时,请使用 gcc: +```shell +export DORIS_TOOLCHAIN=gcc +``` +在 ARM 平台编译 Doris 时,**请关闭 AVX2 和 LIBUNWIND 三方库**: ```shell -# 支持 AVX2 指令集的机器,直接编译即可 -sh build.sh -# 不支持 AVX2 指令集的机器,使用如下命令编译 -USE_AVX2=OFF sh build.sh +export USE_AVX2=OFF +export USE_UNWIND=OFF ``` -### 常见问题 +如在编译、启动时仍遇到问题,请查阅[常见问题](#常见问题)。如果没有相关解决方案,欢迎[提出issue](https://github.com/apache/doris/issues)。 + +## 常见问题 + +### 编译问题 1. 编译第三方库 libhdfs3.a ,找不到文件夹 @@ -373,10 +378,10 @@ USE_AVX2=OFF sh build.sh - 使用第三方下载仓库 - ```shell +```shell export REPOSITORY_URL=https://doris-thirdparty-repo.bj.bcebos.com/thirdparty sh /opt/doris/thirdparty/build-thirdparty.sh - ``` +``` REPOSITORY_URL 中包含所有第三方库源码包和他们的历史版本。 @@ -398,12 +403,12 @@ USE_AVX2=OFF sh build.sh 建立 `\usr\bin` 中 `python` 命令的软连接 - ```shell +```shell # 查看python安装目录 whereis python # 建立软连接 sudo ln -s /usr/bin/python2.7 /usr/bin/python - ``` +``` 3. 编译结束后没有 output 目录 @@ -417,9 +422,9 @@ USE_AVX2=OFF sh build.sh - 解决方案 - ```shell +```shell sh build.sh --clean - ``` +``` 4. spark-dpp 编译失败 @@ -459,43 +464,7 @@ USE_AVX2=OFF sh build.sh 扩大设备剩余空间,如删除不需要的文件等 -6. 启动FE失败,事务-20 问题 - - - 问题描述 - - 在启动 FE 时,报事务错误 20 问题,状态为 UNKNOWN - - > [BDBEnvironment.setup():198] error to open replicated environment. will exit. - > com.sleepycat.je.rep.ReplicaWriteException: (JE 18.3.12) Problem closing transaction 20. The current state is:UNKNOWN. The node transitioned to this state at:Fri Apr 22 12:48:08 CST 2022 - - - 问题原因 - - 硬盘空间不足,需更多空间 - - - 解决方案 - - 释放硬盘空间或者挂载新硬盘 - -7. BDB 环境设置异常,磁盘寻找错误 - - - 问题描述 - - 在迁移 FE 所在的盘符后启动 FE 报异常 - - > 2022-04-22 16:21:44,092 ERROR (MASTER 172.28.7.231_9010_1650606822109(-1)|1) [BDBJEJournal.open():306] catch an exception when setup bdb environment. will exit. - > com.sleepycat.je.DiskLimitException: (JE 18.3.12) Disk usage is not within je.maxDisk or je.freeDisk limits and write operations are prohibited: maxDiskLimit=0 freeDiskLimit=5,368,709,120 adjustedMaxDiskLimit=0 maxDiskOverage=0 freeDiskShortage=1,536,552,960 diskFreeSpace=3,832,156,160 availableLogSize=-1,536,552,960 totalLogSize=4,665 activeLogSize=4,665 reservedLogSize=0 protectedLogSize=0 protectedLogSizeMap={} - - - 问题原因 - - 迁移了 FE 所在的位置,元数据存储的硬盘信息无法匹配到,或者该硬盘损坏或未挂载 - - - 解决方案 - - - 检查硬盘是否正常,是否初始化并正确挂载 - - 修复 FE 元数据 - - 若为测试机器,则可以删除元数据目录重新启动 - -8. 在 pkg.config 中找不到 pkg.m4 文件 +6. 在 pkg.config 中找不到 pkg.m4 文件 - 问题描述 @@ -518,12 +487,12 @@ USE_AVX2=OFF sh build.sh 将 ldb/aclocal 目录下的 `pkg.m4` 文件拷贝至 libxml2/m4 目录下,重新编译第三方库 - ```shell +```shell cp /opt/software/ldb_toolchain/share/aclocal/pkg.m4 /opt/doris/thirdparty/src/libxml2-v2.9.10/m4 sh /opt/doris/thirdparty/build-thirdparty.sh - ``` +``` -9. 执行测试 CURL_HAS_TLS_PROXY 失败 +7. 执行测试 CURL_HAS_TLS_PROXY 失败 - 问题描述 @@ -549,7 +518,7 @@ USE_AVX2=OFF sh build.sh 配置 ldb 环境变量 - ```shell +```shell # 配置环境变量 vim /etc/profile.d/ldb.sh export LDB_HOME=/opt/software/ldb_toolchain @@ -559,14 +528,59 @@ USE_AVX2=OFF sh build.sh # 测试 gcc --version > gcc-11 - ``` +``` + +### 启动问题 + +1. 启动FE失败,事务-20 问题 + + - 问题描述 + + 在启动 FE 时,报事务错误 20 问题,状态为 UNKNOWN + + > [BDBEnvironment.setup():198] error to open replicated environment. will exit. + > com.sleepycat.je.rep.ReplicaWriteException: (JE 18.3.12) Problem closing transaction 20. The current state is:UNKNOWN. The node transitioned to this state at:Fri Apr 22 12:48:08 CST 2022 + + - 问题原因 + + 硬盘空间不足,需更多空间 + + - 解决方案 + + 释放硬盘空间或者挂载新硬盘 -10. 其他异常问题 +2. BDB 环境设置异常,磁盘寻找错误 - 问题描述 + 在迁移 FE 所在的盘符后启动 FE 报异常 + > 2022-04-22 16:21:44,092 ERROR (MASTER 172.28.7.231_9010_1650606822109(-1)|1) [BDBJEJournal.open():306] catch an exception when setup bdb environment. will exit. + > com.sleepycat.je.DiskLimitException: (JE 18.3.12) Disk usage is not within je.maxDisk or je.freeDisk limits and write operations are prohibited: maxDiskLimit=0 freeDiskLimit=5,368,709,120 adjustedMaxDiskLimit=0 maxDiskOverage=0 freeDiskShortage=1,536,552,960 diskFreeSpace=3,832,156,160 availableLogSize=-1,536,552,960 totalLogSize=4,665 activeLogSize=4,665 reservedLogSize=0 protectedLogSize=0 protectedLogSizeMap={} - 如有以下组件的错误提示,则统一以该方案解决 + - 问题原因 + 迁移了 FE 所在的位置,元数据存储的硬盘信息无法匹配到,或者该硬盘损坏或未挂载 + + - 解决方案 + - 检查硬盘是否正常,是否初始化并正确挂载 + - 修复 FE 元数据 + - 若为测试机器,则可以删除元数据目录重新启动 + +3. BE启动时 coredump,core 栈中可见 `libc.so` 或 `glibc-compatibility/` 字样 + + - 问题原因 + 当前机器 glibc 版本过低。可以通过 `ldd --version` 确认,如果版本小于 2.27 则可能发生这一问题。 + + - 解决方案 + 重新编译 BE,编译时增加环境变量: +```shell + export GLIBC_COMPATIBILITY=OFF +``` + +### 其他组件问题 + +1. + - 问题描述 + 如有以下组件的错误提示,则统一以该方案解决: - bison 相关 1. 安装 bison-3.0.4 时报 fseterr.c 错误 - flex 相关 @@ -585,11 +599,11 @@ USE_AVX2=OFF sh build.sh - 问题原因 - 未使用 ldb-toolchain 进行编译 + 未使用正确的 ldb-toolchain 进行编译 - 解决方案 - 检查 ldb-toolchain 环境变量是否配置 - - 查看 gcc 版本是否是 `gcc-11` + - 查看 gcc 版本是否与[使用ldb-toolchain编译](./compilation-with-ldb-toolchain)文档中推荐一致 - 删除 `ldb_toolchain_gen.aarch64.sh` 脚本执行后的 ldb 目录,重新执行并配置环境变量,验证 gcc 版本 From c6adca4e48d2a1d38f6a5bd81a471216a29d4e93 Mon Sep 17 00:00:00 2001 From: yujun Date: Tue, 16 Jan 2024 17:56:05 +0800 Subject: [PATCH 037/200] [feature](merge-cloud) Add create cloud table (#30001) --- .../java/org/apache/doris/common/Config.java | 7 + .../doris/alter/MaterializedViewHandler.java | 3 +- .../doris/alter/SchemaChangeHandler.java | 3 +- .../org/apache/doris/backup/RestoreJob.java | 3 +- .../java/org/apache/doris/catalog/Column.java | 143 +++++ .../catalog/DynamicPartitionProperty.java | 45 +- .../java/org/apache/doris/catalog/Env.java | 21 +- .../org/apache/doris/catalog/EnvFactory.java | 34 ++ .../java/org/apache/doris/catalog/Index.java | 42 ++ .../org/apache/doris/catalog/OlapTable.java | 26 +- .../apache/doris/catalog/TableProperty.java | 10 + .../java/org/apache/doris/catalog/Tablet.java | 32 +- .../doris/cloud/catalog/CloudTablet.java | 82 +++ .../datasource/CloudInternalCatalog.java | 492 ++++++++++++++++++ .../doris/common/util/PropertyAnalyzer.java | 20 + .../doris/datasource/InternalCatalog.java | 71 ++- .../doris/service/FrontendServiceImpl.java | 9 +- .../doris/system/SystemInfoService.java | 3 + .../apache/doris/catalog/EnvFactoryTest.java | 4 + 19 files changed, 996 insertions(+), 54 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudTablet.java diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index 1b1d8906d96b9e..909e1bf2d7ae72 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -2440,6 +2440,10 @@ public static boolean isCloudMode() { return !cloud_unique_id.isEmpty(); } + public static boolean isNotCloudMode() { + return cloud_unique_id.isEmpty(); + } + /** * MetaService endpoint, ip:port, such as meta_service_endpoint = "192.0.0.10:8866" */ @@ -2452,6 +2456,9 @@ public static boolean isCloudMode() { @ConfField(mutable = true) public static int meta_service_connection_pool_size = 20; + @ConfField(mutable = true) + public static int meta_service_rpc_retry_times = 200; + // A connection will expire after a random time during [base, 2*base), so that the FE // has a chance to connect to a new RS. Set zero to disable it. @ConfField(mutable = true) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java index 6425230b147992..3f7122f36505e9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java @@ -31,6 +31,7 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.EnvFactory; import org.apache.doris.catalog.KeysType; import org.apache.doris.catalog.MaterializedIndex; import org.apache.doris.catalog.MaterializedIndex.IndexState; @@ -397,7 +398,7 @@ private RollupJobV2 createMaterializedViewJob(String rawSql, String mvName, Stri long baseTabletId = baseTablet.getId(); long mvTabletId = idGeneratorBuffer.getNextId(); - Tablet newTablet = new Tablet(mvTabletId); + Tablet newTablet = EnvFactory.createTablet(mvTabletId); mvIndex.addTablet(newTablet, mvTabletMeta); addedTablets.add(newTablet); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index 206f028b02339e..b5895ead65360b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -41,6 +41,7 @@ import org.apache.doris.catalog.DistributionInfo; import org.apache.doris.catalog.DistributionInfo.DistributionInfoType; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.EnvFactory; import org.apache.doris.catalog.HashDistributionInfo; import org.apache.doris.catalog.Index; import org.apache.doris.catalog.KeysType; @@ -1547,7 +1548,7 @@ private void createJob(String rawSql, long dbId, OlapTable olapTable, Map bfColumns, List indexes) throws DdlException { + OlapFile.ColumnPB.Builder builder = OlapFile.ColumnPB.newBuilder(); + + // when doing schema change, some modified column has a prefix in name. + // this prefix is only used in FE, not visible to BE, so we should remove this prefix. + builder.setName(name.startsWith(SchemaChangeHandler.SHADOW_NAME_PREFIX) + ? name.substring(SchemaChangeHandler.SHADOW_NAME_PREFIX.length()) : name); + + builder.setUniqueId(uniqueId); + builder.setType(this.getDataType().toThrift().name()); + builder.setIsKey(this.isKey); + if (null != this.aggregationType) { + if (type.isAggStateType()) { + AggStateType aggState = (AggStateType) type; + builder.setAggregation(aggState.getFunctionName()); + builder.setResultIsNullable(aggState.getResultIsNullable()); + for (Column column : children) { + builder.addChildrenColumns(column.toPb(Sets.newHashSet(), Lists.newArrayList())); + } + } else { + builder.setAggregation(this.aggregationType.toString()); + } + } else { + builder.setAggregation("NONE"); + } + builder.setIsNullable(this.isAllowNull); + if (this.defaultValue != null) { + builder.setDefaultValue(ByteString.copyFrom(this.defaultValue.getBytes())); + } + builder.setPrecision(this.getPrecision()); + builder.setFrac(this.getScale()); + + int length = getFieldLengthByType(this.getDataType().toThrift(), this.getStrLen()); + + builder.setLength(length); + builder.setIndexLength(length); + if (this.getDataType().toThrift() == TPrimitiveType.VARCHAR + || this.getDataType().toThrift() == TPrimitiveType.STRING) { + builder.setIndexLength(this.getOlapColumnIndexSize()); + } + + if (bfColumns != null && bfColumns.contains(this.name)) { + builder.setIsBfColumn(true); + } else { + builder.setIsBfColumn(false); + } + builder.setVisible(visible); + + if (indexes != null) { + for (Index index : indexes) { + if (index.getIndexType() == IndexDef.IndexType.BITMAP) { + List columns = index.getColumns(); + if (this.name.equalsIgnoreCase(columns.get(0))) { + builder.setHasBitmapIndex(true); + break; + } + } + } + } + + if (this.type.isArrayType()) { + Column child = this.getChildren().get(0); + builder.addChildrenColumns(child.toPb(Sets.newHashSet(), Lists.newArrayList())); + } else if (this.type.isMapType()) { + Column k = this.getChildren().get(0); + builder.addChildrenColumns(k.toPb(Sets.newHashSet(), Lists.newArrayList())); + Column v = this.getChildren().get(1); + builder.addChildrenColumns(v.toPb(Sets.newHashSet(), Lists.newArrayList())); + } else if (this.type.isStructType()) { + List childrenColumns = this.getChildren(); + for (Column c : childrenColumns) { + builder.addChildrenColumns(c.toPb(Sets.newHashSet(), Lists.newArrayList())); + } + } + + OlapFile.ColumnPB col = builder.build(); + return col; + } + // CLOUD_CODE_END public void checkSchemaChangeAllowed(Column other) throws DdlException { if (Strings.isNullOrEmpty(other.name)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/DynamicPartitionProperty.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/DynamicPartitionProperty.java index 54c49c1ee8dce7..2daeb7bc72cdba 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/DynamicPartitionProperty.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/DynamicPartitionProperty.java @@ -19,6 +19,7 @@ import org.apache.doris.analysis.TimestampArithmeticExpr.TimeUnit; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; import org.apache.doris.common.FeConstants; import org.apache.doris.common.util.DynamicPartitionUtil; @@ -216,28 +217,34 @@ public String getSortedReservedHistoryPeriods(String reservedHistoryPeriods, Str * use table replication_num as dynamic_partition.replication_num default value */ public String getProperties(ReplicaAllocation tableReplicaAlloc) { - ReplicaAllocation tmpAlloc = this.replicaAlloc.isNotSet() ? tableReplicaAlloc : this.replicaAlloc; - String res = ",\n\"" + ENABLE + "\" = \"" + enable + "\"" - + ",\n\"" + TIME_UNIT + "\" = \"" + timeUnit + "\"" - + ",\n\"" + TIME_ZONE + "\" = \"" + tz.getID() + "\"" - + ",\n\"" + START + "\" = \"" + start + "\"" - + ",\n\"" + END + "\" = \"" + end + "\"" - + ",\n\"" + PREFIX + "\" = \"" + prefix + "\"" - + ",\n\"" + REPLICATION_ALLOCATION + "\" = \"" + tmpAlloc.toCreateStmt() + "\"" - + ",\n\"" + BUCKETS + "\" = \"" + buckets + "\"" - + ",\n\"" + CREATE_HISTORY_PARTITION + "\" = \"" + createHistoryPartition + "\"" - + ",\n\"" + HISTORY_PARTITION_NUM + "\" = \"" + historyPartitionNum + "\"" - + ",\n\"" + HOT_PARTITION_NUM + "\" = \"" + hotPartitionNum + "\"" - + ",\n\"" + RESERVED_HISTORY_PERIODS + "\" = \"" + reservedHistoryPeriods + "\"" - + ",\n\"" + STORAGE_POLICY + "\" = \"" + storagePolicy + "\""; - if (!Strings.isNullOrEmpty(storageMedium)) { - res += ",\n\"" + STORAGE_MEDIUM + "\" = \"" + storageMedium + "\""; + StringBuilder sb = new StringBuilder(); + sb.append(",\n\"" + ENABLE + "\" = \"" + enable + "\""); + sb.append(",\n\"" + TIME_UNIT + "\" = \"" + timeUnit + "\""); + sb.append(",\n\"" + TIME_ZONE + "\" = \"" + tz.getID() + "\""); + sb.append(",\n\"" + START + "\" = \"" + start + "\""); + sb.append(",\n\"" + END + "\" = \"" + end + "\""); + sb.append(",\n\"" + PREFIX + "\" = \"" + prefix + "\""); + if (Config.isNotCloudMode()) { + ReplicaAllocation tmpAlloc = this.replicaAlloc.isNotSet() ? tableReplicaAlloc : this.replicaAlloc; + sb.append(",\n\"" + REPLICATION_ALLOCATION + "\" = \"" + tmpAlloc.toCreateStmt() + "\""); + } + sb.append(",\n\"" + BUCKETS + "\" = \"" + buckets + "\""); + sb.append(",\n\"" + CREATE_HISTORY_PARTITION + "\" = \"" + createHistoryPartition + "\""); + sb.append(",\n\"" + HISTORY_PARTITION_NUM + "\" = \"" + historyPartitionNum + "\""); + sb.append(",\n\"" + HOT_PARTITION_NUM + "\" = \"" + hotPartitionNum + "\""); + sb.append(",\n\"" + RESERVED_HISTORY_PERIODS + "\" = \"" + reservedHistoryPeriods + "\""); + if (Config.isNotCloudMode()) { + sb.append(",\n\"" + STORAGE_POLICY + "\" = \"" + storagePolicy + "\""); + if (!Strings.isNullOrEmpty(storageMedium)) { + sb.append(",\n\"" + STORAGE_MEDIUM + "\" = \"" + storageMedium + "\""); + } } if (getTimeUnit().equalsIgnoreCase(TimeUnit.WEEK.toString())) { - res += ",\n\"" + START_DAY_OF_WEEK + "\" = \"" + startOfWeek.dayOfWeek + "\""; + sb.append(",\n\"" + START_DAY_OF_WEEK + "\" = \"" + startOfWeek.dayOfWeek + "\""); } else if (getTimeUnit().equalsIgnoreCase(TimeUnit.MONTH.toString())) { - res += ",\n\"" + START_DAY_OF_MONTH + "\" = \"" + startOfMonth.day + "\""; + sb.append(",\n\"" + START_DAY_OF_MONTH + "\" = \"" + startOfMonth.day + "\""); } - return res; + return sb.toString(); } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index 5b9676a201de2d..b72544365aa38a 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -3243,12 +3243,18 @@ public static void getDdlStmt(DdlStmt ddlStmt, String dbName, TableIf table, Lis // replicationNum ReplicaAllocation replicaAlloc = olapTable.getDefaultReplicaAllocation(); - sb.append("\"").append(PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION).append("\" = \""); - sb.append(replicaAlloc.toCreateStmt()).append("\""); - // min load replica num - sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_MIN_LOAD_REPLICA_NUM).append("\" = \""); - sb.append(olapTable.getMinLoadReplicaNum()).append("\""); + if (Config.isCloudMode()) { + sb.append("\"").append(PropertyAnalyzer.PROPERTIES_FILE_CACHE_TTL_SECONDS).append("\" = \""); + sb.append(olapTable.getTTLSeconds()).append("\""); + } else { + sb.append("\"").append(PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION).append("\" = \""); + sb.append(replicaAlloc.toCreateStmt()).append("\""); + + // min load replica num + sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_MIN_LOAD_REPLICA_NUM).append("\" = \""); + sb.append(olapTable.getMinLoadReplicaNum()).append("\""); + } // bloom filter Set bfColumnNames = olapTable.getCopiedBfColumns(); @@ -3296,6 +3302,11 @@ public static void getDdlStmt(DdlStmt ddlStmt, String dbName, TableIf table, Lis sb.append(olapTable.getDataSortInfo().toSql()); } + if (Config.isCloudMode() && olapTable.getTTLSeconds() != 0) { + sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_FILE_CACHE_TTL_SECONDS).append("\" = \""); + sb.append(olapTable.getTTLSeconds()).append("\""); + } + // in memory if (olapTable.isInMemory()) { sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_INMEMORY).append("\" = \""); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/EnvFactory.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/EnvFactory.java index a44e9da9c775b7..c653159d7b6820 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/EnvFactory.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/EnvFactory.java @@ -20,10 +20,12 @@ import org.apache.doris.cloud.catalog.CloudEnv; import org.apache.doris.cloud.catalog.CloudPartition; import org.apache.doris.cloud.catalog.CloudReplica; +import org.apache.doris.cloud.catalog.CloudTablet; import org.apache.doris.cloud.datasource.CloudInternalCatalog; import org.apache.doris.common.Config; import org.apache.doris.datasource.InternalCatalog; +import java.lang.reflect.Type; public class EnvFactory { @@ -43,6 +45,14 @@ public static InternalCatalog createInternalCatalog() { } } + public static Type getPartitionClass() { + if (Config.isCloudMode()) { + return CloudPartition.class; + } else { + return Partition.class; + } + } + public static Partition createPartition() { if (Config.isCloudMode()) { return new CloudPartition(); @@ -51,6 +61,30 @@ public static Partition createPartition() { } } + public static Type getTabletClass() { + if (Config.isCloudMode()) { + return CloudTablet.class; + } else { + return Tablet.class; + } + } + + public static Tablet createTablet() { + if (Config.isCloudMode()) { + return new CloudTablet(); + } else { + return new Tablet(); + } + } + + public static Tablet createTablet(long tabletId) { + if (Config.isCloudMode()) { + return new CloudTablet(tabletId); + } else { + return new Tablet(tabletId); + } + } + public static Replica createReplica() { if (Config.isCloudMode()) { return new CloudReplica(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Index.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Index.java index e2235868a1ce17..4ec72e0c23289d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Index.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Index.java @@ -24,6 +24,7 @@ import org.apache.doris.common.io.Writable; import org.apache.doris.common.util.PrintableMap; import org.apache.doris.persist.gson.GsonUtils; +import org.apache.doris.proto.OlapFile; import org.apache.doris.thrift.TIndexType; import org.apache.doris.thrift.TOlapTableIndex; @@ -211,6 +212,47 @@ public TOlapTableIndex toThrift() { return tIndex; } + public OlapFile.TabletIndexPB toPb(List schemaColumns) { + OlapFile.TabletIndexPB.Builder builder = OlapFile.TabletIndexPB.newBuilder(); + builder.setIndexId(indexId); + builder.setIndexName(indexName); + for (String columnName : columns) { + for (Column column : schemaColumns) { + if (column.getName().equals(columnName)) { + builder.addColUniqueId(column.getUniqueId()); + } + } + } + + switch (indexType) { + case BITMAP: + builder.setIndexType(OlapFile.IndexType.BITMAP); + break; + + case INVERTED: + builder.setIndexType(OlapFile.IndexType.INVERTED); + break; + + case NGRAM_BF: + builder.setIndexType(OlapFile.IndexType.NGRAM_BF); + break; + + case BLOOMFILTER: + builder.setIndexType(OlapFile.IndexType.BLOOMFILTER); + break; + + default: + throw new RuntimeException("indexType " + indexType + " is not processed in toPb"); + } + + if (properties != null) { + builder.putAllProperties(properties); + } + + OlapFile.TabletIndexPB index = builder.build(); + return index; + } + public static void checkConflict(Collection indices, Set bloomFilters) throws AnalysisException { indices = indices == null ? Collections.emptyList() : indices; bloomFilters = bloomFilters == null ? Collections.emptySet() : bloomFilters; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index 3236fc09ec9010..5e36886e56a11e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -609,7 +609,7 @@ public Status resetIdsForRestore(Env env, Database db, ReplicaAllocation restore idx.clearTabletsForRestore(); for (int i = 0; i < tabletNum; i++) { long newTabletId = env.getNextId(); - Tablet newTablet = new Tablet(newTabletId); + Tablet newTablet = EnvFactory.createTablet(newTabletId); idx.addTablet(newTablet, null /* tablet meta */, true /* is restore */); // replicas @@ -668,6 +668,14 @@ public List getIndexIdListExceptBaseIndex() { return result; } + public List getIndexIdList() { + List result = Lists.newArrayList(); + for (Long indexId : indexIdToMeta.keySet()) { + result.add(indexId); + } + return result; + } + // schema public Map> getIndexIdToSchema() { return getIndexIdToSchema(Util.showHiddenColumns()); @@ -1902,6 +1910,22 @@ public String getEstimatePartitionSize() { return ""; } + public long getTTLSeconds() { + if (tableProperty != null) { + return tableProperty.getTTLSeconds(); + } + return 0L; + } + + public void setTTLSeconds(long ttlSeconds) { + if (tableProperty == null) { + tableProperty = new TableProperty(new HashMap<>()); + } + tableProperty.modifyTableProperties(PropertyAnalyzer.PROPERTIES_FILE_CACHE_TTL_SECONDS, + Long.valueOf(ttlSeconds).toString()); + tableProperty.buildTTLSeconds(); + } + public boolean getEnableLightSchemaChange() { if (tableProperty != null) { return tableProperty.getUseSchemaLightChange(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java index 3d5215bc483349..5e857acf884a96 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java @@ -60,6 +60,7 @@ public class TableProperty implements Writable { private ReplicaAllocation replicaAlloc = ReplicaAllocation.DEFAULT_ALLOCATION; private boolean isInMemory = false; private short minLoadReplicaNum = -1; + private long ttlSeconds = 0L; private String storagePolicy = ""; private Boolean isBeingSynced = null; @@ -188,6 +189,15 @@ public TableProperty buildInMemory() { return this; } + public TableProperty buildTTLSeconds() { + ttlSeconds = Long.parseLong(properties.getOrDefault(PropertyAnalyzer.PROPERTIES_FILE_CACHE_TTL_SECONDS, "0")); + return this; + } + + public long getTTLSeconds() { + return ttlSeconds; + } + public TableProperty buildEnableLightSchemaChange() { enableLightSchemaChange = Boolean.parseBoolean( properties.getOrDefault(PropertyAnalyzer.PROPERTIES_ENABLE_LIGHT_SCHEMA_CHANGE, "false")); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java index e579c6b9a4894c..0ab869ec803ff9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java @@ -24,6 +24,7 @@ import org.apache.doris.common.FeConstants; import org.apache.doris.common.FeMetaVersion; import org.apache.doris.common.Pair; +import org.apache.doris.common.UserException; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.persist.gson.GsonUtils; @@ -84,7 +85,7 @@ public enum TabletStatus { @SerializedName(value = "id") private long id; @SerializedName(value = "replicas") - private List replicas; + protected List replicas; @SerializedName(value = "checkedVersion") private long checkedVersion; @Deprecated @@ -112,7 +113,7 @@ public Tablet(long tabletId) { this(tabletId, new ArrayList<>()); } - public Tablet(long tabletId, List replicas) { + private Tablet(long tabletId, List replicas) { this.id = tabletId; this.replicas = replicas; if (this.replicas == null) { @@ -168,13 +169,14 @@ public Pair getCooldownConf() { } } - private boolean deleteRedundantReplica(long backendId, long version) { + protected boolean isLatestReplicaAndDeleteOld(Replica newReplica) { boolean delete = false; boolean hasBackend = false; + long version = newReplica.getVersion(); Iterator iterator = replicas.iterator(); while (iterator.hasNext()) { Replica replica = iterator.next(); - if (replica.getBackendId() == backendId) { + if (replica.getBackendId() == newReplica.getBackendId()) { hasBackend = true; if (replica.getVersion() <= version) { iterator.remove(); @@ -187,7 +189,7 @@ private boolean deleteRedundantReplica(long backendId, long version) { } public void addReplica(Replica replica, boolean isRestore) { - if (deleteRedundantReplica(replica.getBackendId(), replica.getVersion())) { + if (isLatestReplicaAndDeleteOld(replica)) { replicas.add(replica); if (!isRestore) { Env.getCurrentInvertedIndex().addReplica(id, replica); @@ -212,16 +214,22 @@ public Set getBackendIds() { } public List getNormalReplicaBackendIds() { - return Lists.newArrayList(getNormalReplicaBackendPathMap().keySet()); + try { + return Lists.newArrayList(getNormalReplicaBackendPathMap().keySet()); + } catch (Exception e) { + LOG.warn("failed to getNormalReplicaBackendIds", e); + return Lists.newArrayList(); + } } // return map of (BE id -> path hash) of normal replicas // for load plan. - public Multimap getNormalReplicaBackendPathMap() { + public Multimap getNormalReplicaBackendPathMap() throws UserException { Multimap map = HashMultimap.create(); SystemInfoService infoService = Env.getCurrentSystemInfo(); for (Replica replica : replicas) { - if (!infoService.checkBackendAlive(replica.getBackendId())) { + long backendId = replica.getBackendId(); + if (!infoService.checkBackendAlive(backendId)) { continue; } @@ -232,7 +240,7 @@ public Multimap getNormalReplicaBackendPathMap() { ReplicaState state = replica.getState(); if (state.canLoad() || (state == ReplicaState.DECOMMISSION && replica.getPostWatermarkTxnId() < 0)) { - map.put(replica.getBackendId(), replica.getPathHash()); + map.put(backendId, replica.getPathHash()); } } return map; @@ -383,7 +391,7 @@ public void readFields(DataInput in) throws IOException { int replicaCount = in.readInt(); for (int i = 0; i < replicaCount; ++i) { Replica replica = Replica.read(in); - if (deleteRedundantReplica(replica.getBackendId(), replica.getVersion())) { + if (isLatestReplicaAndDeleteOld(replica)) { replicas.add(replica); } } @@ -396,10 +404,10 @@ public void readFields(DataInput in) throws IOException { public static Tablet read(DataInput in) throws IOException { if (Env.getCurrentEnvJournalVersion() >= FeMetaVersion.VERSION_115) { String json = Text.readString(in); - return GsonUtils.GSON.fromJson(json, Tablet.class); + return GsonUtils.GSON.fromJson(json, EnvFactory.getTabletClass()); } - Tablet tablet = new Tablet(); + Tablet tablet = EnvFactory.createTablet(); tablet.readFields(in); return tablet; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudTablet.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudTablet.java new file mode 100644 index 00000000000000..be8a2599dde51c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudTablet.java @@ -0,0 +1,82 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.cloud.catalog; + +import org.apache.doris.catalog.Replica; +import org.apache.doris.catalog.Tablet; +import org.apache.doris.common.InternalErrorCode; +import org.apache.doris.common.UserException; +import org.apache.doris.system.SystemInfoService; + +import com.google.common.collect.Multimap; + +import java.util.Iterator; + +public class CloudTablet extends Tablet { + + public CloudTablet() { + super(); + } + + public CloudTablet(long tabletId) { + super(tabletId); + } + + @Override + public Replica getReplicaByBackendId(long backendId) { + if (!replicas.isEmpty()) { + return replicas.get(0); + } + return null; + } + + @Override + public Multimap getNormalReplicaBackendPathMap() throws UserException { + Multimap pathMap = super.getNormalReplicaBackendPathMap(); + + if (pathMap.containsKey(-1L)) { + pathMap.removeAll(-1L); + if (pathMap.isEmpty()) { + throw new UserException(InternalErrorCode.META_NOT_FOUND_ERR, + SystemInfoService.NOT_USING_VALID_CLUSTER_MSG); + } + } + + return pathMap; + } + + @Override + protected boolean isLatestReplicaAndDeleteOld(Replica newReplica) { + boolean delete = false; + boolean hasBackend = false; + long version = newReplica.getVersion(); + Iterator iterator = replicas.iterator(); + while (iterator.hasNext()) { + hasBackend = true; + Replica replica = iterator.next(); + if (replica.getVersion() <= version) { + iterator.remove(); + delete = true; + } + } + + return delete || !hasBackend; + } + +} + diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java index 5db352f85263e6..13f04fd36df825 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java @@ -17,13 +17,505 @@ package org.apache.doris.cloud.datasource; +import org.apache.doris.analysis.DataSortInfo; +import org.apache.doris.catalog.BinlogConfig; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.DistributionInfo; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.EnvFactory; +import org.apache.doris.catalog.Index; +import org.apache.doris.catalog.KeysType; +import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MaterializedIndex.IndexState; +import org.apache.doris.catalog.MaterializedIndexMeta; +import org.apache.doris.catalog.MetaIdGenerator.IdGeneratorBuffer; +import org.apache.doris.catalog.Partition; +import org.apache.doris.catalog.Replica; +import org.apache.doris.catalog.Replica.ReplicaState; +import org.apache.doris.catalog.ReplicaAllocation; +import org.apache.doris.catalog.Tablet; +import org.apache.doris.catalog.TabletMeta; +import org.apache.doris.cloud.catalog.CloudPartition; +import org.apache.doris.cloud.catalog.CloudReplica; +import org.apache.doris.cloud.proto.Cloud; +import org.apache.doris.cloud.rpc.MetaServiceProxy; +import org.apache.doris.common.Config; +import org.apache.doris.common.DdlException; import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.proto.OlapCommon; +import org.apache.doris.proto.OlapFile; +import org.apache.doris.proto.Types; +import org.apache.doris.rpc.RpcException; +import org.apache.doris.thrift.TCompressionType; +import org.apache.doris.thrift.TSortType; +import org.apache.doris.thrift.TStorageFormat; +import org.apache.doris.thrift.TStorageMedium; +import org.apache.doris.thrift.TTabletType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Maps; +import doris.segment_v2.SegmentV2; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; public class CloudInternalCatalog extends InternalCatalog { + private static final Logger LOG = LogManager.getLogger(CloudInternalCatalog.class); public CloudInternalCatalog() { super(); } + // BEGIN CREATE TABLE + + // TODO(merge-cloud): merge code with InternalCatalog + @Override + protected Partition createPartitionWithIndices(long dbId, long tableId, String tableName, + long baseIndexId, long partitionId, String partitionName, Map indexIdToMeta, + DistributionInfo distributionInfo, TStorageMedium storageMedium, ReplicaAllocation replicaAlloc, + Long versionInfo, Set bfColumns, double bfFpp, Set tabletIdSet, List indexes, + boolean isInMemory, TStorageFormat storageFormat, TTabletType tabletType, TCompressionType compressionType, + DataSortInfo dataSortInfo, boolean enableUniqueKeyMergeOnWrite, String storagePolicy, + IdGeneratorBuffer idGeneratorBuffer, boolean disableAutoCompaction, + boolean enableSingleReplicaCompaction, boolean skipWriteIndexOnLoad, + String compactionPolicy, Long timeSeriesCompactionGoalSizeMbytes, + Long timeSeriesCompactionFileCountThreshold, Long timeSeriesCompactionTimeThresholdSeconds, + Long timeSeriesCompactionEmptyRowsetsThreshold, + boolean storeRowColumn, BinlogConfig binlogConfig, + boolean isStorageMediumSpecified, List clusterKeyIndexes, + long ttlSeconds) throws DdlException { + // create base index first. + Preconditions.checkArgument(baseIndexId != -1); + MaterializedIndex baseIndex = new MaterializedIndex(baseIndexId, IndexState.NORMAL); + + LOG.info("begin create cloud partition"); + // create partition with base index + Partition partition = new CloudPartition(partitionId, partitionName, baseIndex, + distributionInfo, dbId, tableId); + + // add to index map + Map indexMap = Maps.newHashMap(); + indexMap.put(baseIndexId, baseIndex); + + // create rollup index if has + for (long indexId : indexIdToMeta.keySet()) { + if (indexId == baseIndexId) { + continue; + } + + MaterializedIndex rollup = new MaterializedIndex(indexId, IndexState.NORMAL); + indexMap.put(indexId, rollup); + } + + // version and version hash + if (versionInfo != null) { + partition.updateVisibleVersion(versionInfo); + } + long version = partition.getVisibleVersion(); + + // short totalReplicaNum = replicaAlloc.getTotalReplicaNum(); + for (Map.Entry entry : indexMap.entrySet()) { + long indexId = entry.getKey(); + MaterializedIndex index = entry.getValue(); + MaterializedIndexMeta indexMeta = indexIdToMeta.get(indexId); + + // create tablets + int schemaHash = indexMeta.getSchemaHash(); + TabletMeta tabletMeta = new TabletMeta(dbId, tableId, partitionId, indexId, schemaHash, storageMedium); + createCloudTablets(index, ReplicaState.NORMAL, distributionInfo, version, replicaAlloc, + tabletMeta, tabletIdSet); + + short shortKeyColumnCount = indexMeta.getShortKeyColumnCount(); + // TStorageType storageType = indexMeta.getStorageType(); + List columns = indexMeta.getSchema(); + KeysType keysType = indexMeta.getKeysType(); + + Cloud.CreateTabletsRequest.Builder requestBuilder = Cloud.CreateTabletsRequest.newBuilder(); + for (Tablet tablet : index.getTablets()) { + OlapFile.TabletMetaCloudPB.Builder builder = createTabletMetaBuilder(tableId, indexId, + partitionId, tablet, tabletType, schemaHash, keysType, shortKeyColumnCount, + bfColumns, bfFpp, indexes, columns, dataSortInfo, compressionType, + storagePolicy, isInMemory, false, tableName, ttlSeconds, + enableUniqueKeyMergeOnWrite, storeRowColumn, indexMeta.getSchemaVersion()); + requestBuilder.addTabletMetas(builder); + } + + LOG.info("create tablets, dbId: {}, tableId: {}, tableName: {}, partitionId: {}, partitionName: {}, " + + "indexId: {}", + dbId, tableId, tableName, partitionId, partitionName, indexId); + sendCreateTabletsRpc(requestBuilder); + if (index.getId() != baseIndexId) { + // add rollup index to partition + partition.createRollupIndex(index); + } + } + + LOG.info("succeed in creating partition[{}-{}], table : [{}-{}]", partitionId, partitionName, + tableId, tableName); + + return partition; + } + + private OlapFile.TabletMetaCloudPB.Builder createTabletMetaBuilder(long tableId, long indexId, + long partitionId, Tablet tablet, TTabletType tabletType, int schemaHash, KeysType keysType, + short shortKeyColumnCount, Set bfColumns, double bfFpp, List indexes, + List schemaColumns, DataSortInfo dataSortInfo, TCompressionType compressionType, + String storagePolicy, boolean isInMemory, boolean isShadow, + String tableName, long ttlSeconds, boolean enableUniqueKeyMergeOnWrite, + boolean storeRowColumn, int schemaVersion) throws DdlException { + OlapFile.TabletMetaCloudPB.Builder builder = OlapFile.TabletMetaCloudPB.newBuilder(); + builder.setTableId(tableId); + builder.setIndexId(indexId); + builder.setPartitionId(partitionId); + builder.setTabletId(tablet.getId()); + builder.setSchemaHash(schemaHash); + builder.setTableName(tableName); + builder.setCreationTime(System.currentTimeMillis() / 1000); + builder.setCumulativeLayerPoint(-1); + builder.setTabletState(isShadow ? OlapFile.TabletStatePB.PB_NOTREADY : OlapFile.TabletStatePB.PB_RUNNING); + builder.setIsInMemory(isInMemory); + builder.setTtlSeconds(ttlSeconds); + builder.setSchemaVersion(schemaVersion); + + UUID uuid = UUID.randomUUID(); + Types.PUniqueId tabletUid = Types.PUniqueId.newBuilder() + .setHi(uuid.getMostSignificantBits()) + .setLo(uuid.getLeastSignificantBits()) + .build(); + builder.setTabletUid(tabletUid); + + builder.setPreferredRowsetType(OlapFile.RowsetTypePB.BETA_ROWSET); + builder.setTabletType(tabletType == TTabletType.TABLET_TYPE_DISK + ? OlapFile.TabletTypePB.TABLET_TYPE_DISK : OlapFile.TabletTypePB.TABLET_TYPE_MEMORY); + + builder.setReplicaId(tablet.getReplicas().get(0).getId()); + builder.setEnableUniqueKeyMergeOnWrite(enableUniqueKeyMergeOnWrite); + + OlapFile.TabletSchemaCloudPB.Builder schemaBuilder = OlapFile.TabletSchemaCloudPB.newBuilder(); + schemaBuilder.setSchemaVersion(schemaVersion); + + if (keysType == KeysType.DUP_KEYS) { + schemaBuilder.setKeysType(OlapFile.KeysType.DUP_KEYS); + } else if (keysType == KeysType.UNIQUE_KEYS) { + schemaBuilder.setKeysType(OlapFile.KeysType.UNIQUE_KEYS); + } else if (keysType == KeysType.AGG_KEYS) { + schemaBuilder.setKeysType(OlapFile.KeysType.AGG_KEYS); + } else { + throw new DdlException("invalid key types"); + } + schemaBuilder.setNumShortKeyColumns(shortKeyColumnCount); + schemaBuilder.setNumRowsPerRowBlock(1024); + schemaBuilder.setCompressKind(OlapCommon.CompressKind.COMPRESS_LZ4); + schemaBuilder.setBfFpp(bfFpp); + + int deleteSign = -1; + int sequenceCol = -1; + for (int i = 0; i < schemaColumns.size(); i++) { + Column column = schemaColumns.get(i); + if (column.isDeleteSignColumn()) { + deleteSign = i; + } + if (column.isSequenceColumn()) { + sequenceCol = i; + } + } + schemaBuilder.setDeleteSignIdx(deleteSign); + schemaBuilder.setSequenceColIdx(sequenceCol); + schemaBuilder.setStoreRowColumn(storeRowColumn); + + if (dataSortInfo.getSortType() == TSortType.LEXICAL) { + schemaBuilder.setSortType(OlapFile.SortType.LEXICAL); + } else if (dataSortInfo.getSortType() == TSortType.ZORDER) { + schemaBuilder.setSortType(OlapFile.SortType.ZORDER); + } else { + LOG.warn("invalid sort types:{}", dataSortInfo.getSortType()); + throw new DdlException("invalid sort types"); + } + + switch (compressionType) { + case NO_COMPRESSION: + schemaBuilder.setCompressionType(SegmentV2.CompressionTypePB.NO_COMPRESSION); + break; + case SNAPPY: + schemaBuilder.setCompressionType(SegmentV2.CompressionTypePB.SNAPPY); + break; + case LZ4: + schemaBuilder.setCompressionType(SegmentV2.CompressionTypePB.LZ4); + break; + case LZ4F: + schemaBuilder.setCompressionType(SegmentV2.CompressionTypePB.LZ4F); + break; + case ZLIB: + schemaBuilder.setCompressionType(SegmentV2.CompressionTypePB.ZLIB); + break; + case ZSTD: + schemaBuilder.setCompressionType(SegmentV2.CompressionTypePB.ZSTD); + break; + default: + schemaBuilder.setCompressionType(SegmentV2.CompressionTypePB.LZ4F); + break; + } + + schemaBuilder.setSortColNum(dataSortInfo.getColNum()); + for (int i = 0; i < schemaColumns.size(); i++) { + Column column = schemaColumns.get(i); + schemaBuilder.addColumn(column.toPb(bfColumns, indexes)); + } + + if (indexes != null) { + for (int i = 0; i < indexes.size(); i++) { + Index index = indexes.get(i); + schemaBuilder.addIndex(index.toPb(schemaColumns)); + } + } + OlapFile.TabletSchemaCloudPB schema = schemaBuilder.build(); + builder.setSchema(schema); + // rowset + OlapFile.RowsetMetaCloudPB.Builder rowsetBuilder = createInitialRowset(tablet, partitionId, + schemaHash, schema); + builder.addRsMetas(rowsetBuilder); + return builder; + } + + private OlapFile.RowsetMetaCloudPB.Builder createInitialRowset(Tablet tablet, long partitionId, + int schemaHash, OlapFile.TabletSchemaCloudPB schema) { + OlapFile.RowsetMetaCloudPB.Builder rowsetBuilder = OlapFile.RowsetMetaCloudPB.newBuilder(); + rowsetBuilder.setRowsetId(0); + rowsetBuilder.setPartitionId(partitionId); + rowsetBuilder.setTabletId(tablet.getId()); + rowsetBuilder.setTabletSchemaHash(schemaHash); + rowsetBuilder.setRowsetType(OlapFile.RowsetTypePB.BETA_ROWSET); + rowsetBuilder.setRowsetState(OlapFile.RowsetStatePB.VISIBLE); + rowsetBuilder.setStartVersion(0); + rowsetBuilder.setEndVersion(1); + rowsetBuilder.setNumRows(0); + rowsetBuilder.setTotalDiskSize(0); + rowsetBuilder.setDataDiskSize(0); + rowsetBuilder.setIndexDiskSize(0); + rowsetBuilder.setSegmentsOverlapPb(OlapFile.SegmentsOverlapPB.NONOVERLAPPING); + rowsetBuilder.setNumSegments(0); + rowsetBuilder.setEmpty(true); + + UUID uuid = UUID.randomUUID(); + String rowsetIdV2Str = String.format("%016X", 2L << 56) + + String.format("%016X", uuid.getMostSignificantBits()) + + String.format("%016X", uuid.getLeastSignificantBits()); + rowsetBuilder.setRowsetIdV2(rowsetIdV2Str); + + rowsetBuilder.setTabletSchema(schema); + return rowsetBuilder; + } + + private void createCloudTablets(MaterializedIndex index, ReplicaState replicaState, + DistributionInfo distributionInfo, long version, ReplicaAllocation replicaAlloc, + TabletMeta tabletMeta, Set tabletIdSet) throws DdlException { + for (int i = 0; i < distributionInfo.getBucketNum(); ++i) { + Tablet tablet = EnvFactory.createTablet(Env.getCurrentEnv().getNextId()); + + // add tablet to inverted index first + index.addTablet(tablet, tabletMeta); + tabletIdSet.add(tablet.getId()); + + long replicaId = Env.getCurrentEnv().getNextId(); + Replica replica = new CloudReplica(replicaId, null, replicaState, version, + tabletMeta.getOldSchemaHash(), tabletMeta.getDbId(), tabletMeta.getTableId(), + tabletMeta.getPartitionId(), tabletMeta.getIndexId(), i); + tablet.addReplica(replica); + } + } + + @Override + protected void beforeCreatePartitions(long dbId, long tableId, List partitionIds, List indexIds) + throws DdlException { + if (partitionIds == null) { + prepareMaterializedIndex(tableId, indexIds); + } else { + preparePartition(dbId, tableId, partitionIds, indexIds); + } + } + + @Override + protected void afterCreatePartitions(long tableId, List partitionIds, List indexIds) + throws DdlException { + if (partitionIds == null) { + commitMaterializedIndex(tableId, indexIds); + } else { + commitPartition(tableId, partitionIds, indexIds); + } + } + + private void preparePartition(long dbId, long tableId, List partitionIds, List indexIds) + throws DdlException { + Cloud.PartitionRequest.Builder partitionRequestBuilder = Cloud.PartitionRequest.newBuilder(); + partitionRequestBuilder.setCloudUniqueId(Config.cloud_unique_id); + partitionRequestBuilder.setTableId(tableId); + partitionRequestBuilder.addAllPartitionIds(partitionIds); + partitionRequestBuilder.addAllIndexIds(indexIds); + partitionRequestBuilder.setExpiration(0); + if (dbId > 0) { + partitionRequestBuilder.setDbId(dbId); + } + final Cloud.PartitionRequest partitionRequest = partitionRequestBuilder.build(); + + Cloud.PartitionResponse response = null; + int tryTimes = 0; + while (tryTimes++ < Config.meta_service_rpc_retry_times) { + try { + response = MetaServiceProxy.getInstance().preparePartition(partitionRequest); + if (response.getStatus().getCode() != Cloud.MetaServiceCode.KV_TXN_CONFLICT) { + break; + } + } catch (RpcException e) { + LOG.warn("tryTimes:{}, preparePartition RpcException", tryTimes, e); + if (tryTimes + 1 >= Config.meta_service_rpc_retry_times) { + throw new DdlException(e.getMessage()); + } + } + sleepSeveralMs(); + } + + if (response.getStatus().getCode() != Cloud.MetaServiceCode.OK) { + LOG.warn("preparePartition response: {} ", response); + throw new DdlException(response.getStatus().getMsg()); + } + } + + private void commitPartition(long tableId, List partitionIds, List indexIds) throws DdlException { + Cloud.PartitionRequest.Builder partitionRequestBuilder = Cloud.PartitionRequest.newBuilder(); + partitionRequestBuilder.setCloudUniqueId(Config.cloud_unique_id); + partitionRequestBuilder.addAllPartitionIds(partitionIds); + partitionRequestBuilder.addAllIndexIds(indexIds); + partitionRequestBuilder.setTableId(tableId); + final Cloud.PartitionRequest partitionRequest = partitionRequestBuilder.build(); + + Cloud.PartitionResponse response = null; + int tryTimes = 0; + while (tryTimes++ < Config.meta_service_rpc_retry_times) { + try { + response = MetaServiceProxy.getInstance().commitPartition(partitionRequest); + if (response.getStatus().getCode() != Cloud.MetaServiceCode.KV_TXN_CONFLICT) { + break; + } + } catch (RpcException e) { + LOG.warn("tryTimes:{}, commitPartition RpcException", tryTimes, e); + if (tryTimes + 1 >= Config.meta_service_rpc_retry_times) { + throw new DdlException(e.getMessage()); + } + } + sleepSeveralMs(); + } + + if (response.getStatus().getCode() != Cloud.MetaServiceCode.OK) { + LOG.warn("commitPartition response: {} ", response); + throw new DdlException(response.getStatus().getMsg()); + } + } + + private void prepareMaterializedIndex(Long tableId, List indexIds) throws DdlException { + Cloud.IndexRequest.Builder indexRequestBuilder = Cloud.IndexRequest.newBuilder(); + indexRequestBuilder.setCloudUniqueId(Config.cloud_unique_id); + indexRequestBuilder.addAllIndexIds(indexIds); + indexRequestBuilder.setTableId(tableId); + indexRequestBuilder.setExpiration(0); + final Cloud.IndexRequest indexRequest = indexRequestBuilder.build(); + + Cloud.IndexResponse response = null; + int tryTimes = 0; + while (tryTimes++ < Config.meta_service_rpc_retry_times) { + try { + response = MetaServiceProxy.getInstance().prepareIndex(indexRequest); + if (response.getStatus().getCode() != Cloud.MetaServiceCode.KV_TXN_CONFLICT) { + break; + } + } catch (RpcException e) { + LOG.warn("tryTimes:{}, prepareIndex RpcException", tryTimes, e); + if (tryTimes + 1 >= Config.meta_service_rpc_retry_times) { + throw new DdlException(e.getMessage()); + } + } + sleepSeveralMs(); + } + + if (response.getStatus().getCode() != Cloud.MetaServiceCode.OK) { + LOG.warn("prepareIndex response: {} ", response); + throw new DdlException(response.getStatus().getMsg()); + } + } + + private void commitMaterializedIndex(Long tableId, List indexIds) throws DdlException { + Cloud.IndexRequest.Builder indexRequestBuilder = Cloud.IndexRequest.newBuilder(); + indexRequestBuilder.setCloudUniqueId(Config.cloud_unique_id); + indexRequestBuilder.addAllIndexIds(indexIds); + indexRequestBuilder.setTableId(tableId); + final Cloud.IndexRequest indexRequest = indexRequestBuilder.build(); + + Cloud.IndexResponse response = null; + int tryTimes = 0; + while (tryTimes++ < Config.meta_service_rpc_retry_times) { + try { + response = MetaServiceProxy.getInstance().commitIndex(indexRequest); + if (response.getStatus().getCode() != Cloud.MetaServiceCode.KV_TXN_CONFLICT) { + break; + } + } catch (RpcException e) { + LOG.warn("tryTimes:{}, commitIndex RpcException", tryTimes, e); + if (tryTimes + 1 >= Config.meta_service_rpc_retry_times) { + throw new DdlException(e.getMessage()); + } + } + sleepSeveralMs(); + } + + if (response.getStatus().getCode() != Cloud.MetaServiceCode.OK) { + LOG.warn("commitIndex response: {} ", response); + throw new DdlException(response.getStatus().getMsg()); + } + } + + private void sendCreateTabletsRpc(Cloud.CreateTabletsRequest.Builder requestBuilder) throws DdlException { + requestBuilder.setCloudUniqueId(Config.cloud_unique_id); + Cloud.CreateTabletsRequest createTabletsReq = requestBuilder.build(); + + LOG.debug("send create tablets rpc, createTabletsReq: {}", createTabletsReq); + Cloud.CreateTabletsResponse response; + try { + response = MetaServiceProxy.getInstance().createTablets(createTabletsReq); + } catch (RpcException e) { + LOG.warn("failed to send create tablets rpc {}", e.getMessage()); + throw new RuntimeException(e); + } + LOG.info("create tablets response: {}", response); + + if (response.getStatus().getCode() != Cloud.MetaServiceCode.OK) { + throw new DdlException(response.getStatus().getMsg()); + } + } + + // END CREATE TABLE + + @Override + protected void checkAvailableCapacity(Database db) throws DdlException { + // check cluster capacity + Env.getCurrentSystemInfo().checkAvailableCapacity(); + // check db quota + db.checkQuota(); + } + + private void sleepSeveralMs() { + // sleep random millis [20, 200] ms, avoid txn conflict + int randomMillis = 20 + (int) (Math.random() * (200 - 20)); + LOG.debug("randomMillis:{}", randomMillis); + try { + Thread.sleep(randomMillis); + } catch (InterruptedException e) { + LOG.info("ignore InterruptedException: ", e); + } + } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java index 8a4051c9af3420..f1e457336f2594 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java @@ -101,6 +101,8 @@ public class PropertyAnalyzer { public static final String PROPERTIES_INMEMORY = "in_memory"; + public static final String PROPERTIES_FILE_CACHE_TTL_SECONDS = "file_cache_ttl_seconds"; + // _auto_bucket can only set in create table stmt rewrite bucket and can not be changed public static final String PROPERTIES_AUTO_BUCKET = "_auto_bucket"; public static final String PROPERTIES_ESTIMATE_PARTITION_SIZE = "estimate_partition_size"; @@ -245,6 +247,7 @@ public static DataProperty analyzeDataProperty(Map properties, f properties.remove(PROPERTIES_STORAGE_COOLDOWN_TIME); properties.remove(PROPERTIES_STORAGE_POLICY); properties.remove(PROPERTIES_DATA_BASE_TIME); + properties.remove(PROPERTIES_FILE_CACHE_TTL_SECONDS); Preconditions.checkNotNull(storageMedium); @@ -449,6 +452,23 @@ public static long analyzeVersionInfo(Map properties) throws Ana return version; } + public static long analyzeTTL(Map properties) throws AnalysisException { + long ttlSeconds = 0; + if (properties != null && properties.containsKey(PROPERTIES_FILE_CACHE_TTL_SECONDS)) { + String ttlSecondsStr = properties.get(PROPERTIES_FILE_CACHE_TTL_SECONDS); + try { + ttlSeconds = Long.parseLong(ttlSecondsStr); + if (ttlSeconds < 0) { + throw new NumberFormatException(); + } + } catch (NumberFormatException e) { + throw new AnalysisException("The value " + ttlSecondsStr + " formats error or is out of range " + + "(0 < integer < Long.MAX_VALUE)"); + } + } + return ttlSeconds; + } + public static int analyzeSchemaVersion(Map properties) throws AnalysisException { int schemaVersion = 0; if (properties != null && properties.containsKey(PROPERTIES_SCHEMA_VERSION)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java index b5e68de5b2e787..4671daba0cb10f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java @@ -65,6 +65,7 @@ import org.apache.doris.catalog.DistributionInfo; import org.apache.doris.catalog.DistributionInfo.DistributionInfoType; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.EnvFactory; import org.apache.doris.catalog.EsTable; import org.apache.doris.catalog.HashDistributionInfo; import org.apache.doris.catalog.HiveTable; @@ -1068,10 +1069,7 @@ public void createTable(CreateTableStmt stmt) throws UserException { // only internal table should check quota and cluster capacity if (!stmt.isExternal()) { - // check cluster capacity - Env.getCurrentSystemInfo().checkAvailableCapacity(); - // check db quota - db.checkQuota(); + checkAvailableCapacity(db); } // check if table exists in db @@ -1532,6 +1530,9 @@ public void addPartition(Database db, String tableName, AddPartitionClause addPa } try { long partitionId = idGeneratorBuffer.getNextId(); + List partitionIds = Lists.newArrayList(partitionId); + List indexIds = indexIdToMeta.keySet().stream().collect(Collectors.toList()); + beforeCreatePartitions(db.getId(), olapTable.getId(), partitionIds, indexIds); Partition partition = createPartitionWithIndices(db.getId(), olapTable.getId(), olapTable.getName(), olapTable.getBaseIndexId(), partitionId, partitionName, indexIdToMeta, distributionInfo, dataProperty.getStorageMedium(), singlePartitionDesc.getReplicaAlloc(), @@ -1546,7 +1547,9 @@ public void addPartition(Database db, String tableName, AddPartitionClause addPa olapTable.getTimeSeriesCompactionTimeThresholdSeconds(), olapTable.getTimeSeriesCompactionEmptyRowsetsThreshold(), olapTable.storeRowColumn(), - binlogConfig, dataProperty.isStorageMediumSpecified(), null); + binlogConfig, dataProperty.isStorageMediumSpecified(), null, + olapTable.getTTLSeconds()); + afterCreatePartitions(olapTable.getId(), partitionIds, indexIds); // TODO cluster key ids // check again @@ -1791,7 +1794,7 @@ public void replayRecoverPartition(RecoverInfo info) throws MetaNotFoundExceptio } } - private Partition createPartitionWithIndices(long dbId, long tableId, String tableName, + protected Partition createPartitionWithIndices(long dbId, long tableId, String tableName, long baseIndexId, long partitionId, String partitionName, Map indexIdToMeta, DistributionInfo distributionInfo, TStorageMedium storageMedium, ReplicaAllocation replicaAlloc, Long versionInfo, Set bfColumns, double bfFpp, Set tabletIdSet, List indexes, @@ -1803,7 +1806,8 @@ private Partition createPartitionWithIndices(long dbId, long tableId, String tab Long timeSeriesCompactionFileCountThreshold, Long timeSeriesCompactionTimeThresholdSeconds, Long timeSeriesCompactionEmptyRowsetsThreshold, boolean storeRowColumn, BinlogConfig binlogConfig, - boolean isStorageMediumSpecified, List clusterKeyIndexes) throws DdlException { + boolean isStorageMediumSpecified, List clusterKeyIndexes, + long ttlSeconds) throws DdlException { // create base index first. Preconditions.checkArgument(baseIndexId != -1); MaterializedIndex baseIndex = new MaterializedIndex(baseIndexId, IndexState.NORMAL); @@ -1947,6 +1951,21 @@ private Partition createPartitionWithIndices(long dbId, long tableId, String tab return partition; } + protected void beforeCreatePartitions(long dbId, long tableId, List partitionIds, List indexIds) + throws DdlException { + } + + protected void afterCreatePartitions(long tableId, List partitionIds, List indexIds) + throws DdlException { + } + + protected void checkAvailableCapacity(Database db) throws DdlException { + // check cluster capacity + Env.getCurrentSystemInfo().checkAvailableCapacity(); + // check db quota + db.checkQuota(); + } + // Create olap table and related base index synchronously. private void createOlapTable(Database db, CreateTableStmt stmt) throws UserException { String tableName = stmt.getTableName(); @@ -2265,6 +2284,9 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep boolean isMutable = PropertyAnalyzer.analyzeBooleanProp(properties, PropertyAnalyzer.PROPERTIES_MUTABLE, true); + Long ttlSeconds = PropertyAnalyzer.analyzeTTL(properties); + olapTable.setTTLSeconds(ttlSeconds); + // set storage policy String storagePolicy = PropertyAnalyzer.analyzeStoragePolicy(properties); Env.getCurrentEnv().getPolicyMgr().checkStoragePolicyExist(storagePolicy); @@ -2481,6 +2503,7 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep "Database " + db.getFullName() + " create unpartitioned table " + tableName + " increasing " + totalReplicaNum + " of replica exceeds quota[" + db.getReplicaQuota() + "]"); } + beforeCreatePartitions(db.getId(), olapTable.getId(), null, olapTable.getIndexIdList()); Partition partition = createPartitionWithIndices(db.getId(), olapTable.getId(), olapTable.getName(), olapTable.getBaseIndexId(), partitionId, partitionName, olapTable.getIndexIdToMeta(), partitionDistributionInfo, @@ -2496,7 +2519,9 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep olapTable.getTimeSeriesCompactionEmptyRowsetsThreshold(), storeRowColumn, binlogConfigForTask, partitionInfo.getDataProperty(partitionId).isStorageMediumSpecified(), - keysDesc.getClusterKeysColumnIds()); + keysDesc.getClusterKeysColumnIds(), + olapTable.getTTLSeconds()); + afterCreatePartitions(olapTable.getId(), null, olapTable.getIndexIdList()); olapTable.addPartition(partition); } else if (partitionInfo.getType() == PartitionType.RANGE || partitionInfo.getType() == PartitionType.LIST) { @@ -2539,6 +2564,8 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep + totalReplicaNum + " of replica exceeds quota[" + db.getReplicaQuota() + "]"); } + beforeCreatePartitions(db.getId(), olapTable.getId(), null, olapTable.getIndexIdList()); + // this is a 2-level partitioned tables for (Map.Entry entry : partitionNameToId.entrySet()) { DataProperty dataProperty = partitionInfo.getDataProperty(entry.getValue()); @@ -2558,7 +2585,6 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep partionStoragePolicy = storagePolicy; } Env.getCurrentEnv().getPolicyMgr().checkStoragePolicyExist(partionStoragePolicy); - Partition partition = createPartitionWithIndices(db.getId(), olapTable.getId(), olapTable.getName(), olapTable.getBaseIndexId(), entry.getValue(), entry.getKey(), olapTable.getIndexIdToMeta(), partitionDistributionInfo, @@ -2573,11 +2599,14 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep olapTable.getTimeSeriesCompactionTimeThresholdSeconds(), olapTable.getTimeSeriesCompactionEmptyRowsetsThreshold(), storeRowColumn, binlogConfigForTask, - dataProperty.isStorageMediumSpecified(), keysDesc.getClusterKeysColumnIds()); + dataProperty.isStorageMediumSpecified(), + keysDesc.getClusterKeysColumnIds(), + olapTable.getTTLSeconds()); olapTable.addPartition(partition); olapTable.getPartitionInfo().getDataProperty(partition.getId()) .setStoragePolicy(partionStoragePolicy); } + afterCreatePartitions(olapTable.getId(), null, olapTable.getIndexIdList()); } else { throw new DdlException("Unsupported partition method: " + partitionInfo.getType().name()); } @@ -2810,7 +2839,7 @@ public void createTablets(MaterializedIndex index, ReplicaState replicaState, for (int i = 0; i < distributionInfo.getBucketNum(); ++i) { // create a new tablet with random chosen backends - Tablet tablet = new Tablet(idGeneratorBuffer.getNextId()); + Tablet tablet = EnvFactory.createTablet(idGeneratorBuffer.getNextId()); // add tablet to inverted index first index.addTablet(tablet, tabletMeta); @@ -2997,6 +3026,19 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti long bufferSize = IdGeneratorUtil.getBufferSizeForTruncateTable(copiedTbl, origPartitions.values()); IdGeneratorBuffer idGeneratorBuffer = origPartitions.isEmpty() ? null : Env.getCurrentEnv().getIdGeneratorBuffer(bufferSize); + + Map oldToNewPartitionId = new HashMap(); + List newPartitionIds = new ArrayList(); + for (Map.Entry entry : origPartitions.entrySet()) { + long oldPartitionId = entry.getValue(); + long newPartitionId = idGeneratorBuffer.getNextId(); + oldToNewPartitionId.put(oldPartitionId, newPartitionId); + newPartitionIds.add(newPartitionId); + } + + List indexIds = copiedTbl.getIndexIdToMeta().keySet().stream().collect(Collectors.toList()); + beforeCreatePartitions(db.getId(), copiedTbl.getId(), newPartitionIds, indexIds); + for (Map.Entry entry : origPartitions.entrySet()) { // the new partition must use new id // If we still use the old partition id, the behavior of current load jobs on this partition @@ -3004,7 +3046,7 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti // By using a new id, load job will be aborted(just like partition is dropped), // which is the right behavior. long oldPartitionId = entry.getValue(); - long newPartitionId = idGeneratorBuffer.getNextId(); + long newPartitionId = oldToNewPartitionId.get(oldPartitionId); Partition newPartition = createPartitionWithIndices(db.getId(), copiedTbl.getId(), copiedTbl.getName(), copiedTbl.getBaseIndexId(), newPartitionId, entry.getKey(), copiedTbl.getIndexIdToMeta(), partitionsDistributionInfo.get(oldPartitionId), @@ -3023,9 +3065,12 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti olapTable.getTimeSeriesCompactionEmptyRowsetsThreshold(), olapTable.storeRowColumn(), binlogConfig, copiedTbl.getPartitionInfo().getDataProperty(oldPartitionId).isStorageMediumSpecified(), - clusterKeyIdxes); + clusterKeyIdxes, olapTable.getTTLSeconds()); newPartitions.add(newPartition); } + + afterCreatePartitions(copiedTbl.getId(), newPartitionIds, indexIds); + } catch (DdlException e) { // create partition failed, remove all newly created tablets for (Long tabletId : tabletIdSet) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index c02bce86b89fb2..280cf5ebf77c93 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -3153,7 +3153,14 @@ public TCreatePartitionResult createPartition(TCreatePartitionRequest request) t // we should ensure the replica backend is alive // otherwise, there will be a 'unknown node id, id=xxx' error for stream load // BE id -> path hash - Multimap bePathsMap = tablet.getNormalReplicaBackendPathMap(); + Multimap bePathsMap; + try { + bePathsMap = tablet.getNormalReplicaBackendPathMap(); + } catch (UserException ex) { + errorStatus.setErrorMsgs(Lists.newArrayList(ex.getMessage())); + result.setStatus(errorStatus); + return result; + } if (bePathsMap.keySet().size() < quorum) { LOG.warn("auto go quorum exception"); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java index 78297aae83a88f..62511e25e47d9b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java @@ -83,6 +83,9 @@ public class SystemInfoService { public static final String NO_SCAN_NODE_BACKEND_AVAILABLE_MSG = "There is no scanNode Backend available."; + public static final String NOT_USING_VALID_CLUSTER_MSG = "Not using valid cloud clusters, " + + "please use a cluster before issuing any queries"; + private volatile ImmutableMap idToBackendRef = ImmutableMap.of(); private volatile ImmutableMap idToReportVersionRef = ImmutableMap.of(); // TODO(gavin): use {clusterId -> List} instead to reduce risk of inconsistency diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/EnvFactoryTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/EnvFactoryTest.java index ddd3b9441a59ed..1a939457f5ea74 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/EnvFactoryTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/EnvFactoryTest.java @@ -20,6 +20,7 @@ import org.apache.doris.cloud.catalog.CloudEnv; import org.apache.doris.cloud.catalog.CloudPartition; import org.apache.doris.cloud.catalog.CloudReplica; +import org.apache.doris.cloud.catalog.CloudTablet; import org.apache.doris.cloud.datasource.CloudInternalCatalog; import org.apache.doris.common.Config; import org.apache.doris.datasource.InternalCatalog; @@ -42,6 +43,8 @@ public void testCreate() throws Exception { Assert.assertFalse(EnvFactory.createInternalCatalog() instanceof CloudInternalCatalog); Assert.assertTrue(EnvFactory.createPartition() instanceof Partition); Assert.assertFalse(EnvFactory.createPartition() instanceof CloudPartition); + Assert.assertTrue(EnvFactory.createTablet() instanceof Tablet); + Assert.assertFalse(EnvFactory.createTablet() instanceof CloudTablet); Assert.assertTrue(EnvFactory.createReplica() instanceof Replica); Assert.assertFalse(EnvFactory.createReplica() instanceof CloudReplica); @@ -49,6 +52,7 @@ public void testCreate() throws Exception { Assert.assertTrue(EnvFactory.createEnv(false) instanceof CloudEnv); Assert.assertTrue(EnvFactory.createInternalCatalog() instanceof CloudInternalCatalog); Assert.assertTrue(EnvFactory.createPartition() instanceof CloudPartition); + Assert.assertTrue(EnvFactory.createTablet() instanceof CloudTablet); Assert.assertTrue(EnvFactory.createReplica() instanceof CloudReplica); } From 4c7d4d1c3d121f17128efa05d324824a2a287982 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=A2=E5=81=A5?= Date: Tue, 16 Jan 2024 18:21:40 +0800 Subject: [PATCH 038/200] [feat](Nereids) persist constraint in table (#29767) --- .../apache/doris/common/FeMetaVersion.java | 4 +- .../java/org/apache/doris/catalog/Table.java | 20 +- .../apache/doris/catalog/TableAttributes.java | 53 +++ .../org/apache/doris/catalog/TableIf.java | 34 +- .../doris/catalog/constraint/Constraint.java | 31 +- .../constraint/ForeignKeyConstraint.java | 12 +- .../constraint/PrimaryKeyConstraint.java | 9 +- .../catalog/constraint/TableIdentifier.java | 26 +- .../catalog/constraint/UniqueConstraint.java | 8 +- .../doris/catalog/external/ExternalTable.java | 10 + .../doris/datasource/ExternalSchemaCache.java | 5 + .../apache/doris/journal/JournalEntity.java | 7 + .../exceptions/MetaNotFoundException.java | 74 +++++ .../plans/commands/AddConstraintCommand.java | 15 +- .../plans/commands/DropConstraintCommand.java | 9 +- .../commands/ShowConstraintsCommand.java | 2 +- .../doris/persist/AlterConstraintLog.java | 61 ++++ .../org/apache/doris/persist/EditLog.java | 18 ++ .../apache/doris/persist/OperationType.java | 3 + .../apache/doris/persist/gson/GsonUtils.java | 11 + .../constraint/ConstraintPersistTest.java | 305 ++++++++++++++++++ .../doris/utframe/TestWithFeService.java | 11 + 22 files changed, 683 insertions(+), 45 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/catalog/TableAttributes.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/MetaNotFoundException.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/persist/AlterConstraintLog.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/catalog/constraint/ConstraintPersistTest.java diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java b/fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java index 650a831640bfb1..387ce91e2c68e2 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java @@ -72,9 +72,11 @@ public final class FeMetaVersion { public static final int VERSION_125 = 125; // For write/read function nullable mode info public static final int VERSION_126 = 126; + // For constraints + public static final int VERSION_127 = 127; // note: when increment meta version, should assign the latest version to VERSION_CURRENT - public static final int VERSION_CURRENT = VERSION_126; + public static final int VERSION_CURRENT = VERSION_127; // all logs meta version should >= the minimum version, so that we could remove many if clause, for example // if (FE_METAVERSION < VERSION_94) ... diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java index 3395da6b58eeb2..783016e7d80d92 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java @@ -22,12 +22,14 @@ import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.FeMetaVersion; import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.common.util.QueryableReentrantReadWriteLock; import org.apache.doris.common.util.SqlUtils; import org.apache.doris.common.util.Util; +import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.BaseAnalysisTask; import org.apache.doris.statistics.ColumnStatistic; @@ -50,7 +52,6 @@ import java.io.IOException; import java.time.Instant; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -116,8 +117,8 @@ public abstract class Table extends MetaObject implements Writable, TableIf { @SerializedName(value = "comment") protected String comment = ""; - @SerializedName(value = "constraints") - private HashMap constraintsMap = new HashMap<>(); + @SerializedName(value = "ta") + private TableAttributes tableAttributes = new TableAttributes(); // check read lock leaky private Map readLockThreads = null; @@ -334,12 +335,12 @@ public String getQualifiedName() { } public Constraint getConstraint(String name) { - return constraintsMap.get(name); + return getConstraintsMap().get(name); } @Override public Map getConstraintsMapUnsafe() { - return constraintsMap; + return tableAttributes.getConstraintsMap(); } public TableType getType() { @@ -455,9 +456,9 @@ public void write(DataOutput out) throws IOException { for (Column column : fullSchema) { column.write(out); } - Text.writeString(out, comment); - + // write table attributes + Text.writeString(out, GsonUtils.GSON.toJson(tableAttributes)); // write create time out.writeLong(createTime); } @@ -488,7 +489,12 @@ public void readFields(DataInput in) throws IOException { hasCompoundKey = true; } comment = Text.readString(in); + // table attribute only support after version 127 + if (FeMetaVersion.VERSION_127 <= Env.getCurrentEnvJournalVersion()) { + String json = Text.readString(in); + this.tableAttributes = GsonUtils.GSON.fromJson(json, TableAttributes.class); + } // read create time this.createTime = in.readLong(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableAttributes.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableAttributes.java new file mode 100644 index 00000000000000..2ac86fada5146a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableAttributes.java @@ -0,0 +1,53 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.catalog; + +import org.apache.doris.catalog.constraint.Constraint; +import org.apache.doris.common.io.Text; +import org.apache.doris.common.io.Writable; +import org.apache.doris.persist.gson.GsonUtils; + +import com.google.gson.annotations.SerializedName; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +/** + * TableAttributes contains additional information about all table + */ +public class TableAttributes implements Writable { + @SerializedName(value = "constraints") + private final Map constraintsMap = new HashMap<>(); + + + @Override + public void write(DataOutput out) throws IOException { + Text.writeString(out, GsonUtils.GSON.toJson(this)); + } + + public TableAttributes read(DataInput in) throws IOException { + return GsonUtils.GSON.fromJson(Text.readString(in), TableAttributes.class); + } + + public Map getConstraintsMap() { + return constraintsMap; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java index bb6abcdd5092c0..b59aedfdbf3c1f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java @@ -183,7 +183,7 @@ default Set getForeignKeyConstraints() { } } - default Map getConstraintMap() { + default Map getConstraintsMap() { readLock(); try { return ImmutableMap.copyOf(getConstraintsMapUnsafe()); @@ -236,25 +236,27 @@ default void checkConstraintNotExistence(String name, Constraint primaryKeyConst } } - default void addUniqueConstraint(String name, ImmutableList columns) { + default Constraint addUniqueConstraint(String name, ImmutableList columns) { writeLock(); try { Map constraintMap = getConstraintsMapUnsafe(); UniqueConstraint uniqueConstraint = new UniqueConstraint(name, ImmutableSet.copyOf(columns)); checkConstraintNotExistence(name, uniqueConstraint, constraintMap); constraintMap.put(name, uniqueConstraint); + return uniqueConstraint; } finally { writeUnlock(); } } - default void addPrimaryKeyConstraint(String name, ImmutableList columns) { + default Constraint addPrimaryKeyConstraint(String name, ImmutableList columns) { writeLock(); try { Map constraintMap = getConstraintsMapUnsafe(); PrimaryKeyConstraint primaryKeyConstraint = new PrimaryKeyConstraint(name, ImmutableSet.copyOf(columns)); checkConstraintNotExistence(name, primaryKeyConstraint, constraintMap); constraintMap.put(name, primaryKeyConstraint); + return primaryKeyConstraint; } finally { writeUnlock(); } @@ -277,7 +279,7 @@ default void updatePrimaryKeyForForeignKey(PrimaryKeyConstraint requirePrimaryKe } } - default void addForeignConstraint(String name, ImmutableList columns, + default Constraint addForeignConstraint(String name, ImmutableList columns, TableIf referencedTable, ImmutableList referencedColumns) { writeLock(); try { @@ -289,12 +291,32 @@ default void addForeignConstraint(String name, ImmutableList columns, foreignKeyConstraint.getReferencedColumnNames()); updatePrimaryKeyForForeignKey(requirePrimaryKey, referencedTable); constraintMap.put(name, foreignKeyConstraint); + return foreignKeyConstraint; } finally { writeUnlock(); } } - default void dropConstraint(String name) { + default void replayAddConstraint(Constraint constraint) { + if (constraint instanceof UniqueConstraint) { + UniqueConstraint uniqueConstraint = (UniqueConstraint) constraint; + this.addUniqueConstraint(constraint.getName(), + ImmutableList.copyOf(uniqueConstraint.getUniqueColumnNames())); + } else if (constraint instanceof PrimaryKeyConstraint) { + PrimaryKeyConstraint primaryKeyConstraint = (PrimaryKeyConstraint) constraint; + this.addPrimaryKeyConstraint(primaryKeyConstraint.getName(), + ImmutableList.copyOf(primaryKeyConstraint.getPrimaryKeyNames())); + } else if (constraint instanceof ForeignKeyConstraint) { + ForeignKeyConstraint foreignKey = (ForeignKeyConstraint) constraint; + this.addForeignConstraint(foreignKey.getName(), + ImmutableList.copyOf(foreignKey.getForeignKeyNames()), + foreignKey.getReferencedTable(), + ImmutableList.copyOf(foreignKey.getReferencedColumnNames())); + } + } + + default Constraint dropConstraint(String name) { + Constraint dropConstraint; writeLock(); try { Map constraintMap = getConstraintsMapUnsafe(); @@ -308,9 +330,11 @@ default void dropConstraint(String name) { ((PrimaryKeyConstraint) constraint).getForeignTables() .forEach(t -> t.dropFKReferringPK(this, (PrimaryKeyConstraint) constraint)); } + dropConstraint = constraint; } finally { writeUnlock(); } + return dropConstraint; } default void dropFKReferringPK(TableIf table, PrimaryKeyConstraint constraint) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/Constraint.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/Constraint.java index 9faa80f8c1e536..01686aadaacb80 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/Constraint.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/Constraint.java @@ -17,15 +17,25 @@ package org.apache.doris.catalog.constraint; -public abstract class Constraint { +import org.apache.doris.common.io.Text; +import org.apache.doris.common.io.Writable; +import org.apache.doris.persist.gson.GsonUtils; + +import com.google.gson.annotations.SerializedName; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +public abstract class Constraint implements Writable { public enum ConstraintType { FOREIGN_KEY("FOREIGN KEY"), PRIMARY_KEY("PRIMARY KEY"), UNIQUE("UNIQUE"); - + @SerializedName(value = "tn") private final String name; - private ConstraintType(String stringValue) { + ConstraintType(String stringValue) { this.name = stringValue; } @@ -34,7 +44,9 @@ public String getName() { } } + @SerializedName(value = "n") private final String name; + @SerializedName(value = "ty") private final ConstraintType type; @@ -50,4 +62,17 @@ public String getName() { public ConstraintType getType() { return type; } + + @Override + public void write(DataOutput out) throws IOException { + Text.writeString(out, GsonUtils.GSON.toJson(this)); + } + + /** + * Read Constraint. + **/ + public static Constraint read(DataInput in) throws IOException { + String json = Text.readString(in); + return GsonUtils.GSON.fromJson(json, Constraint.class); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/ForeignKeyConstraint.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/ForeignKeyConstraint.java index 01dc00052b9fa4..66d8e0a3706b97 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/ForeignKeyConstraint.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/ForeignKeyConstraint.java @@ -24,13 +24,17 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap.Builder; import com.google.common.collect.ImmutableSet; +import com.google.gson.annotations.SerializedName; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; public class ForeignKeyConstraint extends Constraint { - private final ImmutableMap foreignToReference; + @SerializedName(value = "ftr") + private final Map foreignToReference; + @SerializedName(value = "rt") private final TableIdentifier referencedTable; public ForeignKeyConstraint(String name, List columns, @@ -50,11 +54,11 @@ public ForeignKeyConstraint(String name, List columns, this.foreignToReference = builder.build(); } - public ImmutableSet getForeignKeyNames() { + public Set getForeignKeyNames() { return foreignToReference.keySet(); } - public ImmutableSet getReferencedColumnNames() { + public Set getReferencedColumnNames() { return ImmutableSet.copyOf(foreignToReference.values()); } @@ -62,7 +66,7 @@ public String getReferencedColumnName(String column) { return foreignToReference.get(column); } - public ImmutableMap getForeignToReference() { + public Map getForeignToReference() { return foreignToReference; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/PrimaryKeyConstraint.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/PrimaryKeyConstraint.java index c868d3a84f8395..47c822ad597139 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/PrimaryKeyConstraint.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/PrimaryKeyConstraint.java @@ -23,15 +23,18 @@ import com.google.common.base.Objects; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import com.google.gson.annotations.SerializedName; import java.util.HashSet; import java.util.List; import java.util.Set; public class PrimaryKeyConstraint extends Constraint { - private final ImmutableSet columns; + @SerializedName(value = "cols") + private final Set columns; // record the foreign table which references the primary key + @SerializedName(value = "ft") private final Set foreignTables = new HashSet<>(); public PrimaryKeyConstraint(String name, Set columns) { @@ -39,11 +42,11 @@ public PrimaryKeyConstraint(String name, Set columns) { this.columns = ImmutableSet.copyOf(columns); } - public ImmutableSet getPrimaryKeyNames() { + public Set getPrimaryKeyNames() { return columns; } - public ImmutableSet getPrimaryKeys(TableIf table) { + public Set getPrimaryKeys(TableIf table) { return columns.stream().map(table::getColumn).collect(ImmutableSet.toImmutableSet()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/TableIdentifier.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/TableIdentifier.java index c4db65c14e8874..2688fd5784db7a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/TableIdentifier.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/TableIdentifier.java @@ -20,30 +20,42 @@ import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.TableIf; +import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.nereids.exceptions.MetaNotFoundException; import com.google.common.base.Preconditions; +import com.google.gson.annotations.SerializedName; import java.util.Objects; -class TableIdentifier { +public class TableIdentifier { + @SerializedName(value = "dbId") private final long databaseId; + @SerializedName(value = "tId") private final long tableId; + @SerializedName(value = "cId") + private final long catalogId; - TableIdentifier(TableIf tableIf) { + public TableIdentifier(TableIf tableIf) { Preconditions.checkArgument(tableIf != null, "Table can not be null in constraint"); - databaseId = tableIf.getDatabase().getId(); tableId = tableIf.getId(); + databaseId = tableIf.getDatabase().getId(); + catalogId = tableIf.getDatabase().getCatalog().getId(); } - TableIf toTableIf() { - DatabaseIf databaseIf = Env.getCurrentEnv().getCurrentCatalog().getDbNullable(databaseId); + public TableIf toTableIf() { + CatalogIf catalogIf = Env.getCurrentEnv().getCatalogMgr().getCatalog(catalogId); + if (catalogIf == null) { + throw new MetaNotFoundException(String.format("Can not find catalog %s in constraint", catalogId)); + } + DatabaseIf databaseIf = catalogIf.getDbNullable(databaseId); if (databaseIf == null) { - throw new RuntimeException(String.format("Can not find database %s in constraint", databaseId)); + throw new MetaNotFoundException(String.format("Can not find database %s in constraint", databaseId)); } TableIf tableIf = databaseIf.getTableNullable(tableId); if (tableIf == null) { - throw new RuntimeException(String.format("Can not find table %s in constraint", databaseId)); + throw new MetaNotFoundException(String.format("Can not find table %s in constraint", databaseId)); } return tableIf; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/UniqueConstraint.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/UniqueConstraint.java index 3e8042dc8e02e6..0b54f13bdee721 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/UniqueConstraint.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/UniqueConstraint.java @@ -22,22 +22,24 @@ import com.google.common.base.Objects; import com.google.common.collect.ImmutableSet; +import com.google.gson.annotations.SerializedName; import java.util.Set; public class UniqueConstraint extends Constraint { - private final ImmutableSet columns; + @SerializedName(value = "cols") + private final Set columns; public UniqueConstraint(String name, Set columns) { super(ConstraintType.UNIQUE, name); this.columns = ImmutableSet.copyOf(columns); } - public ImmutableSet getUniqueColumnNames() { + public Set getUniqueColumnNames() { return columns; } - public ImmutableSet getUniqueKeys(TableIf table) { + public Set getUniqueKeys(TableIf table) { return columns.stream().map(table::getColumn).collect(ImmutableSet.toImmutableSet()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalTable.java index 49bf28589055db..1af013fcbfe721 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalTable.java @@ -21,7 +21,9 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.TableAttributes; import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.constraint.Constraint; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; @@ -77,6 +79,9 @@ public class ExternalTable implements TableIf, Writable, GsonPostProcessable { protected long timestamp; @SerializedName(value = "dbName") protected String dbName; + @SerializedName(value = "ta") + private final TableAttributes tableAttributes = new TableAttributes(); + // this field will be refreshed after reloading schema protected volatile long schemaUpdateTime; @@ -272,6 +277,11 @@ public Column getColumn(String name) { return null; } + @Override + public Map getConstraintsMapUnsafe() { + return tableAttributes.getConstraintsMap(); + } + @Override public String getEngine() { return getType().toEngineName(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalSchemaCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalSchemaCache.java index 3eab6028d4607e..962f9d977c9ae0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalSchemaCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalSchemaCache.java @@ -94,6 +94,11 @@ public List getSchema(String dbName, String tblName) { } } + public void addSchemaForTest(String dbName, String tblName, ImmutableList schema) { + SchemaCacheKey key = new SchemaCacheKey(dbName, tblName); + schemaCache.put(key, schema); + } + public void invalidateTableCache(String dbName, String tblName) { SchemaCacheKey key = new SchemaCacheKey(dbName, tblName); schemaCache.invalidate(key); diff --git a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java index 6f6bc945096ea5..1bedda0a7e9d7e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java +++ b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java @@ -58,6 +58,7 @@ import org.apache.doris.load.routineload.RoutineLoadJob; import org.apache.doris.load.sync.SyncJob; import org.apache.doris.mysql.privilege.UserPropertyInfo; +import org.apache.doris.persist.AlterConstraintLog; import org.apache.doris.persist.AlterDatabasePropertyInfo; import org.apache.doris.persist.AlterLightSchemaChangeInfo; import org.apache.doris.persist.AlterMTMV; @@ -793,6 +794,12 @@ public void readFields(DataInput in) throws IOException { isRead = true; break; } + case OperationType.OP_DROP_CONSTRAINT: + case OperationType.OP_ADD_CONSTRAINT: { + data = AlterConstraintLog.read(in); + isRead = true; + break; + } case OperationType.OP_ALTER_USER: { data = AlterUserOperationLog.read(in); isRead = true; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/MetaNotFoundException.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/MetaNotFoundException.java new file mode 100644 index 00000000000000..f7d19c3f844ddd --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/MetaNotFoundException.java @@ -0,0 +1,74 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.nereids.exceptions; + +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; + +import java.util.Optional; + +/** Nereids's AnalysisException. */ +public class MetaNotFoundException extends RuntimeException { + private final String message; + private final Optional line; + private final Optional startPosition; + private final Optional plan; + + public MetaNotFoundException(String message, Throwable cause, Optional line, + Optional startPosition, Optional plan) { + super(message, cause); + this.message = message; + this.line = line; + this.startPosition = startPosition; + this.plan = plan; + } + + public MetaNotFoundException(String message, Optional line, + Optional startPosition, Optional plan) { + super(message); + this.message = message; + this.line = line; + this.startPosition = startPosition; + this.plan = plan; + } + + public MetaNotFoundException(String message, Throwable cause) { + this(message, cause, Optional.empty(), Optional.empty(), Optional.empty()); + } + + public MetaNotFoundException(String message) { + this(message, Optional.empty(), Optional.empty(), Optional.empty()); + } + + @Override + public String getMessage() { + String planAnnotation = plan.map(p -> ";\n" + p.treeString()).orElse(""); + return getSimpleMessage() + planAnnotation; + } + + private String getSimpleMessage() { + if (line.isPresent() || startPosition.isPresent()) { + String lineAnnotation = line.map(l -> "line " + l).orElse(""); + String positionAnnotation = startPosition.map(s -> " pos " + s).orElse(""); + return message + ";" + lineAnnotation + positionAnnotation; + } else { + return message; + } + } + + // TODO: support ErrorCode +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AddConstraintCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AddConstraintCommand.java index 4086758e74fe65..8c90bc0f9144b1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AddConstraintCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AddConstraintCommand.java @@ -17,7 +17,7 @@ package org.apache.doris.nereids.trees.plans.commands; -import org.apache.doris.catalog.Table; +import org.apache.doris.catalog.Env; import org.apache.doris.catalog.TableIf; import org.apache.doris.common.Pair; import org.apache.doris.nereids.NereidsPlanner; @@ -30,6 +30,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalCatalogRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.persist.AlterConstraintLog; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.StmtExecutor; @@ -46,6 +47,7 @@ public class AddConstraintCommand extends Command implements ForwardWithSync { public static final Logger LOG = LogManager.getLogger(AddConstraintCommand.class); + private final String name; private final Constraint constraint; @@ -61,16 +63,19 @@ public AddConstraintCommand(String name, Constraint constraint) { @Override public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { Pair, TableIf> columnsAndTable = extractColumnsAndTable(ctx, constraint.toProject()); + org.apache.doris.catalog.constraint.Constraint catalogConstraint = null; if (constraint.isForeignKey()) { Pair, TableIf> referencedColumnsAndTable = extractColumnsAndTable(ctx, constraint.toReferenceProject()); - columnsAndTable.second.addForeignConstraint(name, columnsAndTable.first, + catalogConstraint = columnsAndTable.second.addForeignConstraint(name, columnsAndTable.first, referencedColumnsAndTable.second, referencedColumnsAndTable.first); } else if (constraint.isPrimaryKey()) { - columnsAndTable.second.addPrimaryKeyConstraint(name, columnsAndTable.first); + catalogConstraint = columnsAndTable.second.addPrimaryKeyConstraint(name, columnsAndTable.first); } else if (constraint.isUnique()) { - columnsAndTable.second.addUniqueConstraint(name, columnsAndTable.first); + catalogConstraint = columnsAndTable.second.addUniqueConstraint(name, columnsAndTable.first); } + Env.getCurrentEnv().getEditLog().logAddConstraint( + new AlterConstraintLog(catalogConstraint, columnsAndTable.second)); } private Pair, TableIf> extractColumnsAndTable(ConnectContext ctx, LogicalPlan plan) { @@ -82,8 +87,6 @@ private Pair, TableIf> extractColumnsAndTable(ConnectConte throw new AnalysisException("Can not found table in constraint " + constraint.toString()); } LogicalCatalogRelation catalogRelation = logicalCatalogRelationSet.iterator().next(); - Preconditions.checkArgument(catalogRelation.getTable() instanceof Table, - "We only support table now but we meet ", catalogRelation.getTable()); ImmutableList columns = analyzedPlan.getOutput().stream() .map(s -> { Preconditions.checkArgument(s instanceof SlotReference diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropConstraintCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropConstraintCommand.java index 202f53197d2bec..84143f234c33f0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropConstraintCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropConstraintCommand.java @@ -17,7 +17,7 @@ package org.apache.doris.nereids.trees.plans.commands; -import org.apache.doris.catalog.Table; +import org.apache.doris.catalog.Env; import org.apache.doris.catalog.TableIf; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.exceptions.AnalysisException; @@ -28,10 +28,10 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalCatalogRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.persist.AlterConstraintLog; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.StmtExecutor; -import com.google.common.base.Preconditions; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -58,7 +58,8 @@ public DropConstraintCommand(String name, LogicalPlan plan) { @Override public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { TableIf table = extractTable(ctx, plan); - table.dropConstraint(name); + org.apache.doris.catalog.constraint.Constraint catalogConstraint = table.dropConstraint(name); + Env.getCurrentEnv().getEditLog().logDropConstraint(new AlterConstraintLog(catalogConstraint, table)); } private TableIf extractTable(ConnectContext ctx, LogicalPlan plan) { @@ -70,8 +71,6 @@ private TableIf extractTable(ConnectContext ctx, LogicalPlan plan) { throw new AnalysisException("Can not found table when dropping constraint"); } LogicalCatalogRelation catalogRelation = logicalCatalogRelationSet.iterator().next(); - Preconditions.checkArgument(catalogRelation.getTable() instanceof Table, - "Don't support table ", catalogRelation.getTable()); return catalogRelation.getTable(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowConstraintsCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowConstraintsCommand.java index 32950308e01885..13d1f0c5f6d2a2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowConstraintsCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowConstraintsCommand.java @@ -51,7 +51,7 @@ public ShowConstraintsCommand(List nameParts) { public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { TableIf tableIf = RelationUtil.getDbAndTable( RelationUtil.getQualifierName(ctx, nameParts), ctx.getEnv()).value(); - List> res = tableIf.getConstraintMap().entrySet().stream() + List> res = tableIf.getConstraintsMap().entrySet().stream() .map(e -> Lists.newArrayList(e.getKey(), e.getValue().getType().getName(), e.getValue().toString())) diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/AlterConstraintLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/AlterConstraintLog.java new file mode 100644 index 00000000000000..eef8b2cd2b55bc --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/AlterConstraintLog.java @@ -0,0 +1,61 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.persist; + +import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.constraint.Constraint; +import org.apache.doris.catalog.constraint.TableIdentifier; +import org.apache.doris.common.io.Text; +import org.apache.doris.common.io.Writable; +import org.apache.doris.persist.gson.GsonUtils; + +import com.google.gson.annotations.SerializedName; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +public class AlterConstraintLog implements Writable { + @SerializedName("ct") + final Constraint constraint; + @SerializedName("tid") + final TableIdentifier tableIdentifier; + + public AlterConstraintLog(Constraint constraint, TableIf table) { + this.constraint = constraint; + this.tableIdentifier = new TableIdentifier(table); + } + + public TableIf getTableIf() { + return tableIdentifier.toTableIf(); + } + + public Constraint getConstraint() { + return constraint; + } + + @Override + public void write(DataOutput out) throws IOException { + Text.writeString(out, GsonUtils.GSON.toJson(this)); + } + + public static AlterConstraintLog read(DataInput in) throws IOException { + String json = Text.readString(in); + return GsonUtils.GSON.fromJson(json, AlterConstraintLog.class); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index 6eae5a688647ba..fb97c02b00ccd8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -973,6 +973,16 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { case OperationType.OP_ALTER_MTMV_STMT: { break; } + case OperationType.OP_ADD_CONSTRAINT: { + final AlterConstraintLog log = (AlterConstraintLog) journal.getData(); + log.getTableIf().replayAddConstraint(log.getConstraint()); + break; + } + case OperationType.OP_DROP_CONSTRAINT: { + final AlterConstraintLog log = (AlterConstraintLog) journal.getData(); + log.getTableIf().dropConstraint(log.getConstraint().getName()); + break; + } case OperationType.OP_ALTER_USER: { final AlterUserOperationLog log = (AlterUserOperationLog) journal.getData(); env.getAuth().replayAlterUser(log); @@ -1969,6 +1979,14 @@ public void logAlterMTMV(AlterMTMV log) { } + public void logAddConstraint(AlterConstraintLog log) { + logEdit(OperationType.OP_ADD_CONSTRAINT, log); + } + + public void logDropConstraint(AlterConstraintLog log) { + logEdit(OperationType.OP_DROP_CONSTRAINT, log); + } + public void logInsertOverwrite(InsertOverwriteLog log) { logEdit(OperationType.OP_INSERT_OVERWRITE, log); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java index d26ddc12f4901f..42312297b917d7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java @@ -285,6 +285,8 @@ public class OperationType { public static final short OP_CHANGE_MTMV_TASK = 342; @Deprecated public static final short OP_ALTER_MTMV_STMT = 345; + public static final short OP_ADD_CONSTRAINT = 346; + public static final short OP_DROP_CONSTRAINT = 347; public static final short OP_DROP_EXTERNAL_TABLE = 350; public static final short OP_DROP_EXTERNAL_DB = 351; @@ -355,6 +357,7 @@ public class OperationType { public static final short OP_INSERT_OVERWRITE = 461; + /** * Get opcode name by op code. **/ diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java index db49ecdc9920a1..54c109b61a56eb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java @@ -44,6 +44,10 @@ import org.apache.doris.catalog.SparkResource; import org.apache.doris.catalog.StructType; import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.constraint.Constraint; +import org.apache.doris.catalog.constraint.ForeignKeyConstraint; +import org.apache.doris.catalog.constraint.PrimaryKeyConstraint; +import org.apache.doris.catalog.constraint.UniqueConstraint; import org.apache.doris.catalog.external.EsExternalDatabase; import org.apache.doris.catalog.external.EsExternalTable; import org.apache.doris.catalog.external.ExternalDatabase; @@ -202,6 +206,12 @@ public class GsonUtils { Policy.class, "clazz").registerSubtype(RowPolicy.class, RowPolicy.class.getSimpleName()) .registerSubtype(StoragePolicy.class, StoragePolicy.class.getSimpleName()); + private static RuntimeTypeAdapterFactory constraintTypeAdapterFactory = RuntimeTypeAdapterFactory.of( + Constraint.class, "clazz") + .registerSubtype(PrimaryKeyConstraint.class, PrimaryKeyConstraint.class.getSimpleName()) + .registerSubtype(ForeignKeyConstraint.class, ForeignKeyConstraint.class.getSimpleName()) + .registerSubtype(UniqueConstraint.class, UniqueConstraint.class.getSimpleName()); + private static RuntimeTypeAdapterFactory dsTypeAdapterFactory = RuntimeTypeAdapterFactory.of( CatalogIf.class, "clazz") .registerSubtype(InternalCatalog.class, InternalCatalog.class.getSimpleName()) @@ -287,6 +297,7 @@ public class GsonUtils { .registerTypeAdapterFactory(hbResponseTypeAdapterFactory) .registerTypeAdapterFactory(rdsTypeAdapterFactory) .registerTypeAdapterFactory(jobExecutorRuntimeTypeAdapterFactory) + .registerTypeAdapterFactory(constraintTypeAdapterFactory) .registerTypeAdapter(ImmutableMap.class, new ImmutableMapDeserializer()) .registerTypeAdapter(AtomicBoolean.class, new AtomicBooleanAdapter()) .registerTypeAdapter(PartitionKey.class, new PartitionKey.PartitionKeySerializer()) diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/constraint/ConstraintPersistTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/constraint/ConstraintPersistTest.java new file mode 100644 index 00000000000000..64f3db583ad0c5 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/constraint/ConstraintPersistTest.java @@ -0,0 +1,305 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.catalog.constraint; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.catalog.Table; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.external.EsExternalDatabase; +import org.apache.doris.catalog.external.EsExternalTable; +import org.apache.doris.catalog.external.ExternalTable; +import org.apache.doris.common.Config; +import org.apache.doris.datasource.EsExternalCatalog; +import org.apache.doris.journal.JournalEntity; +import org.apache.doris.nereids.util.PlanPatternMatchSupported; +import org.apache.doris.nereids.util.RelationUtil; +import org.apache.doris.persist.AlterConstraintLog; +import org.apache.doris.persist.EditLog; +import org.apache.doris.persist.OperationType; +import org.apache.doris.utframe.TestWithFeService; + +import com.google.common.collect.ImmutableList; +import mockit.Mock; +import mockit.MockUp; +import org.apache.hadoop.util.Lists; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInput; +import java.io.DataInputStream; +import java.io.DataOutput; +import java.io.DataOutputStream; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +class ConstraintPersistTest extends TestWithFeService implements PlanPatternMatchSupported { + + @Override + public void runBeforeAll() throws Exception { + createDatabase("test"); + connectContext.setDatabase("test"); + createTable("create table t1 (\n" + + " k1 int,\n" + + " k2 int\n" + + ")\n" + + "unique key(k1, k2)\n" + + "distributed by hash(k1) buckets 4\n" + + "properties(\n" + + " \"replication_num\"=\"1\"\n" + + ")"); + createTable("create table t2 (\n" + + " k1 int,\n" + + " k2 int\n" + + ")\n" + + "unique key(k1, k2)\n" + + "distributed by hash(k1) buckets 4\n" + + "properties(\n" + + " \"replication_num\"=\"1\"\n" + + ")"); + } + + @Test + void addConstraintLogPersistTest() throws Exception { + Config.edit_log_type = "local"; + addConstraint("alter table t1 add constraint pk primary key (k1)"); + addConstraint("alter table t2 add constraint pk primary key (k1)"); + addConstraint("alter table t1 add constraint uk unique (k1)"); + addConstraint("alter table t1 add constraint fk foreign key (k1) references t2(k1)"); + TableIf tableIf = RelationUtil.getTable( + RelationUtil.getQualifierName(connectContext, Lists.newArrayList("test", "t1")), + connectContext.getEnv()); + Map constraintMap = tableIf.getConstraintsMap(); + tableIf.getConstraintsMapUnsafe().clear(); + Assertions.assertTrue(tableIf.getConstraintsMap().isEmpty()); + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + DataOutput output = new DataOutputStream(outputStream); + for (Constraint value : constraintMap.values()) { + JournalEntity journalEntity = new JournalEntity(); + journalEntity.setData(new AlterConstraintLog(value, tableIf)); + journalEntity.setOpCode(OperationType.OP_ADD_CONSTRAINT); + journalEntity.write(output); + } + InputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray()); + DataInput input = new DataInputStream(inputStream); + for (int i = 0; i < constraintMap.values().size(); i++) { + JournalEntity journalEntity = new JournalEntity(); + journalEntity.readFields(input); + EditLog.loadJournal(Env.getCurrentEnv(), 0L, journalEntity); + } + Assertions.assertEquals(tableIf.getConstraintsMap(), constraintMap); + dropConstraint("alter table t1 drop constraint fk"); + dropConstraint("alter table t1 drop constraint pk"); + dropConstraint("alter table t2 drop constraint pk"); + dropConstraint("alter table t1 drop constraint uk"); + } + + @Test + void dropConstraintLogPersistTest() throws Exception { + Config.edit_log_type = "local"; + addConstraint("alter table t1 add constraint pk primary key (k1)"); + addConstraint("alter table t2 add constraint pk primary key (k1)"); + addConstraint("alter table t1 add constraint uk unique (k1)"); + addConstraint("alter table t1 add constraint fk foreign key (k1) references t2(k1)"); + TableIf tableIf = RelationUtil.getTable( + RelationUtil.getQualifierName(connectContext, Lists.newArrayList("test", "t1")), + connectContext.getEnv()); + Map constraintMap = tableIf.getConstraintsMap(); + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + DataOutput output = new DataOutputStream(outputStream); + for (Constraint value : constraintMap.values()) { + JournalEntity journalEntity = new JournalEntity(); + journalEntity.setData(new AlterConstraintLog(value, tableIf)); + journalEntity.setOpCode(OperationType.OP_DROP_CONSTRAINT); + journalEntity.write(output); + } + InputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray()); + DataInput input = new DataInputStream(inputStream); + for (int i = 0; i < constraintMap.values().size(); i++) { + JournalEntity journalEntity = new JournalEntity(); + journalEntity.readFields(input); + EditLog.loadJournal(Env.getCurrentEnv(), 0L, journalEntity); + } + Assertions.assertTrue(tableIf.getConstraintsMap().isEmpty()); + } + + @Test + void constraintWithTablePersistTest() throws Exception { + addConstraint("alter table t1 add constraint pk primary key (k1)"); + addConstraint("alter table t2 add constraint pk primary key (k1)"); + addConstraint("alter table t1 add constraint uk unique (k1)"); + addConstraint("alter table t1 add constraint fk foreign key (k1) references t2(k1)"); + TableIf tableIf = RelationUtil.getTable( + RelationUtil.getQualifierName(connectContext, Lists.newArrayList("test", "t1")), + connectContext.getEnv()); + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + DataOutput output = new DataOutputStream(outputStream); + tableIf.write(output); + InputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray()); + DataInput input = new DataInputStream(inputStream); + TableIf loadTable = Table.read(input); + Assertions.assertEquals(loadTable.getConstraintsMap(), tableIf.getConstraintsMap()); + dropConstraint("alter table t1 drop constraint fk"); + dropConstraint("alter table t1 drop constraint pk"); + dropConstraint("alter table t2 drop constraint pk"); + dropConstraint("alter table t1 drop constraint uk"); + } + + @Test + void externalTableTest() throws Exception { + ExternalTable externalTable = new ExternalTable(); + externalTable.addPrimaryKeyConstraint("pk", ImmutableList.of("col")); + + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + DataOutput output = new DataOutputStream(outputStream); + externalTable.write(output); + InputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray()); + DataInput input = new DataInputStream(inputStream); + TableIf loadTable = ExternalTable.read(input); + Assertions.assertEquals(1, loadTable.getConstraintsMap().size()); + } + + @Test + void addConstraintLogPersistForExternalTableTest() throws Exception { + Config.edit_log_type = "local"; + createCatalog("create catalog es properties('type' = 'es', 'elasticsearch.hosts' = 'http://192.168.0.1'," + + " 'elasticsearch.username' = 'user1');"); + + Env.getCurrentEnv().changeCatalog(connectContext, "es"); + EsExternalCatalog esCatalog = (EsExternalCatalog) getCatalog("es"); + EsExternalDatabase db = new EsExternalDatabase(esCatalog, 10002, "es_db1"); + EsExternalTable tbl = new EsExternalTable(10003, "es_tbl1", "es_db1", esCatalog); + ImmutableList schema = ImmutableList.of(new Column("k1", PrimitiveType.INT)); + tbl.setNewFullSchema(schema); + db.addTableForTest(tbl); + esCatalog.addDatabaseForTest(db); + Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(esCatalog).addSchemaForTest(db.getFullName(), tbl.getName(), schema); + new MockUp() { + @Mock + public TableIf getTable(List qualifierName, Env env) { + return tbl; + } + }; + + new MockUp() { + @Mock + public DatabaseIf getDatabase() { + return db; + } + }; + + new MockUp() { + @Mock + public TableIf toTableIf() { + return tbl; + } + }; + + addConstraint("alter table es.es_db1.es_tbl1 add constraint pk primary key (k1)"); + addConstraint("alter table es.es_db1.es_tbl1 add constraint uk unique (k1)"); + TableIf tableIf = RelationUtil.getTable( + RelationUtil.getQualifierName(connectContext, Lists.newArrayList("test", "t1")), + connectContext.getEnv()); + Map constraintMap = tableIf.getConstraintsMap(); + tableIf.getConstraintsMapUnsafe().clear(); + Assertions.assertTrue(tableIf.getConstraintsMap().isEmpty()); + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + DataOutput output = new DataOutputStream(outputStream); + for (Constraint value : new ArrayList<>(constraintMap.values())) { + JournalEntity journalEntity = new JournalEntity(); + journalEntity.setData(new AlterConstraintLog(value, tableIf)); + journalEntity.setOpCode(OperationType.OP_ADD_CONSTRAINT); + journalEntity.write(output); + } + InputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray()); + DataInput input = new DataInputStream(inputStream); + for (int i = 0; i < constraintMap.values().size(); i++) { + JournalEntity journalEntity = new JournalEntity(); + journalEntity.readFields(input); + EditLog.loadJournal(Env.getCurrentEnv(), 0L, journalEntity); + } + Assertions.assertEquals(tableIf.getConstraintsMap(), constraintMap); + Env.getCurrentEnv().changeCatalog(connectContext, "internal"); + } + + @Test + void dropConstraintLogPersistForExternalTest() throws Exception { + Config.edit_log_type = "local"; + createCatalog("create catalog es2 properties('type' = 'es', 'elasticsearch.hosts' = 'http://192.168.0.1'," + + " 'elasticsearch.username' = 'user1');"); + + Env.getCurrentEnv().changeCatalog(connectContext, "es2"); + EsExternalCatalog esCatalog = (EsExternalCatalog) getCatalog("es2"); + EsExternalDatabase db = new EsExternalDatabase(esCatalog, 10002, "es_db1"); + EsExternalTable tbl = new EsExternalTable(10003, "es_tbl1", "es_db1", esCatalog); + ImmutableList schema = ImmutableList.of(new Column("k1", PrimitiveType.INT)); + tbl.setNewFullSchema(schema); + db.addTableForTest(tbl); + esCatalog.addDatabaseForTest(db); + Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(esCatalog).addSchemaForTest(db.getFullName(), tbl.getName(), schema); + new MockUp() { + @Mock + public TableIf getTable(List qualifierName, Env env) { + return tbl; + } + }; + + new MockUp() { + @Mock + public DatabaseIf getDatabase() { + return db; + } + }; + + new MockUp() { + @Mock + public TableIf toTableIf() { + return tbl; + } + }; + addConstraint("alter table es.es_db1.es_tbl1 add constraint pk primary key (k1)"); + addConstraint("alter table es.es_db1.es_tbl1 add constraint uk unique (k1)"); + TableIf tableIf = RelationUtil.getTable( + RelationUtil.getQualifierName(connectContext, Lists.newArrayList("test", "t1")), + connectContext.getEnv()); + Map constraintMap = tableIf.getConstraintsMap(); + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + DataOutput output = new DataOutputStream(outputStream); + for (Constraint value : constraintMap.values()) { + JournalEntity journalEntity = new JournalEntity(); + journalEntity.setData(new AlterConstraintLog(value, tableIf)); + journalEntity.setOpCode(OperationType.OP_DROP_CONSTRAINT); + journalEntity.write(output); + } + InputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray()); + DataInput input = new DataInputStream(inputStream); + for (int i = 0; i < constraintMap.values().size(); i++) { + JournalEntity journalEntity = new JournalEntity(); + journalEntity.readFields(input); + EditLog.loadJournal(Env.getCurrentEnv(), 0L, journalEntity); + } + Assertions.assertTrue(tableIf.getConstraintsMap().isEmpty()); + Env.getCurrentEnv().changeCatalog(connectContext, "internal"); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java b/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java index 85f6e22e6f46c2..1f57040d305b8f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java +++ b/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java @@ -21,6 +21,7 @@ import org.apache.doris.analysis.AlterSqlBlockRuleStmt; import org.apache.doris.analysis.AlterTableStmt; import org.apache.doris.analysis.Analyzer; +import org.apache.doris.analysis.CreateCatalogStmt; import org.apache.doris.analysis.CreateDbStmt; import org.apache.doris.analysis.CreateFunctionStmt; import org.apache.doris.analysis.CreateMaterializedViewStmt; @@ -55,6 +56,7 @@ import org.apache.doris.common.FeConstants; import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.util.SqlParserUtils; +import org.apache.doris.datasource.CatalogIf; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.glue.LogicalPlanAdapter; @@ -650,6 +652,15 @@ public void createTableAsSelect(String sql) throws Exception { Env.getCurrentEnv().createTableAsSelect(createTableAsSelectStmt); } + public void createCatalog(String sql) throws Exception { + CreateCatalogStmt stmt = (CreateCatalogStmt) parseAndAnalyzeStmt(sql, connectContext); + Env.getCurrentEnv().getCatalogMgr().createCatalog(stmt); + } + + public CatalogIf getCatalog(String name) throws Exception { + return Env.getCurrentEnv().getCatalogMgr().getCatalog(name); + } + public void createTables(String... sqls) throws Exception { createTables(false, sqls); } From 68286c4542178a9ee3408ab47e5d41970bd21c9b Mon Sep 17 00:00:00 2001 From: Ashin Gau Date: Tue, 16 Jan 2024 19:13:51 +0800 Subject: [PATCH 039/200] [opt](scanner) optimize the number of threads of scanners, follow up #28640 (#30030) follow up #28640, doris_scanner_thread_pool_thread_num may be to large in machines with 128 cores. --- be/src/common/config.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 2d418b2bf2412f..f7baa43ce85219 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -236,7 +236,7 @@ DEFINE_Int32(doris_scanner_thread_pool_thread_num, "-1"); DEFINE_Validator(doris_scanner_thread_pool_thread_num, [](const int config) -> bool { if (config == -1) { CpuInfo::init(); - doris_scanner_thread_pool_thread_num = std::max(48, CpuInfo::num_cores() * 4); + doris_scanner_thread_pool_thread_num = std::max(48, CpuInfo::num_cores() * 2); } return true; }); From 9b68bc211f12811066b5957960818e84fcaeb015 Mon Sep 17 00:00:00 2001 From: HHoflittlefish777 <77738092+HHoflittlefish777@users.noreply.github.com> Date: Tue, 16 Jan 2024 20:07:28 +0800 Subject: [PATCH 040/200] [fix](test) fix disableDebugPointForAllBEs do not execute (#30023) --- .../test_load_stream_back_pressure_fault_injection.groovy | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/regression-test/suites/fault_injection_p0/test_load_stream_back_pressure_fault_injection.groovy b/regression-test/suites/fault_injection_p0/test_load_stream_back_pressure_fault_injection.groovy index 57825c80d17d0e..fc2abe7fb0ad6f 100644 --- a/regression-test/suites/fault_injection_p0/test_load_stream_back_pressure_fault_injection.groovy +++ b/regression-test/suites/fault_injection_p0/test_load_stream_back_pressure_fault_injection.groovy @@ -78,8 +78,6 @@ suite("test_load_stream_back_pressure_fault_injection", "nonConcurrent") { } catch(Exception e) { logger.info(e.getMessage()) assertTrue(e.getMessage().contains("Communications link failure")) - } finally { - GetDebugPoint().disableDebugPointForAllBEs("TabletStream.append_data.long_wait") } }) thread1.start() @@ -98,6 +96,8 @@ suite("test_load_stream_back_pressure_fault_injection", "nonConcurrent") { } } catch(Exception e) { logger.info(e.getMessage()) + } finally { + GetDebugPoint().disableDebugPointForAllBEs("TabletStream.append_data.long_wait") } try { @@ -109,8 +109,6 @@ suite("test_load_stream_back_pressure_fault_injection", "nonConcurrent") { } catch(Exception e) { logger.info(e.getMessage()) assertTrue(e.getMessage().contains("Communications link failure")) - } finally { - GetDebugPoint().disableDebugPointForAllBEs("TabletStream.add_segment.long_wait") } }) thread1.start() @@ -129,6 +127,8 @@ suite("test_load_stream_back_pressure_fault_injection", "nonConcurrent") { } } catch(Exception e) { logger.info(e.getMessage()) + } finally { + GetDebugPoint().disableDebugPointForAllBEs("TabletStream.add_segment.long_wait") } sql """ DROP TABLE IF EXISTS `baseall` """ From b60a8be7bba8b67d96dd2cb971d7922c75b6ff99 Mon Sep 17 00:00:00 2001 From: Kaijie Chen Date: Tue, 16 Jan 2024 20:08:26 +0800 Subject: [PATCH 041/200] [improve](move-memtable) add cancel method to load stream stub (#29994) --- be/src/vec/sink/load_stream_stub.cpp | 25 +++++++++++++++++-- be/src/vec/sink/load_stream_stub.h | 15 +++++++++++ be/src/vec/sink/load_stream_stub_pool.cpp | 2 +- be/src/vec/sink/load_stream_stub_pool.h | 4 +-- be/src/vec/sink/writer/vtablet_writer_v2.cpp | 7 ++++-- .../vec/exec/load_stream_stub_pool_test.cpp | 7 +++--- 6 files changed, 49 insertions(+), 11 deletions(-) diff --git a/be/src/vec/sink/load_stream_stub.cpp b/be/src/vec/sink/load_stream_stub.cpp index 40ce75d24e6248..347acb1b6f6f0f 100644 --- a/be/src/vec/sink/load_stream_stub.cpp +++ b/be/src/vec/sink/load_stream_stub.cpp @@ -294,6 +294,7 @@ Status LoadStreamStub::wait_for_schema(int64_t partition_id, int64_t index_id, i watch.start(); while (!_tablet_schema_for_index->contains(index_id) && watch.elapsed_time() / 1000 / 1000 < timeout_ms) { + RETURN_IF_ERROR(_check_cancel()); static_cast(wait_for_new_schema(100)); } @@ -308,8 +309,12 @@ Status LoadStreamStub::close_wait(int64_t timeout_ms) { while (true) { }; }); - if (!_is_init.load() || _is_closed.load()) { - return Status::OK(); + if (!_is_init.load()) { + return Status::InternalError("stream {} is not opened, load_id={}", _stream_id, + print_id(_load_id)); + } + if (_is_closed.load()) { + return _check_cancel(); } if (timeout_ms <= 0) { timeout_ms = config::close_load_stream_timeout_ms; @@ -324,6 +329,7 @@ Status LoadStreamStub::close_wait(int64_t timeout_ms) { print_id(_load_id), _dst_id, _stream_id); } } + RETURN_IF_ERROR(_check_cancel()); if (!_is_eos.load()) { return Status::InternalError( "stream closed without eos, load_id={}, dst_id={}, stream_id={}", @@ -332,6 +338,20 @@ Status LoadStreamStub::close_wait(int64_t timeout_ms) { return Status::OK(); } +void LoadStreamStub::cancel(Status reason) { + LOG(WARNING) << *this << " is cancelled because of " << reason; + { + std::lock_guard lock(_cancel_mutex); + _cancel_reason = reason; + _is_cancelled.store(true); + } + { + std::lock_guard lock(_close_mutex); + _is_closed.store(true); + _close_cv.notify_all(); + } +} + Status LoadStreamStub::_encode_and_send(PStreamHeader& header, std::span data) { butil::IOBuf buf; size_t header_len = header.ByteSizeLong(); @@ -365,6 +385,7 @@ Status LoadStreamStub::_send_with_buffer(butil::IOBuf& buf, bool sync) { Status LoadStreamStub::_send_with_retry(butil::IOBuf& buf) { for (;;) { + RETURN_IF_ERROR(_check_cancel()); int ret; { SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(ExecEnv::GetInstance()->orphan_mem_tracker()); diff --git a/be/src/vec/sink/load_stream_stub.h b/be/src/vec/sink/load_stream_stub.h index 6aae778dc93551..c91f1016d35646 100644 --- a/be/src/vec/sink/load_stream_stub.h +++ b/be/src/vec/sink/load_stream_stub.h @@ -153,6 +153,9 @@ class LoadStreamStub { // if timeout_ms <= 0, will fallback to config::close_load_stream_timeout_ms Status close_wait(int64_t timeout_ms = 0); + // cancel the stream, abort close_wait, mark _is_closed and _is_cancelled + void cancel(Status reason); + Status wait_for_schema(int64_t partition_id, int64_t index_id, int64_t tablet_id, int64_t timeout_ms = 60000); @@ -197,9 +200,19 @@ class LoadStreamStub { Status _send_with_buffer(butil::IOBuf& buf, bool sync = false); Status _send_with_retry(butil::IOBuf& buf); + Status _check_cancel() { + if (!_is_cancelled.load()) { + return Status::OK(); + } + std::lock_guard lock(_cancel_mutex); + return Status::Cancelled("load_id={}, reason: {}", print_id(_load_id), + _cancel_reason.to_string_no_stack()); + } + protected: std::atomic _is_init; std::atomic _is_closed; + std::atomic _is_cancelled; std::atomic _is_eos; std::atomic _use_cnt; @@ -207,9 +220,11 @@ class LoadStreamStub { brpc::StreamId _stream_id; int64_t _src_id = -1; // source backend_id int64_t _dst_id = -1; // destination backend_id + Status _cancel_reason; bthread::Mutex _open_mutex; bthread::Mutex _close_mutex; + bthread::Mutex _cancel_mutex; bthread::ConditionVariable _close_cv; std::mutex _tablets_to_commit_mutex; diff --git a/be/src/vec/sink/load_stream_stub_pool.cpp b/be/src/vec/sink/load_stream_stub_pool.cpp index 1baa903f2eeba8..d76402b57d5020 100644 --- a/be/src/vec/sink/load_stream_stub_pool.cpp +++ b/be/src/vec/sink/load_stream_stub_pool.cpp @@ -26,7 +26,7 @@ class TExpr; LoadStreams::LoadStreams(UniqueId load_id, int64_t dst_id, int num_use, LoadStreamStubPool* pool) : _load_id(load_id), _dst_id(dst_id), _use_cnt(num_use), _pool(pool) {} -void LoadStreams::release(Status status) { +void LoadStreams::release() { int num_use = --_use_cnt; DBUG_EXECUTE_IF("LoadStreams.release.keeping_streams", { num_use = 1; }); if (num_use == 0) { diff --git a/be/src/vec/sink/load_stream_stub_pool.h b/be/src/vec/sink/load_stream_stub_pool.h index b34383b25f9e2d..662fc5bc1a143d 100644 --- a/be/src/vec/sink/load_stream_stub_pool.h +++ b/be/src/vec/sink/load_stream_stub_pool.h @@ -76,7 +76,7 @@ class LoadStreams { public: LoadStreams(UniqueId load_id, int64_t dst_id, int num_use, LoadStreamStubPool* pool); - void release(Status status); + void release(); Streams& streams() { return _streams; } @@ -116,4 +116,4 @@ class LoadStreamStubPool { std::unordered_map, std::shared_ptr> _pool; }; -} // namespace doris \ No newline at end of file +} // namespace doris diff --git a/be/src/vec/sink/writer/vtablet_writer_v2.cpp b/be/src/vec/sink/writer/vtablet_writer_v2.cpp index 02b40549253a88..e23fe761ecf1d6 100644 --- a/be/src/vec/sink/writer/vtablet_writer_v2.cpp +++ b/be/src/vec/sink/writer/vtablet_writer_v2.cpp @@ -457,7 +457,10 @@ Status VTabletWriterV2::_cancel(Status status) { _delta_writer_for_tablet.reset(); } for (const auto& [_, streams] : _streams_for_node) { - streams->release(status); + for (const auto& stream : streams->streams()) { + stream->cancel(status); + } + streams->release(); } return Status::OK(); } @@ -514,7 +517,7 @@ Status VTabletWriterV2::close(Status exec_status) { // defer stream release to prevent memory leak Defer defer([&] { for (const auto& [_, streams] : _streams_for_node) { - streams->release(status); + streams->release(); } _streams_for_node.clear(); }); diff --git a/be/test/vec/exec/load_stream_stub_pool_test.cpp b/be/test/vec/exec/load_stream_stub_pool_test.cpp index bea5443b4ff7fd..24da3bb6999f36 100644 --- a/be/test/vec/exec/load_stream_stub_pool_test.cpp +++ b/be/test/vec/exec/load_stream_stub_pool_test.cpp @@ -32,7 +32,6 @@ TEST_F(LoadStreamStubPoolTest, test) { LoadStreamStubPool pool; int64_t src_id = 100; PUniqueId load_id; - Status st = Status::OK(); load_id.set_hi(1); load_id.set_hi(2); auto streams1 = pool.get_or_create(load_id, src_id, 101, 5, 1); @@ -42,9 +41,9 @@ TEST_F(LoadStreamStubPoolTest, test) { EXPECT_EQ(1, pool.templates_size()); EXPECT_EQ(streams1, streams3); EXPECT_NE(streams1, streams2); - streams1->release(st); - streams2->release(st); - streams3->release(st); + streams1->release(); + streams2->release(); + streams3->release(); EXPECT_EQ(0, pool.size()); EXPECT_EQ(0, pool.templates_size()); } From c3e4f0c402358a402d1e05a20b5358ea7fe58d5b Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Tue, 16 Jan 2024 20:16:53 +0800 Subject: [PATCH 042/200] [fix](Nereids) remove useless substr of string like type cast (#29997) --- .../org/apache/doris/nereids/rules/analysis/BindSink.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java index 4ce7350cf2d9fd..7dbb6b3e1e10c0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java @@ -297,10 +297,13 @@ public List buildRules() { DataType inputType = expr.getDataType(); DataType targetType = DataType.fromCatalogType(table.getFullSchema().get(i).getType()); Expression castExpr = expr; - if (isSourceAndTargetStringLikeType(inputType, targetType)) { + // TODO move string like type logic into TypeCoercionUtils#castIfNotSameType + if (isSourceAndTargetStringLikeType(inputType, targetType) && !inputType.equals(targetType)) { int sourceLength = ((CharacterType) inputType).getLen(); int targetLength = ((CharacterType) targetType).getLen(); - if (sourceLength >= targetLength && targetLength >= 0) { + if (sourceLength == targetLength) { + castExpr = TypeCoercionUtils.castIfNotSameType(castExpr, targetType); + } else if (sourceLength > targetLength && targetLength >= 0) { castExpr = new Substring(castExpr, Literal.of(1), Literal.of(targetLength)); } else if (targetType.isStringType()) { castExpr = new Cast(castExpr, StringType.INSTANCE); From dfac651af8aa4b97ff05cb19c5a15b6f194aca4d Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Tue, 16 Jan 2024 20:19:16 +0800 Subject: [PATCH 043/200] [chore](test) update delete complex type case to make Nereids happy (#30022) --- .../suites/delete_p0/test_array_column_delete.groovy | 2 +- regression-test/suites/delete_p0/test_map_column_delete.groovy | 2 +- .../suites/delete_p0/test_struct_column_delete.groovy | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/regression-test/suites/delete_p0/test_array_column_delete.groovy b/regression-test/suites/delete_p0/test_array_column_delete.groovy index 442b4f5995765b..6dc2d1d982bfe0 100644 --- a/regression-test/suites/delete_p0/test_array_column_delete.groovy +++ b/regression-test/suites/delete_p0/test_array_column_delete.groovy @@ -23,7 +23,7 @@ suite("test_array_column_delete") { sql """ insert into ${tableName} values(1, NULL),(2,[12,3]),(3,[]),(4,NULL),(5,NULL) """ test { sql """ DELETE FROM ${tableName} WHERE c_array is NULL """ - exception("java.sql.SQLException: errCode = 2, detailMessage = errCode = 2, detailMessage = Can not apply delete condition to column type: ARRAY") + exception("Can not apply delete condition to column type: ARRAY") } sql """ DELETE FROM ${tableName} WHERE id = 1; """ qt_sql """ SELECT * FROM ${tableName} order by id """ diff --git a/regression-test/suites/delete_p0/test_map_column_delete.groovy b/regression-test/suites/delete_p0/test_map_column_delete.groovy index e82c6885cd56d0..ce00b77f1534df 100644 --- a/regression-test/suites/delete_p0/test_map_column_delete.groovy +++ b/regression-test/suites/delete_p0/test_map_column_delete.groovy @@ -24,7 +24,7 @@ suite("test_map_column_delete") { sql """ insert into ${tableName} values(1, {1:'a', 2:"doris"}),(2,{}),(3,NULL),(4,NULL),(5,NULL) """ test { sql """ DELETE FROM ${tableName} WHERE m_map is NULL """ - exception("java.sql.SQLException: errCode = 2, detailMessage = errCode = 2, detailMessage = Can not apply delete condition to column type: MAP") + exception("Can not apply delete condition to column type: MAP") } sql """ DELETE FROM ${tableName} WHERE id = 1; """ qt_sql """ SELECT * FROM ${tableName} order by id """ diff --git a/regression-test/suites/delete_p0/test_struct_column_delete.groovy b/regression-test/suites/delete_p0/test_struct_column_delete.groovy index 26f24837a539ad..7b82cda668cc15 100644 --- a/regression-test/suites/delete_p0/test_struct_column_delete.groovy +++ b/regression-test/suites/delete_p0/test_struct_column_delete.groovy @@ -23,7 +23,7 @@ suite("test_struct_column_delete") { sql """ insert into ${tableName} values(1, {1, 'a'}),(2,NULL),(3,NULL),(4,NULL),(5,NULL) """ test { sql """ DELETE FROM ${tableName} WHERE s_struct is NULL """ - exception("java.sql.SQLException: errCode = 2, detailMessage = errCode = 2, detailMessage = Can not apply delete condition to column type: STRUCT") + exception("Can not apply delete condition to column type: STRUCT") } sql """ DELETE FROM ${tableName} WHERE id = 1; """ qt_sql """ SELECT * FROM ${tableName} order by id """ From 3c93448899ba6b71b6111c72563eb59b3df99310 Mon Sep 17 00:00:00 2001 From: HHoflittlefish777 <77738092+HHoflittlefish777@users.noreply.github.com> Date: Tue, 16 Jan 2024 20:44:54 +0800 Subject: [PATCH 044/200] [improve](multi-table-load) avoid plan and execute too many plan at once (#29951) --- be/src/common/config.cpp | 7 +++++++ be/src/common/config.h | 7 +++++++ be/src/io/fs/multi_table_pipe.cpp | 10 ++++++---- be/src/io/fs/multi_table_pipe.h | 3 +++ docs/en/docs/admin-manual/config/be-config.md | 6 ++++++ docs/zh-CN/docs/admin-manual/config/be-config.md | 7 +++++++ 6 files changed, 36 insertions(+), 4 deletions(-) diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index f7baa43ce85219..14efa8a4b9da6b 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -838,6 +838,13 @@ DEFINE_Int32(routine_load_consumer_pool_size, "10"); // if the size of batch is more than this threshold, we will request plans for all related tables. DEFINE_Int32(multi_table_batch_plan_threshold, "200"); +// Used in single-stream-multi-table load. When receiving a batch of messages from Kafka, +// if the size of the table wait for plan is more than this threshold, we will request plans for all related tables. +// The param is aimed to avoid requesting and executing too many plans at once. +// Performing small batch processing on multiple tables during the loaded process can reduce the pressure of a single RPC +// and improve the real-time processing of data. +DEFINE_Int32(multi_table_max_wait_tables, "5"); + // When the timeout of a load task is less than this threshold, // Doris treats it as a high priority task. // high priority tasks use a separate thread pool for flush and do not block rpc by memory cleanup logic. diff --git a/be/src/common/config.h b/be/src/common/config.h index 7ae583cedb6e2c..9c0b766f4090be 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -893,6 +893,13 @@ DECLARE_Int32(routine_load_consumer_pool_size); // if the size of batch is more than this threshold, we will request plans for all related tables. DECLARE_Int32(multi_table_batch_plan_threshold); +// Used in single-stream-multi-table load. When receiving a batch of messages from Kafka, +// if the size of the table wait for plan is more than this threshold, we will request plans for all related tables. +// The param is aimed to avoid requesting and executing too many plans at once. +// Performing small batch processing on multiple tables during the loaded process can reduce the pressure of a single RPC +// and improve the real-time processing of data. +DECLARE_Int32(multi_table_max_wait_tables); + // When the timeout of a load task is less than this threshold, // Doris treats it as a high priority task. // high priority tasks use a separate thread pool for flush and do not block rpc by memory cleanup logic. diff --git a/be/src/io/fs/multi_table_pipe.cpp b/be/src/io/fs/multi_table_pipe.cpp index da46645fd4ff34..36976a239725bf 100644 --- a/be/src/io/fs/multi_table_pipe.cpp +++ b/be/src/io/fs/multi_table_pipe.cpp @@ -118,10 +118,12 @@ Status MultiTablePipe::dispatch(const std::string& table, const char* data, size "append failed in unplanned kafka pipe"); ++_unplanned_row_cnt; - size_t threshold = config::multi_table_batch_plan_threshold; - if (_unplanned_row_cnt >= threshold) { - LOG(INFO) << fmt::format("unplanned row cnt={} reach threshold={}, plan them", - _unplanned_row_cnt, threshold); + if (_unplanned_row_cnt >= _row_threshold || + _unplanned_pipes.size() >= _wait_tables_threshold) { + LOG(INFO) << fmt::format( + "unplanned row cnt={} reach row_threshold={} or wait_plan_table_threshold={}, " + "plan them", + _unplanned_row_cnt, _row_threshold, _wait_tables_threshold); Status st = request_and_exec_plans(); _unplanned_row_cnt = 0; if (!st.ok()) { diff --git a/be/src/io/fs/multi_table_pipe.h b/be/src/io/fs/multi_table_pipe.h index 3ded0fc608d4d8..694794638afbe2 100644 --- a/be/src/io/fs/multi_table_pipe.h +++ b/be/src/io/fs/multi_table_pipe.h @@ -92,6 +92,9 @@ class MultiTablePipe : public KafkaConsumerPipe { #endif std::mutex _pipe_map_lock; std::unordered_map> _pipe_map; + + uint32_t _row_threshold = config::multi_table_batch_plan_threshold; + uint32_t _wait_tables_threshold = config::multi_table_max_wait_tables; }; } // namespace io } // end namespace doris diff --git a/docs/en/docs/admin-manual/config/be-config.md b/docs/en/docs/admin-manual/config/be-config.md index 62329d214c3af7..3f00e62d38bf26 100644 --- a/docs/en/docs/admin-manual/config/be-config.md +++ b/docs/en/docs/admin-manual/config/be-config.md @@ -779,6 +779,12 @@ BaseCompaction:546859: * Description: For single-stream-multi-table load. When receive a batch of messages from kafka, if the size of batch is more than this threshold, we will request plans for all related tables. * Default value: 200 +#### `multi_table_max_wait_tables` + +* Type: int32 +* Description: Used in single-stream-multi-table load. When receiving a batch of messages from Kafka, if the size of the table wait for plan is more than this threshold, we will request plans for all related tables.The param is aimed to avoid requesting and executing too many plans at once. Performing small batch processing on multiple tables during the loaded process can reduce the pressure of a single RPC and improve the real-time processing of data. +* Default value: 5 + #### `single_replica_load_download_num_workers` * Type: int32 diff --git a/docs/zh-CN/docs/admin-manual/config/be-config.md b/docs/zh-CN/docs/admin-manual/config/be-config.md index 26e47822b7795b..dc8418d548bb2b 100644 --- a/docs/zh-CN/docs/admin-manual/config/be-config.md +++ b/docs/zh-CN/docs/admin-manual/config/be-config.md @@ -805,6 +805,13 @@ BaseCompaction:546859: * 描述:一流多表使用该配置,表示攒多少条数据再进行规划。过小的值会导致规划频繁,多大的值会增加内存压力和导入延迟。 * 默认值:200 +#### `multi_table_max_wait_tables` + +* 类型:int32 +* 描述:一流多表使用该配置,如果等待执行的表的数量大于此阈值,将请求并执行所有相关表的计划。该参数旨在避免一次同时请求和执行过多的计划。 +将导入过程的多表进行小批处理,可以减少单次rpc的压力,同时可以提高导入数据处理的实时性。 +* 默认值:5 + #### `single_replica_load_download_num_workers` * 类型: int32 * 描述: 单副本数据导入功能中,Slave副本通过HTTP从Master副本下载数据文件的工作线程数。导入并发增大时,可以适当调大该参数来保证Slave副本及时同步Master副本数据。必要时也应相应地调大`webserver_num_workers`来提高IO效率。 From d6a7d634b26ed912245c3145e078bbc49c8b50ef Mon Sep 17 00:00:00 2001 From: zy-kkk Date: Tue, 16 Jan 2024 21:07:59 +0800 Subject: [PATCH 045/200] [improvement](catalog) Change the push-down parameters of the predicate function of the table query SQL into variables (#30028) In this PR, we will control whether the external data source query is a push-down function parameter in the filter condition, changing the enable_fun_pushdown of fe conf to the enable_ext_func_pred_pushdown of the variable --- docs/en/docs/lakehouse/multi-catalog/jdbc.md | 2 +- docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md | 2 +- .../main/java/org/apache/doris/common/Config.java | 7 ------- .../doris/planner/external/jdbc/JdbcScanNode.java | 8 ++++++-- .../doris/planner/external/odbc/OdbcScanNode.java | 7 +++++-- .../java/org/apache/doris/qe/SessionVariable.java | 12 +++++++++++- .../jdbc/test_mysql_jdbc_catalog.groovy | 11 ++++++++--- 7 files changed, 32 insertions(+), 17 deletions(-) diff --git a/docs/en/docs/lakehouse/multi-catalog/jdbc.md b/docs/en/docs/lakehouse/multi-catalog/jdbc.md index 662fcc81118063..1a401146df9034 100644 --- a/docs/en/docs/lakehouse/multi-catalog/jdbc.md +++ b/docs/en/docs/lakehouse/multi-catalog/jdbc.md @@ -114,7 +114,7 @@ In some cases, the keywords in the database might be used as the field names. Fo 1. When executing a query like `where dt = '2022-01-01'`, Doris can push down these filtering conditions to the external data source, thereby directly excluding data that does not meet the conditions at the data source level, reducing the number of unqualified Necessary data acquisition and transfer. This greatly improves query performance while also reducing the load on external data sources. -2. When `enable_func_pushdown` is set to true, the function conditions after where will also be pushed down to the external data source. Currently, only MySQL, ClickHouse, and Oracle are supported. If you encounter functions that are not supported by MySQL, ClickHouse, and Oracle, you can use this The parameter is set to false. At present, Doris will automatically identify some functions that are not supported by MySQL and functions supported by CLickHouse and Oracle for push-down condition filtering. You can view them through explain sql. +2. When variable `enable_ext_func_pred_pushdown` is set to true, the function conditions after where will also be pushed down to the external data source. Currently, only MySQL, ClickHouse, and Oracle are supported. If you encounter functions that are not supported by MySQL, ClickHouse, and Oracle, you can use this The parameter is set to false. At present, Doris will automatically identify some functions that are not supported by MySQL and functions supported by CLickHouse and Oracle for push-down condition filtering. You can view them through explain sql. Functions that are currently not pushed down include: diff --git a/docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md b/docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md index e3eb90d7f40472..a86ab26c33dd90 100644 --- a/docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md +++ b/docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md @@ -114,7 +114,7 @@ select * from mysql_catalog.mysql_database.mysql_table where k1 > 1000 and k3 =' 1. 当执行类似于 `where dt = '2022-01-01'` 这样的查询时,Doris 能够将这些过滤条件下推到外部数据源,从而直接在数据源层面排除不符合条件的数据,减少了不必要的数据获取和传输。这大大提高了查询性能,同时也降低了对外部数据源的负载。 -2. 当 `enable_func_pushdown` 设置为true,会将 where 之后的函数条件也下推到外部数据源,目前仅支持 MySQL、ClickHouse、Oracle,如遇到 MySQL、ClickHouse、Oracle 不支持的函数,可以将此参数设置为 false,目前 Doris 会自动识别部分 MySQL 不支持的函数以及 CLickHouse、Oracle 支持的函数进行下推条件过滤,可通过 explain sql 查看。 +2. 当变量 `enable_ext_func_pred_pushdown` 设置为true,会将 where 之后的函数条件也下推到外部数据源,目前仅支持 MySQL、ClickHouse、Oracle,如遇到 MySQL、ClickHouse、Oracle 不支持的函数,可以将此参数设置为 false,目前 Doris 会自动识别部分 MySQL 不支持的函数以及 CLickHouse、Oracle 支持的函数进行下推条件过滤,可通过 explain sql 查看。 目前不会下推的函数有: diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index 909e1bf2d7ae72..2a8a2ed53adc8d 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -1921,13 +1921,6 @@ public class Config extends ConfigBase { @ConfField(mutable = false, varType = VariableAnnotation.EXPERIMENTAL) public static boolean enable_fqdn_mode = false; - /** - * This is used whether to push down function to MYSQL in external Table with query sql - * like odbc, jdbc for mysql table - */ - @ConfField(mutable = true) - public static boolean enable_func_pushdown = true; - /** * If set to true, doris will try to parse the ddl of a hive view and try to execute the query * otherwise it will throw an AnalysisException. diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/jdbc/JdbcScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/jdbc/JdbcScanNode.java index 774d4284bce2b0..d2731d33e137a3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/jdbc/JdbcScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/jdbc/JdbcScanNode.java @@ -37,7 +37,6 @@ import org.apache.doris.catalog.TableIf.TableType; import org.apache.doris.catalog.external.JdbcExternalTable; import org.apache.doris.common.AnalysisException; -import org.apache.doris.common.Config; import org.apache.doris.common.UserException; import org.apache.doris.nereids.glue.translator.PlanTranslatorContext; import org.apache.doris.planner.PlanNodeId; @@ -223,6 +222,7 @@ private String getJdbcQueryStr() { } if (jdbcType == TOdbcTableType.CLICKHOUSE + && ConnectContext.get() != null && ConnectContext.get().getSessionVariable().jdbcClickhouseQueryFinal) { sql.append(" SETTINGS final = 1"); } @@ -313,7 +313,11 @@ private static boolean shouldPushDownConjunct(TOdbcTableType tableType, Expr exp if (containsFunctionCallExpr(expr)) { if (tableType.equals(TOdbcTableType.MYSQL) || tableType.equals(TOdbcTableType.CLICKHOUSE) || tableType.equals(TOdbcTableType.ORACLE)) { - return Config.enable_func_pushdown; + if (ConnectContext.get() != null) { + return ConnectContext.get().getSessionVariable().enableExtFuncPredPushdown; + } else { + return true; + } } else { return false; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/odbc/OdbcScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/odbc/OdbcScanNode.java index f3ba2a5add81f1..891cd8240b0c98 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/odbc/OdbcScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/odbc/OdbcScanNode.java @@ -30,7 +30,6 @@ import org.apache.doris.catalog.JdbcTable; import org.apache.doris.catalog.OdbcTable; import org.apache.doris.common.AnalysisException; -import org.apache.doris.common.Config; import org.apache.doris.common.UserException; import org.apache.doris.nereids.glue.translator.PlanTranslatorContext; import org.apache.doris.planner.PlanNodeId; @@ -267,6 +266,10 @@ public static boolean shouldPushDownConjunct(TOdbcTableType tableType, Expr expr return false; } } - return Config.enable_func_pushdown; + if (ConnectContext.get() != null) { + return ConnectContext.get().getSessionVariable().enableExtFuncPredPushdown; + } else { + return true; + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 5c1a4120dc43d6..1a995801d40261 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -302,6 +302,8 @@ public class SessionVariable implements Serializable, Writable { public static final String ENABLE_FUNCTION_PUSHDOWN = "enable_function_pushdown"; + public static final String ENABLE_EXT_FUNC_PRED_PUSHDOWN = "enable_ext_func_pred_pushdown"; + public static final String ENABLE_COMMON_EXPR_PUSHDOWN = "enable_common_expr_pushdown"; public static final String FRAGMENT_TRANSMISSION_COMPRESSION_CODEC = "fragment_transmission_compression_codec"; @@ -524,7 +526,9 @@ public class SessionVariable implements Serializable, Writable { @VariableMgr.VarAttr(name = EXPAND_RUNTIME_FILTER_BY_INNER_JION) public boolean expandRuntimeFilterByInnerJoin = true; - @VariableMgr.VarAttr(name = JDBC_CLICKHOUSE_QUERY_FINAL) + @VariableMgr.VarAttr(name = JDBC_CLICKHOUSE_QUERY_FINAL, needForward = true, + description = {"是否在查询 ClickHouse JDBC 外部表时,对查询 SQL 添加 FINAL 关键字。", + "Whether to add the FINAL keyword to the query SQL when querying ClickHouse JDBC external tables."}) public boolean jdbcClickhouseQueryFinal = false; @VariableMgr.VarAttr(name = ROUND_PRECISE_DECIMALV2_VALUE) @@ -1090,6 +1094,12 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { @VariableMgr.VarAttr(name = ENABLE_FUNCTION_PUSHDOWN, fuzzy = true) public boolean enableFunctionPushdown = false; + @VariableMgr.VarAttr(name = ENABLE_EXT_FUNC_PRED_PUSHDOWN, needForward = true, + description = {"启用外部表(如通过ODBC或JDBC访问的表)查询中谓词的函数下推", + "Enable function pushdown for predicates in queries to external tables " + + "(such as tables accessed via ODBC or JDBC)"}) + public boolean enableExtFuncPredPushdown = true; + @VariableMgr.VarAttr(name = FORBID_UNKNOWN_COLUMN_STATS) public boolean forbidUnknownColStats = false; diff --git a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy index fdc1f84e382eb2..86afd749e56eae 100644 --- a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy +++ b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy @@ -186,6 +186,11 @@ suite("test_mysql_jdbc_catalog", "p0,external,mysql,external_docker,external_doc contains """ SELECT `datetime` FROM `doris_test`.`all_types` WHERE (date(`datetime`) = '2012-10-25')""" } + explain { + sql("select /*+ SET_VAR(enable_ext_func_pred_pushdown = false) */ `datetime` from all_types where to_date(`datetime`) = '2012-10-25';") + contains """SELECT `datetime` FROM `doris_test`.`all_types`""" + } + // test insert String uuid1 = UUID.randomUUID().toString(); connect(user=user, password="${pwd}", url=url) { @@ -378,7 +383,7 @@ suite("test_mysql_jdbc_catalog", "p0,external,mysql,external_docker,external_doc } try { sql """ use ${ex_db_name}""" - sql """ admin set frontend config ("enable_func_pushdown" = "true"); """ + sql """ set enable_ext_func_pred_pushdown = "true"; """ order_qt_filter1 """select * from ${ex_tb17} where id = 1; """ order_qt_filter2 """select * from ${ex_tb17} where 1=1 order by 1; """ order_qt_filter3 """select * from ${ex_tb17} where id = 1 and 1 = 1; """ @@ -409,13 +414,13 @@ suite("test_mysql_jdbc_catalog", "p0,external,mysql,external_docker,external_doc contains "QUERY: SELECT `k6`, `k8` FROM `doris_test`.`test1` WHERE (ifnull(ifnull(`k6`, NULL), NULL) = 1)" } - sql """ admin set frontend config ("enable_func_pushdown" = "false"); """ + sql """ set enable_ext_func_pred_pushdown = "false"; """ explain { sql ("select k6, k8 from test1 where nvl(k6, null) = 1 and k8 = 1;") contains "QUERY: SELECT `k6`, `k8` FROM `doris_test`.`test1` WHERE (`k8` = 1)" } - sql """ admin set frontend config ("enable_func_pushdown" = "true"); """ + sql """ set enable_ext_func_pred_pushdown = "true"; """ } finally { res_dbs_log = sql "show databases;" for(int i = 0;i < res_dbs_log.size();i++) { From 65661ce38e33ece79384d0b3dff29ba7152fb601 Mon Sep 17 00:00:00 2001 From: wangbo Date: Tue, 16 Jan 2024 21:24:30 +0800 Subject: [PATCH 046/200] [Feature](profile)Support active_queries TVF (#29999) --- be/src/runtime/fragment_mgr.cpp | 3 + be/src/runtime/memory/mem_tracker.h | 1 + be/src/runtime/query_statistics.cpp | 18 +++- be/src/runtime/query_statistics.h | 23 ++++- .../runtime/runtime_query_statistics_mgr.cpp | 9 ++ be/src/runtime/runtime_query_statistics_mgr.h | 4 + .../table-functions/active_queries.md | 83 ++++++++++++++++ .../sql-functions/table-functions/queries.md | 79 --------------- docs/sidebars.json | 2 +- .../table-functions/active_queries.md | 83 ++++++++++++++++ .../sql-functions/table-functions/queries.md | 79 --------------- .../catalog/BuiltinTableValuedFunctions.java | 4 +- .../doris/httpv2/rest/QueryDetailAction.java | 63 ------------ .../{Queries.java => ActiveQueries.java} | 10 +- .../visitor/TableValuedFunctionVisitor.java | 4 +- .../org/apache/doris/qe/AuditLogHelper.java | 12 --- .../java/org/apache/doris/qe/QeProcessor.java | 3 + .../org/apache/doris/qe/QeProcessorImpl.java | 13 +++ .../org/apache/doris/qe/StmtExecutor.java | 9 -- .../WorkloadRuntimeStatusMgr.java | 4 + ... => ActiveQueriesTableValuedFunction.java} | 29 +++--- .../tablefunction/MetadataGenerator.java | 96 +++++++++++++++---- .../MetadataTableValuedFunction.java | 2 +- .../tablefunction/TableValuedFunctionIf.java | 4 +- gensrc/thrift/FrontendService.thrift | 2 + .../tvf/queries/test_queries_tvf.groovy | 4 +- .../suites/nereids_function_p0/tvf/tvf.groovy | 2 +- 27 files changed, 348 insertions(+), 297 deletions(-) create mode 100644 docs/en/docs/sql-manual/sql-functions/table-functions/active_queries.md delete mode 100644 docs/en/docs/sql-manual/sql-functions/table-functions/queries.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/table-functions/active_queries.md delete mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/table-functions/queries.md delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/QueryDetailAction.java rename fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/{Queries.java => ActiveQueries.java} (87%) rename fe/fe-core/src/main/java/org/apache/doris/tablefunction/{QueriesTableValuedFunction.java => ActiveQueriesTableValuedFunction.java} (71%) diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index f0282e9d7c1b6f..7d3dba810205d9 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -65,6 +65,7 @@ #include "runtime/primitive_type.h" #include "runtime/query_context.h" #include "runtime/runtime_filter_mgr.h" +#include "runtime/runtime_query_statistics_mgr.h" #include "runtime/runtime_state.h" #include "runtime/stream_load/new_load_stream_mgr.h" #include "runtime/stream_load/stream_load_context.h" @@ -707,6 +708,8 @@ Status FragmentMgr::_get_query_ctx(const Params& params, TUniqueId query_id, boo query_ctx->use_task_group_for_cpu_limit.store(true); } LOG(INFO) << ss.str(); + _exec_env->runtime_query_statistics_mgr()->set_workload_group_id( + print_id(query_id), tg_id); } else { VLOG_DEBUG << "Query/load id: " << print_id(query_ctx->query_id()) << " no task group found, does not use task group."; diff --git a/be/src/runtime/memory/mem_tracker.h b/be/src/runtime/memory/mem_tracker.h index 9ce12f1532f6eb..de7628c17490bd 100644 --- a/be/src/runtime/memory/mem_tracker.h +++ b/be/src/runtime/memory/mem_tracker.h @@ -141,6 +141,7 @@ class MemTracker { _consumption->add(bytes); if (_query_statistics) { _query_statistics->set_max_peak_memory_bytes(_consumption->peak_value()); + _query_statistics->set_current_used_memory_bytes(_consumption->current_value()); } } diff --git a/be/src/runtime/query_statistics.cpp b/be/src/runtime/query_statistics.cpp index f71bad24e81e45..ab49b02ad4320b 100644 --- a/be/src/runtime/query_statistics.cpp +++ b/be/src/runtime/query_statistics.cpp @@ -44,9 +44,17 @@ void QueryStatistics::merge(const QueryStatistics& other) { scan_bytes += other.scan_bytes; int64_t other_cpu_time = other.cpu_nanos.load(std::memory_order_relaxed); cpu_nanos += other_cpu_time; - if (other.max_peak_memory_bytes > this->max_peak_memory_bytes) { - this->max_peak_memory_bytes = other.max_peak_memory_bytes.load(std::memory_order_relaxed); + + int64_t other_peak_mem = other.max_peak_memory_bytes.load(std::memory_order_relaxed); + if (other_peak_mem > this->max_peak_memory_bytes) { + this->max_peak_memory_bytes = other_peak_mem; + } + + int64_t other_memory_used = other.current_used_memory_bytes.load(std::memory_order_relaxed); + if (other_memory_used > 0) { + this->current_used_memory_bytes = other_memory_used; } + for (auto& other_node_statistics : other._nodes_statistics_map) { int64_t node_id = other_node_statistics.first; auto node_statistics = add_nodes_statistics(node_id); @@ -70,11 +78,13 @@ void QueryStatistics::to_pb(PQueryStatistics* statistics) { void QueryStatistics::to_thrift(TQueryStatistics* statistics) const { DCHECK(statistics != nullptr); - statistics->__set_scan_bytes(scan_bytes); - statistics->__set_scan_rows(scan_rows); + statistics->__set_scan_bytes(scan_bytes.load(std::memory_order_relaxed)); + statistics->__set_scan_rows(scan_rows.load(std::memory_order_relaxed)); statistics->__set_cpu_ms(cpu_nanos.load(std::memory_order_relaxed) / NANOS_PER_MILLIS); statistics->__set_returned_rows(returned_rows); statistics->__set_max_peak_memory_bytes(max_peak_memory_bytes.load(std::memory_order_relaxed)); + statistics->__set_current_used_memory_bytes( + current_used_memory_bytes.load(std::memory_order_relaxed)); } void QueryStatistics::from_pb(const PQueryStatistics& statistics) { diff --git a/be/src/runtime/query_statistics.h b/be/src/runtime/query_statistics.h index 7f11daf6ec3959..abaf0a251a8245 100644 --- a/be/src/runtime/query_statistics.h +++ b/be/src/runtime/query_statistics.h @@ -64,14 +64,19 @@ class QueryStatistics { scan_bytes(0), cpu_nanos(0), returned_rows(0), - max_peak_memory_bytes(0) {} + max_peak_memory_bytes(0), + current_used_memory_bytes(0) {} virtual ~QueryStatistics(); void merge(const QueryStatistics& other); - void add_scan_rows(int64_t scan_rows) { this->scan_rows += scan_rows; } + void add_scan_rows(int64_t delta_scan_rows) { + this->scan_rows.fetch_add(delta_scan_rows, std::memory_order_relaxed); + } - void add_scan_bytes(int64_t scan_bytes) { this->scan_bytes += scan_bytes; } + void add_scan_bytes(int64_t delta_scan_bytes) { + this->scan_bytes.fetch_add(delta_scan_bytes, std::memory_order_relaxed); + } void add_cpu_nanos(int64_t delta_cpu_time) { this->cpu_nanos.fetch_add(delta_cpu_time, std::memory_order_relaxed); @@ -95,6 +100,10 @@ class QueryStatistics { this->max_peak_memory_bytes.store(max_peak_memory_bytes, std::memory_order_relaxed); } + void set_current_used_memory_bytes(int64_t current_used_memory) { + this->current_used_memory_bytes.store(current_used_memory, std::memory_order_relaxed); + } + void merge(QueryStatisticsRecvr* recvr); void merge(QueryStatisticsRecvr* recvr, int sender_id); @@ -121,8 +130,11 @@ class QueryStatistics { bool collected() const { return _collected; } void set_collected() { _collected = true; } - int64_t get_scan_rows() { return scan_rows.load(); } - int64_t get_scan_bytes() { return scan_bytes.load(); } + int64_t get_scan_rows() { return scan_rows.load(std::memory_order_relaxed); } + int64_t get_scan_bytes() { return scan_bytes.load(std::memory_order_relaxed); } + int64_t get_current_used_memory_bytes() { + return current_used_memory_bytes.load(std::memory_order_relaxed); + } private: friend class QueryStatisticsRecvr; @@ -139,6 +151,7 @@ class QueryStatistics { using NodeStatisticsMap = std::unordered_map; NodeStatisticsMap _nodes_statistics_map; bool _collected = false; + std::atomic current_used_memory_bytes; }; using QueryStatisticsPtr = std::shared_ptr; // It is used for collecting sub plan query statistics in DataStreamRecvr. diff --git a/be/src/runtime/runtime_query_statistics_mgr.cpp b/be/src/runtime/runtime_query_statistics_mgr.cpp index 6df9c0b858dd13..a658e527f61a6f 100644 --- a/be/src/runtime/runtime_query_statistics_mgr.cpp +++ b/be/src/runtime/runtime_query_statistics_mgr.cpp @@ -30,6 +30,7 @@ void QueryStatisticsCtx::collect_query_statistics(TQueryStatistics* tq_s) { tmp_qs.merge(*qs_ptr); } tmp_qs.to_thrift(tq_s); + tq_s->__set_workload_group_id(_wg_id); } void RuntimeQueryStatiticsMgr::register_query_statistics(std::string query_id, @@ -171,4 +172,12 @@ std::shared_ptr RuntimeQueryStatiticsMgr::get_runtime_query_sta return qs_ptr; } +void RuntimeQueryStatiticsMgr::set_workload_group_id(std::string query_id, int64_t wg_id) { + // wg id just need eventual consistency, read lock is ok + std::shared_lock read_lock(_qs_ctx_map_lock); + if (_query_statistics_ctx_map.find(query_id) != _query_statistics_ctx_map.end()) { + _query_statistics_ctx_map.at(query_id)->_wg_id = wg_id; + } +} + } // namespace doris \ No newline at end of file diff --git a/be/src/runtime/runtime_query_statistics_mgr.h b/be/src/runtime/runtime_query_statistics_mgr.h index 6f1ea11a61add8..98d4f554728ec5 100644 --- a/be/src/runtime/runtime_query_statistics_mgr.h +++ b/be/src/runtime/runtime_query_statistics_mgr.h @@ -28,6 +28,7 @@ class QueryStatisticsCtx { public: QueryStatisticsCtx(TNetworkAddress fe_addr) : _fe_addr(fe_addr) { this->_is_query_finished = false; + this->_wg_id = -1; } ~QueryStatisticsCtx() = default; @@ -38,6 +39,7 @@ class QueryStatisticsCtx { bool _is_query_finished; TNetworkAddress _fe_addr; int64_t _query_finish_time; + int64_t _wg_id; }; class RuntimeQueryStatiticsMgr { @@ -54,6 +56,8 @@ class RuntimeQueryStatiticsMgr { std::shared_ptr get_runtime_query_statistics(std::string query_id); + void set_workload_group_id(std::string query_id, int64_t wg_id); + private: std::shared_mutex _qs_ctx_map_lock; std::map> _query_statistics_ctx_map; diff --git a/docs/en/docs/sql-manual/sql-functions/table-functions/active_queries.md b/docs/en/docs/sql-manual/sql-functions/table-functions/active_queries.md new file mode 100644 index 00000000000000..35a71b5eb60bce --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/table-functions/active_queries.md @@ -0,0 +1,83 @@ +--- +{ + "title": "ACTIVE_QUERIES", + "language": "en" +} +--- + + + +## `active_queries` + +### Name + + + +active_queries + + + +### description + +Table-Value-Function, generate a temporary table named active_queries. This tvf is used to view the information of running queries in doris cluster. + +This function is used in FROM clauses. + +#### syntax +`active_queries()` + +active_queries() table schema: +``` +mysql [(none)]> desc function active_queries(); ++------------------------+--------+------+-------+---------+-------+ +| Field | Type | Null | Key | Default | Extra | ++------------------------+--------+------+-------+---------+-------+ +| BeHost | TEXT | No | false | NULL | NONE | +| BePort | BIGINT | No | false | NULL | NONE | +| QueryId | TEXT | No | false | NULL | NONE | +| StartTime | TEXT | No | false | NULL | NONE | +| QueryTimeMs | BIGINT | No | false | NULL | NONE | +| WorkloadGroupId | BIGINT | No | false | NULL | NONE | +| QueryCpuTimeMs | BIGINT | No | false | NULL | NONE | +| ScanRows | BIGINT | No | false | NULL | NONE | +| ScanBytes | BIGINT | No | false | NULL | NONE | +| BePeakMemoryBytes | BIGINT | No | false | NULL | NONE | +| CurrentUsedMemoryBytes | BIGINT | No | false | NULL | NONE | +| Database | TEXT | No | false | NULL | NONE | +| FrontendInstance | TEXT | No | false | NULL | NONE | +| Sql | TEXT | No | false | NULL | NONE | ++------------------------+--------+------+-------+---------+-------+ +14 rows in set (0.00 sec) +``` + +### example +``` +mysql [(none)]>select * from active_queries(); ++------------+--------+----------------------------------+---------------------+-------------+-----------------+----------------+----------+------------+-------------------+------------------------+----------+------------------+-------+ +| BeHost | BePort | QueryId | StartTime | QueryTimeMs | WorkloadGroupId | QueryCpuTimeMs | ScanRows | ScanBytes | BePeakMemoryBytes | CurrentUsedMemoryBytes | Database | FrontendInstance | Sql | ++------------+--------+----------------------------------+---------------------+-------------+-----------------+----------------+----------+------------+-------------------+------------------------+----------+------------------+-------+ +| 127.0.0.1 | 6090 | 71fd11b7b0e438c-bc98434b97b8cb98 | 2024-01-16 16:21:15 | 7260 | 10002 | 8392 | 16082249 | 4941889536 | 360470040 | 360420915 | hits | localhost | SELECT xxxx | ++------------+--------+----------------------------------+---------------------+-------------+-----------------+----------------+----------+------------+-------------------+------------------------+----------+------------------+-------+ +1 row in set (0.01 sec) +``` + +### keywords + + active_queries diff --git a/docs/en/docs/sql-manual/sql-functions/table-functions/queries.md b/docs/en/docs/sql-manual/sql-functions/table-functions/queries.md deleted file mode 100644 index ebc2cb3ebe4de1..00000000000000 --- a/docs/en/docs/sql-manual/sql-functions/table-functions/queries.md +++ /dev/null @@ -1,79 +0,0 @@ ---- -{ - "title": "QUERIES", - "language": "en" -} ---- - - - -## `queries` - -### Name - - - -queries - - - -### description - -Table-Value-Function, generate a temporary table named queries. This tvf is used to view the information of running queries and history queries in doris cluster. - -This function is used in FROM clauses. - -#### syntax -`queries()` - -queries() table schema: -``` -mysql> desc function queries(); -+------------------+--------+------+-------+---------+-------+ -| Field | Type | Null | Key | Default | Extra | -+------------------+--------+------+-------+---------+-------+ -| QueryId | TEXT | No | false | NULL | NONE | -| StartTime | BIGINT | No | false | NULL | NONE | -| EndTime | BIGINT | No | false | NULL | NONE | -| EventTime | BIGINT | No | false | NULL | NONE | -| Latency | BIGINT | No | false | NULL | NONE | -| State | TEXT | No | false | NULL | NONE | -| Database | TEXT | No | false | NULL | NONE | -| Sql | TEXT | No | false | NULL | NONE | -| FrontendInstance | TEXT | No | false | NULL | NONE | -+------------------+--------+------+-------+---------+-------+ -9 rows in set (0.00 sec) -``` - -### example -``` -mysql> select* from queries(); -+-----------------------------------+---------------+---------------+---------------+---------+----------+----------+------------------------+------------------+ -| QueryId | StartTime | EndTime | EventTime | Latency | State | Database | Sql | FrontendInstance | -+-----------------------------------+---------------+---------------+---------------+---------+----------+----------+------------------------+------------------+ -| e1293f2ed2a5427a-982301c462586043 | 1699255138730 | 1699255139823 | 1699255139823 | 1093 | FINISHED | demo | select* from queries() | localhost | -| 46fa3ad0e7814ebd-b1cd34940a29b1e9 | 1699255143588 | -1 | 1699255143588 | 20 | RUNNING | demo | select* from queries() | localhost | -+-----------------------------------+---------------+---------------+---------------+---------+----------+----------+------------------------+------------------+ -2 rows in set (0.04 sec) -``` - -### keywords - - queries diff --git a/docs/sidebars.json b/docs/sidebars.json index 5d6a34e2a0a466..42b442c597ebb4 100644 --- a/docs/sidebars.json +++ b/docs/sidebars.json @@ -771,7 +771,7 @@ "sql-manual/sql-functions/table-functions/workload-group", "sql-manual/sql-functions/table-functions/catalogs", "sql-manual/sql-functions/table-functions/frontends_disks", - "sql-manual/sql-functions/table-functions/queries", + "sql-manual/sql-functions/table-functions/active_queries", "sql-manual/sql-functions/table-functions/jobs", "sql-manual/sql-functions/table-functions/mv_infos", "sql-manual/sql-functions/table-functions/tasks" diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/active_queries.md b/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/active_queries.md new file mode 100644 index 00000000000000..bdae08285f285c --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/active_queries.md @@ -0,0 +1,83 @@ +--- +{ + "title": "ACTIVE_QUERIES", + "language": "zh-CN" +} +--- + + + +## `active_queries` + +### Name + + + +active_queries + + + +### description + +表函数,生成active_queries临时表,可以查看当前doris集群中正在运行的 query 信息。 + +该函数用于from子句中。 + +#### syntax +`active_queries()` + +active_queries()表结构: +``` +mysql [(none)]> desc function active_queries(); ++------------------------+--------+------+-------+---------+-------+ +| Field | Type | Null | Key | Default | Extra | ++------------------------+--------+------+-------+---------+-------+ +| BeHost | TEXT | No | false | NULL | NONE | +| BePort | BIGINT | No | false | NULL | NONE | +| QueryId | TEXT | No | false | NULL | NONE | +| StartTime | TEXT | No | false | NULL | NONE | +| QueryTimeMs | BIGINT | No | false | NULL | NONE | +| WorkloadGroupId | BIGINT | No | false | NULL | NONE | +| QueryCpuTimeMs | BIGINT | No | false | NULL | NONE | +| ScanRows | BIGINT | No | false | NULL | NONE | +| ScanBytes | BIGINT | No | false | NULL | NONE | +| BePeakMemoryBytes | BIGINT | No | false | NULL | NONE | +| CurrentUsedMemoryBytes | BIGINT | No | false | NULL | NONE | +| Database | TEXT | No | false | NULL | NONE | +| FrontendInstance | TEXT | No | false | NULL | NONE | +| Sql | TEXT | No | false | NULL | NONE | ++------------------------+--------+------+-------+---------+-------+ +14 rows in set (0.00 sec) +``` + +### example +``` +mysql [(none)]>select * from active_queries(); ++------------+--------+----------------------------------+---------------------+-------------+-----------------+----------------+----------+------------+-------------------+------------------------+----------+------------------+-------+ +| BeHost | BePort | QueryId | StartTime | QueryTimeMs | WorkloadGroupId | QueryCpuTimeMs | ScanRows | ScanBytes | BePeakMemoryBytes | CurrentUsedMemoryBytes | Database | FrontendInstance | Sql | ++------------+--------+----------------------------------+---------------------+-------------+-----------------+----------------+----------+------------+-------------------+------------------------+----------+------------------+-------+ +| 127.0.0.1 | 6090 | 71fd11b7b0e438c-bc98434b97b8cb98 | 2024-01-16 16:21:15 | 7260 | 10002 | 8392 | 16082249 | 4941889536 | 360470040 | 360420915 | hits | localhost | SELECT xxxx | ++------------+--------+----------------------------------+---------------------+-------------+-----------------+----------------+----------+------------+-------------------+------------------------+----------+------------------+-------+ +1 row in set (0.01 sec) +``` + +### keywords + + active_queries diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/queries.md b/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/queries.md deleted file mode 100644 index e3f22da7ad522a..00000000000000 --- a/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/queries.md +++ /dev/null @@ -1,79 +0,0 @@ ---- -{ - "title": "QUERIES", - "language": "zh-CN" -} ---- - - - -## `queries` - -### Name - - - -queries - - - -### description - -表函数,生成queries临时表,可以查看当前doris集群中正在运行的以及历史的 query 信息。 - -该函数用于from子句中。 - -#### syntax -`queries()` - -queries()表结构: -``` -mysql> desc function queries(); -+------------------+--------+------+-------+---------+-------+ -| Field | Type | Null | Key | Default | Extra | -+------------------+--------+------+-------+---------+-------+ -| QueryId | TEXT | No | false | NULL | NONE | -| StartTime | BIGINT | No | false | NULL | NONE | -| EndTime | BIGINT | No | false | NULL | NONE | -| EventTime | BIGINT | No | false | NULL | NONE | -| Latency | BIGINT | No | false | NULL | NONE | -| State | TEXT | No | false | NULL | NONE | -| Database | TEXT | No | false | NULL | NONE | -| Sql | TEXT | No | false | NULL | NONE | -| FrontendInstance | TEXT | No | false | NULL | NONE | -+------------------+--------+------+-------+---------+-------+ -9 rows in set (0.00 sec) -``` - -### example -``` -mysql> select* from queries(); -+-----------------------------------+---------------+---------------+---------------+---------+----------+----------+------------------------+------------------+ -| QueryId | StartTime | EndTime | EventTime | Latency | State | Database | Sql | FrontendInstance | -+-----------------------------------+---------------+---------------+---------------+---------+----------+----------+------------------------+------------------+ -| e1293f2ed2a5427a-982301c462586043 | 1699255138730 | 1699255139823 | 1699255139823 | 1093 | FINISHED | demo | select* from queries() | localhost | -| 46fa3ad0e7814ebd-b1cd34940a29b1e9 | 1699255143588 | -1 | 1699255143588 | 20 | RUNNING | demo | select* from queries() | localhost | -+-----------------------------------+---------------+---------------+---------------+---------+----------+----------+------------------------+------------------+ -2 rows in set (0.04 sec) -``` - -### keywords - - queries diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinTableValuedFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinTableValuedFunctions.java index d1b63fe6237019..b45847088ddf77 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinTableValuedFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinTableValuedFunctions.java @@ -17,6 +17,7 @@ package org.apache.doris.catalog; +import org.apache.doris.nereids.trees.expressions.functions.table.ActiveQueries; import org.apache.doris.nereids.trees.expressions.functions.table.Backends; import org.apache.doris.nereids.trees.expressions.functions.table.Catalogs; import org.apache.doris.nereids.trees.expressions.functions.table.Frontends; @@ -29,7 +30,6 @@ import org.apache.doris.nereids.trees.expressions.functions.table.Local; import org.apache.doris.nereids.trees.expressions.functions.table.MvInfos; import org.apache.doris.nereids.trees.expressions.functions.table.Numbers; -import org.apache.doris.nereids.trees.expressions.functions.table.Queries; import org.apache.doris.nereids.trees.expressions.functions.table.S3; import org.apache.doris.nereids.trees.expressions.functions.table.Tasks; import org.apache.doris.nereids.trees.expressions.functions.table.WorkloadGroups; @@ -54,7 +54,7 @@ public class BuiltinTableValuedFunctions implements FunctionHelper { tableValued(Hdfs.class, "hdfs"), tableValued(HttpStream.class, "http_stream"), tableValued(Numbers.class, "numbers"), - tableValued(Queries.class, "queries"), + tableValued(ActiveQueries.class, "active_queries"), tableValued(S3.class, "s3"), tableValued(MvInfos.class, "mv_infos"), tableValued(Jobs.class, "jobs"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/QueryDetailAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/QueryDetailAction.java deleted file mode 100644 index 508f8aade70ef6..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/QueryDetailAction.java +++ /dev/null @@ -1,63 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you 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 org.apache.doris.httpv2.rest; - -import org.apache.doris.httpv2.entity.ResponseEntityBuilder; -import org.apache.doris.mysql.privilege.PrivPredicate; -import org.apache.doris.qe.ConnectContext; -import org.apache.doris.qe.QueryDetail; -import org.apache.doris.qe.QueryDetailQueue; - -import com.google.common.base.Strings; -import com.google.common.collect.Maps; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.springframework.web.bind.annotation.RequestMapping; -import org.springframework.web.bind.annotation.RequestMethod; -import org.springframework.web.bind.annotation.RestController; - -import java.util.List; -import java.util.Map; -import javax.servlet.http.HttpServletRequest; -import javax.servlet.http.HttpServletResponse; - -// This class is used to get current query_id of connection_id. -// Every connection holds at most one query at every point. -// Some we can get query_id firstly, and get query by query_id. -@RestController -public class QueryDetailAction extends RestBaseController { - private static final Logger LOG = LogManager.getLogger(QueryDetailAction.class); - - @RequestMapping(path = "/api/query_detail", method = RequestMethod.GET) - protected Object query_detail(HttpServletRequest request, HttpServletResponse response) { - executeCheckPassword(request, response); - checkGlobalAuth(ConnectContext.get().getCurrentUserIdentity(), PrivPredicate.ADMIN); - - String eventTimeStr = request.getParameter("event_time"); - if (Strings.isNullOrEmpty(eventTimeStr)) { - return ResponseEntityBuilder.badRequest("Missing event_time"); - } - - long eventTime = Long.valueOf(eventTimeStr.trim()); - List queryDetails = QueryDetailQueue.getQueryDetails(eventTime); - - Map> result = Maps.newHashMap(); - result.put("query_details", queryDetails); - return ResponseEntityBuilder.ok(result); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/Queries.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/ActiveQueries.java similarity index 87% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/Queries.java rename to fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/ActiveQueries.java index dd90853bf7ccad..f8dcaa4a7eca6a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/Queries.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/ActiveQueries.java @@ -22,7 +22,7 @@ import org.apache.doris.nereids.trees.expressions.Properties; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.coercion.AnyDataType; -import org.apache.doris.tablefunction.QueriesTableValuedFunction; +import org.apache.doris.tablefunction.ActiveQueriesTableValuedFunction; import org.apache.doris.tablefunction.TableValuedFunctionIf; import java.util.Map; @@ -30,10 +30,10 @@ /** * queries tvf */ -public class Queries extends TableValuedFunction { +public class ActiveQueries extends TableValuedFunction { - public Queries(Properties properties) { - super("queries", properties); + public ActiveQueries(Properties properties) { + super("active_queries", properties); } @Override @@ -45,7 +45,7 @@ public FunctionSignature customSignature() { protected TableValuedFunctionIf toCatalogFunction() { try { Map arguments = getTVFProperties().getMap(); - return new QueriesTableValuedFunction(arguments); + return new ActiveQueriesTableValuedFunction(arguments); } catch (Throwable t) { throw new AnalysisException("Can not build FrontendsTableValuedFunction by " + this + ": " + t.getMessage(), t); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/TableValuedFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/TableValuedFunctionVisitor.java index fe07097bc81fca..fba34d48168da9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/TableValuedFunctionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/TableValuedFunctionVisitor.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.expressions.visitor; +import org.apache.doris.nereids.trees.expressions.functions.table.ActiveQueries; import org.apache.doris.nereids.trees.expressions.functions.table.Backends; import org.apache.doris.nereids.trees.expressions.functions.table.Catalogs; import org.apache.doris.nereids.trees.expressions.functions.table.Frontends; @@ -29,7 +30,6 @@ import org.apache.doris.nereids.trees.expressions.functions.table.Local; import org.apache.doris.nereids.trees.expressions.functions.table.MvInfos; import org.apache.doris.nereids.trees.expressions.functions.table.Numbers; -import org.apache.doris.nereids.trees.expressions.functions.table.Queries; import org.apache.doris.nereids.trees.expressions.functions.table.S3; import org.apache.doris.nereids.trees.expressions.functions.table.TableValuedFunction; import org.apache.doris.nereids.trees.expressions.functions.table.Tasks; @@ -91,7 +91,7 @@ default R visitNumbers(Numbers numbers, C context) { return visitTableValuedFunction(numbers, context); } - default R visitQueries(Queries queries, C context) { + default R visitQueries(ActiveQueries queries, C context) { return visitTableValuedFunction(queries, context); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java b/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java index 7b6e86ca3a4c94..9311b4ca8e85ff 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java @@ -89,18 +89,6 @@ public static void logAuditLog(ConnectContext ctx, String origStmt, StatementBas } } auditEventBuilder.setIsQuery(true); - if (ctx.getQueryDetail() != null) { - ctx.getQueryDetail().setEventTime(endTime); - ctx.getQueryDetail().setEndTime(endTime); - ctx.getQueryDetail().setLatency(elapseMs); - if (ctx.isKilled()) { - ctx.getQueryDetail().setState(QueryDetail.QueryMemState.CANCELLED); - } else { - ctx.getQueryDetail().setState(QueryDetail.QueryMemState.FINISHED); - } - QueryDetailQueue.addOrUpdateQueryDetail(ctx.getQueryDetail()); - ctx.setQueryDetail(null); - } } else { auditEventBuilder.setIsQuery(false); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/QeProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/QeProcessor.java index a2a23488cf4108..44999ecef64caf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/QeProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/QeProcessor.java @@ -18,6 +18,7 @@ package org.apache.doris.qe; import org.apache.doris.common.UserException; +import org.apache.doris.qe.QeProcessorImpl.QueryInfo; import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TReportExecStatusParams; import org.apache.doris.thrift.TReportExecStatusResult; @@ -45,4 +46,6 @@ public interface QeProcessor { Coordinator getCoordinator(TUniqueId queryId); List getAllCoordinators(); + + Map getQueryInfoMap(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/QeProcessorImpl.java b/fe/fe-core/src/main/java/org/apache/doris/qe/QeProcessorImpl.java index b6d902b76cc61d..97a0a95d24e84c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/QeProcessorImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/QeProcessorImpl.java @@ -40,6 +40,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicInteger; @@ -246,6 +247,18 @@ public String getCurrentQueryByQueryId(TUniqueId queryId) { return ""; } + public Map getQueryInfoMap() { + Map retQueryInfoMap = Maps.newHashMap(); + Set queryIdSet = coordinatorMap.keySet(); + for (TUniqueId qid : queryIdSet) { + QueryInfo queryInfo = coordinatorMap.get(qid); + if (queryInfo != null) { + retQueryInfoMap.put(DebugUtil.printId(qid), queryInfo); + } + } + return retQueryInfoMap; + } + public static final class QueryInfo { private final ConnectContext connectContext; private final Coordinator coord; diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java index 9c234698865f74..767d03f43d7f44 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -1447,15 +1447,6 @@ private void handleQueryStmt() throws Exception { Queriable queryStmt = (Queriable) parsedStmt; - QueryDetail queryDetail = new QueryDetail(context.getStartTime(), - DebugUtil.printId(context.queryId()), - context.getStartTime(), -1, -1, - QueryDetail.QueryMemState.RUNNING, - context.getDatabase(), - originStmt.originStmt); - context.setQueryDetail(queryDetail); - QueryDetailQueue.addOrUpdateQueryDetail(queryDetail); - if (queryStmt.isExplain()) { String explainString = planner.getExplainString(queryStmt.getExplainOptions()); handleExplainStmt(explainString, false); diff --git a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadschedpolicy/WorkloadRuntimeStatusMgr.java b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadschedpolicy/WorkloadRuntimeStatusMgr.java index 7fac98ca73aeff..3c5d7fc8bf1e73 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadschedpolicy/WorkloadRuntimeStatusMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadschedpolicy/WorkloadRuntimeStatusMgr.java @@ -167,6 +167,10 @@ public Map getQueryStatisticsMap() { return retQueryMap; } + public Map> getBeQueryStatsMap() { + return beToQueryStatsMap; + } + private void mergeQueryStatistics(TQueryStatistics dst, TQueryStatistics src) { dst.scan_rows += src.scan_rows; dst.scan_bytes += src.scan_bytes; diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/QueriesTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ActiveQueriesTableValuedFunction.java similarity index 71% rename from fe/fe-core/src/main/java/org/apache/doris/tablefunction/QueriesTableValuedFunction.java rename to fe/fe-core/src/main/java/org/apache/doris/tablefunction/ActiveQueriesTableValuedFunction.java index e6004cfb62c6fe..0839ae56a676a0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/QueriesTableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ActiveQueriesTableValuedFunction.java @@ -31,19 +31,24 @@ import java.util.List; import java.util.Map; -public class QueriesTableValuedFunction extends MetadataTableValuedFunction { - public static final String NAME = "queries"; +public class ActiveQueriesTableValuedFunction extends MetadataTableValuedFunction { + public static final String NAME = "active_queries"; private static final ImmutableList SCHEMA = ImmutableList.of( - new Column("QueryId", ScalarType.createStringType()), - new Column("StartTime", PrimitiveType.BIGINT), - new Column("EndTime", PrimitiveType.BIGINT), - new Column("EventTime", PrimitiveType.BIGINT), - new Column("Latency", PrimitiveType.BIGINT), - new Column("State", ScalarType.createStringType()), - new Column("Database", ScalarType.createStringType()), - new Column("Sql", ScalarType.createStringType()), - new Column("FrontendInstance", ScalarType.createStringType())); + new Column("BeHost", ScalarType.createStringType()), + new Column("BePort", PrimitiveType.BIGINT), + new Column("QueryId", ScalarType.createStringType()), + new Column("StartTime", ScalarType.createStringType()), + new Column("QueryTimeMs", PrimitiveType.BIGINT), + new Column("WorkloadGroupId", PrimitiveType.BIGINT), + new Column("QueryCpuTimeMs", PrimitiveType.BIGINT), + new Column("ScanRows", PrimitiveType.BIGINT), + new Column("ScanBytes", PrimitiveType.BIGINT), + new Column("BePeakMemoryBytes", PrimitiveType.BIGINT), + new Column("CurrentUsedMemoryBytes", PrimitiveType.BIGINT), + new Column("Database", ScalarType.createStringType()), + new Column("FrontendInstance", ScalarType.createStringType()), + new Column("Sql", ScalarType.createStringType())); private static final ImmutableMap COLUMN_TO_INDEX; @@ -59,7 +64,7 @@ public static Integer getColumnIndexFromColumnName(String columnName) { return COLUMN_TO_INDEX.get(columnName.toLowerCase()); } - public QueriesTableValuedFunction(Map params) throws AnalysisException { + public ActiveQueriesTableValuedFunction(Map params) throws AnalysisException { if (params.size() != 0) { throw new AnalysisException("Queries table-valued-function does not support any params"); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java index 2d0c7ec66e4358..e4768660698b45 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java @@ -37,8 +37,8 @@ import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.planner.external.iceberg.IcebergMetadataCache; import org.apache.doris.qe.ConnectContext; -import org.apache.doris.qe.QueryDetail; -import org.apache.doris.qe.QueryDetailQueue; +import org.apache.doris.qe.QeProcessorImpl; +import org.apache.doris.qe.QeProcessorImpl.QueryInfo; import org.apache.doris.system.Backend; import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.FrontendService; @@ -54,6 +54,7 @@ import org.apache.doris.thrift.TMetadataType; import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TQueriesMetadataParams; +import org.apache.doris.thrift.TQueryStatistics; import org.apache.doris.thrift.TRow; import org.apache.doris.thrift.TStatus; import org.apache.doris.thrift.TStatusCode; @@ -70,11 +71,14 @@ import org.apache.thrift.TException; import org.jetbrains.annotations.NotNull; +import java.text.SimpleDateFormat; import java.time.Instant; import java.time.LocalDateTime; import java.util.ArrayList; +import java.util.Date; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.TimeUnit; public class MetadataGenerator { @@ -414,6 +418,49 @@ private static TFetchSchemaTableDataResult workloadSchedPolicyMetadataResult(TMe return result; } + private static TRow makeQueryStatisticsTRow(SimpleDateFormat sdf, String queryId, Backend be, + String selfNode, QueryInfo queryInfo, TQueryStatistics qs) { + TRow trow = new TRow(); + if (be != null) { + trow.addToColumnValue(new TCell().setStringVal(be.getHost())); + trow.addToColumnValue(new TCell().setLongVal(be.getBePort())); + } else { + trow.addToColumnValue(new TCell().setStringVal("invalid host")); + trow.addToColumnValue(new TCell().setLongVal(-1)); + } + trow.addToColumnValue(new TCell().setStringVal(queryId)); + + String strDate = sdf.format(new Date(queryInfo.getStartExecTime())); + trow.addToColumnValue(new TCell().setStringVal(strDate)); + trow.addToColumnValue(new TCell().setLongVal(System.currentTimeMillis() - queryInfo.getStartExecTime())); + + if (qs != null) { + trow.addToColumnValue(new TCell().setLongVal(qs.workload_group_id)); + trow.addToColumnValue(new TCell().setLongVal(qs.cpu_ms)); + trow.addToColumnValue(new TCell().setLongVal(qs.scan_rows)); + trow.addToColumnValue(new TCell().setLongVal(qs.scan_bytes)); + trow.addToColumnValue(new TCell().setLongVal(qs.max_peak_memory_bytes)); + trow.addToColumnValue(new TCell().setLongVal(qs.current_used_memory_bytes)); + } else { + trow.addToColumnValue(new TCell().setLongVal(0L)); + trow.addToColumnValue(new TCell().setLongVal(0L)); + trow.addToColumnValue(new TCell().setLongVal(0L)); + trow.addToColumnValue(new TCell().setLongVal(0L)); + trow.addToColumnValue(new TCell().setLongVal(0L)); + trow.addToColumnValue(new TCell().setLongVal(0L)); + } + + if (queryInfo.getConnectContext() != null) { + trow.addToColumnValue(new TCell().setStringVal(queryInfo.getConnectContext().getDatabase())); + } else { + trow.addToColumnValue(new TCell().setStringVal("")); + } + trow.addToColumnValue(new TCell().setStringVal(selfNode)); + trow.addToColumnValue(new TCell().setStringVal(queryInfo.getSql())); + + return trow; + } + private static TFetchSchemaTableDataResult queriesMetadataResult(TMetadataTableRequestParams params, TFetchSchemaTableDataRequest parentRequest) { if (!params.isSetQueriesMetadataParams()) { @@ -429,24 +476,37 @@ private static TFetchSchemaTableDataResult queriesMetadataResult(TMetadataTableR } selfNode = NetUtils.getHostnameByIp(selfNode); + // get query + Map> beQsMap = Env.getCurrentEnv().getWorkloadRuntimeStatusMgr() + .getBeQueryStatsMap(); + Set beIdSet = beQsMap.keySet(); + List dataBatch = Lists.newArrayList(); - List queries = QueryDetailQueue.getQueryDetails(0L); - for (QueryDetail query : queries) { - TRow trow = new TRow(); - trow.addToColumnValue(new TCell().setStringVal(query.getQueryId())); - trow.addToColumnValue(new TCell().setLongVal(query.getStartTime())); - trow.addToColumnValue(new TCell().setLongVal(query.getEndTime())); - trow.addToColumnValue(new TCell().setLongVal(query.getEventTime())); - if (query.getState() == QueryDetail.QueryMemState.RUNNING) { - trow.addToColumnValue(new TCell().setLongVal(System.currentTimeMillis() - query.getStartTime())); - } else { - trow.addToColumnValue(new TCell().setLongVal(query.getLatency())); + Map queryInfoMap = QeProcessorImpl.INSTANCE.getQueryInfoMap(); + + SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + for (Long beId : beIdSet) { + Map qsMap = beQsMap.get(beId); + if (qsMap == null) { + continue; + } + Set queryIdSet = qsMap.keySet(); + for (String queryId : queryIdSet) { + QueryInfo queryInfo = queryInfoMap.get(queryId); + if (queryInfo == null) { + continue; + } + //todo(wb) add connect context for insert select + if (queryInfo.getConnectContext() != null && !Env.getCurrentEnv().getAccessManager() + .checkDbPriv(queryInfo.getConnectContext(), queryInfo.getConnectContext().getDatabase(), + PrivPredicate.SELECT)) { + continue; + } + TQueryStatistics qs = qsMap.get(queryId); + Backend be = Env.getCurrentEnv().getClusterInfo().getBackend(beId); + TRow tRow = makeQueryStatisticsTRow(sdf, queryId, be, selfNode, queryInfo, qs); + dataBatch.add(tRow); } - trow.addToColumnValue(new TCell().setStringVal(query.getState().toString())); - trow.addToColumnValue(new TCell().setStringVal(query.getDatabase())); - trow.addToColumnValue(new TCell().setStringVal(query.getSql())); - trow.addToColumnValue(new TCell().setStringVal(selfNode)); - dataBatch.add(trow); } /* Get the query results from other FE also */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataTableValuedFunction.java index 53a0b7ee5b801e..b56aabc504e845 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataTableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataTableValuedFunction.java @@ -50,7 +50,7 @@ public static Integer getColumnIndexFromColumnName(TMetadataType type, String co case TASKS: return TasksTableValuedFunction.getColumnIndexFromColumnName(columnName, params); case QUERIES: - return QueriesTableValuedFunction.getColumnIndexFromColumnName(columnName); + return ActiveQueriesTableValuedFunction.getColumnIndexFromColumnName(columnName); case WORKLOAD_SCHED_POLICY: return WorkloadSchedPolicyTableValuedFunction.getColumnIndexFromColumnName(columnName); default: diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java index c9547c91bd229c..f9fb76a9666abc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java @@ -74,8 +74,8 @@ public static TableValuedFunctionIf getTableFunction(String funcName, Map Date: Tue, 16 Jan 2024 21:46:37 +0800 Subject: [PATCH 047/200] [doc](Variant) add doc for Variant type (#28951) --- .../sql-reference/Data-Types/VARIANT.md | 359 +++++++++++++++++ docs/sidebars.json | 3 +- .../sql-reference/Data-Types/VARIANT.md | 367 ++++++++++++++++++ 3 files changed, 728 insertions(+), 1 deletion(-) create mode 100644 docs/en/docs/sql-manual/sql-reference/Data-Types/VARIANT.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/VARIANT.md diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Types/VARIANT.md b/docs/en/docs/sql-manual/sql-reference/Data-Types/VARIANT.md new file mode 100644 index 00000000000000..5f745c21948c63 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-reference/Data-Types/VARIANT.md @@ -0,0 +1,359 @@ +--- +{ + "title": "VARIANT", + "language": "en" +} +--- + + + +## VARIANT + +### Description + +VARIANT Type +Introduced a new data type VARIANT in Doris 2.1, which can store semi-structured JSON data. It allows storing complex data structures containing different data types (such as integers, strings, boolean values, etc.) without the need to define specific columns in the table structure beforehand. The VARIANT type is particularly useful for handling complex nested structures that may change at any time. During the writing process, this type can automatically infer column information based on the structure and types of the columns, dynamicly merge written schemas. It stores JSON keys and their corresponding values as columns and dynamic sub-columns. + +### Note + +Advantages over JSON Type: + +1. Different storage methods: The JSON type is stored in binary JSONB format, and the entire JSON is stored row by row in segment files. In contrast, the VARIANT type infers types during writing and stores the written JSON columns. It has a higher compression ratio compared to the JSON type, providing better storage efficiency. +2. Query: Querying does not require parsing. VARIANT fully utilizes columnar storage, vectorized engines, optimizers, and other components in Doris, providing users with extremely high query performance. +Below are test results based on clickbench data: + +| | Storage Space | +|--------------|------------| +| Predefined Static Columns | 24.329 GB | +| VARIANT Type | 24.296 GB | +| JSON Type | 46.730 GB | + +**Saves approximately 50% storage capacity** + +| Query Counts | Predefined Static Columns | VARIANT Type | JSON Type | +|---------------------|---------------------------|--------------|-----------------| +| First Query (cold) | 233.79s | 248.66s | **Most queries timeout** | +| Second Query (hot) | 86.02s | 94.82s | 789.24s | +| Third Query (hot) | 83.03s | 92.29s | 743.69s | + +[test case](https://github.com/ClickHouse/ClickBench/blob/main/doris/queries.sql) contains 43 queries + +**8x faster query, query performance comparable to static columns** + +### Example + +Demonstrate the functionality and usage of VARIANT with an example covering table creation, data import, and query cycle. + +**Table Creation Syntax** +Create a table, using the `variant` keyword in the syntax. + +``` sql +-- Without index +CREATE TABLE IF NOT EXISTS ${table_name} ( + k BIGINT, + v VARIANT +) +table_properties; + +-- Create an index on the v column, optionally specify the tokenize method, default is untokenized +CREATE TABLE IF NOT EXISTS ${table_name} ( + k BIGINT, + v VARIANT, + INDEX idx_var(v) USING INVERTED [PROPERTIES("parser" = "english|unicode|chinese")] [COMMENT 'your comment'] +) +table_properties; + +``` + +**Query Syntax** + +``` sql +-- use v['a']['b'] format for example +SELECT v['properties']['title'] from ${table_name} + +``` + +**Example based on the GitHub events dataset** + +Here, github events data is used to demonstrate the table creation, data import, and query using variant. +The below is a formatted line of data: + +``` json +{ + "id": "14186154924", + "type": "PushEvent", + "actor": { + "id": 282080, + "login": "brianchandotcom", + "display_login": "brianchandotcom", + "gravatar_id": "", + "url": "https://api.github.com/users/brianchandotcom", + "avatar_url": "https://avatars.githubusercontent.com/u/282080?" + }, + "repo": { + "id": 1920851, + "name": "brianchandotcom/liferay-portal", + "url": "https://api.github.com/repos/brianchandotcom/liferay-portal" + }, + "payload": { + "push_id": 6027092734, + "size": 4, + "distinct_size": 4, + "ref": "refs/heads/master", + "head": "91edd3c8c98c214155191feb852831ec535580ba", + "before": "abb58cc0db673a0bd5190000d2ff9c53bb51d04d", + "commits": [""] + }, + "public": true, + "created_at": "2020-11-13T18:00:00Z" +} +``` + +**Table Creation** + +- Created three columns of VARIANT type: `actor`, `repo`, and `payload`. +- Simultaneously created an inverted index, `idx_payload`, for the `payload` column while creating the table. +- Specified the index type as inverted using `USING INVERTED`, aimed at accelerating conditional filtering of sub-columns. +- `PROPERTIES("parser" = "english")` specified the adoption of English tokenization. + +``` sql +CREATE DATABASE test_variant; +USE test_variant; +CREATE TABLE IF NOT EXISTS github_events ( + id BIGINT NOT NULL, + type VARCHAR(30) NULL, + actor VARIANT NULL, + repo VARIANT NULL, + payload VARIANT NULL, + public BOOLEAN NULL, + created_at DATETIME NULL, + INDEX idx_payload (`payload`) USING INVERTED PROPERTIES("parser" = "english") COMMENT 'inverted index for payload' +) +DUPLICATE KEY(`id`) +DISTRIBUTED BY HASH(id) BUCKETS 10 +properties("replication_num" = "1"); +``` + +::: tip + +1. Creating an index on VARIANT columns, such as when there are numerous sub-columns in payload, might lead to an excessive number of index columns, impacting write performance. +2. The tokenization properties for the same VARIANT column are uniform. If you have varied tokenization requirements, consider creating multiple VARIANT columns and specifying index properties separately for each. + +::: + + +**Using Streamload for Import** + +Importing gh_2022-11-07-3.json, which contains one hour's worth of GitHub events data. + +``` shell +wget http://doris-build-hk-1308700295.cos.ap-hongkong.myqcloud.com/regression/variant/gh_2022-11-07-3.json + +curl --location-trusted -u root: -T gh_2022-11-07-3.json -H "read_json_by_line:true" -H "format:json" http://127.0.0.1:18148/api/test_variant/github_events/_strea +m_load + +{ + "TxnId": 2, + "Label": "086fd46a-20e6-4487-becc-9b6ca80281bf", + "Comment": "", + "TwoPhaseCommit": "false", + "Status": "Success", + "Message": "OK", + "NumberTotalRows": 139325, + "NumberLoadedRows": 139325, + "NumberFilteredRows": 0, + "NumberUnselectedRows": 0, + "LoadBytes": 633782875, + "LoadTimeMs": 7870, + "BeginTxnTimeMs": 19, + "StreamLoadPutTimeMs": 162, + "ReadDataTimeMs": 2416, + "WriteDataTimeMs": 7634, + "CommitAndPublishTimeMs": 55 +} +``` + +Confirm the successful import. + +``` sql +-- View the number of rows. +mysql> select count() from github_events; ++----------+ +| count(*) | ++----------+ +| 139325 | ++----------+ +1 row in set (0.25 sec) + +-- Random select one row +mysql> select * from github_events limit 1; ++-------------+-----------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--------+---------------------+ +| id | type | actor | repo | payload | public | created_at | ++-------------+-----------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--------+---------------------+ +| 25061821748 | PushEvent | {"gravatar_id":"","display_login":"jfrog-pipelie-intg","url":"https://api.github.com/users/jfrog-pipelie-intg","id":98024358,"login":"jfrog-pipelie-intg","avatar_url":"https://avatars.githubusercontent.com/u/98024358?"} | {"url":"https://api.github.com/repos/jfrog-pipelie-intg/jfinte2e_1667789956723_16","id":562683829,"name":"jfrog-pipelie-intg/jfinte2e_1667789956723_16"} | {"commits":[{"sha":"334433de436baa198024ef9f55f0647721bcd750","author":{"email":"98024358+jfrog-pipelie-intg@users.noreply.github.com","name":"jfrog-pipelie-intg"},"message":"commit message 10238493157623136117","distinct":true,"url":"https://api.github.com/repos/jfrog-pipelie-intg/jfinte2e_1667789956723_16/commits/334433de436baa198024ef9f55f0647721bcd750"}],"before":"f84a26792f44d54305ddd41b7e3a79d25b1a9568","head":"334433de436baa198024ef9f55f0647721bcd750","size":1,"push_id":11572649828,"ref":"refs/heads/test-notification-sent-branch-10238493157623136113","distinct_size":1} | 1 | 2022-11-07 11:00:00 | ++-------------+-----------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--------+---------------------+ +1 row in set (0.23 sec) +``` +Running desc command to view schema information, sub-columns will automatically expand at the storage layer and undergo type inference. + +``` sql +mysql> desc github_events; ++------------------------------------------------------------+------------+------+-------+---------+-------+ +| Field | Type | Null | Key | Default | Extra | ++------------------------------------------------------------+------------+------+-------+---------+-------+ +| id | BIGINT | No | true | NULL | | +| type | VARCHAR(*) | Yes | false | NULL | NONE | +| actor | VARIANT | Yes | false | NULL | NONE | +| created_at | DATETIME | Yes | false | NULL | NONE | +| payload | VARIANT | Yes | false | NULL | NONE | +| public | BOOLEAN | Yes | false | NULL | NONE | ++------------------------------------------------------------+------------+------+-------+---------+-------+ +6 rows in set (0.07 sec) + +mysql> set describe_extend_variant_column = true; +Query OK, 0 rows affected (0.01 sec) + +mysql> desc github_events; ++------------------------------------------------------------+------------+------+-------+---------+-------+ +| Field | Type | Null | Key | Default | Extra | ++------------------------------------------------------------+------------+------+-------+---------+-------+ +| id | BIGINT | No | true | NULL | | +| type | VARCHAR(*) | Yes | false | NULL | NONE | +| actor | VARIANT | Yes | false | NULL | NONE | +| actor.avatar_url | TEXT | Yes | false | NULL | NONE | +| actor.display_login | TEXT | Yes | false | NULL | NONE | +| actor.id | INT | Yes | false | NULL | NONE | +| actor.login | TEXT | Yes | false | NULL | NONE | +| actor.url | TEXT | Yes | false | NULL | NONE | +| created_at | DATETIME | Yes | false | NULL | NONE | +| payload | VARIANT | Yes | false | NULL | NONE | +| payload.action | TEXT | Yes | false | NULL | NONE | +| payload.before | TEXT | Yes | false | NULL | NONE | +| payload.comment.author_association | TEXT | Yes | false | NULL | NONE | +| payload.comment.body | TEXT | Yes | false | NULL | NONE | +.... ++------------------------------------------------------------+------------+------+-------+---------+-------+ +406 rows in set (0.07 sec) +``` +DESC can be used to specify partition and view the schema of a particular partition. The syntax is as follows: + +``` sql +DESCRIBE ${table_name} PARTITION ($partition_name); +``` + +**Querying** + +::: warning + +When utilizing filtering and aggregation functionalities to query sub-columns, additional casting operations need to be performed on sub-columns (because the storage types are not necessarily fixed and require a unified SQL type). +For instance, `SELECT * FROM tbl where CAST(var['titile'] as text) MATCH "hello world"` +The simplified examples below illustrate how to use VARIANT for querying: +The following are three typical query scenarios + +::: + +1. Retrieve the top 5 repositories based on star count from the `github_events` table. + +``` sql +mysql> SELECT + -> cast(repo['name'] as text) as repo_name, count() AS stars + -> FROM github_events + -> WHERE type = 'WatchEvent' + -> GROUP BY + -> ORDER BY stars DESC LIMIT 5; ++--------------------------+-------+ +| repo_name | stars | ++--------------------------+-------+ +| aplus-framework/app | 78 | +| lensterxyz/lenster | 77 | +| aplus-framework/database | 46 | +| stashapp/stash | 42 | +| aplus-framework/image | 34 | ++--------------------------+-------+ +5 rows in set (0.03 sec) +``` + +2. Retrieve the count of comments containing "doris". + +``` sql +mysql> SELECT + -> count() FROM github_events + -> WHERE cast(payload['comment']['body'] as text) MATCH 'doris'; ++---------+ +| count() | ++---------+ +| 3 | ++---------+ +1 row in set (0.04 sec) +``` + +3. Query the issue number with the highest number of comments along with its corresponding repository. + +``` sql +mysql> SELECT + -> cast(repo['name'] as string) as repo_name, + -> cast(payload['issue']['number'] as int) as issue_number, + -> count() AS comments, + -> count( + -> distinct cast(actor['login'] as string) + -> ) AS authors + -> FROM github_events + -> WHERE type = 'IssueCommentEvent' AND (cast(payload["action"] as string) = 'created') AND (cast(payload["issue"]["number"] as int) > 10) + -> GROUP BY repo_name, issue_number + -> HAVING authors >= 4 + -> ORDER BY comments DESC, repo_name + -> LIMIT 50; ++--------------------------------------+--------------+----------+---------+ +| repo_name | issue_number | comments | authors | ++--------------------------------------+--------------+----------+---------+ +| facebook/react-native | 35228 | 5 | 4 | +| swsnu/swppfall2022-team4 | 27 | 5 | 4 | +| belgattitude/nextjs-monorepo-example | 2865 | 4 | 4 | ++--------------------------------------+--------------+----------+---------+ +3 rows in set (0.03 sec) +``` + +### Usage Restrictions and Best Practices + +**There are several limitations when using the VARIANT type:** +Dynamic columns of VARIANT are nearly as efficient as predefined static columns. When dealing with data like logs, where fields are often added dynamically (such as container labels in Kubernetes), parsing JSON and inferring types can generate additional costs during write operations. Therefore, it's recommended to keep the number of columns for a single import below 1000. + +Ensure consistency in types whenever possible. Doris automatically performs compatible type conversions. When a field cannot undergo compatible type conversion, it is uniformly converted to JSONB type. The performance of JSONB columns may degrade compared to columns like int or text. + +1. tinyint -> smallint -> int -> bigint, integer types can be promoted following the direction of the arrows. +2. float -> double, floating-point numbers can be promoted following the direction of the arrow. +3. text, string type. +4. JSON, binary JSON type. + +When the above types cannot be compatible, they will be transformed into JSON type to prevent loss of type information. If you need to set a strict schema in VARIANT, the VARIANT MAPPING mechanism will be introduced soon. + +**Other limitations include:** + +- Aggregate models are currently not supported. +- VARIANT columns can only create inverted indexes. +- Using the **RANDOM** mode is recommended for higher write performance. +- Non-standard JSON types such as date and decimal should ideally use static types for better performance, since these types are infered to text type. +- Arrays with dimensions of 2 or higher will be stored as JSONB encoding, which might perform less efficiently than native arrays. +- Not supported as primary or sort keys. +- Queries with filters or aggregations require casting. The storage layer eliminates cast operations based on storage type and the target type of the cast, speeding up queries. + +### Keywords + + VARIANT \ No newline at end of file diff --git a/docs/sidebars.json b/docs/sidebars.json index 42b442c597ebb4..c6b5efc203f106 100644 --- a/docs/sidebars.json +++ b/docs/sidebars.json @@ -923,7 +923,8 @@ "sql-manual/sql-reference/Data-Types/MAP", "sql-manual/sql-reference/Data-Types/STRUCT", "sql-manual/sql-reference/Data-Types/JSON", - "sql-manual/sql-reference/Data-Types/AGG_STATE" + "sql-manual/sql-reference/Data-Types/AGG_STATE", + "sql-manual/sql-reference/Data-Types/VARIANT" ] }, { diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/VARIANT.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/VARIANT.md new file mode 100644 index 00000000000000..dca483fe230d1e --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/VARIANT.md @@ -0,0 +1,367 @@ +--- +{ + "title": "VARIANT", + "language": "zh-CN" +} +--- + + + +## VARIANT + +### Description + +VARIANT类型 +在 Doris 2.1 中引入一种新的数据类型 VARIANT,它可以存储半结构化 JSON 数据。它允许存储包含不同数据类型(如整数、字符串、布尔值等)的复杂数据结构,而无需在表结构中提前定义具体的列。VARIANT 类型特别适用于处理复杂的嵌套结构,而这些结构可能随时会发生变化。在写入过程中,该类型可以自动根据列的结构、类型推断列信息,动态合并写入的 schema,并通过将 JSON 键及其对应的值存储为列和动态子列。 + +### Note + +相比 JSON 类型有有以下优势: + +1. 存储方式不同, JSON 类型是以二进制 JSONB 格式进行存储,整行 JSON 以行存的形式存储到 segment 文件中。而 VARIANT 类型在写入的时候进行类型推断,将写入的 JSON 列存化。比JSON类型有更高的压缩比, 存储空间更小。 +2. 查询方式不同,查询不需要进行解析。VARIANT 充分利用 Doris 中列式存储、向量化引擎、优化器等组件给用户带来极高的查询性能。 +下面是基于 clickbench 数据测试的结果: + +| | 存储空间 | +|--------------|------------| +| 预定义静态列 | 24.329 GB | +| variant 类型 | 24.296 GB | +| json 类型 | 46.730 GB | + + + +**节省约 50%存储容量** + +| 查询次数 | 预定义静态列 | variant 类型 | json 类型 | +|----------------|--------------|--------------|-----------------| +| 第一次查询 (cold) | 233.79s | 248.66s | **大部分查询超时** | +| 第二次查询 (hot) | 86.02s | 94.82s | 789.24s | +| 第三次查询 (hot) | 83.03s | 92.29s | 743.69s | + +[测试集](https://github.com/ClickHouse/ClickBench/blob/main/doris/queries.sql) 一共43个查询语句 + +**查询提速 8+倍, 查询性能与静态列相当** + +### Example + +``` +用一个从建表、导数据、查询全周期的例子说明VARIANT的功能和用法。 +``` + +**建表语法** + +建表,建表语法关键字 variant + +``` sql +-- 无索引 +CREATE TABLE IF NOT EXISTS ${table_name} ( + k BIGINT, + v VARIANT +) +table_properties; + +-- 在v列创建索引,可选指定分词方式,默认不分词 +CREATE TABLE IF NOT EXISTS ${table_name} ( + k BIGINT, + v VARIANT, + INDEX idx_var(v) USING INVERTED [PROPERTIES("parser" = "english|unicode|chinese")] [COMMENT 'your comment'] +) +table_properties; +``` + +**查询语法** + +``` sql +-- 使用 v['a']['b'] 形式例如 +SELECT v['properties']['title'] from ${table_name} +``` + +### 基于 github events 数据集示例 + +这里用 github events 数据展示 variant 的建表、导入、查询。 +下面是格式化后的一行数据 + +``` json +{ + "id": "14186154924", + "type": "PushEvent", + "actor": { + "id": 282080, + "login": "brianchandotcom", + "display_login": "brianchandotcom", + "gravatar_id": "", + "url": "https://api.github.com/users/brianchandotcom", + "avatar_url": "https://avatars.githubusercontent.com/u/282080?" + }, + "repo": { + "id": 1920851, + "name": "brianchandotcom/liferay-portal", + "url": "https://api.github.com/repos/brianchandotcom/liferay-portal" + }, + "payload": { + "push_id": 6027092734, + "size": 4, + "distinct_size": 4, + "ref": "refs/heads/master", + "head": "91edd3c8c98c214155191feb852831ec535580ba", + "before": "abb58cc0db673a0bd5190000d2ff9c53bb51d04d", + "commits": [""] + }, + "public": true, + "created_at": "2020-11-13T18:00:00Z" +} +``` + +**建表** + +- 创建了三个 VARIANT 类型的列, `actor`,`repo` 和 `payload` +- 创建表的同时创建了 `payload` 列的倒排索引 `idx_payload` +- USING INVERTED 指定索引类型是倒排索引,用于加速子列的条件过滤 +- `PROPERTIES("parser" = "english")` 指定采用 english 分词 + +``` sql +CREATE DATABASE test_variant; +USE test_variant; +CREATE TABLE IF NOT EXISTS github_events ( + id BIGINT NOT NULL, + type VARCHAR(30) NULL, + actor VARIANT NULL, + repo VARIANT NULL, + payload VARIANT NULL, + public BOOLEAN NULL, + created_at DATETIME NULL, + INDEX idx_payload (`payload`) USING INVERTED PROPERTIES("parser" = "english") COMMENT 'inverted index for payload' +) +DUPLICATE KEY(`id`) +DISTRIBUTED BY HASH(id) BUCKETS 10 +properties("replication_num" = "1"); +``` + +**需要注意的是:** + +::: tip + +1. 在 VARIANT 列上创建索引,比如 payload 的子列很多时,可能会造成索引列过多,影响写入性能 +2. 同一个 VARIANT 列的分词属性是相同的,如果您有不同的分词需求,那么可以创建多个 VARIANT 然后分别指定索引属性 + +::: + +**使用 streamload 导入** + +导入gh_2022-11-07-3.json,这是 github events 一个小时的数据 + +``` shell +wget http://doris-build-hk-1308700295.cos.ap-hongkong.myqcloud.com/regression/variant/gh_2022-11-07-3.json + +curl --location-trusted -u root: -T gh_2022-11-07-3.json -H "read_json_by_line:true" -H "format:json" http://127.0.0.1:18148/api/test_variant/github_events/_strea +m_load + +{ + "TxnId": 2, + "Label": "086fd46a-20e6-4487-becc-9b6ca80281bf", + "Comment": "", + "TwoPhaseCommit": "false", + "Status": "Success", + "Message": "OK", + "NumberTotalRows": 139325, + "NumberLoadedRows": 139325, + "NumberFilteredRows": 0, + "NumberUnselectedRows": 0, + "LoadBytes": 633782875, + "LoadTimeMs": 7870, + "BeginTxnTimeMs": 19, + "StreamLoadPutTimeMs": 162, + "ReadDataTimeMs": 2416, + "WriteDataTimeMs": 7634, + "CommitAndPublishTimeMs": 55 +} +``` + +确认导入成功 + +``` sql +-- 查看行数 +mysql> select count() from github_events; ++----------+ +| count(*) | ++----------+ +| 139325 | ++----------+ +1 row in set (0.25 sec) + +-- 随机看一条数据 +mysql> select * from github_events limit 1; ++-------------+-----------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--------+---------------------+ +| id | type | actor | repo | payload | public | created_at | ++-------------+-----------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--------+---------------------+ +| 25061821748 | PushEvent | {"gravatar_id":"","display_login":"jfrog-pipelie-intg","url":"https://api.github.com/users/jfrog-pipelie-intg","id":98024358,"login":"jfrog-pipelie-intg","avatar_url":"https://avatars.githubusercontent.com/u/98024358?"} | {"url":"https://api.github.com/repos/jfrog-pipelie-intg/jfinte2e_1667789956723_16","id":562683829,"name":"jfrog-pipelie-intg/jfinte2e_1667789956723_16"} | {"commits":[{"sha":"334433de436baa198024ef9f55f0647721bcd750","author":{"email":"98024358+jfrog-pipelie-intg@users.noreply.github.com","name":"jfrog-pipelie-intg"},"message":"commit message 10238493157623136117","distinct":true,"url":"https://api.github.com/repos/jfrog-pipelie-intg/jfinte2e_1667789956723_16/commits/334433de436baa198024ef9f55f0647721bcd750"}],"before":"f84a26792f44d54305ddd41b7e3a79d25b1a9568","head":"334433de436baa198024ef9f55f0647721bcd750","size":1,"push_id":11572649828,"ref":"refs/heads/test-notification-sent-branch-10238493157623136113","distinct_size":1} | 1 | 2022-11-07 11:00:00 | ++-------------+-----------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--------+---------------------+ +1 row in set (0.23 sec) +``` + +desc 查看 schema 信息,子列会在存储层自动扩展、并进行类型推导 + +``` sql +mysql> desc github_events; ++------------------------------------------------------------+------------+------+-------+---------+-------+ +| Field | Type | Null | Key | Default | Extra | ++------------------------------------------------------------+------------+------+-------+---------+-------+ +| id | BIGINT | No | true | NULL | | +| type | VARCHAR(*) | Yes | false | NULL | NONE | +| actor | VARIANT | Yes | false | NULL | NONE | +| created_at | DATETIME | Yes | false | NULL | NONE | +| payload | VARIANT | Yes | false | NULL | NONE | +| public | BOOLEAN | Yes | false | NULL | NONE | ++------------------------------------------------------------+------------+------+-------+---------+-------+ +6 rows in set (0.07 sec) + +mysql> set describe_extend_variant_column = true; +Query OK, 0 rows affected (0.01 sec) + +mysql> desc github_events; ++------------------------------------------------------------+------------+------+-------+---------+-------+ +| Field | Type | Null | Key | Default | Extra | ++------------------------------------------------------------+------------+------+-------+---------+-------+ +| id | BIGINT | No | true | NULL | | +| type | VARCHAR(*) | Yes | false | NULL | NONE | +| actor | VARIANT | Yes | false | NULL | NONE | +| actor.avatar_url | TEXT | Yes | false | NULL | NONE | +| actor.display_login | TEXT | Yes | false | NULL | NONE | +| actor.id | INT | Yes | false | NULL | NONE | +| actor.login | TEXT | Yes | false | NULL | NONE | +| actor.url | TEXT | Yes | false | NULL | NONE | +| created_at | DATETIME | Yes | false | NULL | NONE | +| payload | VARIANT | Yes | false | NULL | NONE | +| payload.action | TEXT | Yes | false | NULL | NONE | +| payload.before | TEXT | Yes | false | NULL | NONE | +| payload.comment.author_association | TEXT | Yes | false | NULL | NONE | +| payload.comment.body | TEXT | Yes | false | NULL | NONE | +.... ++------------------------------------------------------------+------------+------+-------+---------+-------+ +406 rows in set (0.07 sec) +``` + +desc 可以指定 partition 查看某个 partition 的 schema, 语法如下 + +``` +DESCRIBE ${table_name} PARTITION ($partition_name); +``` + +**查询** + +::: warning + +**注意** +如使用过滤和聚合等功能来查询子列, 需要对子列进行额外的 cast 操作(因为存储类型不一定是固定的,需要有一个 SQL 统一的类型)。 +例如 SELECT * FROM tbl where CAST(var['titile'] as text) MATCH "hello world" +以下简化的示例说明了如何使用 VARIANT 进行查询 + +::: + +下面是典型的三个查询场景: + +1. 从 github_events 表中获取 top 5 star 数的代码库 + +``` sql +mysql> SELECT + -> cast(repo['name'] as text) as repo_name, count() AS stars + -> FROM github_events + -> WHERE type = 'WatchEvent' + -> GROUP BY stars + -> ORDER BY stars DESC LIMIT 5; ++--------------------------+-------+ +| repo_name | stars | ++--------------------------+-------+ +| aplus-framework/app | 78 | +| lensterxyz/lenster | 77 | +| aplus-framework/database | 46 | +| stashapp/stash | 42 | +| aplus-framework/image | 34 | ++--------------------------+-------+ +5 rows in set (0.03 sec) +``` + +2. 获取评论中包含 doris 的数量 + +``` sql +mysql> SELECT + -> count() FROM github_events + -> WHERE cast(payload['comment']['body'] as text) MATCH 'doris'; ++---------+ +| count() | ++---------+ +| 3 | ++---------+ +1 row in set (0.04 sec) +``` + +3. 查询 comments 最多的 issue 号以及对应的库 + +``` sql +mysql> SELECT + -> cast(repo['name'] as string) as repo_name, + -> cast(payload['issue']['number'] as int) as issue_number, + -> count() AS comments, + -> count( + -> distinct cast(actor['login'] as string) + -> ) AS authors + -> FROM github_events + -> WHERE type = 'IssueCommentEvent' AND (cast(payload['action'] as string) = 'created') AND (cast(payload['issue']['number'] as int) > 10) + -> GROUP BY repo_name, issue_number + -> HAVING authors >= 4 + -> ORDER BY comments DESC, repo_name + -> LIMIT 50; ++--------------------------------------+--------------+----------+---------+ +| repo_name | issue_number | comments | authors | ++--------------------------------------+--------------+----------+---------+ +| facebook/react-native | 35228 | 5 | 4 | +| swsnu/swppfall2022-team4 | 27 | 5 | 4 | +| belgattitude/nextjs-monorepo-example | 2865 | 4 | 4 | ++--------------------------------------+--------------+----------+---------+ +3 rows in set (0.03 sec) +``` + +### 使用限制和最佳实践 + +**VARIANT 类型的使用有以下限制:** +VARIANT 动态列与预定义静态列几乎一样高效。处理诸如日志之类的数据 ,在这类数据中,经常通过动态属性添加字段(例如 Kubernetes 中的容器标签)。但是解析 JSON 和推断类型会在写入时产生额外开销。因此,我们建议保持单次导入列数在 1000 以下。 + +尽可能保证类型一致, Doris 会自动进行如下兼容类型转换,当字段无法进行兼容类型转换时会统一转换成 JSONB 类型。JSONB 列的性能与 int、text 等列性能会有所退化。 + +1. tinyint->smallint->int->bigint, 整形可以按照箭头做类型提升 +2. float->double,浮点数按照箭头做类型提升 +3. text,字符串类型 +4. JSON, 二进制JSON类型 + +上诉类型无法兼容时, 会变成JSON类型防止类型信息丢失, 如果您需要在 VARIANT 中设置严格的schema,即将推出 VARIANT MAPPING机制 + +其它限制如下: + +- 目前不支持 Aggregate 模型 +- VARIANT 列只能创建倒排索引 +- **推荐使用 RANDOM 模式, 写入性能更高效** +- 日期、decimal 等非标准 JSON 类型会被默认推断成字符串类型,所以尽可能从 VARIANT 中提取出来,用静态类型,性能更好 +- 2 维及其以上的数组列存化会被存成 JSONB 编码,性能不如原生数组 +- 不支持作为主键或者排序键 +- 查询过滤、聚合需要带 cast, 存储层会根据存储类型和 cast 目标类型来消除 cast 操作,加速查询。 + +### Keywords + + VARIANT \ No newline at end of file From 33702aac565e531fa5e7b7c68be0225ee75c7af0 Mon Sep 17 00:00:00 2001 From: Mingyu Chen Date: Tue, 16 Jan 2024 22:41:18 +0800 Subject: [PATCH 048/200] [refactor](insert) remove unused insert code in FE #29924 --- docs/en/docs/ecosystem/audit-plugin.md | 4 +- .../apache/doris/analysis/BrokerLoadStmt.java | 125 ----- .../apache/doris/analysis/MysqlLoadStmt.java | 100 ---- .../doris/analysis/NativeInsertStmt.java | 1 + .../apache/doris/analysis/S3TvfLoadStmt.java | 437 ------------------ .../apache/doris/analysis/SparkLoadStmt.java | 99 ---- .../doris/analysis/UnifiedLoadStmt.java | 15 - .../org/apache/doris/qe/SessionVariable.java | 10 - .../doris/analysis/S3TvfLoadStmtTest.java | 244 ---------- .../test_partial_update_broker_load.groovy | 1 - .../broker_load/test_array_load.groovy | 10 - .../mysql_load/test_mysql_load_unified.groovy | 2 - .../test_tvf_based_broker_load.groovy | 4 +- 13 files changed, 4 insertions(+), 1048 deletions(-) delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/analysis/BrokerLoadStmt.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/analysis/MysqlLoadStmt.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/analysis/S3TvfLoadStmt.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/analysis/SparkLoadStmt.java delete mode 100644 fe/fe-core/src/test/java/org/apache/doris/analysis/S3TvfLoadStmtTest.java diff --git a/docs/en/docs/ecosystem/audit-plugin.md b/docs/en/docs/ecosystem/audit-plugin.md index 3698caeb18e691..fc8e7560d1038a 100644 --- a/docs/en/docs/ecosystem/audit-plugin.md +++ b/docs/en/docs/ecosystem/audit-plugin.md @@ -112,7 +112,7 @@ After executing install, the AuditLoader directory will be automatically generat zip -r -q -m auditloader.zip auditloader.jar plugin.properties plugin.conf ``` -### 创建库表 +### Create Audit Table In Doris, you need to create the library and table of the audit log. The table structure is as follows: @@ -206,7 +206,7 @@ You can place the packaged auditloader.zip on an http server, or copy `auditload ### Installation -通过以下语句安装 Audit Loader 插件: +Install the audit loader plugin: ```sql INSTALL PLUGIN FROM [source] [PROPERTIES ("key"="value", ...)] diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/BrokerLoadStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/BrokerLoadStmt.java deleted file mode 100644 index 55f2113504a1a2..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/BrokerLoadStmt.java +++ /dev/null @@ -1,125 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you 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 org.apache.doris.analysis; - -import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.OlapTable; -import org.apache.doris.common.AnalysisException; -import org.apache.doris.common.DdlException; -import org.apache.doris.common.UserException; -import org.apache.doris.common.util.PrintableMap; - -import com.google.common.base.Joiner; -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import org.apache.commons.collections.CollectionUtils; - -import java.util.List; -import java.util.Map; - -public class BrokerLoadStmt extends InsertStmt { - - private final List dataDescList; - - private final BrokerDesc brokerDesc; - - private String cluster; - - public BrokerLoadStmt(LabelName label, List dataDescList, BrokerDesc brokerDesc, - Map properties, String comments) { - super(label, properties, comments); - this.dataDescList = dataDescList; - this.brokerDesc = brokerDesc; - } - - @Override - public List getDataDescList() { - return dataDescList; - } - - @Override - public BrokerDesc getResourceDesc() { - return brokerDesc; - } - - @Override - public LoadType getLoadType() { - return LoadType.BROKER_LOAD; - } - - @Override - public void analyzeProperties() throws DdlException { - // public check should be in base class - } - - @Override - public void analyze(Analyzer analyzer) throws UserException { - super.analyze(analyzer); - label.analyze(analyzer); - Preconditions.checkState(!CollectionUtils.isEmpty(dataDescList), - new AnalysisException("No data file in load statement.")); - Preconditions.checkNotNull(brokerDesc, "No broker desc found."); - // check data descriptions - for (DataDescription dataDescription : dataDescList) { - final String fullDbName = dataDescription.analyzeFullDbName(label.getDbName(), analyzer); - dataDescription.analyze(fullDbName); - Preconditions.checkState(!dataDescription.isLoadFromTable(), - new AnalysisException("Load from table should use Spark Load")); - Database db = analyzer.getEnv().getInternalCatalog().getDbOrAnalysisException(fullDbName); - OlapTable table = db.getOlapTableOrAnalysisException(dataDescription.getTableName()); - dataDescription.checkKeyTypeForLoad(table); - if (!brokerDesc.isMultiLoadBroker()) { - for (int i = 0; i < dataDescription.getFilePaths().size(); i++) { - String location = brokerDesc.getFileLocation(dataDescription.getFilePaths().get(i)); - dataDescription.getFilePaths().set(i, location); - StorageBackend.checkPath(dataDescription.getFilePaths().get(i), - brokerDesc.getStorageType(), "DATA INFILE must be specified."); - dataDescription.getFilePaths().set(i, dataDescription.getFilePaths().get(i)); - } - } - } - } - - @Override - public boolean needAuditEncryption() { - return true; - } - - @Override - public String toSql() { - StringBuilder sb = new StringBuilder(); - sb.append("LOAD LABEL ").append(label.toSql()).append("\n"); - sb.append("("); - Joiner.on(",\n").appendTo(sb, Lists.transform(dataDescList, DataDesc::toSql)).append(")"); - if (cluster != null) { - sb.append("\nBY '"); - sb.append(cluster); - sb.append("'"); - } - if (brokerDesc != null) { - sb.append("\n").append(brokerDesc.toSql()); - } - - if (properties != null && !properties.isEmpty()) { - sb.append("\nPROPERTIES ("); - sb.append(new PrintableMap<>(properties, "=", true, false, true)); - sb.append(")"); - } - return sb.toString(); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/MysqlLoadStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/MysqlLoadStmt.java deleted file mode 100644 index 29c7f52f50b5ba..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/MysqlLoadStmt.java +++ /dev/null @@ -1,100 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you 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. -// This file is copied from -// https://github.com/apache/impala/blob/branch-2.9.0/fe/src/main/java/org/apache/impala/StatementBase.java -// and modified by Doris - -package org.apache.doris.analysis; - -import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.OlapTable; -import org.apache.doris.common.AnalysisException; -import org.apache.doris.common.Config; -import org.apache.doris.common.DdlException; -import org.apache.doris.common.UserException; - -import java.io.File; -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -public class MysqlLoadStmt extends InsertStmt { - - private final DataDescription dataDescription; - - public MysqlLoadStmt(DataDescription dataDescription, Map properties, String comments) { - super(new LabelName(), properties, comments); - this.dataDescription = dataDescription; - } - - @Override - public List getDataDescList() { - return Collections.singletonList(dataDescription); - } - - @Override - public ResourceDesc getResourceDesc() { - // mysql load does not have resource desc - return null; - } - - @Override - public LoadType getLoadType() { - return LoadType.MYSQL_LOAD; - } - - @Override - public void analyzeProperties() throws DdlException { - - } - - @Override - public void analyze(Analyzer analyzer) throws AnalysisException, UserException { - super.analyze(analyzer); - String fullDbName = dataDescription.analyzeFullDbName(label.getDbName(), analyzer); - dataDescription.analyze(fullDbName); - Database db = analyzer.getEnv().getInternalCatalog().getDbOrAnalysisException(fullDbName); - OlapTable table = db.getOlapTableOrAnalysisException(dataDescription.getTableName()); - dataDescription.checkKeyTypeForLoad(table); - if (!dataDescription.isClientLocal()) { - for (String path : dataDescription.getFilePaths()) { - if (Config.mysql_load_server_secure_path.isEmpty()) { - throw new AnalysisException("Load local data from fe local is not enabled. If you want to use it," - + " plz set the `mysql_load_server_secure_path` for FE to be a right path."); - } else { - File file = new File(path); - try { - if (!(file.getCanonicalPath().startsWith(Config.mysql_load_server_secure_path))) { - throw new AnalysisException("Local file should be under the secure path of FE."); - } - } catch (IOException e) { - throw new RuntimeException(e); - } - if (!file.exists()) { - throw new AnalysisException("File: " + path + " is not exists."); - } - } - } - } - } - - @Override - public RedirectStatus getRedirectStatus() { - return RedirectStatus.NO_FORWARD; - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java index 17d3a9ffd0094d..d69ca40cecad0d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java @@ -187,6 +187,7 @@ public NativeInsertStmt(InsertTarget target, String label, List cols, In && ((SelectStmt) queryStmt).getTableRefs().isEmpty()); } + // Ctor of group commit in sql parser public NativeInsertStmt(long tableId, String label, List cols, InsertSource source, List hints) { this(new InsertTarget(new TableName(null, null, null), null), label, cols, source, hints); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/S3TvfLoadStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/S3TvfLoadStmt.java deleted file mode 100644 index ac83f26d49b8b5..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/S3TvfLoadStmt.java +++ /dev/null @@ -1,437 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you 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 org.apache.doris.analysis; - -import org.apache.doris.analysis.CompoundPredicate.Operator; -import org.apache.doris.analysis.StorageBackend.StorageType; -import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.TableIf; -import org.apache.doris.common.AnalysisException; -import org.apache.doris.common.DdlException; -import org.apache.doris.common.UserException; -import org.apache.doris.common.util.FileFormatConstants; -import org.apache.doris.datasource.property.constants.S3Properties; -import org.apache.doris.datasource.property.constants.S3Properties.Env; -import org.apache.doris.tablefunction.S3TableValuedFunction; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import com.google.common.base.Predicates; -import com.google.common.base.Strings; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Consumer; -import java.util.function.Function; -import java.util.stream.Collectors; - -/** - * S3 Load based on S3 TVF - */ -public class S3TvfLoadStmt extends NativeInsertStmt { - - private static final Logger LOG = LogManager.getLogger(S3TvfLoadStmt.class); - - private static final String FORMAT_CSV = "csv"; - - private static final String DEFAULT_FORMAT = FORMAT_CSV; - - private final DataDescription dataDescription; - - /** - * for csv format, we need some particular process - */ - private final boolean isCsvFormat; - - /** - * only used for loading from csv format tvf - * with mapping from col name to csv-format-tvf-style col name (c1, c2, c3...) - */ - private Map selectColNameToCsvColName; - - @VisibleForTesting - private Set functionGenTableColNames; - - public S3TvfLoadStmt(LabelName label, List dataDescList, BrokerDesc brokerDesc, - Map properties, String comments) throws DdlException { - super(buildInsertTarget(dataDescList.get(0)), - label.getLabelName(), /*insert all columns by default*/null, - buildInsertSource(dataDescList.get(0), brokerDesc), null); - this.label = label; - this.dataDescription = dataDescList.get(0); - this.properties = properties; - this.comments = comments; - this.isCsvFormat = isCsvFormat(dataDescription.getFileFormat()); - } - - // ------------------------------------ init helpers ------------------------------------ - - private static InsertTarget buildInsertTarget(DataDescription dataDescription) { - final TableName tableName = new TableName(null, null, dataDescription.getTableName()); - return new InsertTarget(tableName, dataDescription.getPartitionNames()); - } - - private static InsertSource buildInsertSource(DataDescription dataDescription, BrokerDesc brokerDesc) - throws DdlException { - final SelectList selectList = new SelectList(); - // use `select *` by default - final SelectListItem item = new SelectListItem(SelectListItem.createStarItem(null)); - selectList.addItem(item); - - // build from - final FromClause fromClause = new FromClause( - Collections.singletonList(buildTvfRef(dataDescription, brokerDesc)) - ); - - // trans order by in load stmt - final String sequenceCol = dataDescription.getSequenceCol(); - final ArrayList orderByElementList = Lists.newArrayList(); - if (!Strings.isNullOrEmpty(sequenceCol)) { - final OrderByElement orderByElement = new OrderByElement( - new SlotRef(null, sequenceCol), - true, null - ); - orderByElementList.add(orderByElement); - } - - - // merge preceding filter and where expr - final BoolLiteral trueLiteral = new BoolLiteral(true); - final Expr whereExpr = Optional.ofNullable(dataDescription.getWhereExpr()).orElse(trueLiteral); - final Expr precdingFilterExpr = - Optional.ofNullable(dataDescription.getPrecdingFilterExpr()).orElse(trueLiteral); - final Expr compoundPredicate = new CompoundPredicate(Operator.AND, precdingFilterExpr, whereExpr); - - final SelectStmt selectStmt = new SelectStmt( - selectList, fromClause, compoundPredicate, - null, null, - orderByElementList, LimitElement.NO_LIMIT - ); - return new InsertSource(selectStmt); - } - - private static TableRef buildTvfRef(DataDescription dataDescription, BrokerDesc brokerDesc) throws DdlException { - final Map params = Maps.newHashMap(); - - final List filePaths = dataDescription.getFilePaths(); - Preconditions.checkState(filePaths.size() == 1, "there should be only one file path"); - final String s3FilePath = filePaths.get(0); - params.put(S3TableValuedFunction.PROP_URI, s3FilePath); - - final Map dataDescProp = dataDescription.getProperties(); - if (dataDescProp != null) { - params.putAll(dataDescProp); - } - - final String format = Optional.ofNullable(dataDescription.getFileFormat()).orElse(DEFAULT_FORMAT); - params.put(FileFormatConstants.PROP_FORMAT, format); - - if (isCsvFormat(format)) { - parseSeparator(dataDescription.getColumnSeparatorObj(), params); - parseSeparator(dataDescription.getLineDelimiterObj(), params); - } - - List columnsFromPath = dataDescription.getColumnsFromPath(); - if (columnsFromPath != null) { - params.put(FileFormatConstants.PROP_PATH_PARTITION_KEYS, - String.join(",", columnsFromPath)); - } - - Preconditions.checkState(!brokerDesc.isMultiLoadBroker(), "do not support multi broker load currently"); - Preconditions.checkState(brokerDesc.getStorageType() == StorageType.S3, "only support S3 load"); - - final Map s3ResourceProp = brokerDesc.getProperties(); - S3Properties.convertToStdProperties(s3ResourceProp); - s3ResourceProp.keySet().removeIf(Env.FS_KEYS::contains); - params.putAll(s3ResourceProp); - - try { - return new TableValuedFunctionRef(S3TableValuedFunction.NAME, null, params); - } catch (AnalysisException e) { - throw new DdlException("failed to create s3 tvf ref", e); - } - } - - private static void parseSeparator(Separator separator, Map tvfParams) throws DdlException { - if (separator == null) { - return; - } - try { - separator.analyze(); - } catch (AnalysisException e) { - throw new DdlException(String.format("failed to parse separator:%s", separator), e); - } - tvfParams.put(FileFormatConstants.PROP_COLUMN_SEPARATOR, separator.getSeparator()); - } - - private static boolean isCsvFormat(String format) { - return Strings.isNullOrEmpty(format) || StringUtils.equalsIgnoreCase(format, FORMAT_CSV); - } - - // -------------------------------------------------------------------------------------- - - @Override - public void convertSemantic(Analyzer analyzer) throws UserException { - label.analyze(analyzer); - initTargetTable(analyzer); - analyzeColumns(analyzer); - } - - @Override - public void getTables(Analyzer analyzer, Map tableMap, Set parentViewNameSet) - throws AnalysisException { - - super.getTables(analyzer, tableMap, parentViewNameSet); - final List tables = Lists.newArrayList(tableMap.values()); - Preconditions.checkState(tables.size() == 2, - "table map should only contain the unique function generated table and the unique target table"); - functionGenTableColNames = tables.get(0) - .getBaseSchema() - .stream() - .map(Column::getName) - .collect(Collectors.toSet()); - } - - // ------------------------------------ columns mapping ------------------------------------ - - private void analyzeColumns(Analyzer analyzer) throws AnalysisException { - final String fullDbName = dataDescription.analyzeFullDbName(label.getDbName(), analyzer); - dataDescription.analyze(fullDbName); - // copy a list for analyzing - List columnExprList = Lists.newArrayList(dataDescription.getParsedColumnExprList()); - if (CollectionUtils.isEmpty(columnExprList)) { - padExprListIfNeeded(columnExprList); - } - rewriteExpr(columnExprList); - boolean isFileFieldSpecified = columnExprList.stream().anyMatch(ImportColumnDesc::isColumn); - if (!isFileFieldSpecified) { - return; - } - if (isCsvFormat) { - // in tvf, csv format column names are like "c1, c2, c3", record for correctness of select list - recordCsvColNames(columnExprList); - } - columnExprList = filterColumns(columnExprList); - if (CollectionUtils.isEmpty(columnExprList)) { - return; - } - resetTargetColumnNames(columnExprList); - resetSelectList(columnExprList); - } - - /** - * deal with the case that not all columns in table are in file - */ - private void padExprListIfNeeded(List columnExprList) { - if (isCsvFormat) { - return; - } - columnExprList.addAll( - functionGenTableColNames - .stream() - .map(ImportColumnDesc::new) - .collect(Collectors.toList()) - ); - } - - /** - * find and rewrite the derivative columns - * e.g. (v1,v2=v1+1,v3=v2+1) --> (v1, v2=v1+1, v3=v1+1+1) - */ - private void rewriteExpr(List columnDescList) { - Preconditions.checkNotNull(columnDescList, "columns should be not null"); - Preconditions.checkNotNull(targetTable, "target table is unset"); - LOG.debug("original columnExpr:{}", columnDescList); - Map derivativeColumns = Maps.newHashMap(); - columnDescList - .stream() - .filter(Predicates.not(ImportColumnDesc::isColumn)) - .forEach(desc -> { - final Expr expr = desc.getExpr(); - if (expr instanceof SlotRef) { - final String columnName = ((SlotRef) expr).getColumnName(); - if (derivativeColumns.containsKey(columnName)) { - desc.setExpr(derivativeColumns.get(columnName)); - } - } else { - recursiveRewrite(expr, derivativeColumns); - } - derivativeColumns.put(desc.getColumnName(), expr); - }); - // `tmp` columns with expr can be removed after expr rewritten - columnDescList.removeIf( - Predicates.not(columnDesc -> - columnDesc.isColumn() || Objects.nonNull(targetTable.getColumn(columnDesc.getColumnName())) - ) - ); - LOG.debug("rewrite result:{}", columnDescList); - } - - private void recursiveRewrite(Expr expr, Map derivativeColumns) { - final ArrayList children = expr.getChildren(); - if (CollectionUtils.isEmpty(children)) { - return; - } - for (int i = 0; i < children.size(); i++) { - Expr child = expr.getChild(i); - if (child instanceof SlotRef) { - final String columnName = ((SlotRef) child).getColumnName(); - if (derivativeColumns.containsKey(columnName)) { - expr.setChild(i, derivativeColumns.get(columnName)); - } - continue; - } - recursiveRewrite(child, derivativeColumns); - } - } - - /** - * record mapping from col name to csv-format-tvf-style col name - * - * @see selectColNameToCsvColName - */ - private void recordCsvColNames(List columnDescList) { - AtomicInteger counter = new AtomicInteger(1); - selectColNameToCsvColName = columnDescList.stream() - .filter(ImportColumnDesc::isColumn) - .collect(Collectors.toMap( - ImportColumnDesc::getColumnName, - name -> "c" + counter.getAndIncrement(), - (v1, v2) -> v1, - LinkedHashMap::new - )); - LOG.debug("select column name to csv colum name:{}", selectColNameToCsvColName); - } - - private List filterColumns(List columnExprList) { - Preconditions.checkNotNull(targetTable, "target table is unset"); - - // remove all `tmp` columns, which are not in target table - columnExprList.removeIf( - Predicates.and( - ImportColumnDesc::isColumn, - desc -> Objects.isNull(targetTable.getColumn(desc.getColumnName())), - desc -> functionGenTableColNames.contains(desc.getColumnName()) - ) - ); - - // deal with the case like: - // (k1, k2) SET(k1 = `upper(k1)`) - columnExprList = Lists.newArrayList(columnExprList.stream() - .collect(Collectors.toMap( - ImportColumnDesc::getColumnName, - Function.identity(), - (lhs, rhs) -> { - if (lhs.getExpr() != null && rhs.getExpr() == null) { - return lhs; - } - if (lhs.getExpr() == null && rhs.getExpr() != null) { - return rhs; - } - throw new IllegalArgumentException( - String.format("column `%s` specified twice", lhs.getColumnName())); - } - )).values()); - - // deal with the case that column in target table but not in tvf - columnExprList.removeIf(desc -> - !functionGenTableColNames.contains(desc.getColumnName()) - && Objects.nonNull(targetTable.getColumn(desc.getColumnName())) - && desc.isColumn() - ); - - LOG.debug("filtered result:{}", columnExprList); - return columnExprList; - } - - private void resetTargetColumnNames(List columnExprList) { - targetColumnNames = columnExprList - .stream() - .map(ImportColumnDesc::getColumnName) - .collect(Collectors.toList()); - LOG.debug("target cols:{}", targetColumnNames); - } - - private void resetSelectList(List columnExprList) { - if (isCsvFormat) { - rewriteExprColNameToCsvStyle(columnExprList); - } - LOG.debug("select list:{}", columnExprList); - final SelectList selectList = new SelectList(); - columnExprList.forEach(desc -> { - if (!desc.isColumn()) { - selectList.addItem(new SelectListItem(desc.getExpr(), desc.getColumnName())); - return; - } - - if (isCsvFormat) { - // use csv-style-column name and target column name as alias - final Expr slotRef = new SlotRef(null, selectColNameToCsvColName.get(desc.getColumnName())); - selectList.addItem(new SelectListItem(slotRef, desc.getColumnName())); - } else { - selectList.addItem(new SelectListItem(new SlotRef(null, desc.getColumnName()), null)); - } - }); - ((SelectStmt) getQueryStmt()).resetSelectList(selectList); - } - - private void rewriteExprColNameToCsvStyle(List columnExprList) { - Preconditions.checkNotNull(selectColNameToCsvColName, - "SelectColName To CsvColName is not recorded"); - columnExprList - .stream() - .filter(Predicates.not(ImportColumnDesc::isColumn)) - .forEach(desc -> rewriteSlotRefInExpr(desc.getExpr())); - - // rewrite where predicate and order by elements - final SelectStmt selectStmt = (SelectStmt) getQueryStmt(); - rewriteSlotRefInExpr(selectStmt.getWhereClause()); - if (selectStmt.getOrderByElements() != null) { - selectStmt.getOrderByElements().forEach(orderByElement -> rewriteSlotRefInExpr(orderByElement.getExpr())); - } - } - - private void rewriteSlotRefInExpr(Expr expr) { - final Predicate rewritePredicate = e -> e instanceof SlotRef - && selectColNameToCsvColName.containsKey(((SlotRef) e).getColumnName()); - final Consumer rewriteOperation = e -> { - final SlotRef slotRef = (SlotRef) e; - slotRef.setCol(selectColNameToCsvColName.get(slotRef.getColumnName())); - }; - - List slotRefs = Lists.newArrayList(); - expr.collect(rewritePredicate, slotRefs); - slotRefs.forEach(rewriteOperation); - } - - // ----------------------------------------------------------------------------------------- -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SparkLoadStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SparkLoadStmt.java deleted file mode 100644 index f75468f10ed7e7..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SparkLoadStmt.java +++ /dev/null @@ -1,99 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you 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. -// This file is copied from -// https://github.com/apache/impala/blob/branch-2.9.0/fe/src/main/java/org/apache/impala/StatementBase.java -// and modified by Doris - -package org.apache.doris.analysis; - -import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.Env; -import org.apache.doris.catalog.OlapTable; -import org.apache.doris.common.AnalysisException; -import org.apache.doris.common.DdlException; -import org.apache.doris.common.UserException; -import org.apache.doris.mysql.privilege.PrivPredicate; -import org.apache.doris.qe.ConnectContext; - -import com.google.common.base.Preconditions; - -import java.util.Collections; -import java.util.List; -import java.util.Map; - -public class SparkLoadStmt extends InsertStmt { - - private final DataDescription dataDescription; - - private final ResourceDesc resourceDesc; - - public SparkLoadStmt(LabelName label, List dataDescList, ResourceDesc resourceDesc, - Map properties, String comments) { - super(label, properties, comments); - Preconditions.checkState(dataDescList.size() == 1, - "spark load could only have one desc"); - this.dataDescription = dataDescList.get(0); - this.resourceDesc = resourceDesc; - } - - @Override - public List getDataDescList() { - return Collections.singletonList(dataDescription); - } - - @Override - public ResourceDesc getResourceDesc() { - return resourceDesc; - } - - @Override - public LoadType getLoadType() { - return LoadType.SPARK_LOAD; - } - - @Override - public void analyzeProperties() throws DdlException { - - } - - @Override - public void analyze(Analyzer analyzer) throws UserException { - super.analyze(analyzer); - label.analyze(analyzer); - Preconditions.checkNotNull(dataDescription, new AnalysisException("No data file in load statement.")); - Preconditions.checkNotNull(resourceDesc, new AnalysisException("Resource desc not found")); - String fullDbName = dataDescription.analyzeFullDbName(label.getDbName(), analyzer); - dataDescription.analyze(fullDbName); - resourceDesc.analyze(); - Database db = analyzer.getEnv().getInternalCatalog().getDbOrAnalysisException(fullDbName); - OlapTable table = db.getOlapTableOrAnalysisException(dataDescription.getTableName()); - dataDescription.checkKeyTypeForLoad(table); - // check resource usage privilege - if (!Env.getCurrentEnv().getAccessManager().checkResourcePriv(ConnectContext.get(), - resourceDesc.getName(), - PrivPredicate.USAGE)) { - throw new AnalysisException("USAGE denied to user '" + ConnectContext.get().getQualifiedUser() - + "'@'" + ConnectContext.get().getRemoteIP() - + "' for resource '" + resourceDesc.getName() + "'"); - } - } - - @Override - public String toSql() { - return super.toSql(); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/UnifiedLoadStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/UnifiedLoadStmt.java index 351fa116888ed2..e8d4e513223be6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/UnifiedLoadStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/UnifiedLoadStmt.java @@ -17,9 +17,7 @@ package org.apache.doris.analysis; -import org.apache.doris.analysis.StorageBackend.StorageType; import org.apache.doris.common.DdlException; -import org.apache.doris.qe.ConnectContext; import com.google.common.base.Preconditions; @@ -46,25 +44,12 @@ public void init() { public static UnifiedLoadStmt buildMysqlLoadStmt(DataDescription dataDescription, Map properties, String comment) { - final ConnectContext connectContext = ConnectContext.get(); - if (connectContext != null && connectContext.getSessionVariable().isEnableUnifiedLoad()) { - return new UnifiedLoadStmt(new MysqlLoadStmt(dataDescription, properties, comment)); - } return new UnifiedLoadStmt(new LoadStmt(dataDescription, properties, comment)); } public static UnifiedLoadStmt buildBrokerLoadStmt(LabelName label, List dataDescriptions, BrokerDesc brokerDesc, Map properties, String comment) throws DdlException { - - final ConnectContext connectContext = ConnectContext.get(); - if (connectContext != null && connectContext.getSessionVariable().isEnableUnifiedLoad()) { - if (brokerDesc != null && brokerDesc.getStorageType() == StorageType.S3) { - // for tvf solution validation - return new UnifiedLoadStmt(new S3TvfLoadStmt(label, dataDescriptions, brokerDesc, properties, comment)); - } - return new UnifiedLoadStmt(new BrokerLoadStmt(label, dataDescriptions, brokerDesc, properties, comment)); - } return new UnifiedLoadStmt(new LoadStmt(label, dataDescriptions, brokerDesc, properties, comment)); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 1a995801d40261..6bf7de619dbadb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -1319,12 +1319,6 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { @VariableMgr.VarAttr(name = FILE_SPLIT_SIZE, needForward = true) public long fileSplitSize = 0; - /** - * determine should we enable unified load (use insert stmt as the backend for all load) - */ - @VariableMgr.VarAttr(name = ENABLE_UNIFIED_LOAD, needForward = true) - public boolean enableUnifiedLoad = false; - @VariableMgr.VarAttr( name = ENABLE_PARQUET_LAZY_MAT, description = {"控制 parquet reader 是否启用延迟物化技术。默认为 true。", @@ -3158,10 +3152,6 @@ public int getVariableNumByVariableAnnotation(VariableAnnotation type) { return num; } - public boolean isEnableUnifiedLoad() { - return enableUnifiedLoad; - } - public boolean getEnablePipelineEngine() { return enablePipelineEngine || enablePipelineXEngine; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/S3TvfLoadStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/S3TvfLoadStmtTest.java deleted file mode 100644 index a6eb67b0616325..00000000000000 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/S3TvfLoadStmtTest.java +++ /dev/null @@ -1,244 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you 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 org.apache.doris.analysis; - -import org.apache.doris.analysis.BinaryPredicate.Operator; -import org.apache.doris.analysis.StorageBackend.StorageType; -import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.PrimitiveType; -import org.apache.doris.catalog.ScalarType; -import org.apache.doris.catalog.Table; -import org.apache.doris.common.AnalysisException; -import org.apache.doris.common.FeConstants; -import org.apache.doris.common.UserException; -import org.apache.doris.common.jmockit.Deencapsulation; -import org.apache.doris.common.util.SqlParserUtils; -import org.apache.doris.datasource.property.constants.S3Properties; -import org.apache.doris.datasource.property.constants.S3Properties.Env; -import org.apache.doris.load.loadv2.LoadTask.MergeType; -import org.apache.doris.tablefunction.S3TableValuedFunction; -import org.apache.doris.utframe.TestWithFeService; - -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; -import mockit.Expectations; -import mockit.Injectable; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; - -import java.io.StringReader; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Set; - -public class S3TvfLoadStmtTest extends TestWithFeService { - - private static final String ACCESS_KEY_VALUE = "ak"; - - private static final String SECRET_KEY_VALUE = "sk"; - - private static final String ENDPOINT_VALUE = "cos.ap-beijing.myqcloud.com"; - - private static final String REGION_VALUE = "ap-beijing"; - - private static final String DATA_URI = "s3://doris-build-1308700295/regression/load/data/part*"; - - private static final String FORMAT = "parquet"; - - private static final String TARGET_TABLE_NAME = "target"; - - private LabelName labelName; - - private BrokerDesc brokerDesc; - - private Set colNames; - - @BeforeAll - public void setUp() throws AnalysisException { - FeConstants.runningUnitTest = true; - - labelName = new LabelName("testDb", "testTbl"); - - final Map brokerProperties = Maps.newHashMap(); - brokerProperties.put(Env.ACCESS_KEY, ACCESS_KEY_VALUE); - brokerProperties.put(Env.SECRET_KEY, SECRET_KEY_VALUE); - brokerProperties.put(Env.ENDPOINT, ENDPOINT_VALUE); - brokerProperties.put(Env.REGION, REGION_VALUE); - brokerDesc = new BrokerDesc("s3", StorageType.S3, brokerProperties); - - colNames = Sets.newHashSet("k1", "k2", "k3", "k4"); - } - - @Test - public void testClauses() throws UserException { - final BinaryPredicate greater = new BinaryPredicate(Operator.GT, new IntLiteral(1), new IntLiteral(0)); - final BinaryPredicate less = new BinaryPredicate(Operator.LT, new IntLiteral(1), new IntLiteral(0)); - DataDescription dataDescription = buildDataDesc( - Lists.newArrayList(colNames), - greater, - less, - null - ); - final S3TvfLoadStmt s3TvfLoadStmt = new S3TvfLoadStmt(labelName, Collections.singletonList(dataDescription), - brokerDesc, - Maps.newHashMap(), "comment"); - final SelectStmt selectStmt = (SelectStmt) s3TvfLoadStmt.getQueryStmt(); - final Expr whereClause = Deencapsulation.getField(selectStmt, "whereClause"); - Assertions.assertEquals(whereClause, new CompoundPredicate(CompoundPredicate.Operator.AND, greater, less)); - } - - @Test - public void testTvfGeneration() { - DataDescription dataDescription = buildDataDesc( - Lists.newArrayList(colNames), - null, - null, - null - ); - final TableRef tvfRef = Deencapsulation.invoke(S3TvfLoadStmt.class, - "buildTvfRef", - dataDescription, brokerDesc); - Assertions.assertTrue(tvfRef instanceof TableValuedFunctionRef); - final S3TableValuedFunction tableFunction - = (S3TableValuedFunction) ((TableValuedFunctionRef) tvfRef).getTableFunction(); - final Map locationProperties = tableFunction.getLocationProperties(); - Assertions.assertEquals(locationProperties.get(S3Properties.ENDPOINT), ENDPOINT_VALUE); - Assertions.assertEquals(locationProperties.get(S3Properties.ACCESS_KEY), ACCESS_KEY_VALUE); - Assertions.assertEquals(locationProperties.get(S3Properties.SECRET_KEY), SECRET_KEY_VALUE); - Assertions.assertEquals(locationProperties.get(S3Properties.REGION), REGION_VALUE); - Assertions.assertEquals(tableFunction.getFilePath(), DATA_URI); - } - - @Injectable - Table targetTable; - - @Test - public void testColumnMappings() throws Exception { - // c1/c2/c3 in both file and table, and c5 is only in table - final List columnsDescList = getColumnsDescList( - "c1,c2,c3,c1=upper(c1), tmp_c4=c1 + 1, c5 = tmp_c4+1"); - DataDescription dataDescription = buildDataDesc(colNames, null, null, null); - new Expectations() { - { - dataDescription.getParsedColumnExprList(); - minTimes = 0; - result = columnsDescList; - - dataDescription.getFilePaths(); - minTimes = 0; - result = Collections.singletonList(DATA_URI); - - targetTable.getBaseSchema(); - minTimes = 0; - result = getBaseSchema(); - - targetTable.getColumn("c1"); - minTimes = 0; - result = new Column(); - - targetTable.getColumn("c2"); - minTimes = 0; - result = new Column(); - - targetTable.getColumn("c3"); - minTimes = 0; - result = new Column(); - - targetTable.getColumn("c5"); - minTimes = 0; - result = new Column(); - - targetTable.getColumn("tmp_c4"); - minTimes = 0; - result = null; - } - }; - final S3TvfLoadStmt s3TvfLoadStmt = new S3TvfLoadStmt(labelName, Collections.singletonList(dataDescription), - brokerDesc, null, "comment"); - s3TvfLoadStmt.setTargetTable(targetTable); - Deencapsulation.setField(s3TvfLoadStmt, "functionGenTableColNames", Sets.newHashSet("c1", "c2", "c3")); - - Deencapsulation.invoke(s3TvfLoadStmt, "rewriteExpr", columnsDescList); - Assertions.assertEquals(columnsDescList.size(), 5); - final String orig4 = "((upper(`c1`) + 1) + 1)"; - Assertions.assertEquals(orig4, columnsDescList.get(4).getExpr().toString()); - - final List filterColumns = Deencapsulation.invoke(s3TvfLoadStmt, - "filterColumns", columnsDescList); - Assertions.assertEquals(filterColumns.size(), 4); - } - - private static DataDescription buildDataDesc(Iterable columns, Expr fileFilter, Expr wherePredicate, - List mappingList) { - - return new DataDescription( - TARGET_TABLE_NAME, - null, - Collections.singletonList(DATA_URI), - Lists.newArrayList(columns), - null, - FORMAT, - null, - false, - mappingList, - fileFilter, - wherePredicate, - MergeType.APPEND, - null, - null, - null - ); - } - - private static List getColumnsDescList(String columns) throws Exception { - String columnsSQL = "COLUMNS (" + columns + ")"; - return ((ImportColumnsStmt) SqlParserUtils.getFirstStmt( - new org.apache.doris.analysis.SqlParser( - new org.apache.doris.analysis.SqlScanner(new StringReader(columnsSQL))))).getColumns(); - } - - private static List getBaseSchema() { - List columns = Lists.newArrayList(); - - Column c1 = new Column("c1", PrimitiveType.BIGINT); - c1.setIsKey(true); - c1.setIsAllowNull(false); - columns.add(c1); - - Column c2 = new Column("c2", ScalarType.createVarchar(25)); - c2.setIsKey(true); - c2.setIsAllowNull(true); - columns.add(c2); - - Column c3 = new Column("c3", PrimitiveType.BIGINT); - c3.setIsKey(true); - c3.setIsAllowNull(false); - columns.add(c3); - - Column c5 = new Column("c5", PrimitiveType.BIGINT); - c5.setIsKey(true); - c5.setIsAllowNull(true); - columns.add(c5); - - return columns; - } - -} diff --git a/regression-test/suites/external_table_p0/hive/test_partial_update_broker_load.groovy b/regression-test/suites/external_table_p0/hive/test_partial_update_broker_load.groovy index c225ca02758e0f..c0920d68a94f3b 100644 --- a/regression-test/suites/external_table_p0/hive/test_partial_update_broker_load.groovy +++ b/regression-test/suites/external_table_p0/hive/test_partial_update_broker_load.groovy @@ -81,7 +81,6 @@ suite("test_primary_key_partial_update_broker_load", "p0,external,hive,external_ wait_for_load_result(test_load_label, tableName) qt_sql """select * from ${tableName} order by id;""" - sql "set enable_unified_load=true;" sql "sync;" def test_load_label2 = UUID.randomUUID().toString().replaceAll("-", "") load_from_hdfs(tableName, test_load_label2, "hdfs://${externalEnvIp}:${hdfs_port}/user/doris/preinstalled_data/data_case/partial_update/update2.csv", "csv", brokerName, hdfsUser, hdfsPasswd) diff --git a/regression-test/suites/load_p0/broker_load/test_array_load.groovy b/regression-test/suites/load_p0/broker_load/test_array_load.groovy index 5ef0bf7d18d26d..fe2985aff4aa0c 100644 --- a/regression-test/suites/load_p0/broker_load/test_array_load.groovy +++ b/regression-test/suites/load_p0/broker_load/test_array_load.groovy @@ -274,7 +274,6 @@ suite("test_array_load", "load_p0") { // test unified load try { sql "DROP TABLE IF EXISTS ${testTable}" - sql """ set enable_unified_load=true; """ create_test_table.call(testTable) @@ -286,7 +285,6 @@ suite("test_array_load", "load_p0") { } finally { try_sql("DROP TABLE IF EXISTS ${testTable}") - sql """ set enable_unified_load=false; """ } // case7: import array data by hdfs in csv format and enable vectorized @@ -307,7 +305,6 @@ suite("test_array_load", "load_p0") { // test unified load try { sql "DROP TABLE IF EXISTS ${testTable}" - sql """ set enable_unified_load=true; """ create_test_table.call(testTable) @@ -319,7 +316,6 @@ suite("test_array_load", "load_p0") { } finally { try_sql("DROP TABLE IF EXISTS ${testTable}") - sql """ set enable_unified_load=false; """ } // case9: import array data by hdfs in orc format and enable vectorized @@ -340,7 +336,6 @@ suite("test_array_load", "load_p0") { // test unified load try { sql "DROP TABLE IF EXISTS ${testTable}" - sql """ set enable_unified_load=true; """ create_test_table.call(testTable) @@ -352,7 +347,6 @@ suite("test_array_load", "load_p0") { } finally { try_sql("DROP TABLE IF EXISTS ${testTable}") - sql """ set enable_unified_load=false; """ } // case11: import array data by hdfs in parquet format and enable vectorized @@ -373,7 +367,6 @@ suite("test_array_load", "load_p0") { // test unified load try { sql "DROP TABLE IF EXISTS ${testTable}" - sql """ set enable_unified_load=true; """ create_test_table.call(testTable) @@ -385,7 +378,6 @@ suite("test_array_load", "load_p0") { } finally { try_sql("DROP TABLE IF EXISTS ${testTable}") - sql """ set enable_unified_load=false; """ } // case13: import array data by hdfs in orc format(with array type) and enable vectorized @@ -406,7 +398,6 @@ suite("test_array_load", "load_p0") { // test unified load try { sql "DROP TABLE IF EXISTS ${testTable}" - sql """ set enable_unified_load=true; """ create_test_table.call(testTable) @@ -418,7 +409,6 @@ suite("test_array_load", "load_p0") { } finally { try_sql("DROP TABLE IF EXISTS ${testTable}") - sql """ set enable_unified_load=false; """ } } } diff --git a/regression-test/suites/load_p0/mysql_load/test_mysql_load_unified.groovy b/regression-test/suites/load_p0/mysql_load/test_mysql_load_unified.groovy index e4c51db1104171..c438d41e2ecf9c 100644 --- a/regression-test/suites/load_p0/mysql_load/test_mysql_load_unified.groovy +++ b/regression-test/suites/load_p0/mysql_load/test_mysql_load_unified.groovy @@ -54,7 +54,6 @@ suite("test_mysql_load_unified", "p0") { // test unified load try { sql "DROP TABLE IF EXISTS ${tableName}" - sql """ set enable_unified_load=true; """ sql """ CREATE TABLE IF NOT EXISTS ${tableName} ( @@ -86,7 +85,6 @@ suite("test_mysql_load_unified", "p0") { } finally { try_sql("DROP TABLE IF EXISTS ${tableName}") - sql """ set enable_unified_load=false; """ } } diff --git a/regression-test/suites/load_p2/broker_load/test_tvf_based_broker_load.groovy b/regression-test/suites/load_p2/broker_load/test_tvf_based_broker_load.groovy index a6af747de02973..35ce6bb6e27c93 100644 --- a/regression-test/suites/load_p2/broker_load/test_tvf_based_broker_load.groovy +++ b/regression-test/suites/load_p2/broker_load/test_tvf_based_broker_load.groovy @@ -201,9 +201,8 @@ suite("test_tvf_based_broker_load_p2", "p2") { logger.info("Submit load with lable: $uuid, table: $table, path: $path") } - // test unified load + // test load if (enabled != null && enabled.equalsIgnoreCase("true")) { - sql """ set enable_unified_load=true; """ def uuids = [] try { def i = 0 @@ -237,7 +236,6 @@ suite("test_tvf_based_broker_load_p2", "p2") { for (String table in tables) { sql new File("""${context.file.parent}/ddl/${table}_drop.sql""").text } - sql """ set enable_unified_load=false; """ } } } From b8a337468c3063854e0df43bfb01e93e214fc3c2 Mon Sep 17 00:00:00 2001 From: yujun Date: Wed, 17 Jan 2024 09:02:37 +0800 Subject: [PATCH 049/200] [feature](merge-cloud) Add drop cloud table (#30032) Co-authored-by: Luwei <814383175@qq.com> Co-authored-by: deardeng <565620795@qq.com> Co-authored-by: Gavin Chou Co-authored-by: Lightman <31928846+Lchangliang@users.noreply.github.com> Co-authored-by: zhengyu Co-authored-by: Lei Zhang <1091517373@qq.com> Co-authored-by: AlexYue Co-authored-by: Xiaocc <598887962@qq.com> Co-authored-by: panDing19 <56944854+panDing19@users.noreply.github.com> Co-authored-by: plat1ko Co-authored-by: zhangdong <493738387@qq.com> Co-authored-by: walter --- .../java/org/apache/doris/common/Config.java | 3 + .../java/org/apache/doris/catalog/Env.java | 29 +--- .../apache/doris/catalog/TempPartitions.java | 8 +- .../apache/doris/cloud/catalog/CloudEnv.java | 5 + .../datasource/CloudInternalCatalog.java | 163 +++++++++++++++++- .../doris/datasource/InternalCatalog.java | 58 ++++++- 6 files changed, 228 insertions(+), 38 deletions(-) diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index 2a8a2ed53adc8d..baf46177bc1035 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -2463,6 +2463,9 @@ public static boolean isNotCloudMode() { @ConfField(mutable = true) public static int sts_duration = 3600; + @ConfField(mutable = true) + public static int drop_rpc_retry_num = 200; + @ConfField public static int cloud_meta_service_rpc_failed_retry_times = 200; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index b72544365aa38a..222eb195b3480c 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -251,7 +251,6 @@ import org.apache.doris.task.AgentBatchTask; import org.apache.doris.task.AgentTaskExecutor; import org.apache.doris.task.CompactionTask; -import org.apache.doris.task.DropReplicaTask; import org.apache.doris.task.MasterTaskExecutor; import org.apache.doris.task.PriorityMasterTaskExecutor; import org.apache.doris.thrift.BackendService; @@ -5684,33 +5683,11 @@ public void onEraseOlapTable(OlapTable olapTable, boolean isReplay) { } } - if (!isReplay && !Env.isCheckpointThread()) { - // drop all replicas - AgentBatchTask batchTask = new AgentBatchTask(); - for (Partition partition : olapTable.getAllPartitions()) { - List allIndices = partition.getMaterializedIndices(IndexExtState.ALL); - for (MaterializedIndex materializedIndex : allIndices) { - long indexId = materializedIndex.getId(); - int schemaHash = olapTable.getSchemaHashByIndexId(indexId); - for (Tablet tablet : materializedIndex.getTablets()) { - long tabletId = tablet.getId(); - List replicas = tablet.getReplicas(); - for (Replica replica : replicas) { - long backendId = replica.getBackendId(); - long replicaId = replica.getId(); - DropReplicaTask dropTask = new DropReplicaTask(backendId, tabletId, - replicaId, schemaHash, true); - batchTask.addTask(dropTask); - } // end for replicas - } // end for tablets - } // end for indices - } // end for partitions - AgentTaskExecutor.submit(batchTask); - } - // TODO: does checkpoint need update colocate index ? // colocation Env.getCurrentColocateIndex().removeTable(olapTable.getId()); + + getInternalCatalog().eraseTableDropBackendReplicas(olapTable, isReplay); } public void onErasePartition(Partition partition) { @@ -5721,6 +5698,8 @@ public void onErasePartition(Partition partition) { invertedIndex.deleteTablet(tablet.getId()); } } + + getInternalCatalog().erasePartitionDropBackendReplicas(Lists.newArrayList(partition)); } public void cleanTrash(AdminCleanTrashStmt stmt) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TempPartitions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TempPartitions.java index 9cd2d61bf91bd1..aebcae2e51d5ef 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TempPartitions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TempPartitions.java @@ -17,7 +17,6 @@ package org.apache.doris.catalog; -import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.persist.gson.GsonPostProcessable; @@ -78,12 +77,7 @@ public void dropPartition(String partitionName, boolean needDropTablet) { idToPartition.remove(partition.getId()); nameToPartition.remove(partitionName); if (needDropTablet) { - TabletInvertedIndex invertedIndex = Env.getCurrentInvertedIndex(); - for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { - for (Tablet tablet : index.getTablets()) { - invertedIndex.deleteTablet(tablet.getId()); - } - } + Env.getCurrentEnv().onErasePartition(partition); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java index 8d49783f3ca2c0..85162fec825246 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java @@ -19,8 +19,13 @@ import org.apache.doris.catalog.Env; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + public class CloudEnv extends Env { + private static final Logger LOG = LogManager.getLogger(CloudEnv.class); + public CloudEnv(boolean isCheckpointCatalog) { super(isCheckpointCatalog); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java index 13f04fd36df825..f2e3f10c2800fe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java @@ -27,9 +27,11 @@ import org.apache.doris.catalog.Index; import org.apache.doris.catalog.KeysType; import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.MaterializedIndex.IndexState; import org.apache.doris.catalog.MaterializedIndexMeta; import org.apache.doris.catalog.MetaIdGenerator.IdGeneratorBuffer; +import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Replica; import org.apache.doris.catalog.Replica.ReplicaState; @@ -54,6 +56,7 @@ import org.apache.doris.thrift.TTabletType; import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import doris.segment_v2.SegmentV2; import org.apache.logging.log4j.LogManager; @@ -498,12 +501,164 @@ private void sendCreateTabletsRpc(Cloud.CreateTabletsRequest.Builder requestBuil // END CREATE TABLE + // BEGIN DROP TABLE + + @Override + public void eraseTableDropBackendReplicas(OlapTable olapTable, boolean isReplay) { + if (!Env.getCurrentEnv().isMaster()) { + return; + } + + List indexs = Lists.newArrayList(); + for (Partition partition : olapTable.getAllPartitions()) { + List allIndices = partition.getMaterializedIndices(IndexExtState.ALL); + for (MaterializedIndex materializedIndex : allIndices) { + long indexId = materializedIndex.getId(); + indexs.add(indexId); + } + } + + int tryCnt = 0; + while (true) { + if (tryCnt++ > Config.drop_rpc_retry_num) { + LOG.warn("failed to drop index {} of table {}, try cnt {} reaches maximum retry count", + indexs, olapTable.getId(), tryCnt); + break; + } + + try { + if (indexs.isEmpty()) { + break; + } + dropMaterializedIndex(olapTable.getId(), indexs); + } catch (Exception e) { + LOG.warn("failed to drop index {} of table {}, try cnt {}, execption {}", + indexs, olapTable.getId(), tryCnt, e); + try { + Thread.sleep(3000); + } catch (InterruptedException ie) { + LOG.warn("Thread sleep is interrupted"); + } + continue; + } + break; + } + } + + @Override + public void erasePartitionDropBackendReplicas(List partitions) { + if (!Env.getCurrentEnv().isMaster() || partitions.isEmpty()) { + return; + } + + long tableId = -1; + List partitionIds = Lists.newArrayList(); + List indexIds = Lists.newArrayList(); + for (Partition partition : partitions) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { + indexIds.add(index.getId()); + if (tableId == -1) { + tableId = ((CloudReplica) index.getTablets().get(0).getReplicas().get(0)).getTableId(); + } + } + partitionIds.add(partition.getId()); + } + + CloudPartition partition0 = (CloudPartition) partitions.get(0); + + int tryCnt = 0; + while (true) { + if (tryCnt++ > Config.drop_rpc_retry_num) { + LOG.warn("failed to drop partition {} of table {}, try cnt {} reaches maximum retry count", + partitionIds, tableId, tryCnt); + break; + } + try { + dropCloudPartition(partition0.getDbId(), tableId, partitionIds, indexIds); + } catch (Exception e) { + LOG.warn("failed to drop partition {} of table {}, try cnt {}, execption {}", + partitionIds, tableId, tryCnt, e); + try { + Thread.sleep(3000); + } catch (InterruptedException ie) { + LOG.warn("Thread sleep is interrupted"); + } + continue; + } + break; + } + } + + private void dropCloudPartition(long dbId, long tableId, List partitionIds, List indexIds) + throws DdlException { + Cloud.PartitionRequest.Builder partitionRequestBuilder = + Cloud.PartitionRequest.newBuilder(); + partitionRequestBuilder.setCloudUniqueId(Config.cloud_unique_id); + partitionRequestBuilder.setTableId(tableId); + partitionRequestBuilder.addAllPartitionIds(partitionIds); + partitionRequestBuilder.addAllIndexIds(indexIds); + if (dbId > 0) { + partitionRequestBuilder.setDbId(dbId); + } + final Cloud.PartitionRequest partitionRequest = partitionRequestBuilder.build(); + + Cloud.PartitionResponse response = null; + int tryTimes = 0; + while (tryTimes++ < Config.meta_service_rpc_retry_times) { + try { + response = MetaServiceProxy.getInstance().dropPartition(partitionRequest); + if (response.getStatus().getCode() != Cloud.MetaServiceCode.KV_TXN_CONFLICT) { + break; + } + } catch (RpcException e) { + LOG.warn("tryTimes:{}, dropPartition RpcException", tryTimes, e); + if (tryTimes + 1 >= Config.meta_service_rpc_retry_times) { + throw new DdlException(e.getMessage()); + } + } + sleepSeveralMs(); + } + + if (response.getStatus().getCode() != Cloud.MetaServiceCode.OK) { + LOG.warn("dropPartition response: {} ", response); + throw new DdlException(response.getStatus().getMsg()); + } + } + + private void dropMaterializedIndex(Long tableId, List indexIds) throws DdlException { + Cloud.IndexRequest.Builder indexRequestBuilder = Cloud.IndexRequest.newBuilder(); + indexRequestBuilder.setCloudUniqueId(Config.cloud_unique_id); + indexRequestBuilder.addAllIndexIds(indexIds); + indexRequestBuilder.setTableId(tableId); + final Cloud.IndexRequest indexRequest = indexRequestBuilder.build(); + + Cloud.IndexResponse response = null; + int tryTimes = 0; + while (tryTimes++ < Config.meta_service_rpc_retry_times) { + try { + response = MetaServiceProxy.getInstance().dropIndex(indexRequest); + if (response.getStatus().getCode() != Cloud.MetaServiceCode.KV_TXN_CONFLICT) { + break; + } + } catch (RpcException e) { + LOG.warn("tryTimes:{}, dropIndex RpcException", tryTimes, e); + if (tryTimes + 1 >= Config.meta_service_rpc_retry_times) { + throw new DdlException(e.getMessage()); + } + } + sleepSeveralMs(); + } + + if (response.getStatus().getCode() != Cloud.MetaServiceCode.OK) { + LOG.warn("dropIndex response: {} ", response); + throw new DdlException(response.getStatus().getMsg()); + } + } + + // END DROP TABLE + @Override protected void checkAvailableCapacity(Database db) throws DdlException { - // check cluster capacity - Env.getCurrentSystemInfo().checkAvailableCapacity(); - // check db quota - db.checkQuota(); } private void sleepSeveralMs() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java index 4671daba0cb10f..cf270a2c091684 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java @@ -156,6 +156,7 @@ import org.apache.doris.task.AgentTaskExecutor; import org.apache.doris.task.AgentTaskQueue; import org.apache.doris.task.CreateReplicaTask; +import org.apache.doris.task.DropReplicaTask; import org.apache.doris.thrift.TCompressionType; import org.apache.doris.thrift.TStatusCode; import org.apache.doris.thrift.TStorageFormat; @@ -965,6 +966,38 @@ public void replayRecoverTable(RecoverInfo info) throws MetaNotFoundException, D } } + public void eraseTableDropBackendReplicas(OlapTable olapTable, boolean isReplay) { + if (isReplay || Env.isCheckpointThread()) { + return; + } + + // drop all replicas + AgentBatchTask batchTask = new AgentBatchTask(); + for (Partition partition : olapTable.getAllPartitions()) { + List allIndices = partition.getMaterializedIndices(IndexExtState.ALL); + for (MaterializedIndex materializedIndex : allIndices) { + long indexId = materializedIndex.getId(); + int schemaHash = olapTable.getSchemaHashByIndexId(indexId); + for (Tablet tablet : materializedIndex.getTablets()) { + long tabletId = tablet.getId(); + List replicas = tablet.getReplicas(); + for (Replica replica : replicas) { + long backendId = replica.getBackendId(); + long replicaId = replica.getId(); + DropReplicaTask dropTask = new DropReplicaTask(backendId, tabletId, + replicaId, schemaHash, true); + batchTask.addTask(dropTask); + } // end for replicas + } // end for tablets + } // end for indices + } // end for partitions + AgentTaskExecutor.submit(batchTask); + } + + public void erasePartitionDropBackendReplicas(List partitions) { + // no need send be delete task, when be report its tablets, fe will send delete task then. + } + private void unprotectAddReplica(OlapTable olapTable, ReplicaPersistInfo info) { LOG.debug("replay add a replica {}", info); Partition partition = olapTable.getPartition(info.getPartitionId()); @@ -3085,6 +3118,7 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti // Things may be changed outside the table lock. olapTable = (OlapTable) db.getTableOrDdlException(copiedTbl.getId()); olapTable.writeLockOrDdlException(); + List oldPartitions = Lists.newArrayList(); try { olapTable.checkNormalStateForAlter(); // check partitions @@ -3141,7 +3175,7 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti } // replace - truncateTableInternal(olapTable, newPartitions, truncateEntireTable); + oldPartitions = truncateTableInternal(olapTable, newPartitions, truncateEntireTable); // write edit log TruncateTableInfo info = @@ -3152,6 +3186,9 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti } finally { olapTable.writeUnlock(); } + + erasePartitionDropBackendReplicas(oldPartitions); + if (truncateEntireTable) { // Drop the whole table stats after truncate the entire table Env.getCurrentEnv().getAnalysisManager().dropStats(olapTable); @@ -3162,11 +3199,14 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti LOG.info("finished to truncate table {}, partitions: {}", tblRef.getName().toSql(), tblRef.getPartitionNames()); } - private void truncateTableInternal(OlapTable olapTable, List newPartitions, boolean isEntireTable) { + private List truncateTableInternal(OlapTable olapTable, List newPartitions, + boolean isEntireTable) { // use new partitions to replace the old ones. + List oldPartitions = Lists.newArrayList(); Set oldTabletIds = Sets.newHashSet(); for (Partition newPartition : newPartitions) { Partition oldPartition = olapTable.replacePartition(newPartition); + oldPartitions.add(oldPartition); // save old tablets to be removed for (MaterializedIndex index : oldPartition.getMaterializedIndices(IndexExtState.ALL)) { index.getTablets().forEach(t -> { @@ -3176,6 +3216,12 @@ private void truncateTableInternal(OlapTable olapTable, List newParti } if (isEntireTable) { + Set oldPartitionsIds = oldPartitions.stream().map(Partition::getId).collect(Collectors.toSet()); + for (Partition partition : olapTable.getTempPartitions()) { + if (!oldPartitionsIds.contains(partition.getId())) { + oldPartitions.add(partition); + } + } // drop all temp partitions olapTable.dropAllTempPartitions(); } @@ -3184,9 +3230,12 @@ private void truncateTableInternal(OlapTable olapTable, List newParti for (Long tabletId : oldTabletIds) { Env.getCurrentInvertedIndex().deleteTablet(tabletId); } + + return oldPartitions; } public void replayTruncateTable(TruncateTableInfo info) throws MetaNotFoundException { + List oldPartitions = Lists.newArrayList(); Database db = (Database) getDbOrMetaException(info.getDbId()); OlapTable olapTable = (OlapTable) db.getTableOrMetaException(info.getTblId(), TableType.OLAP); olapTable.writeLock(); @@ -3196,6 +3245,7 @@ public void replayTruncateTable(TruncateTableInfo info) throws MetaNotFoundExcep // add tablet to inverted index TabletInvertedIndex invertedIndex = Env.getCurrentInvertedIndex(); for (Partition partition : info.getPartitions()) { + oldPartitions.add(partition); long partitionId = partition.getId(); TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty(partitionId) .getStorageMedium(); @@ -3216,6 +3266,10 @@ public void replayTruncateTable(TruncateTableInfo info) throws MetaNotFoundExcep } finally { olapTable.writeUnlock(); } + + if (!Env.isCheckpointThread()) { + erasePartitionDropBackendReplicas(oldPartitions); + } } public void replayAlterExternalTableSchema(String dbName, String tableName, List newSchema) From e556cff8240f1c3ae6470a5e3a107005df5b9f5a Mon Sep 17 00:00:00 2001 From: jakevin Date: Wed, 17 Jan 2024 11:08:18 +0800 Subject: [PATCH 050/200] [fix](Nereids): find hash condition after infer predicate (#30026) --- .../doris/nereids/jobs/executor/Rewriter.java | 3 ++- .../data/nereids_p0/hint/fix_leading.out | 4 ++-- .../data/nereids_p0/hint/test_leading.out | 16 ++++++++-------- 3 files changed, 12 insertions(+), 11 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java index cbfe3d28ac8d6b..b2f1569ea1d7bc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java @@ -226,7 +226,7 @@ public class Rewriter extends AbstractBatchJobExecutor { // but top-down traverse can not cover this case in one iteration, so bottom-up is more // efficient because it can find the new plans and apply transform wherever it is bottomUp(RuleSet.PUSH_DOWN_FILTERS), - //after push down, some new filters are generated, which needs to be optimized. (example: tpch q19) + // after push down, some new filters are generated, which needs to be optimized. (example: tpch q19) topDown(new ExpressionOptimization()), topDown( new MergeFilters(), @@ -263,6 +263,7 @@ public class Rewriter extends AbstractBatchJobExecutor { // after eliminate outer join, we can move some filters to join.otherJoinConjuncts, // this can help to translate plan to backend topDown(new PushFilterInsideJoin()), + topDown(new FindHashConditionForJoin()), topDown(new ExpressionNormalization()) ), diff --git a/regression-test/data/nereids_p0/hint/fix_leading.out b/regression-test/data/nereids_p0/hint/fix_leading.out index 898fe5882b906c..1410c8e0e6e62d 100644 --- a/regression-test/data/nereids_p0/hint/fix_leading.out +++ b/regression-test/data/nereids_p0/hint/fix_leading.out @@ -9,9 +9,9 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------PhysicalOlapScan[t2] --------PhysicalDistribute[DistributionSpecHash] -----------NestedLoopJoin[CROSS_JOIN](t3.c3 = t4.c4) +----------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() ------------PhysicalOlapScan[t3] -------------PhysicalDistribute[DistributionSpecReplicated] +------------PhysicalDistribute[DistributionSpecHash] --------------PhysicalOlapScan[t4] Hint log: diff --git a/regression-test/data/nereids_p0/hint/test_leading.out b/regression-test/data/nereids_p0/hint/test_leading.out index fe3831a9fc4375..29811a95161c56 100644 --- a/regression-test/data/nereids_p0/hint/test_leading.out +++ b/regression-test/data/nereids_p0/hint/test_leading.out @@ -2609,7 +2609,7 @@ PhysicalResultSink ------------PhysicalProject --------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------NestedLoopJoin[CROSS_JOIN](t1.c1 = t3.c3) +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ----------------PhysicalProject ------------------PhysicalOlapScan[t1] ----------------PhysicalDistribute[DistributionSpecReplicated] @@ -2631,7 +2631,7 @@ PhysicalResultSink ------------PhysicalProject --------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------NestedLoopJoin[CROSS_JOIN](t1.c1 = t3.c3) +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ----------------PhysicalProject ------------------PhysicalOlapScan[t3] ----------------PhysicalDistribute[DistributionSpecReplicated] @@ -2745,7 +2745,7 @@ PhysicalResultSink ------------PhysicalProject --------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------NestedLoopJoin[CROSS_JOIN](t1.c1 = t3.c3) +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ----------------PhysicalProject ------------------PhysicalOlapScan[t1] ----------------PhysicalDistribute[DistributionSpecReplicated] @@ -2767,7 +2767,7 @@ PhysicalResultSink ------------PhysicalProject --------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------NestedLoopJoin[CROSS_JOIN](t1.c1 = t3.c3) +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ----------------PhysicalProject ------------------PhysicalOlapScan[t3] ----------------PhysicalDistribute[DistributionSpecReplicated] @@ -2881,10 +2881,10 @@ PhysicalResultSink ------------PhysicalProject --------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecHash] ---------------NestedLoopJoin[CROSS_JOIN](t1.c1 = t3.c3) +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ----------------PhysicalProject ------------------PhysicalOlapScan[t1] -----------------PhysicalDistribute[DistributionSpecReplicated] +----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] @@ -2903,10 +2903,10 @@ PhysicalResultSink ------------PhysicalProject --------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecHash] ---------------NestedLoopJoin[CROSS_JOIN](t1.c1 = t3.c3) +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ----------------PhysicalProject ------------------PhysicalOlapScan[t3] -----------------PhysicalDistribute[DistributionSpecReplicated] +----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t1] From 5f70c0b408c4c98b5fc541c28c8a86264e2647fe Mon Sep 17 00:00:00 2001 From: Gabriel Date: Wed, 17 Jan 2024 12:22:49 +0800 Subject: [PATCH 051/200] [pipelineX](fix) Fix runtime filter dependency DCHECK failed (#30038) --- .../pipeline/pipeline_x/pipeline_x_task.cpp | 22 ++++++++----------- 1 file changed, 9 insertions(+), 13 deletions(-) diff --git a/be/src/pipeline/pipeline_x/pipeline_x_task.cpp b/be/src/pipeline/pipeline_x/pipeline_x_task.cpp index 574f4e7e5cb78b..1b1c04eb8145cb 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_task.cpp +++ b/be/src/pipeline/pipeline_x/pipeline_x_task.cpp @@ -195,20 +195,16 @@ Status PipelineXTask::_open() { _dry_run = _sink->should_dry_run(_state); for (auto& o : _operators) { auto* local_state = _state->get_local_state(o->operator_id()); - for (size_t i = 0; i < 2; i++) { - auto st = local_state->open(_state); - if (st.is()) { - DCHECK(_filter_dependency); - _blocked_dep = _filter_dependency->is_blocked_by(this); - if (_blocked_dep) { - set_state(PipelineTaskState::BLOCKED_FOR_RF); - RETURN_IF_ERROR(st); - } else if (i == 1) { - CHECK(false) << debug_string(); - } - } else { - break; + auto st = local_state->open(_state); + if (st.is()) { + DCHECK(_filter_dependency); + _blocked_dep = _filter_dependency->is_blocked_by(this); + if (_blocked_dep) { + set_state(PipelineTaskState::BLOCKED_FOR_RF); + RETURN_IF_ERROR(st); } + } else { + RETURN_IF_ERROR(st); } } RETURN_IF_ERROR(_state->get_sink_local_state(_sink->operator_id())->open(_state)); From 140bfd128806e24dbefc1b2e2a50ddff44640b37 Mon Sep 17 00:00:00 2001 From: HHoflittlefish777 <77738092+HHoflittlefish777@users.noreply.github.com> Date: Wed, 17 Jan 2024 13:57:53 +0800 Subject: [PATCH 052/200] [fix](multi-table-load) fix multi table load can not finish (#29957) --- be/src/io/fs/multi_table_pipe.cpp | 32 ++++++++++--------- be/src/io/fs/multi_table_pipe.h | 19 ++++++++--- .../routine_load_task_executor.cpp | 2 +- 3 files changed, 33 insertions(+), 20 deletions(-) diff --git a/be/src/io/fs/multi_table_pipe.cpp b/be/src/io/fs/multi_table_pipe.cpp index 36976a239725bf..916f815173987c 100644 --- a/be/src/io/fs/multi_table_pipe.cpp +++ b/be/src/io/fs/multi_table_pipe.cpp @@ -209,7 +209,7 @@ Status MultiTablePipe::exec_plans(ExecEnv* exec_env, std::vector para _unplanned_pipes.size(), _planned_pipes.size(), params.size()); _unplanned_pipes.clear(); - _inflight_plan_cnt += params.size(); + _inflight_cnt += params.size(); for (auto& plan : params) { if (!plan.__isset.table_name || _planned_pipes.find(plan.table_name) == _planned_pipes.end()) { @@ -263,20 +263,9 @@ Status MultiTablePipe::exec_plans(ExecEnv* exec_env, std::vector para _status = *status; } - --_inflight_plan_cnt; - if (_inflight_plan_cnt == 0 && is_consume_finished()) { - _ctx->number_total_rows = _number_total_rows; - _ctx->number_loaded_rows = _number_loaded_rows; - _ctx->number_filtered_rows = _number_filtered_rows; - _ctx->number_unselected_rows = _number_unselected_rows; - _ctx->commit_infos = _tablet_commit_infos; - LOG(INFO) << "all plan for multi-table load complete. number_total_rows=" - << _ctx->number_total_rows - << " number_loaded_rows=" << _ctx->number_loaded_rows - << " number_filtered_rows=" << _ctx->number_filtered_rows - << " number_unselected_rows=" << _ctx->number_unselected_rows; - _ctx->promise.set_value( - _status); // when all done, finish the routine load task + auto inflight_cnt = _inflight_cnt.fetch_sub(1); + if (inflight_cnt == 1 && is_consume_finished()) { + _handle_consumer_finished(); } })); } @@ -303,6 +292,19 @@ Status MultiTablePipe::exec_plans(ExecEnv* exec_env, std::vector para #endif +void MultiTablePipe::_handle_consumer_finished() { + _ctx->number_total_rows = _number_total_rows; + _ctx->number_loaded_rows = _number_loaded_rows; + _ctx->number_filtered_rows = _number_filtered_rows; + _ctx->number_unselected_rows = _number_unselected_rows; + _ctx->commit_infos = _tablet_commit_infos; + LOG(INFO) << "all plan for multi-table load complete. number_total_rows=" + << _ctx->number_total_rows << " number_loaded_rows=" << _ctx->number_loaded_rows + << " number_filtered_rows=" << _ctx->number_filtered_rows + << " number_unselected_rows=" << _ctx->number_unselected_rows; + _ctx->promise.set_value(_status); // when all done, finish the routine load task +} + Status MultiTablePipe::put_pipe(const TUniqueId& pipe_id, std::shared_ptr pipe) { std::lock_guard l(_pipe_map_lock); diff --git a/be/src/io/fs/multi_table_pipe.h b/be/src/io/fs/multi_table_pipe.h index 694794638afbe2..36f6ec68b17f3c 100644 --- a/be/src/io/fs/multi_table_pipe.h +++ b/be/src/io/fs/multi_table_pipe.h @@ -46,7 +46,13 @@ class MultiTablePipe : public KafkaConsumerPipe { // request and execute plans for unplanned pipes Status request_and_exec_plans(); - void set_consume_finished() { _consume_finished.store(true, std::memory_order_release); } + void handle_consume_finished() { + _set_consume_finished(); + auto inflight_cnt = _inflight_cnt.fetch_sub(1); + if (inflight_cnt == 1) { + _handle_consumer_finished(); + } + } bool is_consume_finished() { return _consume_finished.load(std::memory_order_acquire); } @@ -71,25 +77,30 @@ class MultiTablePipe : public KafkaConsumerPipe { template Status exec_plans(ExecEnv* exec_env, std::vector params); + void _set_consume_finished() { _consume_finished.store(true, std::memory_order_release); } + + void _handle_consumer_finished(); + private: std::unordered_map _planned_pipes; std::unordered_map _unplanned_pipes; std::atomic _unplanned_row_cnt {0}; // trigger plan request when exceed threshold - std::atomic _inflight_plan_cnt {0}; // how many plan fragment are executing? + // inflight count, when it is zero, means consume and all plans is finished + std::atomic _inflight_cnt {1}; std::atomic _consume_finished {false}; // note: Use raw pointer here to avoid cycle reference with StreamLoadContext. // Life cycle of MultiTablePipe is under control of StreamLoadContext, which means StreamLoadContext is created // before NultiTablePipe and released after it. It is safe to use raw pointer here. StreamLoadContext* _ctx = nullptr; Status _status; // save the first error status of all executing plan fragment -#ifndef BE_TEST + std::mutex _tablet_commit_infos_lock; std::vector _tablet_commit_infos; // collect from each plan fragment std::atomic _number_total_rows {0}; std::atomic _number_loaded_rows {0}; std::atomic _number_filtered_rows {0}; std::atomic _number_unselected_rows {0}; -#endif + std::mutex _pipe_map_lock; std::unordered_map> _pipe_map; diff --git a/be/src/runtime/routine_load/routine_load_task_executor.cpp b/be/src/runtime/routine_load/routine_load_task_executor.cpp index dc6b855cd5a409..d22f2bb4a8c2ac 100644 --- a/be/src/runtime/routine_load/routine_load_task_executor.cpp +++ b/be/src/runtime/routine_load/routine_load_task_executor.cpp @@ -350,7 +350,7 @@ void RoutineLoadTaskExecutor::exec_task(std::shared_ptr ctx, HANDLE_ERROR(multi_table_pipe->request_and_exec_plans(), "multi tables task executes plan error"); // need memory order - multi_table_pipe->set_consume_finished(); + multi_table_pipe->handle_consume_finished(); HANDLE_ERROR(kafka_pipe->finish(), "finish multi table task failed"); } From 4ba7bf6f5dbc3e874f05f6afc15f6574038e6059 Mon Sep 17 00:00:00 2001 From: AlexYue Date: Wed, 17 Jan 2024 14:21:19 +0800 Subject: [PATCH 053/200] [feature](merge-cloud) Add mixin support for Backend&Internal Service (#30014) --- be/src/service/backend_service.cpp | 128 +++++----- be/src/service/backend_service.h | 65 +++-- be/src/service/brpc_service.cpp | 9 +- be/src/service/internal_service.cpp | 358 ++++++++++++++-------------- be/src/service/internal_service.h | 111 +++++---- 5 files changed, 359 insertions(+), 312 deletions(-) diff --git a/be/src/service/backend_service.cpp b/be/src/service/backend_service.cpp index 306402eca0d514..aa7d5a3c9a9a3d 100644 --- a/be/src/service/backend_service.cpp +++ b/be/src/service/backend_service.cpp @@ -93,7 +93,7 @@ struct IngestBinlogArg { TStatus* tstatus; }; -void _ingest_binlog(IngestBinlogArg* arg) { +void _ingest_binlog(StorageEngine& engine, IngestBinlogArg* arg) { auto txn_id = arg->txn_id; auto partition_id = arg->partition_id; auto local_tablet_id = arg->local_tablet_id; @@ -103,12 +103,12 @@ void _ingest_binlog(IngestBinlogArg* arg) { auto& request = arg->request; TStatus tstatus; - Defer defer {[=, &tstatus, ingest_binlog_tstatus = arg->tstatus]() { + Defer defer {[=, &engine, &tstatus, ingest_binlog_tstatus = arg->tstatus]() { LOG(INFO) << "ingest binlog. result: " << apache::thrift::ThriftDebugString(tstatus); if (tstatus.status_code != TStatusCode::OK) { // abort txn - StorageEngine::instance()->txn_manager()->abort_txn(partition_id, txn_id, - local_tablet_id, local_tablet_uid); + engine.txn_manager()->abort_txn(partition_id, txn_id, local_tablet_id, + local_tablet_uid); } if (ingest_binlog_tstatus) { @@ -187,8 +187,8 @@ void _ingest_binlog(IngestBinlogArg* arg) { status.to_thrift(&tstatus); return; } - RowsetId new_rowset_id = StorageEngine::instance()->next_rowset_id(); - auto pending_rs_guard = StorageEngine::instance()->pending_local_rowsets().add(new_rowset_id); + RowsetId new_rowset_id = engine.next_rowset_id(); + auto pending_rs_guard = engine.pending_local_rowsets().add(new_rowset_id); rowset_meta->set_rowset_id(new_rowset_id); rowset_meta->set_tablet_uid(local_tablet->tablet_uid()); @@ -299,8 +299,7 @@ void _ingest_binlog(IngestBinlogArg* arg) { } // Step 6.2 calculate delete bitmap before commit - auto calc_delete_bitmap_token = - StorageEngine::instance()->calc_delete_bitmap_executor()->create_token(); + auto calc_delete_bitmap_token = engine.calc_delete_bitmap_executor()->create_token(); DeleteBitmapPtr delete_bitmap = std::make_shared(local_tablet_id); RowsetIdUnorderedSet pre_rowset_ids; if (local_tablet->enable_unique_key_merge_on_write()) { @@ -335,7 +334,7 @@ void _ingest_binlog(IngestBinlogArg* arg) { } // Step 6.3: commit txn - Status commit_txn_status = StorageEngine::instance()->txn_manager()->commit_txn( + Status commit_txn_status = engine.txn_manager()->commit_txn( local_tablet->data_dir()->get_meta(), rowset_meta->partition_id(), rowset_meta->txn_id(), rowset_meta->tablet_id(), local_tablet->tablet_uid(), rowset_meta->load_id(), rowset, std::move(pending_rs_guard), false); @@ -351,9 +350,9 @@ void _ingest_binlog(IngestBinlogArg* arg) { } if (local_tablet->enable_unique_key_merge_on_write()) { - StorageEngine::instance()->txn_manager()->set_txn_related_delete_bitmap( - partition_id, txn_id, local_tablet_id, local_tablet->tablet_uid(), true, - delete_bitmap, pre_rowset_ids, nullptr); + engine.txn_manager()->set_txn_related_delete_bitmap(partition_id, txn_id, local_tablet_id, + local_tablet->tablet_uid(), true, + delete_bitmap, pre_rowset_ids, nullptr); } tstatus.__set_status_code(TStatusCode::OK); @@ -366,12 +365,20 @@ using apache::thrift::TMultiplexedProcessor; using apache::thrift::transport::TTransportException; using apache::thrift::concurrency::ThreadFactory; -BackendService::BackendService(ExecEnv* exec_env) +BaseBackendService::BaseBackendService(ExecEnv* exec_env) : _exec_env(exec_env), _agent_server(new AgentServer(exec_env, *exec_env->master_info())) {} -Status BackendService::create_service(ExecEnv* exec_env, int port, - std::unique_ptr* server) { - auto service = std::make_shared(exec_env); +BackendService::BackendService(StorageEngine& engine, ExecEnv* exec_env) + : BaseBackendService(exec_env), _engine(engine) {} + +Status BaseBackendService::create_service(ExecEnv* exec_env, int port, + std::unique_ptr* server) { + if constexpr (!std::is_same_v) { + // TODO(plat1ko): cloud mode + return Status::NotSupported("Currently only support local storage engine"); + } + auto service = std::make_shared(*ExecEnv::GetInstance()->get_storage_engine(), + exec_env); // TODO: do we want a BoostThreadFactory? // TODO: we want separate thread factories here, so that fe requests can't starve // be requests @@ -401,29 +408,30 @@ Status BackendService::create_service(ExecEnv* exec_env, int port, return Status::OK(); } -void BackendService::exec_plan_fragment(TExecPlanFragmentResult& return_val, - const TExecPlanFragmentParams& params) { +void BaseBackendService::exec_plan_fragment(TExecPlanFragmentResult& return_val, + const TExecPlanFragmentParams& params) { LOG(INFO) << "exec_plan_fragment() instance_id=" << print_id(params.params.fragment_instance_id) << " coord=" << params.coord << " backend#=" << params.backend_num; return_val.__set_status(start_plan_fragment_execution(params).to_thrift()); } -Status BackendService::start_plan_fragment_execution(const TExecPlanFragmentParams& exec_params) { +Status BaseBackendService::start_plan_fragment_execution( + const TExecPlanFragmentParams& exec_params) { if (!exec_params.fragment.__isset.output_sink) { return Status::InternalError("missing sink in plan fragment"); } return _exec_env->fragment_mgr()->exec_plan_fragment(exec_params); } -void BackendService::cancel_plan_fragment(TCancelPlanFragmentResult& return_val, - const TCancelPlanFragmentParams& params) { +void BaseBackendService::cancel_plan_fragment(TCancelPlanFragmentResult& return_val, + const TCancelPlanFragmentParams& params) { LOG(INFO) << "cancel_plan_fragment(): instance_id=" << print_id(params.fragment_instance_id); _exec_env->fragment_mgr()->cancel_instance(params.fragment_instance_id, PPlanFragmentCancelReason::INTERNAL_ERROR); } -void BackendService::transmit_data(TTransmitDataResult& return_val, - const TTransmitDataParams& params) { +void BaseBackendService::transmit_data(TTransmitDataResult& return_val, + const TTransmitDataParams& params) { VLOG_ROW << "transmit_data(): instance_id=" << params.dest_fragment_instance_id << " node_id=" << params.dest_node_id << " #rows=" << params.row_batch.num_rows << " eos=" << (params.eos ? "true" : "false"); @@ -463,7 +471,7 @@ void BackendService::transmit_data(TTransmitDataResult& return_val, } } -void BackendService::submit_export_task(TStatus& t_status, const TExportTaskRequest& request) { +void BaseBackendService::submit_export_task(TStatus& t_status, const TExportTaskRequest& request) { // VLOG_ROW << "submit_export_task. request is " // << apache::thrift::ThriftDebugString(request).c_str(); // @@ -479,7 +487,7 @@ void BackendService::submit_export_task(TStatus& t_status, const TExportTaskRequ // status.to_thrift(&t_status); } -void BackendService::get_export_status(TExportStatusResult& result, const TUniqueId& task_id) { +void BaseBackendService::get_export_status(TExportStatusResult& result, const TUniqueId& task_id) { // VLOG_ROW << "get_export_status. task_id is " << task_id; // Status status = _exec_env->export_task_mgr()->get_task_state(task_id, &result); // if (!status.ok()) { @@ -498,7 +506,7 @@ void BackendService::get_export_status(TExportStatusResult& result, const TUniqu // result.__set_state(TExportState::RUNNING); } -void BackendService::erase_export_task(TStatus& t_status, const TUniqueId& task_id) { +void BaseBackendService::erase_export_task(TStatus& t_status, const TUniqueId& task_id) { // VLOG_ROW << "erase_export_task. task_id is " << task_id; // Status status = _exec_env->export_task_mgr()->erase_task(task_id); // if (!status.ok()) { @@ -511,18 +519,17 @@ void BackendService::erase_export_task(TStatus& t_status, const TUniqueId& task_ } void BackendService::get_tablet_stat(TTabletStatResult& result) { - StorageEngine::instance()->tablet_manager()->get_tablet_stat(&result); + _engine.tablet_manager()->get_tablet_stat(&result); } int64_t BackendService::get_trash_used_capacity() { int64_t result = 0; std::vector data_dir_infos; - static_cast(StorageEngine::instance()->get_all_data_dir_info(&data_dir_infos, - false /*do not update */)); + static_cast(_engine.get_all_data_dir_info(&data_dir_infos, false /*do not update */)); // uses excute sql `show trash`, then update backend trash capacity too. - StorageEngine::instance()->notify_listener("REPORT_DISK_STATE"); + _engine.notify_listener("REPORT_DISK_STATE"); for (const auto& root_path_info : data_dir_infos) { result += root_path_info.trash_used_capacity; @@ -533,11 +540,10 @@ int64_t BackendService::get_trash_used_capacity() { void BackendService::get_disk_trash_used_capacity(std::vector& diskTrashInfos) { std::vector data_dir_infos; - static_cast(StorageEngine::instance()->get_all_data_dir_info(&data_dir_infos, - false /*do not update */)); + static_cast(_engine.get_all_data_dir_info(&data_dir_infos, false /*do not update */)); // uses excute sql `show trash on `, then update backend trash capacity too. - StorageEngine::instance()->notify_listener("REPORT_DISK_STATE"); + _engine.notify_listener("REPORT_DISK_STATE"); for (const auto& root_path_info : data_dir_infos) { TDiskTrashInfo diskTrashInfo; @@ -548,8 +554,8 @@ void BackendService::get_disk_trash_used_capacity(std::vector& d } } -void BackendService::submit_routine_load_task(TStatus& t_status, - const std::vector& tasks) { +void BaseBackendService::submit_routine_load_task(TStatus& t_status, + const std::vector& tasks) { for (auto& task : tasks) { Status st = _exec_env->routine_load_task_executor()->submit_task(task); if (!st.ok()) { @@ -566,7 +572,7 @@ void BackendService::submit_routine_load_task(TStatus& t_status, * 1. validate user privilege (todo) * 2. FragmentMgr#exec_plan_fragment */ -void BackendService::open_scanner(TScanOpenResult& result_, const TScanOpenParams& params) { +void BaseBackendService::open_scanner(TScanOpenResult& result_, const TScanOpenParams& params) { TStatus t_status; TUniqueId fragment_instance_id = generate_uuid(); std::shared_ptr p_context; @@ -592,7 +598,7 @@ void BackendService::open_scanner(TScanOpenResult& result_, const TScanOpenParam } // fetch result from polling the queue, should always maintain the context offset, otherwise inconsistent result -void BackendService::get_next(TScanBatchResult& result_, const TScanNextBatchParams& params) { +void BaseBackendService::get_next(TScanBatchResult& result_, const TScanNextBatchParams& params) { std::string context_id = params.context_id; u_int64_t offset = params.offset; TStatus t_status; @@ -644,7 +650,7 @@ void BackendService::get_next(TScanBatchResult& result_, const TScanNextBatchPar context->last_access_time = time(nullptr); } -void BackendService::close_scanner(TScanCloseResult& result_, const TScanCloseParams& params) { +void BaseBackendService::close_scanner(TScanCloseResult& result_, const TScanCloseParams& params) { std::string context_id = params.context_id; TStatus t_status; Status st = _exec_env->external_scan_context_mgr()->clear_scan_context(context_id); @@ -653,8 +659,8 @@ void BackendService::close_scanner(TScanCloseResult& result_, const TScanClosePa } void BackendService::get_stream_load_record(TStreamLoadRecordResult& result, - const int64_t last_stream_record_time) { - auto stream_load_recorder = StorageEngine::instance()->get_stream_load_recorder(); + int64_t last_stream_record_time) { + auto stream_load_recorder = _engine.get_stream_load_recorder(); if (stream_load_recorder != nullptr) { std::map records; auto st = stream_load_recorder->get_batch(std::to_string(last_stream_record_time), @@ -664,7 +670,7 @@ void BackendService::get_stream_load_record(TStreamLoadRecordResult& result, << records.size() << ", last_stream_load_timestamp: " << last_stream_record_time; std::map stream_load_record_batch; - std::map::iterator it = records.begin(); + auto it = records.begin(); for (; it != records.end(); ++it) { TStreamLoadRecord stream_load_item; StreamLoadContext::parse_stream_load_record(it->second, stream_load_item); @@ -678,12 +684,12 @@ void BackendService::get_stream_load_record(TStreamLoadRecordResult& result, } void BackendService::clean_trash() { - static_cast(StorageEngine::instance()->start_trash_sweep(nullptr, true)); - static_cast(StorageEngine::instance()->notify_listener("REPORT_DISK_STATE")); + static_cast(_engine.start_trash_sweep(nullptr, true)); + static_cast(_engine.notify_listener("REPORT_DISK_STATE")); } void BackendService::check_storage_format(TCheckStorageFormatResult& result) { - StorageEngine::instance()->tablet_manager()->get_all_tablets_storage_format(&result); + _engine.tablet_manager()->get_all_tablets_storage_format(&result); } void BackendService::ingest_binlog(TIngestBinlogResult& result, @@ -760,7 +766,7 @@ void BackendService::ingest_binlog(TIngestBinlogResult& result, auto txn_id = request.txn_id; // Step 1: get local tablet auto const& local_tablet_id = request.local_tablet_id; - auto local_tablet = StorageEngine::instance()->tablet_manager()->get_tablet(local_tablet_id); + auto local_tablet = _engine.tablet_manager()->get_tablet(local_tablet_id); if (local_tablet == nullptr) { auto error_msg = fmt::format("tablet {} not found", local_tablet_id); LOG(WARNING) << error_msg; @@ -775,8 +781,8 @@ void BackendService::ingest_binlog(TIngestBinlogResult& result, PUniqueId p_load_id; p_load_id.set_hi(load_id.hi); p_load_id.set_lo(load_id.lo); - auto status = StorageEngine::instance()->txn_manager()->prepare_txn( - partition_id, *local_tablet, txn_id, p_load_id, is_ingrest); + auto status = _engine.txn_manager()->prepare_txn(partition_id, *local_tablet, txn_id, p_load_id, + is_ingrest); if (!status.ok()) { LOG(WARNING) << "prepare txn failed. txn_id=" << txn_id << ", status=" << status.to_string(); @@ -798,7 +804,7 @@ void BackendService::ingest_binlog(TIngestBinlogResult& result, .tstatus = is_async ? nullptr : tstatus, }; - _ingest_binlog(&ingest_binlog_arg); + _ingest_binlog(_engine, &ingest_binlog_arg); }; if (is_async) { @@ -852,7 +858,7 @@ void BackendService::query_ingest_binlog(TQueryIngestBinlogResult& result, auto tablet_id = request.tablet_id; // Step 1: get local tablet - auto local_tablet = StorageEngine::instance()->tablet_manager()->get_tablet(tablet_id); + auto local_tablet = _engine.tablet_manager()->get_tablet(tablet_id); if (local_tablet == nullptr) { auto error_msg = fmt::format("tablet {} not found", tablet_id); LOG(WARNING) << error_msg; @@ -862,8 +868,8 @@ void BackendService::query_ingest_binlog(TQueryIngestBinlogResult& result, // Step 2: get txn state auto tablet_uid = local_tablet->tablet_uid(); - auto txn_state = StorageEngine::instance()->txn_manager()->get_txn_state(partition_id, txn_id, - tablet_id, tablet_uid); + auto txn_state = + _engine.txn_manager()->get_txn_state(partition_id, txn_id, tablet_id, tablet_uid); switch (txn_state) { case TxnState::NOT_FOUND: result.__set_status(TIngestBinlogStatus::NOT_FOUND); @@ -886,28 +892,28 @@ void BackendService::query_ingest_binlog(TQueryIngestBinlogResult& result, } } -void BackendService::pre_cache_async(TPreCacheAsyncResponse& response, - const TPreCacheAsyncRequest& request) { +void BaseBackendService::pre_cache_async(TPreCacheAsyncResponse& response, + const TPreCacheAsyncRequest& request) { LOG(FATAL) << "BackendService is not implemented"; } -void BackendService::check_pre_cache(TCheckPreCacheResponse& response, - const TCheckPreCacheRequest& request) { +void BaseBackendService::check_pre_cache(TCheckPreCacheResponse& response, + const TCheckPreCacheRequest& request) { LOG(FATAL) << "BackendService is not implemented"; } -void BackendService::sync_load_for_tablets(TSyncLoadForTabletsResponse& response, - const TSyncLoadForTabletsRequest& request) { +void BaseBackendService::sync_load_for_tablets(TSyncLoadForTabletsResponse& response, + const TSyncLoadForTabletsRequest& request) { LOG(FATAL) << "BackendService is not implemented"; } -void BackendService::get_top_n_hot_partitions(TGetTopNHotPartitionsResponse& response, - const TGetTopNHotPartitionsRequest& request) { +void BaseBackendService::get_top_n_hot_partitions(TGetTopNHotPartitionsResponse& response, + const TGetTopNHotPartitionsRequest& request) { LOG(FATAL) << "BackendService is not implemented"; } -void BackendService::warm_up_tablets(TWarmUpTabletsResponse& response, - const TWarmUpTabletsRequest& request) { +void BaseBackendService::warm_up_tablets(TWarmUpTabletsResponse& response, + const TWarmUpTabletsRequest& request) { LOG(FATAL) << "BackendService is not implemented"; } diff --git a/be/src/service/backend_service.h b/be/src/service/backend_service.h index fed63454b7bfaa..6388ae6fdf8821 100644 --- a/be/src/service/backend_service.h +++ b/be/src/service/backend_service.h @@ -30,6 +30,7 @@ namespace doris { +class StorageEngine; class ExecEnv; class ThriftServer; class TAgentResult; @@ -63,11 +64,11 @@ class ThreadPool; // This class just forward rpc for actual handler // make this class because we can bind multiple service on single point -class BackendService : public BackendServiceIf { +class BaseBackendService : public BackendServiceIf { public: - BackendService(ExecEnv* exec_env); + BaseBackendService(ExecEnv* exec_env); - ~BackendService() override = default; + ~BaseBackendService() override = default; // NOTE: now we do not support multiple backend in one process static Status create_service(ExecEnv* exec_env, int port, @@ -112,12 +113,6 @@ class BackendService : public BackendServiceIf { void erase_export_task(TStatus& t_status, const TUniqueId& task_id) override; - void get_tablet_stat(TTabletStatResult& result) override; - - int64_t get_trash_used_capacity() override; - - void get_disk_trash_used_capacity(std::vector& diskTrashInfos) override; - void submit_routine_load_task(TStatus& t_status, const std::vector& tasks) override; @@ -130,18 +125,11 @@ class BackendService : public BackendServiceIf { // used for external service, close some context and release resource related with this context void close_scanner(TScanCloseResult& result_, const TScanCloseParams& params) override; - void get_stream_load_record(TStreamLoadRecordResult& result, - const int64_t last_stream_record_time) override; - - void clean_trash() override; - - void check_storage_format(TCheckStorageFormatResult& result) override; - - void ingest_binlog(TIngestBinlogResult& result, const TIngestBinlogRequest& request) override; - - void query_ingest_binlog(TQueryIngestBinlogResult& result, - const TQueryIngestBinlogRequest& request) override; - + // TODO(AlexYue): The below cloud backend functions should be implemented in + // CloudBackendService + //////////////////////////////////////////////////////////////////////////// + // begin cloud backend functions + //////////////////////////////////////////////////////////////////////////// void pre_cache_async(TPreCacheAsyncResponse& response, const TPreCacheAsyncRequest& request) override; @@ -158,11 +146,44 @@ class BackendService : public BackendServiceIf { void warm_up_tablets(TWarmUpTabletsResponse& response, const TWarmUpTabletsRequest& request) override; -private: + //////////////////////////////////////////////////////////////////////////// + // end cloud backend functions + //////////////////////////////////////////////////////////////////////////// +protected: Status start_plan_fragment_execution(const TExecPlanFragmentParams& exec_params); + ExecEnv* _exec_env = nullptr; std::unique_ptr _agent_server; std::unique_ptr _ingest_binlog_workers; }; +// `StorageEngine` mixin for `BaseBackendService` +class BackendService final : public BaseBackendService { +public: + BackendService(StorageEngine& engine, ExecEnv* exec_env); + + ~BackendService() override = default; + + void get_tablet_stat(TTabletStatResult& result) override; + + int64_t get_trash_used_capacity() override; + + void get_stream_load_record(TStreamLoadRecordResult& result, + int64_t last_stream_record_time) override; + + void get_disk_trash_used_capacity(std::vector& diskTrashInfos) override; + + void clean_trash() override; + + void check_storage_format(TCheckStorageFormatResult& result) override; + + void ingest_binlog(TIngestBinlogResult& result, const TIngestBinlogRequest& request) override; + + void query_ingest_binlog(TQueryIngestBinlogResult& result, + const TQueryIngestBinlogRequest& request) override; + +private: + StorageEngine& _engine; +}; + } // namespace doris diff --git a/be/src/service/brpc_service.cpp b/be/src/service/brpc_service.cpp index 57219f584ec753..b58a2b8df5e6b8 100644 --- a/be/src/service/brpc_service.cpp +++ b/be/src/service/brpc_service.cpp @@ -29,6 +29,7 @@ #include "common/config.h" #include "common/logging.h" +#include "runtime/exec_env.h" #include "service/backend_options.h" #include "service/internal_service.h" #include "util/mem_info.h" @@ -56,8 +57,14 @@ BRpcService::~BRpcService() { } Status BRpcService::start(int port, int num_threads) { + if constexpr (!std::is_same_v) { + // TODO(plat1ko): cloud mode + return Status::NotSupported("Currently only support local storage engine"); + } // Add service - _server->AddService(new PInternalServiceImpl(_exec_env), brpc::SERVER_OWNS_SERVICE); + _server->AddService( + new PInternalServiceImpl(*ExecEnv::GetInstance()->get_storage_engine(), _exec_env), + brpc::SERVER_OWNS_SERVICE); // start service brpc::ServerOptions options; if (num_threads != -1) { diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index e7255f017b7b59..713be89104f705 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -79,6 +79,7 @@ #include "olap/segment_loader.h" #include "olap/storage_engine.h" #include "olap/tablet.h" +#include "olap/tablet_fwd.h" #include "olap/tablet_manager.h" #include "olap/tablet_schema.h" #include "olap/txn_manager.h" @@ -197,7 +198,7 @@ void offer_failed(T* response, google::protobuf::Closure* done, const FifoThread LOG(WARNING) << "fail to offer request to the work pool, pool=" << pool.get_info(); } -PInternalServiceImpl::PInternalServiceImpl(ExecEnv* exec_env) +PInternalService::PInternalService(ExecEnv* exec_env) : _exec_env(exec_env), _heavy_work_pool(config::brpc_heavy_work_pool_threads != -1 ? config::brpc_heavy_work_pool_threads @@ -238,7 +239,12 @@ PInternalServiceImpl::PInternalServiceImpl(ExecEnv* exec_env) CHECK_EQ(0, bthread_key_create(&AsyncIO::btls_io_ctx_key, AsyncIO::io_ctx_key_deleter)); } -PInternalServiceImpl::~PInternalServiceImpl() { +PInternalServiceImpl::PInternalServiceImpl(StorageEngine& engine, ExecEnv* exec_env) + : PInternalService(exec_env), _engine(engine) {} + +PInternalServiceImpl::~PInternalServiceImpl() = default; + +PInternalService::~PInternalService() { DEREGISTER_HOOK_METRIC(heavy_work_pool_queue_size); DEREGISTER_HOOK_METRIC(light_work_pool_queue_size); DEREGISTER_HOOK_METRIC(heavy_work_active_threads); @@ -253,26 +259,25 @@ PInternalServiceImpl::~PInternalServiceImpl() { CHECK_EQ(0, bthread_key_delete(AsyncIO::btls_io_ctx_key)); } -void PInternalServiceImpl::transmit_data(google::protobuf::RpcController* controller, - const PTransmitDataParams* request, - PTransmitDataResult* response, - google::protobuf::Closure* done) {} - -void PInternalServiceImpl::transmit_data_by_http(google::protobuf::RpcController* controller, - const PEmptyRequest* request, - PTransmitDataResult* response, - google::protobuf::Closure* done) {} - -void PInternalServiceImpl::_transmit_data(google::protobuf::RpcController* controller, - const PTransmitDataParams* request, - PTransmitDataResult* response, - google::protobuf::Closure* done, - const Status& extract_st) {} - -void PInternalServiceImpl::tablet_writer_open(google::protobuf::RpcController* controller, - const PTabletWriterOpenRequest* request, - PTabletWriterOpenResult* response, - google::protobuf::Closure* done) { +void PInternalService::transmit_data(google::protobuf::RpcController* controller, + const PTransmitDataParams* request, + PTransmitDataResult* response, + google::protobuf::Closure* done) {} + +void PInternalService::transmit_data_by_http(google::protobuf::RpcController* controller, + const PEmptyRequest* request, + PTransmitDataResult* response, + google::protobuf::Closure* done) {} + +void PInternalService::_transmit_data(google::protobuf::RpcController* controller, + const PTransmitDataParams* request, + PTransmitDataResult* response, + google::protobuf::Closure* done, const Status& extract_st) {} + +void PInternalService::tablet_writer_open(google::protobuf::RpcController* controller, + const PTabletWriterOpenRequest* request, + PTabletWriterOpenResult* response, + google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([this, request, response, done]() { VLOG_RPC << "tablet writer open, id=" << request->id() << ", index_id=" << request->index_id() << ", txn_id=" << request->txn_id(); @@ -292,10 +297,10 @@ void PInternalServiceImpl::tablet_writer_open(google::protobuf::RpcController* c } } -void PInternalServiceImpl::exec_plan_fragment(google::protobuf::RpcController* controller, - const PExecPlanFragmentRequest* request, - PExecPlanFragmentResult* response, - google::protobuf::Closure* done) { +void PInternalService::exec_plan_fragment(google::protobuf::RpcController* controller, + const PExecPlanFragmentRequest* request, + PExecPlanFragmentResult* response, + google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([this, controller, request, response, done]() { _exec_plan_fragment_in_pthread(controller, request, response, done); }); @@ -305,9 +310,10 @@ void PInternalServiceImpl::exec_plan_fragment(google::protobuf::RpcController* c } } -void PInternalServiceImpl::_exec_plan_fragment_in_pthread( - google::protobuf::RpcController* controller, const PExecPlanFragmentRequest* request, - PExecPlanFragmentResult* response, google::protobuf::Closure* done) { +void PInternalService::_exec_plan_fragment_in_pthread(google::protobuf::RpcController* controller, + const PExecPlanFragmentRequest* request, + PExecPlanFragmentResult* response, + google::protobuf::Closure* done) { brpc::ClosureGuard closure_guard(done); auto st = Status::OK(); bool compact = request->has_compact() ? request->compact() : false; @@ -327,10 +333,10 @@ void PInternalServiceImpl::_exec_plan_fragment_in_pthread( st.to_protobuf(response->mutable_status()); } -void PInternalServiceImpl::exec_plan_fragment_prepare(google::protobuf::RpcController* controller, - const PExecPlanFragmentRequest* request, - PExecPlanFragmentResult* response, - google::protobuf::Closure* done) { +void PInternalService::exec_plan_fragment_prepare(google::protobuf::RpcController* controller, + const PExecPlanFragmentRequest* request, + PExecPlanFragmentResult* response, + google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([this, controller, request, response, done]() { _exec_plan_fragment_in_pthread(controller, request, response, done); }); @@ -340,10 +346,10 @@ void PInternalServiceImpl::exec_plan_fragment_prepare(google::protobuf::RpcContr } } -void PInternalServiceImpl::exec_plan_fragment_start(google::protobuf::RpcController* /*controller*/, - const PExecPlanFragmentStartRequest* request, - PExecPlanFragmentResult* result, - google::protobuf::Closure* done) { +void PInternalService::exec_plan_fragment_start(google::protobuf::RpcController* /*controller*/, + const PExecPlanFragmentStartRequest* request, + PExecPlanFragmentResult* result, + google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([this, request, result, done]() { brpc::ClosureGuard closure_guard(done); auto st = _exec_env->fragment_mgr()->start_query_execution(request); @@ -355,10 +361,10 @@ void PInternalServiceImpl::exec_plan_fragment_start(google::protobuf::RpcControl } } -void PInternalServiceImpl::open_load_stream(google::protobuf::RpcController* controller, - const POpenLoadStreamRequest* request, - POpenLoadStreamResponse* response, - google::protobuf::Closure* done) { +void PInternalService::open_load_stream(google::protobuf::RpcController* controller, + const POpenLoadStreamRequest* request, + POpenLoadStreamResponse* response, + google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([this, controller, request, response, done]() { signal::set_signal_task_id(request->load_id()); brpc::ClosureGuard done_guard(done); @@ -369,13 +375,14 @@ void PInternalServiceImpl::open_load_stream(google::protobuf::RpcController* con << ", src_id=" << request->src_id(); for (const auto& req : request->tablets()) { - TabletManager* tablet_mgr = StorageEngine::instance()->tablet_manager(); - TabletSharedPtr tablet = tablet_mgr->get_tablet(req.tablet_id()); - if (tablet == nullptr) { - auto st = Status::NotFound("Tablet {} not found", req.tablet_id()); + BaseTabletSPtr tablet; + if (auto res = ExecEnv::get_tablet(req.tablet_id()); !res.has_value()) [[unlikely]] { + auto st = std::move(res).error(); st.to_protobuf(response->mutable_status()); cntl->SetFailed(st.to_string()); return; + } else { + tablet = std::move(res).value(); } auto resp = response->add_tablet_schemas(); resp->set_index_id(req.index_id()); @@ -411,9 +418,10 @@ void PInternalServiceImpl::open_load_stream(google::protobuf::RpcController* con } } -void PInternalServiceImpl::tablet_writer_add_block_by_http( - google::protobuf::RpcController* controller, const ::doris::PEmptyRequest* request, - PTabletWriterAddBlockResult* response, google::protobuf::Closure* done) { +void PInternalService::tablet_writer_add_block_by_http(google::protobuf::RpcController* controller, + const ::doris::PEmptyRequest* request, + PTabletWriterAddBlockResult* response, + google::protobuf::Closure* done) { PTabletWriterAddBlockRequest* new_request = new PTabletWriterAddBlockRequest(); google::protobuf::Closure* new_done = new NewHttpClosure(new_request, done); @@ -427,10 +435,10 @@ void PInternalServiceImpl::tablet_writer_add_block_by_http( } } -void PInternalServiceImpl::tablet_writer_add_block(google::protobuf::RpcController* controller, - const PTabletWriterAddBlockRequest* request, - PTabletWriterAddBlockResult* response, - google::protobuf::Closure* done) { +void PInternalService::tablet_writer_add_block(google::protobuf::RpcController* controller, + const PTabletWriterAddBlockRequest* request, + PTabletWriterAddBlockResult* response, + google::protobuf::Closure* done) { int64_t submit_task_time_ns = MonotonicNanos(); bool ret = _heavy_work_pool.try_offer([request, response, done, submit_task_time_ns, this]() { int64_t wait_execution_time_ns = MonotonicNanos() - submit_task_time_ns; @@ -457,10 +465,10 @@ void PInternalServiceImpl::tablet_writer_add_block(google::protobuf::RpcControll } } -void PInternalServiceImpl::tablet_writer_cancel(google::protobuf::RpcController* controller, - const PTabletWriterCancelRequest* request, - PTabletWriterCancelResult* response, - google::protobuf::Closure* done) { +void PInternalService::tablet_writer_cancel(google::protobuf::RpcController* controller, + const PTabletWriterCancelRequest* request, + PTabletWriterCancelResult* response, + google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([this, request, done]() { VLOG_RPC << "tablet writer cancel, id=" << request->id() << ", index_id=" << request->index_id() << ", sender_id=" << request->sender_id(); @@ -479,7 +487,7 @@ void PInternalServiceImpl::tablet_writer_cancel(google::protobuf::RpcController* } } -Status PInternalServiceImpl::_exec_plan_fragment_impl( +Status PInternalService::_exec_plan_fragment_impl( const std::string& ser_request, PFragmentRequestVersion version, bool compact, const std::function& cb) { // Sometimes the BE do not receive the first heartbeat message and it receives request from FE @@ -561,10 +569,10 @@ Status PInternalServiceImpl::_exec_plan_fragment_impl( } } -void PInternalServiceImpl::cancel_plan_fragment(google::protobuf::RpcController* /*controller*/, - const PCancelPlanFragmentRequest* request, - PCancelPlanFragmentResult* result, - google::protobuf::Closure* done) { +void PInternalService::cancel_plan_fragment(google::protobuf::RpcController* /*controller*/, + const PCancelPlanFragmentRequest* request, + PCancelPlanFragmentResult* result, + google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([this, request, result, done]() { brpc::ClosureGuard closure_guard(done); TUniqueId tid; @@ -601,9 +609,9 @@ void PInternalServiceImpl::cancel_plan_fragment(google::protobuf::RpcController* } } -void PInternalServiceImpl::fetch_data(google::protobuf::RpcController* controller, - const PFetchDataRequest* request, PFetchDataResult* result, - google::protobuf::Closure* done) { +void PInternalService::fetch_data(google::protobuf::RpcController* controller, + const PFetchDataRequest* request, PFetchDataResult* result, + google::protobuf::Closure* done) { bool ret = _heavy_work_pool.try_offer([this, controller, request, result, done]() { brpc::Controller* cntl = static_cast(controller); GetResultBatchCtx* ctx = new GetResultBatchCtx(cntl, result, done); @@ -615,10 +623,10 @@ void PInternalServiceImpl::fetch_data(google::protobuf::RpcController* controlle } } -void PInternalServiceImpl::fetch_table_schema(google::protobuf::RpcController* controller, - const PFetchTableSchemaRequest* request, - PFetchTableSchemaResult* result, - google::protobuf::Closure* done) { +void PInternalService::fetch_table_schema(google::protobuf::RpcController* controller, + const PFetchTableSchemaRequest* request, + PFetchTableSchemaResult* result, + google::protobuf::Closure* done) { bool ret = _heavy_work_pool.try_offer([request, result, done]() { VLOG_RPC << "fetch table schema"; brpc::ClosureGuard closure_guard(done); @@ -723,10 +731,10 @@ void PInternalServiceImpl::fetch_table_schema(google::protobuf::RpcController* c } } -void PInternalServiceImpl::fetch_arrow_flight_schema(google::protobuf::RpcController* controller, - const PFetchArrowFlightSchemaRequest* request, - PFetchArrowFlightSchemaResult* result, - google::protobuf::Closure* done) { +void PInternalService::fetch_arrow_flight_schema(google::protobuf::RpcController* controller, + const PFetchArrowFlightSchemaRequest* request, + PFetchArrowFlightSchemaResult* result, + google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([request, result, done]() { brpc::ClosureGuard closure_guard(done); std::shared_ptr schema = @@ -758,6 +766,7 @@ void PInternalServiceImpl::fetch_arrow_flight_schema(google::protobuf::RpcContro Status PInternalServiceImpl::_tablet_fetch_data(const PTabletKeyLookupRequest* request, PTabletKeyLookupResponse* response) { + // TODO(yuejing): use PointQueryExecutor lookup_util(_engine); instead PointQueryExecutor lookup_util; RETURN_IF_ERROR(lookup_util.init(request, response)); RETURN_IF_ERROR(lookup_util.lookup_up()); @@ -773,7 +782,7 @@ void PInternalServiceImpl::tablet_fetch_data(google::protobuf::RpcController* co PTabletKeyLookupResponse* response, google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([this, controller, request, response, done]() { - [[maybe_unused]] brpc::Controller* cntl = static_cast(controller); + [[maybe_unused]] auto* cntl = static_cast(controller); brpc::ClosureGuard guard(done); Status st = _tablet_fetch_data(request, response); st.to_protobuf(response->mutable_status()); @@ -801,12 +810,12 @@ void PInternalServiceImpl::_get_column_ids_by_tablet_ids( google::protobuf::RpcController* controller, const PFetchColIdsRequest* request, PFetchColIdsResponse* response, google::protobuf::Closure* done) { brpc::ClosureGuard guard(done); - [[maybe_unused]] brpc::Controller* cntl = static_cast(controller); - TabletManager* tablet_mgr = StorageEngine::instance()->tablet_manager(); + [[maybe_unused]] auto* cntl = static_cast(controller); + TabletManager* tablet_mgr = _engine.tablet_manager(); const auto& params = request->params(); for (const auto& param : params) { int64_t index_id = param.indexid(); - auto tablet_ids = param.tablet_ids(); + const auto& tablet_ids = param.tablet_ids(); std::set> filter_set; std::map id_to_column; for (const int64_t tablet_id : tablet_ids) { @@ -826,7 +835,7 @@ void PInternalServiceImpl::_get_column_ids_by_tablet_ids( for (const auto& col : columns) { column_ids.insert(col.unique_id()); } - filter_set.insert(column_ids); + filter_set.insert(std::move(column_ids)); if (id_to_column.empty()) { for (const auto& col : columns) { @@ -889,7 +898,7 @@ void PInternalServiceImpl::fetch_remote_tablet_schema(google::protobuf::RpcContr const PFetchRemoteSchemaRequest* request, PFetchRemoteSchemaResponse* response, google::protobuf::Closure* done) { - bool ret = _heavy_work_pool.try_offer([request, response, done]() { + bool ret = _heavy_work_pool.try_offer([this, request, response, done]() { brpc::ClosureGuard closure_guard(done); Status st = Status::OK(); if (request->is_coordinator()) { @@ -960,9 +969,7 @@ void PInternalServiceImpl::fetch_remote_tablet_schema(google::protobuf::RpcContr if (!target_tablets.empty()) { std::vector tablet_schemas; for (int64_t tablet_id : target_tablets) { - TabletSharedPtr tablet = - StorageEngine::instance()->tablet_manager()->get_tablet(tablet_id, - false); + TabletSharedPtr tablet = _engine.tablet_manager()->get_tablet(tablet_id, false); if (tablet == nullptr) { // just ignore LOG(WARNING) << "tablet does not exist, tablet id is " << tablet_id; @@ -987,10 +994,10 @@ void PInternalServiceImpl::fetch_remote_tablet_schema(google::protobuf::RpcContr } } -void PInternalServiceImpl::report_stream_load_status(google::protobuf::RpcController* controller, - const PReportStreamLoadStatusRequest* request, - PReportStreamLoadStatusResponse* response, - google::protobuf::Closure* done) { +void PInternalService::report_stream_load_status(google::protobuf::RpcController* controller, + const PReportStreamLoadStatusRequest* request, + PReportStreamLoadStatusResponse* response, + google::protobuf::Closure* done) { TUniqueId load_id; load_id.__set_hi(request->load_id().hi()); load_id.__set_lo(request->load_id().lo()); @@ -1003,9 +1010,9 @@ void PInternalServiceImpl::report_stream_load_status(google::protobuf::RpcContro st.to_protobuf(response->mutable_status()); } -void PInternalServiceImpl::get_info(google::protobuf::RpcController* controller, - const PProxyRequest* request, PProxyResult* response, - google::protobuf::Closure* done) { +void PInternalService::get_info(google::protobuf::RpcController* controller, + const PProxyRequest* request, PProxyResult* response, + google::protobuf::Closure* done) { bool ret = _heavy_work_pool.try_offer([this, request, response, done]() { brpc::ClosureGuard closure_guard(done); // PProxyRequest is defined in gensrc/proto/internal_service.proto @@ -1069,9 +1076,9 @@ void PInternalServiceImpl::get_info(google::protobuf::RpcController* controller, } } -void PInternalServiceImpl::update_cache(google::protobuf::RpcController* controller, - const PUpdateCacheRequest* request, - PCacheResponse* response, google::protobuf::Closure* done) { +void PInternalService::update_cache(google::protobuf::RpcController* controller, + const PUpdateCacheRequest* request, PCacheResponse* response, + google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([this, request, response, done]() { brpc::ClosureGuard closure_guard(done); _exec_env->result_cache()->update(request, response); @@ -1082,9 +1089,9 @@ void PInternalServiceImpl::update_cache(google::protobuf::RpcController* control } } -void PInternalServiceImpl::fetch_cache(google::protobuf::RpcController* controller, - const PFetchCacheRequest* request, PFetchCacheResult* result, - google::protobuf::Closure* done) { +void PInternalService::fetch_cache(google::protobuf::RpcController* controller, + const PFetchCacheRequest* request, PFetchCacheResult* result, + google::protobuf::Closure* done) { bool ret = _heavy_work_pool.try_offer([this, request, result, done]() { brpc::ClosureGuard closure_guard(done); _exec_env->result_cache()->fetch(request, result); @@ -1095,9 +1102,9 @@ void PInternalServiceImpl::fetch_cache(google::protobuf::RpcController* controll } } -void PInternalServiceImpl::clear_cache(google::protobuf::RpcController* controller, - const PClearCacheRequest* request, PCacheResponse* response, - google::protobuf::Closure* done) { +void PInternalService::clear_cache(google::protobuf::RpcController* controller, + const PClearCacheRequest* request, PCacheResponse* response, + google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([this, request, response, done]() { brpc::ClosureGuard closure_guard(done); _exec_env->result_cache()->clear(request, response); @@ -1108,10 +1115,10 @@ void PInternalServiceImpl::clear_cache(google::protobuf::RpcController* controll } } -void PInternalServiceImpl::merge_filter(::google::protobuf::RpcController* controller, - const ::doris::PMergeFilterRequest* request, - ::doris::PMergeFilterResponse* response, - ::google::protobuf::Closure* done) { +void PInternalService::merge_filter(::google::protobuf::RpcController* controller, + const ::doris::PMergeFilterRequest* request, + ::doris::PMergeFilterResponse* response, + ::google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([this, controller, request, response, done]() { brpc::ClosureGuard closure_guard(done); auto attachment = static_cast(controller)->request_attachment(); @@ -1128,10 +1135,10 @@ void PInternalServiceImpl::merge_filter(::google::protobuf::RpcController* contr } } -void PInternalServiceImpl::apply_filter(::google::protobuf::RpcController* controller, - const ::doris::PPublishFilterRequest* request, - ::doris::PPublishFilterResponse* response, - ::google::protobuf::Closure* done) { +void PInternalService::apply_filter(::google::protobuf::RpcController* controller, + const ::doris::PPublishFilterRequest* request, + ::doris::PPublishFilterResponse* response, + ::google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([this, controller, request, response, done]() { brpc::ClosureGuard closure_guard(done); auto attachment = static_cast(controller)->request_attachment(); @@ -1150,10 +1157,10 @@ void PInternalServiceImpl::apply_filter(::google::protobuf::RpcController* contr } } -void PInternalServiceImpl::apply_filterv2(::google::protobuf::RpcController* controller, - const ::doris::PPublishFilterRequestV2* request, - ::doris::PPublishFilterResponse* response, - ::google::protobuf::Closure* done) { +void PInternalService::apply_filterv2(::google::protobuf::RpcController* controller, + const ::doris::PPublishFilterRequestV2* request, + ::doris::PPublishFilterResponse* response, + ::google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([this, controller, request, response, done]() { brpc::ClosureGuard closure_guard(done); auto attachment = static_cast(controller)->request_attachment(); @@ -1172,9 +1179,9 @@ void PInternalServiceImpl::apply_filterv2(::google::protobuf::RpcController* con } } -void PInternalServiceImpl::send_data(google::protobuf::RpcController* controller, - const PSendDataRequest* request, PSendDataResult* response, - google::protobuf::Closure* done) { +void PInternalService::send_data(google::protobuf::RpcController* controller, + const PSendDataRequest* request, PSendDataResult* response, + google::protobuf::Closure* done) { bool ret = _heavy_work_pool.try_offer([this, request, response, done]() { brpc::ClosureGuard closure_guard(done); TUniqueId load_id; @@ -1206,9 +1213,9 @@ void PInternalServiceImpl::send_data(google::protobuf::RpcController* controller } } -void PInternalServiceImpl::commit(google::protobuf::RpcController* controller, - const PCommitRequest* request, PCommitResult* response, - google::protobuf::Closure* done) { +void PInternalService::commit(google::protobuf::RpcController* controller, + const PCommitRequest* request, PCommitResult* response, + google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([this, request, response, done]() { brpc::ClosureGuard closure_guard(done); TUniqueId load_id; @@ -1230,9 +1237,9 @@ void PInternalServiceImpl::commit(google::protobuf::RpcController* controller, } } -void PInternalServiceImpl::rollback(google::protobuf::RpcController* controller, - const PRollbackRequest* request, PRollbackResult* response, - google::protobuf::Closure* done) { +void PInternalService::rollback(google::protobuf::RpcController* controller, + const PRollbackRequest* request, PRollbackResult* response, + google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([this, request, response, done]() { brpc::ClosureGuard closure_guard(done); TUniqueId load_id; @@ -1253,10 +1260,10 @@ void PInternalServiceImpl::rollback(google::protobuf::RpcController* controller, } } -void PInternalServiceImpl::fold_constant_expr(google::protobuf::RpcController* controller, - const PConstantExprRequest* request, - PConstantExprResult* response, - google::protobuf::Closure* done) { +void PInternalService::fold_constant_expr(google::protobuf::RpcController* controller, + const PConstantExprRequest* request, + PConstantExprResult* response, + google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([this, request, response, done]() { brpc::ClosureGuard closure_guard(done); Status st = _fold_constant_expr(request->request(), response); @@ -1268,8 +1275,8 @@ void PInternalServiceImpl::fold_constant_expr(google::protobuf::RpcController* c } } -Status PInternalServiceImpl::_fold_constant_expr(const std::string& ser_request, - PConstantExprResult* response) { +Status PInternalService::_fold_constant_expr(const std::string& ser_request, + PConstantExprResult* response) { TFoldConstantParams t_request; { const uint8_t* buf = (const uint8_t*)ser_request.data(); @@ -1285,10 +1292,10 @@ Status PInternalServiceImpl::_fold_constant_expr(const std::string& ser_request, return st; } -void PInternalServiceImpl::transmit_block(google::protobuf::RpcController* controller, - const PTransmitDataParams* request, - PTransmitDataResult* response, - google::protobuf::Closure* done) { +void PInternalService::transmit_block(google::protobuf::RpcController* controller, + const PTransmitDataParams* request, + PTransmitDataResult* response, + google::protobuf::Closure* done) { int64_t receive_time = GetCurrentTimeNanos(); response->set_receive_time(receive_time); @@ -1298,10 +1305,10 @@ void PInternalServiceImpl::transmit_block(google::protobuf::RpcController* contr _transmit_block(controller, request, response, done, Status::OK()); } -void PInternalServiceImpl::transmit_block_by_http(google::protobuf::RpcController* controller, - const PEmptyRequest* request, - PTransmitDataResult* response, - google::protobuf::Closure* done) { +void PInternalService::transmit_block_by_http(google::protobuf::RpcController* controller, + const PEmptyRequest* request, + PTransmitDataResult* response, + google::protobuf::Closure* done) { bool ret = _heavy_work_pool.try_offer([this, controller, response, done]() { PTransmitDataParams* new_request = new PTransmitDataParams(); google::protobuf::Closure* new_done = @@ -1317,11 +1324,10 @@ void PInternalServiceImpl::transmit_block_by_http(google::protobuf::RpcControlle } } -void PInternalServiceImpl::_transmit_block(google::protobuf::RpcController* controller, - const PTransmitDataParams* request, - PTransmitDataResult* response, - google::protobuf::Closure* done, - const Status& extract_st) { +void PInternalService::_transmit_block(google::protobuf::RpcController* controller, + const PTransmitDataParams* request, + PTransmitDataResult* response, + google::protobuf::Closure* done, const Status& extract_st) { std::string query_id; TUniqueId finst_id; if (request->has_query_id()) { @@ -1354,10 +1360,10 @@ void PInternalServiceImpl::_transmit_block(google::protobuf::RpcController* cont } } -void PInternalServiceImpl::check_rpc_channel(google::protobuf::RpcController* controller, - const PCheckRPCChannelRequest* request, - PCheckRPCChannelResponse* response, - google::protobuf::Closure* done) { +void PInternalService::check_rpc_channel(google::protobuf::RpcController* controller, + const PCheckRPCChannelRequest* request, + PCheckRPCChannelResponse* response, + google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([request, response, done]() { brpc::ClosureGuard closure_guard(done); response->mutable_status()->set_status_code(0); @@ -1387,10 +1393,10 @@ void PInternalServiceImpl::check_rpc_channel(google::protobuf::RpcController* co } } -void PInternalServiceImpl::reset_rpc_channel(google::protobuf::RpcController* controller, - const PResetRPCChannelRequest* request, - PResetRPCChannelResponse* response, - google::protobuf::Closure* done) { +void PInternalService::reset_rpc_channel(google::protobuf::RpcController* controller, + const PResetRPCChannelRequest* request, + PResetRPCChannelResponse* response, + google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([request, response, done]() { brpc::ClosureGuard closure_guard(done); response->mutable_status()->set_status_code(0); @@ -1426,10 +1432,9 @@ void PInternalServiceImpl::reset_rpc_channel(google::protobuf::RpcController* co } } -void PInternalServiceImpl::hand_shake(google::protobuf::RpcController* controller, - const PHandShakeRequest* request, - PHandShakeResponse* response, - google::protobuf::Closure* done) { +void PInternalService::hand_shake(google::protobuf::RpcController* controller, + const PHandShakeRequest* request, PHandShakeResponse* response, + google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([request, response, done]() { brpc::ClosureGuard closure_guard(done); if (request->has_hello()) { @@ -1448,14 +1453,14 @@ constexpr char DownloadApiPath[] = "/api/_tablet/_download?token="; constexpr char FileParam[] = "&file="; constexpr auto Permissions = S_IRUSR | S_IWUSR; -std::string construct_url(const std::string& host_port, const std::string& token, - const std::string& path) { +static std::string construct_url(const std::string& host_port, const std::string& token, + const std::string& path) { return fmt::format("{}{}{}{}{}{}", HttpProtocol, host_port, DownloadApiPath, token, FileParam, path); } -std::string construct_file_path(const std::string& tablet_path, const std::string& rowset_id, - int64_t segment) { +static std::string construct_file_path(const std::string& tablet_path, const std::string& rowset_id, + int64_t segment) { return fmt::format("{}/{}_{}.dat", tablet_path, rowset_id, segment); } @@ -1500,7 +1505,7 @@ void PInternalServiceImpl::request_slave_tablet_pull_rowset( int64_t node_id = request->node_id(); bool ret = _heavy_work_pool.try_offer([rowset_meta_pb, host, brpc_port, node_id, segments_size, indices_size, http_port, token, rowset_path, this]() { - TabletSharedPtr tablet = StorageEngine::instance()->tablet_manager()->get_tablet( + TabletSharedPtr tablet = _engine.tablet_manager()->get_tablet( rowset_meta_pb.tablet_id(), rowset_meta_pb.tablet_schema_hash()); if (tablet == nullptr) { LOG(WARNING) << "failed to pull rowset for slave replica. tablet [" @@ -1538,9 +1543,8 @@ void PInternalServiceImpl::request_slave_tablet_pull_rowset( } RowsetId remote_rowset_id = rowset_meta->rowset_id(); // change rowset id because it maybe same as other local rowset - RowsetId new_rowset_id = StorageEngine::instance()->next_rowset_id(); - auto pending_rs_guard = - StorageEngine::instance()->pending_local_rowsets().add(new_rowset_id); + RowsetId new_rowset_id = _engine.next_rowset_id(); + auto pending_rs_guard = _engine.pending_local_rowsets().add(new_rowset_id); rowset_meta->set_rowset_id(new_rowset_id); rowset_meta->set_tablet_uid(tablet->tablet_uid()); VLOG_CRITICAL << "succeed to init rowset meta for slave replica. rowset_id=" @@ -1639,7 +1643,7 @@ void PInternalServiceImpl::request_slave_tablet_pull_rowset( rowset_meta->tablet_id(), node_id, false); return; } - Status commit_txn_status = StorageEngine::instance()->txn_manager()->commit_txn( + Status commit_txn_status = _engine.txn_manager()->commit_txn( tablet->data_dir()->get_meta(), rowset_meta->partition_id(), rowset_meta->txn_id(), rowset_meta->tablet_id(), tablet->tablet_uid(), rowset_meta->load_id(), rowset, std::move(pending_rs_guard), false); @@ -1720,14 +1724,15 @@ void PInternalServiceImpl::_response_pull_slave_rowset(const std::string& remote void PInternalServiceImpl::response_slave_tablet_pull_rowset( google::protobuf::RpcController* controller, const PTabletWriteSlaveDoneRequest* request, PTabletWriteSlaveDoneResult* response, google::protobuf::Closure* done) { - bool ret = _heavy_work_pool.try_offer([request, response, done]() { + bool ret = _heavy_work_pool.try_offer([txn_mgr = _engine.txn_manager(), request, response, + done]() { brpc::ClosureGuard closure_guard(done); VLOG_CRITICAL << "receive the result of slave replica pull rowset from slave replica. " "slave server=" << request->node_id() << ", is_succeed=" << request->is_succeed() << ", tablet_id=" << request->tablet_id() << ", txn_id=" << request->txn_id(); - StorageEngine::instance()->txn_manager()->finish_slave_tablet_pull_rowset( - request->txn_id(), request->tablet_id(), request->node_id(), request->is_succeed()); + txn_mgr->finish_slave_tablet_pull_rowset(request->txn_id(), request->tablet_id(), + request->node_id(), request->is_succeed()); Status::OK().to_protobuf(response->mutable_status()); }); if (!ret) { @@ -1809,8 +1814,8 @@ Status PInternalServiceImpl::_multi_get(const PMultiGetRequest& request, watch.start(); TabletSharedPtr tablet = scope_timer_run( [&]() { - return StorageEngine::instance()->tablet_manager()->get_tablet( - row_loc.tablet_id(), true /*include deleted*/); + return _engine.tablet_manager()->get_tablet(row_loc.tablet_id(), + true /*include deleted*/); }, &acquire_tablet_ms); RowsetId rowset_id; @@ -1820,8 +1825,7 @@ Status PInternalServiceImpl::_multi_get(const PMultiGetRequest& request, } // We ensured it's rowset is not released when init Tablet reader param, rowset->update_delayed_expired_timestamp(); BetaRowsetSharedPtr rowset = std::static_pointer_cast(scope_timer_run( - [&]() { return StorageEngine::instance()->get_quering_rowset(rowset_id); }, - &acquire_rowsets_ms)); + [&]() { return _engine.get_quering_rowset(rowset_id); }, &acquire_rowsets_ms)); if (!rowset) { LOG(INFO) << "no such rowset " << rowset_id; continue; @@ -1947,12 +1951,12 @@ void PInternalServiceImpl::get_tablet_rowset_versions(google::protobuf::RpcContr google::protobuf::Closure* done) { brpc::ClosureGuard closure_guard(done); VLOG_DEBUG << "receive get tablet versions request: " << request->DebugString(); - StorageEngine::instance()->get_tablet_rowset_versions(request, response); + _engine.get_tablet_rowset_versions(request, response); } -void PInternalServiceImpl::glob(google::protobuf::RpcController* controller, - const PGlobRequest* request, PGlobResponse* response, - google::protobuf::Closure* done) { +void PInternalService::glob(google::protobuf::RpcController* controller, + const PGlobRequest* request, PGlobResponse* response, + google::protobuf::Closure* done) { bool ret = _heavy_work_pool.try_offer([request, response, done]() { brpc::ClosureGuard closure_guard(done); std::vector files; @@ -1972,10 +1976,10 @@ void PInternalServiceImpl::glob(google::protobuf::RpcController* controller, } } -void PInternalServiceImpl::group_commit_insert(google::protobuf::RpcController* controller, - const PGroupCommitInsertRequest* request, - PGroupCommitInsertResponse* response, - google::protobuf::Closure* done) { +void PInternalService::group_commit_insert(google::protobuf::RpcController* controller, + const PGroupCommitInsertRequest* request, + PGroupCommitInsertResponse* response, + google::protobuf::Closure* done) { TUniqueId load_id; load_id.__set_hi(request->load_id().hi()); load_id.__set_lo(request->load_id().lo()); @@ -2042,10 +2046,10 @@ void PInternalServiceImpl::group_commit_insert(google::protobuf::RpcController* } }; -void PInternalServiceImpl::get_wal_queue_size(google::protobuf::RpcController* controller, - const PGetWalQueueSizeRequest* request, - PGetWalQueueSizeResponse* response, - google::protobuf::Closure* done) { +void PInternalService::get_wal_queue_size(google::protobuf::RpcController* controller, + const PGetWalQueueSizeRequest* request, + PGetWalQueueSizeResponse* response, + google::protobuf::Closure* done) { bool ret = _light_work_pool.try_offer([this, request, response, done]() { brpc::ClosureGuard closure_guard(done); Status st = Status::OK(); diff --git a/be/src/service/internal_service.h b/be/src/service/internal_service.h index 261a3d161dcb75..b9d3432982c15c 100644 --- a/be/src/service/internal_service.h +++ b/be/src/service/internal_service.h @@ -18,32 +18,30 @@ #pragma once #include -#include #include #include "common/status.h" #include "util/work_thread_pool.hpp" -namespace google { -namespace protobuf { +namespace google::protobuf { class Closure; class RpcController; -} // namespace protobuf -} // namespace google +} // namespace google::protobuf namespace doris { +class StorageEngine; class ExecEnv; class PHandShakeRequest; class PHandShakeResponse; class LoadStreamMgr; class RuntimeState; -class PInternalServiceImpl : public PBackendService { +class PInternalService : public PBackendService { public: - PInternalServiceImpl(ExecEnv* exec_env); - ~PInternalServiceImpl() override; + PInternalService(ExecEnv* exec_env); + ~PInternalService() override; void transmit_data(::google::protobuf::RpcController* controller, const ::doris::PTransmitDataParams* request, @@ -166,31 +164,6 @@ class PInternalServiceImpl : public PBackendService { google::protobuf::Closure* done) override; void hand_shake(google::protobuf::RpcController* controller, const PHandShakeRequest* request, PHandShakeResponse* response, google::protobuf::Closure* done) override; - void request_slave_tablet_pull_rowset(google::protobuf::RpcController* controller, - const PTabletWriteSlaveRequest* request, - PTabletWriteSlaveResult* response, - google::protobuf::Closure* done) override; - void response_slave_tablet_pull_rowset(google::protobuf::RpcController* controller, - const PTabletWriteSlaveDoneRequest* request, - PTabletWriteSlaveDoneResult* response, - google::protobuf::Closure* done) override; - void multiget_data(google::protobuf::RpcController* controller, const PMultiGetRequest* request, - PMultiGetResponse* response, google::protobuf::Closure* done) override; - - void tablet_fetch_data(google::protobuf::RpcController* controller, - const PTabletKeyLookupRequest* request, - PTabletKeyLookupResponse* response, - google::protobuf::Closure* done) override; - - void get_column_ids_by_tablet_ids(google::protobuf::RpcController* controller, - const PFetchColIdsRequest* request, - PFetchColIdsResponse* response, - google::protobuf::Closure* done) override; - - void get_tablet_rowset_versions(google::protobuf::RpcController* controller, - const PGetTabletVersionsRequest* request, - PGetTabletVersionsResponse* response, - google::protobuf::Closure* done) override; void report_stream_load_status(google::protobuf::RpcController* controller, const PReportStreamLoadStatusRequest* request, @@ -204,10 +177,6 @@ class PInternalServiceImpl : public PBackendService { const PGroupCommitInsertRequest* request, PGroupCommitInsertResponse* response, google::protobuf::Closure* done) override; - void fetch_remote_tablet_schema(google::protobuf::RpcController* controller, - const PFetchRemoteSchemaRequest* request, - PFetchRemoteSchemaResponse* response, - google::protobuf::Closure* done) override; void get_wal_queue_size(google::protobuf::RpcController* controller, const PGetWalQueueSizeRequest* request, @@ -227,9 +196,6 @@ class PInternalServiceImpl : public PBackendService { Status _fold_constant_expr(const std::string& ser_request, PConstantExprResult* response); - Status _tablet_fetch_data(const PTabletKeyLookupRequest* request, - PTabletKeyLookupResponse* response); - void _transmit_data(::google::protobuf::RpcController* controller, const ::doris::PTransmitDataParams* request, ::doris::PTransmitDataResult* response, ::google::protobuf::Closure* done, @@ -240,17 +206,7 @@ class PInternalServiceImpl : public PBackendService { ::doris::PTransmitDataResult* response, ::google::protobuf::Closure* done, const Status& extract_st); - void _response_pull_slave_rowset(const std::string& remote_host, int64_t brpc_port, - int64_t txn_id, int64_t tablet_id, int64_t node_id, - bool is_succeed); - Status _multi_get(const PMultiGetRequest& request, PMultiGetResponse* response); - - void _get_column_ids_by_tablet_ids(google::protobuf::RpcController* controller, - const PFetchColIdsRequest* request, - PFetchColIdsResponse* response, - google::protobuf::Closure* done); - -private: +protected: ExecEnv* _exec_env = nullptr; // every brpc service request should put into thread pool @@ -263,4 +219,57 @@ class PInternalServiceImpl : public PBackendService { std::unique_ptr _load_stream_mgr; }; +// `StorageEngine` mixin for `PInternalService` +class PInternalServiceImpl final : public PInternalService { +public: + PInternalServiceImpl(StorageEngine& engine, ExecEnv* exec_env); + + ~PInternalServiceImpl() override; + void request_slave_tablet_pull_rowset(google::protobuf::RpcController* controller, + const PTabletWriteSlaveRequest* request, + PTabletWriteSlaveResult* response, + google::protobuf::Closure* done) override; + void response_slave_tablet_pull_rowset(google::protobuf::RpcController* controller, + const PTabletWriteSlaveDoneRequest* request, + PTabletWriteSlaveDoneResult* response, + google::protobuf::Closure* done) override; + void multiget_data(google::protobuf::RpcController* controller, const PMultiGetRequest* request, + PMultiGetResponse* response, google::protobuf::Closure* done) override; + + void tablet_fetch_data(google::protobuf::RpcController* controller, + const PTabletKeyLookupRequest* request, + PTabletKeyLookupResponse* response, + google::protobuf::Closure* done) override; + + void get_column_ids_by_tablet_ids(google::protobuf::RpcController* controller, + const PFetchColIdsRequest* request, + PFetchColIdsResponse* response, + google::protobuf::Closure* done) override; + + void get_tablet_rowset_versions(google::protobuf::RpcController* controller, + const PGetTabletVersionsRequest* request, + PGetTabletVersionsResponse* response, + google::protobuf::Closure* done) override; + + void fetch_remote_tablet_schema(google::protobuf::RpcController* controller, + const PFetchRemoteSchemaRequest* request, + PFetchRemoteSchemaResponse* response, + google::protobuf::Closure* done) override; + +private: + Status _tablet_fetch_data(const PTabletKeyLookupRequest* request, + PTabletKeyLookupResponse* response); + + void _response_pull_slave_rowset(const std::string& remote_host, int64_t brpc_port, + int64_t txn_id, int64_t tablet_id, int64_t node_id, + bool is_succeed); + Status _multi_get(const PMultiGetRequest& request, PMultiGetResponse* response); + + void _get_column_ids_by_tablet_ids(google::protobuf::RpcController* controller, + const PFetchColIdsRequest* request, + PFetchColIdsResponse* response, + google::protobuf::Closure* done); + + StorageEngine& _engine; +}; } // namespace doris From 4d09fb6bebf771fb01fb708c6df474de98eff7f5 Mon Sep 17 00:00:00 2001 From: yujun Date: Wed, 17 Jan 2024 14:25:47 +0800 Subject: [PATCH 054/200] [fix](fe ut) fix unstable TabletRepairAndBalanceTest (#30041) --- fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java | 4 ++++ .../org/apache/doris/clone/TabletRepairAndBalanceTest.java | 4 +++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java index 0ab869ec803ff9..badeb992465ce9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java @@ -700,6 +700,10 @@ public TabletStatus getColocateHealthStatus(long visibleVersion, * LOW: delay Config.tablet_repair_delay_factor_second * 3; */ public boolean readyToBeRepaired(SystemInfoService infoService, TabletSchedCtx.Priority priority) { + if (FeConstants.runningUnitTest) { + return true; + } + if (priority == Priority.VERY_HIGH) { return true; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/clone/TabletRepairAndBalanceTest.java b/fe/fe-core/src/test/java/org/apache/doris/clone/TabletRepairAndBalanceTest.java index 9f9fd8a09421e0..979943b1609fed 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/clone/TabletRepairAndBalanceTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/clone/TabletRepairAndBalanceTest.java @@ -106,9 +106,11 @@ public static void beforeClass() throws Exception { FeConstants.runningUnitTest = true; System.out.println(runningDir); FeConstants.runningUnitTest = true; - Config.tablet_checker_interval_ms = 1000; + Config.tablet_checker_interval_ms = 100; Config.tablet_repair_delay_factor_second = 1; Config.colocate_group_relocate_delay_second = 1; + Config.schedule_slot_num_per_hdd_path = 1000; + Config.schedule_slot_num_per_ssd_path = 1000; Config.disable_balance = true; // 5 backends: // 127.0.0.1 From 468226c44dd1d3c81dfa4453578de3393cff0219 Mon Sep 17 00:00:00 2001 From: Pxl Date: Wed, 17 Jan 2024 14:33:38 +0800 Subject: [PATCH 055/200] [Feature](materialized-view) support match function with alias in materialized-view (#30025) support match function with alias in materialized-view --- .../org/apache/doris/analysis/Analyzer.java | 2 + .../apache/doris/analysis/FunctionName.java | 4 ++ .../apache/doris/rewrite/FunctionAlias.java | 59 +++++++++++++++++++ gensrc/script/doris_builtins_functions.py | 2 +- .../data/mv_p0/test_substr/test_substr.out | 4 ++ .../mv_p0/test_substr/test_substr.groovy | 56 ++++++++++++++++++ 6 files changed, 126 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/rewrite/FunctionAlias.java create mode 100644 regression-test/data/mv_p0/test_substr/test_substr.out create mode 100644 regression-test/suites/mv_p0/test_substr/test_substr.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java index 5dcb9c193e2db0..b41666820b508f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java @@ -56,6 +56,7 @@ import org.apache.doris.rewrite.ExprRewriter; import org.apache.doris.rewrite.ExtractCommonFactorsRule; import org.apache.doris.rewrite.FoldConstantsRule; +import org.apache.doris.rewrite.FunctionAlias; import org.apache.doris.rewrite.InferFiltersRule; import org.apache.doris.rewrite.MatchPredicateRule; import org.apache.doris.rewrite.NormalizeBinaryPredicatesRule; @@ -458,6 +459,7 @@ public GlobalState(Env env, ConnectContext context) { rules.add(MatchPredicateRule.INSTANCE); rules.add(EliminateUnnecessaryFunctions.INSTANCE); rules.add(ElementAtToSlotRefRule.INSTANCE); + rules.add(FunctionAlias.INSTANCE); List onceRules = Lists.newArrayList(); onceRules.add(ExtractCommonFactorsRule.INSTANCE); onceRules.add(InferFiltersRule.INSTANCE); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionName.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionName.java index 510af96e4a8a70..cb243ef03e3d14 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionName.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionName.java @@ -104,6 +104,10 @@ public void setDb(String db) { this.db = db; } + public void setFn(String fn) { + this.fn = fn; + } + public String getFunction() { return fn; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/rewrite/FunctionAlias.java b/fe/fe-core/src/main/java/org/apache/doris/rewrite/FunctionAlias.java new file mode 100644 index 00000000000000..9e9c6dc9f75a21 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/rewrite/FunctionAlias.java @@ -0,0 +1,59 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.rewrite; + +import org.apache.doris.analysis.Analyzer; +import org.apache.doris.analysis.Expr; +import org.apache.doris.analysis.FunctionCallExpr; +import org.apache.doris.common.AnalysisException; + +import com.google.common.collect.ImmutableMap; + +import java.util.Map; + +/** + * Change functio name to function class name on nereids + * alias list: catalog/BuiltinScalarFunctions.java + */ +public final class FunctionAlias implements ExprRewriteRule { + public static ExprRewriteRule INSTANCE = new FunctionAlias(); + + static final Map aliasToName = ImmutableMap.builder() + .put("array_size", "cardinality").put("size", "cardinality").put("ceiling", "ceil") + .put("char_length", "character_length").put("curdate", "current_date").put("curtime", "current_time") + .put("schema", "database").put("day", "dayofmonth").put("date_add", "days_add").put("adddate", "days_add") + .put("date_sub", "days_sub").put("subdate", "days_sub").put("inet_ntoa", "ipv4_num_to_string") + .put("inet_aton", "ipv4_string_to_num").put("inet6_ntoa", "ipv6_num_to_string") + .put("inet6_aton", "ipv6_string_to_num").put("lcase", "lower").put("add_months", "months_add") + .put("current_timestamp", "now").put("localtime", "now").put("localtimestamp", "now").put("ifnull", "nvl") + .put("rand", "random").put("sha", "sha1").put("substr", "substring").put("ucase", "upper").build(); + + @Override + public Expr apply(Expr expr, Analyzer analyzer, ExprRewriter.ClauseType clauseType) throws AnalysisException { + if (!(expr instanceof FunctionCallExpr)) { + return expr; + } + FunctionCallExpr functionCall = (FunctionCallExpr) expr; + if (aliasToName.containsKey(functionCall.getFnName().getFunction())) { + FunctionCallExpr result = (FunctionCallExpr) functionCall.clone(); + result.getFnName().setFn(aliasToName.get(functionCall.getFnName().getFunction())); + return result; + } + return expr; + } +} diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py index 4f4c2f37b8ee42..0ca8dfad354efa 100644 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -75,7 +75,7 @@ "map": [ [['map'], 'MAP', ['K', 'V', '...'], 'ALWAYS_NOT_NULLABLE', ['K', 'V']], [['element_at', '%element_extract%'], 'V', ['MAP', 'K'], 'ALWAYS_NULLABLE', ['K', 'V']], - [['size', 'map_size'], 'BIGINT', ['MAP'], '', ['K', 'V']], + [['size', 'map_size', 'cardinality'], 'BIGINT', ['MAP'], '', ['K', 'V']], [['map_contains_key'], 'BOOLEAN', ['MAP', 'K'], 'CUSTOM', ['K', 'V']], [['map_contains_value'], 'BOOLEAN', ['MAP', 'V'], 'CUSTOM', ['K', 'V']], #[['map_contains_key_like'], 'BOOLEAN', ['MAP', 'K'], '', ['K', 'V']], diff --git a/regression-test/data/mv_p0/test_substr/test_substr.out b/regression-test/data/mv_p0/test_substr/test_substr.out new file mode 100644 index 00000000000000..8ebcba7eec0be6 --- /dev/null +++ b/regression-test/data/mv_p0/test_substr/test_substr.out @@ -0,0 +1,4 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_mv -- +\N \N + diff --git a/regression-test/suites/mv_p0/test_substr/test_substr.groovy b/regression-test/suites/mv_p0/test_substr/test_substr.groovy new file mode 100644 index 00000000000000..5397bac6dc9546 --- /dev/null +++ b/regression-test/suites/mv_p0/test_substr/test_substr.groovy @@ -0,0 +1,56 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite ("test_substr") { + sql """set enable_nereids_planner=true""" + sql """SET enable_fallback_to_original_planner=false""" + sql """ drop table if exists dwd;""" + + sql """ + CREATE TABLE `dwd` ( + `id` bigint(20) NULL COMMENT 'id', + `created_at` datetime NULL, + `dt` date NULL + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql """insert into dwd(id) values(1);""" + + createMV (""" + create materialized view dwd_mv as + SELECT + substr(created_at,1,10) as statistic_date, + max(dt) as dt + FROM dwd + group by substr(created_at,1,10); + """) + + sql """insert into dwd(id) values(2);""" + + explain { + sql("SELECT substr(created_at,1,10) as statistic_date, max(dt) as dt FROM dwd group by substr(created_at,1,10);") + contains "(dwd_mv)" + } + qt_select_mv "SELECT substr(created_at,1,10) as statistic_date, max(dt) as dt FROM dwd group by substr(created_at,1,10);" +} From a4ad25cabe469db4094dccef0421df6446e3f237 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=A2=E5=81=A5?= Date: Wed, 17 Jan 2024 15:11:20 +0800 Subject: [PATCH 056/200] [fix](Nereids): fix enable mv rewrite is useless in dphyp (#30042) --- .../cascades/OptimizeGroupExpressionJob.java | 23 ++++++++++++++----- .../apache/doris/nereids/rules/RuleSet.java | 1 - 2 files changed, 17 insertions(+), 7 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/OptimizeGroupExpressionJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/OptimizeGroupExpressionJob.java index 16cf90b786eee6..8c8e7e4399884a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/OptimizeGroupExpressionJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/OptimizeGroupExpressionJob.java @@ -24,7 +24,8 @@ import org.apache.doris.nereids.rules.Rule; import org.apache.doris.qe.ConnectContext; -import java.util.ArrayList; +import com.google.common.collect.ImmutableList; + import java.util.Collections; import java.util.List; @@ -48,11 +49,6 @@ public void execute() { countJobExecutionTimesOfGroupExpressions(groupExpression); List implementationRules = getRuleSet().getImplementationRules(); List explorationRules = getExplorationRules(); - ConnectContext connectContext = context.getCascadesContext().getConnectContext(); - if (connectContext.getSessionVariable().isEnableMaterializedViewRewrite()) { - explorationRules = new ArrayList<>(explorationRules); - explorationRules.addAll(getRuleSet().getMaterializedViewRules()); - } for (Rule rule : explorationRules) { if (rule.isInvalid(disableRules, groupExpression)) { @@ -70,6 +66,13 @@ public void execute() { } private List getExplorationRules() { + return ImmutableList.builder() + .addAll(getJoinRules()) + .addAll(getMvRules()) + .build(); + } + + private List getJoinRules() { boolean isDisableJoinReorder = context.getCascadesContext().getConnectContext().getSessionVariable() .isDisableJoinReorder() || context.getCascadesContext().getMemo().getGroupExpressionsSize() > context.getCascadesContext() @@ -96,4 +99,12 @@ private List getExplorationRules() { return getRuleSet().getZigZagTreeJoinReorder(); } } + + private List getMvRules() { + ConnectContext connectContext = context.getCascadesContext().getConnectContext(); + if (connectContext.getSessionVariable().isEnableMaterializedViewRewrite()) { + return getRuleSet().getMaterializedViewRules(); + } + return ImmutableList.of(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java index bb5b2f3dcf813a..de0f3d518fa119 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java @@ -240,7 +240,6 @@ public class RuleSet { .build(); public static final List DPHYP_REORDER_RULES = ImmutableList.builder() - .addAll(MATERIALIZED_VIEW_RULES) .add(JoinCommute.BUSHY.build()) .build(); From c7784b16482c6e59a76f9aed29121dcf1efd39c8 Mon Sep 17 00:00:00 2001 From: Lei Zhang <27994433+SWJTU-ZhangLei@users.noreply.github.com> Date: Wed, 17 Jan 2024 16:40:17 +0800 Subject: [PATCH 057/200] [fix](fe) Fix drop table write too many times editlog (#29966) --- .../apache/doris/datasource/InternalCatalog.java | 9 +++++++-- .../java/org/apache/doris/persist/EditLog.java | 4 +++- .../apache/doris/statistics/AnalysisManager.java | 14 ++------------ 3 files changed, 12 insertions(+), 15 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java index cf270a2c091684..791fcd007d8cfe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java @@ -905,11 +905,15 @@ public void dropTable(DropTableStmt stmt) throws DdlException { } finally { table.writeUnlock(); } - DropInfo info = new DropInfo(db.getId(), table.getId(), tableName, -1L, stmt.isForceDrop(), recycleTime); - Env.getCurrentEnv().getEditLog().logDropTable(info); + Env.getCurrentEnv().getQueryStats().clear(Env.getCurrentEnv().getCurrentCatalog().getId(), db.getId(), table.getId()); + Env.getCurrentEnv().getAnalysisManager().removeTableStats(table.getId()); + + DropInfo info = new DropInfo(db.getId(), table.getId(), tableName, -1L, stmt.isForceDrop(), recycleTime); + Env.getCurrentEnv().getEditLog().logDropTable(info); + Env.getCurrentEnv().getMtmvService().dropTable(table); } catch (UserException e) { throw new DdlException(e.getMessage(), e.getMysqlErrorCode()); @@ -946,6 +950,7 @@ public void replayDropTable(Database db, long tableId, boolean isForceDrop, unprotectDropTable(db, table, isForceDrop, true, recycleTime); Env.getCurrentEnv().getQueryStats().clear(Env.getCurrentInternalCatalog().getId(), db.getId(), tableId); + Env.getCurrentEnv().getAnalysisManager().removeTableStats(table.getId()); } finally { table.writeUnlock(); db.writeUnlock(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index fb97c02b00ccd8..5b2ff25f54e479 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -1135,7 +1135,9 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { break; } case OperationType.OP_DELETE_TABLE_STATS: { - env.getAnalysisManager().replayTableStatsDeletion((TableStatsDeletionLog) journal.getData()); + long tableId = ((TableStatsDeletionLog) journal.getData()).id; + LOG.info("replay delete table stat tableId: {}", tableId); + Env.getCurrentEnv().getAnalysisManager().removeTableStats(tableId); break; } case OperationType.OP_ALTER_MTMV: { diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java index f56e800e83de80..00e858c15369fb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java @@ -46,7 +46,6 @@ import org.apache.doris.datasource.CatalogIf; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.persist.AnalyzeDeletionLog; -import org.apache.doris.persist.TableStatsDeletionLog; import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSet; @@ -1015,17 +1014,8 @@ public void removeColStatsStatus(long tblId, String colName) { } } - public void removeTableStats(long tblId) { - if (!idToTblStats.containsKey(tblId)) { - return; - } - TableStatsDeletionLog log = new TableStatsDeletionLog(tblId); - Env.getCurrentEnv().getEditLog().logDeleteTableStats(log); - replayTableStatsDeletion(log); - } - - public void replayTableStatsDeletion(TableStatsDeletionLog log) { - idToTblStats.remove(log.id); + public void removeTableStats(long tableId) { + idToTblStats.remove(tableId); } public ColStatsMeta findColStatsMeta(long tblId, String colName) { From e2620d78e4060e58f703e92f4bcb70d456ae8525 Mon Sep 17 00:00:00 2001 From: zzzxl <33418555+zzzxl1993@users.noreply.github.com> Date: Wed, 17 Jan 2024 17:04:44 +0800 Subject: [PATCH 058/200] [fix](index_tool) fix index_tool compile error (#30049) --- be/src/index-tools/index_tool.cpp | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/be/src/index-tools/index_tool.cpp b/be/src/index-tools/index_tool.cpp index 1ad624d12dad19..ade72ae6809101 100644 --- a/be/src/index-tools/index_tool.cpp +++ b/be/src/index-tools/index_tool.cpp @@ -17,11 +17,13 @@ #include #include +#include #include #include #include #include +#include #include #include #include @@ -29,12 +31,14 @@ #include "io/fs/local_file_system.h" #include "olap/rowset/segment_v2/inverted_index/query/conjunction_query.h" +#include "olap/rowset/segment_v2/inverted_index/query/query.h" #include "olap/rowset/segment_v2/inverted_index_compound_directory.h" #include "olap/rowset/segment_v2/inverted_index_compound_reader.h" using doris::segment_v2::DorisCompoundReader; using doris::segment_v2::DorisCompoundDirectoryFactory; using doris::io::FileInfo; +using namespace doris::segment_v2; using namespace lucene::analysis; using namespace lucene::index; using namespace lucene::util; @@ -83,7 +87,7 @@ void search(lucene::store::Directory* dir, std::string& field, std::string& toke _CLDELETE(reader); reader = newreader; } - IndexSearcher s(reader); + auto s = std::make_shared(reader); std::unique_ptr query; std::cout << "version: " << (int32_t)(reader->getIndexVersion()) << std::endl; @@ -124,13 +128,16 @@ void search(lucene::store::Directory* dir, std::string& field, std::string& toke if (pred == "match_all") { roaring::Roaring result; std::vector terms = split(token, '|'); - doris::ConjunctionQuery query(s.getReader()); + + doris::TQueryOptions queryOptions; + ConjunctionQuery query(s, queryOptions); query.add(field_ws, terms); query.search(result); + total += result.cardinality(); } else { roaring::Roaring result; - s._search(query.get(), [&result](const int32_t docid, const float_t /*score*/) { + s->_search(query.get(), [&result](const int32_t docid, const float_t /*score*/) { // docid equal to rowid in segment result.add(docid); if (FLAGS_print_row_id) { @@ -141,7 +148,7 @@ void search(lucene::store::Directory* dir, std::string& field, std::string& toke } std::cout << "Term queried count:" << total << std::endl; - s.close(); + s->close(); reader->close(); _CLLDELETE(reader); } From 92d45135bde286b988599e5eec9bea7730c784fc Mon Sep 17 00:00:00 2001 From: shuke <37901441+shuke987@users.noreply.github.com> Date: Wed, 17 Jan 2024 17:06:09 +0800 Subject: [PATCH 059/200] [test](regression-test) fix case, compatible with 3 replicas. (#29905) --- .../doris/regression/suite/Suite.groovy | 27 ++++++++++++++++++- .../test_index_change_with_compaction.groovy | 13 ++++++++- .../test_index_compaction_dup_keys.groovy | 10 ++++++- .../test_index_compaction_unique_keys.groovy | 10 ++++++- ...ompaction_with_multi_index_segments.groovy | 10 ++++++- .../test_load_to_single_tablet.groovy | 4 ++- .../select_tablets/select_with_tablets.groovy | 3 +++ 7 files changed, 71 insertions(+), 6 deletions(-) diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy index f276e29a5308e5..41a99abe7a7ee0 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy @@ -932,5 +932,30 @@ class Suite implements GroovyInterceptable { } Assert.assertEquals(true, !createdTableName.isEmpty()) } -} + String[][] deduplicate_tablets(String[][] tablets) { + def result = [:] + + tablets.each { row -> + def tablet_id = row[0] + if (!result.containsKey(tablet_id)) { + result[tablet_id] = row + } + } + + return result.values().toList() + } + + ArrayList deduplicate_tablets(ArrayList tablets) { + def result = [:] + + tablets.each { row -> + def tablet_id = row[0] + if (!result.containsKey(tablet_id)) { + result[tablet_id] = row + } + } + + return result.values().toList() + } +} diff --git a/regression-test/suites/inverted_index_p0/index_change/test_index_change_with_compaction.groovy b/regression-test/suites/inverted_index_p0/index_change/test_index_change_with_compaction.groovy index b4a42aba2373f5..924ab0a9f4edcf 100644 --- a/regression-test/suites/inverted_index_p0/index_change/test_index_change_with_compaction.groovy +++ b/regression-test/suites/inverted_index_p0/index_change/test_index_change_with_compaction.groovy @@ -226,7 +226,18 @@ suite("test_index_change_with_compaction") { rowCount += Integer.parseInt(rowset.split(" ")[1]) } } - assert (rowCount <= 8) + + String[][] dedup_tablets = deduplicate_tablets(tablets) + + // In the p0 testing environment, there are no expected operations such as scaling down BE (backend) services + // if tablets or dedup_tablets is empty, exception is thrown, and case fail + int replicaNum = Math.floor(tablets.size() / dedup_tablets.size()) + if (replicaNum != 1 && replicaNum != 3) + { + assert(false); + } + + assert (rowCount <= 8*replicaNum) qt_select_default2 """ SELECT * FROM ${tableName} t ORDER BY user_id,date,city,age,sex,last_visit_date,last_update_date,last_visit_date_not_null,cost,max_dwell_time,min_dwell_time; """ } finally { // try_sql("DROP TABLE IF EXISTS ${tableName}") diff --git a/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_dup_keys.groovy b/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_dup_keys.groovy index cd98f58ce4f237..8bbedb977941cc 100644 --- a/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_dup_keys.groovy +++ b/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_dup_keys.groovy @@ -164,8 +164,16 @@ suite("test_index_compaction_dup_keys", "p0") { //TabletId,ReplicaId,BackendId,SchemaHash,Version,LstSuccessVersion,LstFailedVersion,LstFailedTime,LocalDataSize,RemoteDataSize,RowCount,State,LstConsistencyCheckTime,CheckVersion,VersionCount,PathHash,MetaUrl,CompactionStatus String[][] tablets = sql """ show tablets from ${tableName}; """ + String[][] dedup_tablets = deduplicate_tablets(tablets) + + // In the p0 testing environment, there are no expected operations such as scaling down BE (backend) services + // if tablets or dedup_tablets is empty, exception is thrown, and case fail + int replicaNum = Math.floor(tablets.size() / dedup_tablets.size()) + if (replicaNum != 1 && replicaNum != 3) + { + assert(false); + } - int replicaNum = 1 // before full compaction, there are 7 rowsets. int rowsetCount = get_rowset_count.call(tablets); assert (rowsetCount == 7 * replicaNum) diff --git a/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_unique_keys.groovy b/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_unique_keys.groovy index f4f2afb78d664e..64d2ee21647377 100644 --- a/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_unique_keys.groovy +++ b/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_unique_keys.groovy @@ -168,8 +168,16 @@ suite("test_index_compaction_unique_keys", "p0") { //TabletId,ReplicaId,BackendId,SchemaHash,Version,LstSuccessVersion,LstFailedVersion,LstFailedTime,LocalDataSize,RemoteDataSize,RowCount,State,LstConsistencyCheckTime,CheckVersion,VersionCount,PathHash,MetaUrl,CompactionStatus String[][] tablets = sql """ show tablets from ${tableName}; """ + String[][] dedup_tablets = deduplicate_tablets(tablets) + + // In the p0 testing environment, there are no expected operations such as scaling down BE (backend) services + // if tablets or dedup_tablets is empty, exception is thrown, and case fail + int replicaNum = Math.floor(tablets.size() / dedup_tablets.size()) + if (replicaNum != 1 && replicaNum != 3) + { + assert(false); + } - int replicaNum = 1 // before full compaction, there are 7 rowsets. int rowsetCount = get_rowset_count.call(tablets); assert (rowsetCount == 7 * replicaNum) diff --git a/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_with_multi_index_segments.groovy b/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_with_multi_index_segments.groovy index 25ed93aec51c69..dd6e91e53b7cb0 100644 --- a/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_with_multi_index_segments.groovy +++ b/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_with_multi_index_segments.groovy @@ -194,8 +194,16 @@ suite("test_index_compaction_with_multi_index_segments", "p0") { //TabletId,ReplicaId,BackendId,SchemaHash,Version,LstSuccessVersion,LstFailedVersion,LstFailedTime,LocalDataSize,RemoteDataSize,RowCount,State,LstConsistencyCheckTime,CheckVersion,VersionCount,PathHash,MetaUrl,CompactionStatus String[][] tablets = sql """ show tablets from ${tableName}; """ + String[][] dedup_tablets = deduplicate_tablets(tablets) + + // In the p0 testing environment, there are no expected operations such as scaling down BE (backend) services + // if tablets or dedup_tablets is empty, exception is thrown, and case fail + int replicaNum = Math.floor(tablets.size() / dedup_tablets.size()) + if (replicaNum != 1 && replicaNum != 3) + { + assert(false); + } - int replicaNum = 1 // before full compaction, there are 3 rowsets. int rowsetCount = get_rowset_count.call(tablets) assert (rowsetCount == 3 * replicaNum) diff --git a/regression-test/suites/load_p0/stream_load/test_load_to_single_tablet.groovy b/regression-test/suites/load_p0/stream_load/test_load_to_single_tablet.groovy index c35500267b9c58..80b25bd34c3930 100644 --- a/regression-test/suites/load_p0/stream_load/test_load_to_single_tablet.groovy +++ b/regression-test/suites/load_p0/stream_load/test_load_to_single_tablet.groovy @@ -55,7 +55,8 @@ suite("test_load_to_single_tablet", "p0") { sql "sync" def totalCount = sql "select count() from ${tableName}" assertEquals(10, totalCount[0][0]) - def res = sql "show tablets from ${tableName}" + String[][] res = sql "show tablets from ${tableName}" + res = deduplicate_tablets(res) def tablet1 = res[0][0] def tablet2 = res[1][0] def tablet3 = res[2][0] @@ -150,6 +151,7 @@ suite("test_load_to_single_tablet", "p0") { totalCount = sql "select count() from ${tableName}" assertEquals(10, totalCount[0][0]) res = sql "show tablets from ${tableName} partitions(p20231011, p20231012)" + res = deduplicate_tablets(res) tablet1 = res[0][0] tablet2 = res[1][0] tablet3 = res[2][0] diff --git a/regression-test/suites/nereids_p0/select_tablets/select_with_tablets.groovy b/regression-test/suites/nereids_p0/select_tablets/select_with_tablets.groovy index f2c2546b0241d2..86bc846a1105f7 100644 --- a/regression-test/suites/nereids_p0/select_tablets/select_with_tablets.groovy +++ b/regression-test/suites/nereids_p0/select_tablets/select_with_tablets.groovy @@ -45,6 +45,7 @@ suite("select_with_tablets") { order_qt_select1 """ SELECT * FROM ${table_name1} """ def res = sql """ show tablets from ${table_name1} where version = 2 """ + res = deduplicate_tablets(res) assertTrue(res.size() == 1) assertTrue(res[0].size() == 21) assertEquals("2", res[0][4]) @@ -65,6 +66,7 @@ suite("select_with_tablets") { order_qt_select11 """ SELECT * FROM ${table_name1} PARTITION between_20_70 where id < 2""" res = sql """ show tablets from ${table_name1} where version = 1 """ + res = deduplicate_tablets(res) assertTrue(res.size() == 2) assertEquals("1", res[0][4]) assertEquals("1", res[1][4]) @@ -93,6 +95,7 @@ suite("select_with_tablets") { order_qt_no_partition_1 """ SELECT * FROM ${table_no_partition} """ res = sql """ show tablets from ${table_no_partition} where version = 2 """ + res = deduplicate_tablets(res) order_qt_no_partition_2 """ SELECT * FROM ${table_no_partition} TABLET(${res[0][0]}) """ order_qt_no_partition_3 """ SELECT * FROM ${table_no_partition} TABLET(${res[1][0]}) """ From 90dee2d123db81ef8b1addca59a5f9d18763274f Mon Sep 17 00:00:00 2001 From: jakevin Date: Wed, 17 Jan 2024 17:12:04 +0800 Subject: [PATCH 060/200] [config](Nereids): add SessionVariable to control nereids timeout (#30048) --- .../nereids/jobs/scheduler/SimpleJobScheduler.java | 12 ++++++++---- .../java/org/apache/doris/qe/SessionVariable.java | 3 +++ 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/scheduler/SimpleJobScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/scheduler/SimpleJobScheduler.java index c52e2e90f13d60..ec751bdab2d446 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/scheduler/SimpleJobScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/scheduler/SimpleJobScheduler.java @@ -19,6 +19,7 @@ import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.jobs.Job; +import org.apache.doris.qe.SessionVariable; import java.util.concurrent.TimeUnit; @@ -29,11 +30,14 @@ public class SimpleJobScheduler implements JobScheduler { @Override public void executeJobPool(ScheduleContext scheduleContext) { JobPool pool = scheduleContext.getJobPool(); + CascadesContext context = (CascadesContext) scheduleContext; + SessionVariable sessionVariable = context.getConnectContext().getSessionVariable(); while (!pool.isEmpty()) { - CascadesContext context = (CascadesContext) scheduleContext; - if (context.getConnectContext().getSessionVariable().enableNereidsTimeout - && context.getStatementContext().getStopwatch().elapsed(TimeUnit.MILLISECONDS) > 5000) { - throw new RuntimeException("Nereids cost too much time ( > 5s )"); + if (sessionVariable.enableNereidsTimeout + && context.getStatementContext().getStopwatch().elapsed(TimeUnit.MILLISECONDS) + > sessionVariable.nereidsTimeoutSecond * 1000L) { + throw new RuntimeException( + "Nereids cost too much time ( > " + sessionVariable.nereidsTimeoutSecond + "s )"); } Job job = pool.pop(); job.execute(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 6bf7de619dbadb..7cfab79eb25652 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -1155,6 +1155,9 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { @VariableMgr.VarAttr(name = ENABLE_NEREIDS_TIMEOUT, needForward = true) public boolean enableNereidsTimeout = true; + @VariableMgr.VarAttr(name = "nereids_timeout_second", needForward = true) + public int nereidsTimeoutSecond = 5; + @VariableMgr.VarAttr(name = ENABLE_PUSH_DOWN_NO_GROUP_AGG) public boolean enablePushDownNoGroupAgg = true; From 2218e50f2569fbd42f1b1c680209223880a23113 Mon Sep 17 00:00:00 2001 From: zhangstar333 <87313068+zhangstar333@users.noreply.github.com> Date: Wed, 17 Jan 2024 17:15:06 +0800 Subject: [PATCH 061/200] [function](char) change char function behaviour same with mysql (#30034) select char(0) = '\0'; should return true; --- be/src/vec/functions/function_string.h | 6 +++--- .../string_functions/test_string_function.out | Bin 3360 -> 3418 bytes .../test_string_function.groovy | 2 ++ 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/be/src/vec/functions/function_string.h b/be/src/vec/functions/function_string.h index 79b5205640a74d..0a4b3432015ccc 100644 --- a/be/src/vec/functions/function_string.h +++ b/be/src/vec/functions/function_string.h @@ -3243,7 +3243,7 @@ class FunctionIntToChar : public IFunction { void integer_to_char_(int line_num, const int* num, ColumnString::Chars& chars, IColumn::Offsets& offsets) const { if (0 == *num) { - chars.push_back(' '); + chars.push_back('\0'); offsets[line_num] = offsets[line_num - 1] + 1; return; } @@ -3257,7 +3257,7 @@ class FunctionIntToChar : public IFunction { } offsets[line_num] = offsets[line_num - 1] + k + 1; for (; k >= 0; --k) { - chars.push_back(bytes[k] ? bytes[k] : ' '); + chars.push_back(bytes[k] ? bytes[k] : '\0'); } #else int k = 0; @@ -3268,7 +3268,7 @@ class FunctionIntToChar : public IFunction { } offsets[line_num] = offsets[line_num - 1] + 4 - k; for (; k < 4; ++k) { - chars.push_back(bytes[k] ? bytes[k] : ' '); + chars.push_back(bytes[k] ? bytes[k] : '\0'); } #endif } diff --git a/regression-test/data/query_p0/sql_functions/string_functions/test_string_function.out b/regression-test/data/query_p0/sql_functions/string_functions/test_string_function.out index b51fb32d6150ece9f968e91d0d8814bb90e9d173..118c58c491081b02527548938b7f3a3b69a52343 100644 GIT binary patch delta 306 zcmXX>Jq`g;5FQ1^ye$YJSqTcUniWE`iICU21Ig~QTdDk<1DQ^>g#$Q)V`#-K6lTU2 z^L=k-zBk`Y?!WxU8cRSM95PN!ovK)&90Sf`Y||Sveyz-)3yIreECB+E1PC|@ptI=& zs<9%ns&1V0JL;5t;MVth6tbgP3JPpR3@D+<$tOks?shyDbv+7Y3Y^cV##gU zOo_iMCj~*l?q7A{C<{wj?F$w}f48%f<>=uHcaDt{dJev~)8$pz9{e!~oir-O5XLW@S@C`o0R{Vv9ncNkb z*(9@>ZLTt=D%o_Do76l^WnFbLmYW;O&UZJk)HXIV((lHx ZC2l!2)dWW@jI!M;exVn>{95tn^FKrEV7veT diff --git a/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy b/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy index 4f9faac47f93a7..cee30808b84de8 100644 --- a/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy +++ b/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy @@ -280,5 +280,7 @@ suite("test_string_function", "arrow_flight_sql") { qt_sql_func_char6 """ select char(k1) from test_function_char order by k1; """ qt_sql_func_char7 """ select char(k1, k2, k3, k4) from test_function_char order by k1, k2, k3, k4; """ qt_sql_func_char8 """ select char(k1, k2, k3, k4, 65) from test_function_char order by k1, k2, k3, k4; """ + qt_sql_func_char9 """ select char(0) = ' '; """ + qt_sql_func_char10 """ select char(0) = '\0'; """ } From 7ae2b8cbdd5aca32aee1a9cb77ce6830c3d4ad80 Mon Sep 17 00:00:00 2001 From: hechao <73096722+hechao-ustc@users.noreply.github.com> Date: Wed, 17 Jan 2024 17:16:12 +0800 Subject: [PATCH 062/200] [typo](docs) fix typo (#30031) Co-authored-by: hechao --- docs/zh-CN/docs/advanced/variables.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh-CN/docs/advanced/variables.md b/docs/zh-CN/docs/advanced/variables.md index 9c042d5049f2e7..3075c46a1c2aa1 100644 --- a/docs/zh-CN/docs/advanced/variables.md +++ b/docs/zh-CN/docs/advanced/variables.md @@ -565,7 +565,7 @@ try (Connection conn = DriverManager.getConnection("jdbc:mysql://127.0.0.1:9030/ * `skip_missing_version` - 有些极端场景下,表的 Tablet 下的所有的所有副本都有版本缺失,使得这些 Tablet 没有办法被恢复,导致整张表都不能查询。这个变量可以用来控制查询的行为,打设置为`true`时,查询会忽略 FE partition 中记录的 visibleVersion,使用 replica version。如果 Be 上的 Replica 有缺失的版本,则查询会直接跳过这些缺失的版本,只返回仍存在版本的数据。此外,查询将会总是选择所有存活的 BE 中所有 Replica 里 lastSuccessVersion 最大的那一个,这样可以尽可能的恢复更多的数据。这个变量应该只在上述紧急情况下才被设置为`true`,仅用于临时让表恢复查询。注意,此变量与 use_fix_replica 变量冲突,当 use_fix_replica 变量不等于 -1 时,此变量会不起作用 + 有些极端场景下,表的 Tablet 下的所有的所有副本都有版本缺失,使得这些 Tablet 没有办法被恢复,导致整张表都不能查询。这个变量可以用来控制查询的行为,当设置为`true`时,查询会忽略 FE partition 中记录的 visibleVersion,使用 replica version。如果 Be 上的 Replica 有缺失的版本,则查询会直接跳过这些缺失的版本,只返回仍存在版本的数据。此外,查询将会总是选择所有存活的 BE 中所有 Replica 里 lastSuccessVersion 最大的那一个,这样可以尽可能的恢复更多的数据。这个变量应该只在上述紧急情况下才被设置为`true`,仅用于临时让表恢复查询。注意,此变量与 use_fix_replica 变量冲突,当 use_fix_replica 变量不等于 -1 时,此变量会不起作用 * `default_password_lifetime` From 771e350284605422fa2faec29b99fa64676d81ac Mon Sep 17 00:00:00 2001 From: zclllyybb Date: Wed, 17 Jan 2024 17:39:31 +0800 Subject: [PATCH 063/200] [Fix](compile) fix compile failure on newest clang (#30056) fix compile failure on newest clang --- be/src/service/backend_service.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/be/src/service/backend_service.cpp b/be/src/service/backend_service.cpp index aa7d5a3c9a9a3d..3fa1f16e7e90f7 100644 --- a/be/src/service/backend_service.cpp +++ b/be/src/service/backend_service.cpp @@ -793,14 +793,14 @@ void BackendService::ingest_binlog(TIngestBinlogResult& result, bool is_async = (_ingest_binlog_workers != nullptr); result.__set_is_async(is_async); - auto ingest_binlog_func = [=, tstatus = &tstatus]() { + auto ingest_binlog_func = [=, this, tstatus = &tstatus]() { IngestBinlogArg ingest_binlog_arg = { .txn_id = txn_id, .partition_id = partition_id, .local_tablet_id = local_tablet_id, .local_tablet = local_tablet, - .request = std::move(request), + .request = request, .tstatus = is_async ? nullptr : tstatus, }; From eb4fe55350d5791a0a023207aed1a7eaef656d44 Mon Sep 17 00:00:00 2001 From: amory Date: Wed, 17 Jan 2024 17:41:40 +0800 Subject: [PATCH 064/200] [Improve](config)delete confused config for nested complex type (#29988) --- .../java/org/apache/doris/common/Config.java | 11 ---- .../org/apache/doris/analysis/TypeDef.java | 11 ---- .../plans/commands/info/ColumnDefinition.java | 14 ----- ...tvf_with_complex_type_insertinto_doris.out | 3 - ...three_level_nestedtypes_with_s3data.groovy | 2 +- .../two_level_nestedtypes_with_s3data.groovy | 1 - .../datatype_p0/nested_types/load.groovy | 2 +- .../test_basic_map_function.groovy | 1 - .../query/test_nested_type_with_count.groovy | 1 - ...sted_types_insert_into_with_literal.groovy | 1 - ...st_nested_types_insert_into_with_s3.groovy | 1 - ...nestedtypes_csv_insert_into_with_s3.groovy | 1 - ...test_nestedtypes_insert_into_select.groovy | 1 - ...estedtypes_json_insert_into_with_s3.groovy | 1 - ..._with_complex_type_insertinto_doris.groovy | 2 +- .../test_load_with_map_nested_array.groovy | 1 - .../map_agg_nested_insert_doris.groovy | 2 +- .../show/test_nested_complex_switch.groovy | 56 ------------------- 18 files changed, 4 insertions(+), 108 deletions(-) diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index baf46177bc1035..87181e637bf434 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -2089,17 +2089,6 @@ public class Config extends ConfigBase { @ConfField(mutable = true) public static boolean disable_datev1 = true; - /** - * Now we not fully support array/struct/map nesting complex type in many situation, - * so just disable creating nesting complex data type when create table. - * We can make it able after we fully support - */ - @ConfField(mutable = true, masterOnly = true, description = { - "当前默认设置为 true,不支持建表时创建复杂类型(array/struct/map)嵌套复杂类型, 仅支持array类型自身嵌套。", - "Now default set to true, not support create complex type(array/struct/map) nested complex type " - + "when we create table, only support array type nested array"}) - public static boolean disable_nested_complex_type = true; - /* * This variable indicates the number of digits by which to increase the scale * of the result of division operations performed with the `/` operator. The diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/TypeDef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/TypeDef.java index c7199b82b289b1..2e8c1291b8b991 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/TypeDef.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/TypeDef.java @@ -28,7 +28,6 @@ import org.apache.doris.catalog.StructType; import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; -import org.apache.doris.common.Config; import org.apache.doris.qe.SessionVariable; import org.apache.doris.thrift.TColumnDesc; import org.apache.doris.thrift.TPrimitiveType; @@ -130,18 +129,10 @@ private void analyze(Type type) throws AnalysisException { Type itemType = ((ArrayType) type).getItemType(); if (itemType instanceof ScalarType) { analyzeNestedType(type, (ScalarType) itemType); - } else if (Config.disable_nested_complex_type && !(itemType instanceof ArrayType)) { - // now we can array nesting array - throw new AnalysisException("Unsupported data type: ARRAY<" + itemType.toSql() + ">"); } } if (type.isMapType()) { MapType mt = (MapType) type; - if (Config.disable_nested_complex_type && (!(mt.getKeyType() instanceof ScalarType) - || !(mt.getValueType() instanceof ScalarType))) { - throw new AnalysisException("Unsupported data type: MAP<" + mt.getKeyType().toSql() + "," - + mt.getValueType().toSql() + ">"); - } if (mt.getKeyType() instanceof ScalarType) { analyzeNestedType(type, (ScalarType) mt.getKeyType()); } @@ -160,8 +151,6 @@ private void analyze(Type type) throws AnalysisException { throw new AnalysisException("Duplicate field name " + field.getName() + " in struct " + type.toSql()); } - } else if (Config.disable_nested_complex_type) { - throw new AnalysisException("Unsupported field type: " + fieldType.toSql() + " for STRUCT"); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java index 5832a0534c0b8a..bf85d4055037ab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java @@ -24,7 +24,6 @@ import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.Type; -import org.apache.doris.common.Config; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.types.ArrayType; import org.apache.doris.nereids.types.BigIntType; @@ -392,21 +391,11 @@ private void validateDataType(Type catalogType) { Type itemType = ((org.apache.doris.catalog.ArrayType) catalogType).getItemType(); if (itemType instanceof ScalarType) { validateNestedType(catalogType, (ScalarType) itemType); - } else if (Config.disable_nested_complex_type - && !(itemType instanceof org.apache.doris.catalog.ArrayType)) { - // now we can array nesting array - throw new AnalysisException( - "Unsupported data type: ARRAY<" + itemType.toSql() + ">"); } } if (catalogType.isMapType()) { org.apache.doris.catalog.MapType mt = (org.apache.doris.catalog.MapType) catalogType; - if (Config.disable_nested_complex_type && (!(mt.getKeyType() instanceof ScalarType) - || !(mt.getValueType() instanceof ScalarType))) { - throw new AnalysisException("Unsupported data type: MAP<" - + mt.getKeyType().toSql() + "," + mt.getValueType().toSql() + ">"); - } if (mt.getKeyType() instanceof ScalarType) { validateNestedType(catalogType, (ScalarType) mt.getKeyType()); } @@ -426,9 +415,6 @@ private void validateDataType(Type catalogType) { throw new AnalysisException("Duplicate field name " + field.getName() + " in struct " + catalogType.toSql()); } - } else if (Config.disable_nested_complex_type) { - throw new AnalysisException( - "Unsupported field type: " + fieldType.toSql() + " for STRUCT"); } } } diff --git a/regression-test/data/external_table_p0/tvf/test_local_tvf_with_complex_type_insertinto_doris.out b/regression-test/data/external_table_p0/tvf/test_local_tvf_with_complex_type_insertinto_doris.out index ebc6fbe02b87ba..8e99380d460bad 100644 --- a/regression-test/data/external_table_p0/tvf/test_local_tvf_with_complex_type_insertinto_doris.out +++ b/regression-test/data/external_table_p0/tvf/test_local_tvf_with_complex_type_insertinto_doris.out @@ -2,9 +2,6 @@ -- !sql -- 0 --- !sql -- -0 - -- !sql -- 2 {1:[0.35815922932906263, 0.0011899152357573994, 0.28749219850167373, 0.93512930168283781, 0.1552584991620739, 0.73308976093672584, 0.52815960653805338, 0.92936404769642733, 0.89084215295591418, 0.21986459138832559], 2:[0.25076205844319044, 0.54003619330849928, 0.70661164863002113, 0.99472899095144263, 0.078314941019622886, 0.44206845606243961, 0.30857433265707379, 0.048661247184825784, 0.76954870938240083, 0.27253204080482074], 3:[0.687619207224633, 0.81614795840545351, 0.88663481647216158, 0.38735602510379774, 0.80082741637394084, 0.39139397466930215, 0.81747099649629329, 0.17689152877700398, 0.293284479800614, 0.4831799656491037], 4:[0.91830437289915945, 0.073632718338386915, 0.77375510933714242, 0.16816732397102252, 0.43277238052192457, 0.44208980569126877, 0.79186698174595249, 0.16564847296516494, 0.0094630056071454138, 0.35501997374061678], 5:[0.46246293630195257, 0.068748494114144409, 0.99666952760980676, 0.19382320901198535, 0.13487672467906897, 0.33765716511811372, 0.94048529885220122, 0.49731102964787222, 0.32349099284636484, 0.48597339317230759], 6:[0.085073215003979574, 0.28401725721857973, 0.20014761214390719, 0.20524657901039456, 0.11392467553698971, 0.24438834044276414, 0.49910871763701081, 0.33495152703896491, 0.46452858927426788, 0.390595446136895], 7:[0.20961408068486054, 0.67796351365684493, 0.088745389947127551, 0.12660368488966578, 0.505554324463536, 0.27743440100778682, 0.13385620029678247, 0.59106155590166665, 0.10571357063523412, 0.042619186963761324], 8:[0.68279814520834636, 0.94813249012442957, 0.94054706738722227, 0.47041093695670555, 0.58451395465060019, 0.16449033849974781, 0.14272344659751923, 0.66091032176672571, 0.4394562483212725, 0.16905962729691759]} {"G72Qcx-UBjfOr1-Pis-7YGo":{"UsPMIs-ipxhEnU-1EG-RJpe":0.22741640776012562, "gcn7fm-ILbMhQ6-fcz-TndT":0.36370276228098763, "o8dgBn-1bM26Wz-8SJ-xW6u":0.55671646501523719, "wav6ZA-780SwLJ-Vj3-KCv5":0.8665187582647581}} diff --git a/regression-test/suites/datatype_p0/nested_types/base_cases/three_level_nestedtypes_with_s3data.groovy b/regression-test/suites/datatype_p0/nested_types/base_cases/three_level_nestedtypes_with_s3data.groovy index 4258c30e1d4d4b..66897b161d3791 100644 --- a/regression-test/suites/datatype_p0/nested_types/base_cases/three_level_nestedtypes_with_s3data.groovy +++ b/regression-test/suites/datatype_p0/nested_types/base_cases/three_level_nestedtypes_with_s3data.groovy @@ -20,7 +20,7 @@ import org.apache.commons.lang3.StringUtils suite("three_level_nestedtypes_with_s3data") { sql """set enable_nereids_planner=false""" sql """ set enable_fallback_to_original_planner=true;""" - sql """ ADMIN SET FRONTEND CONFIG ('disable_nested_complex_type' = 'false');""" + // this test case aim to test one-level nested type with s3 data diff --git a/regression-test/suites/datatype_p0/nested_types/base_cases/two_level_nestedtypes_with_s3data.groovy b/regression-test/suites/datatype_p0/nested_types/base_cases/two_level_nestedtypes_with_s3data.groovy index c00ccfcf7d6662..d4b701b94b817f 100644 --- a/regression-test/suites/datatype_p0/nested_types/base_cases/two_level_nestedtypes_with_s3data.groovy +++ b/regression-test/suites/datatype_p0/nested_types/base_cases/two_level_nestedtypes_with_s3data.groovy @@ -20,7 +20,6 @@ import org.apache.commons.lang3.StringUtils suite("two_level_nestedtypes_with_s3data") { sql """set enable_nereids_planner=false""" sql """ set enable_fallback_to_original_planner=true;""" - sql """ ADMIN SET FRONTEND CONFIG ('disable_nested_complex_type' = 'false');""" // this test case aim to test one-level nested type with s3 data diff --git a/regression-test/suites/datatype_p0/nested_types/load.groovy b/regression-test/suites/datatype_p0/nested_types/load.groovy index 3ceffc30cd4a23..2bf559425d1cb5 100644 --- a/regression-test/suites/datatype_p0/nested_types/load.groovy +++ b/regression-test/suites/datatype_p0/nested_types/load.groovy @@ -17,7 +17,7 @@ suite("load") { // ddl begin - sql """ADMIN SET FRONTEND CONFIG ('disable_nested_complex_type' = 'false')""" + sql """set enable_nereids_planner=false""" def dataFile = """test_scalar_types_100.csv""" diff --git a/regression-test/suites/datatype_p0/nested_types/query/map_functions/test_basic_map_function.groovy b/regression-test/suites/datatype_p0/nested_types/query/map_functions/test_basic_map_function.groovy index 88993c5aeb0f39..1b4a4e8a919685 100644 --- a/regression-test/suites/datatype_p0/nested_types/query/map_functions/test_basic_map_function.groovy +++ b/regression-test/suites/datatype_p0/nested_types/query/map_functions/test_basic_map_function.groovy @@ -16,7 +16,6 @@ // under the License. suite("test_basic_map_function", "p0") { - sql """ ADMIN SET FRONTEND CONFIG ('disable_nested_complex_type' = 'false'); """ sql """set enable_nereids_planner=false""" // ============ sum(map-value) ============ qt_sql """ SELECT "sum-map-value" """ diff --git a/regression-test/suites/datatype_p0/nested_types/query/test_nested_type_with_count.groovy b/regression-test/suites/datatype_p0/nested_types/query/test_nested_type_with_count.groovy index 2b5bf97795e66b..1bc8285257e2e6 100644 --- a/regression-test/suites/datatype_p0/nested_types/query/test_nested_type_with_count.groovy +++ b/regression-test/suites/datatype_p0/nested_types/query/test_nested_type_with_count.groovy @@ -20,7 +20,6 @@ import org.apache.commons.lang3.StringUtils suite("test_nested_type_with_count") { // this test case aim to test nested type with old planner sql """set enable_nereids_planner=false""" - sql """ ADMIN SET FRONTEND CONFIG ('disable_nested_complex_type' = 'false');""" def table_names = ["test_array_one_level", "test_map_one_level", "test_struct_one_level"] diff --git a/regression-test/suites/datatype_p0/nested_types/query/test_nested_types_insert_into_with_literal.groovy b/regression-test/suites/datatype_p0/nested_types/query/test_nested_types_insert_into_with_literal.groovy index da3262858ae21a..08e794b23f3d0c 100644 --- a/regression-test/suites/datatype_p0/nested_types/query/test_nested_types_insert_into_with_literal.groovy +++ b/regression-test/suites/datatype_p0/nested_types/query/test_nested_types_insert_into_with_literal.groovy @@ -25,7 +25,6 @@ suite("test_nested_types_insert_into_with_literal", "p0") { // old planner does not support cast empty sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - sql """ADMIN SET FRONTEND CONFIG ('disable_nested_complex_type' = 'false')""" def table_names = [ "two_level_array_array_a", diff --git a/regression-test/suites/datatype_p0/nested_types/query/test_nested_types_insert_into_with_s3.groovy b/regression-test/suites/datatype_p0/nested_types/query/test_nested_types_insert_into_with_s3.groovy index 2b59e76aa29acf..fc2d3d2598812a 100644 --- a/regression-test/suites/datatype_p0/nested_types/query/test_nested_types_insert_into_with_s3.groovy +++ b/regression-test/suites/datatype_p0/nested_types/query/test_nested_types_insert_into_with_s3.groovy @@ -24,7 +24,6 @@ suite("test_nested_types_insert_into_with_s3", "p0") { sql 'use regression_test_datatype_p0_nested_types' sql 'set enable_nereids_planner=false' sql 'set max_allowed_packet=4194304' - sql """ADMIN SET FRONTEND CONFIG ('disable_nested_complex_type' = 'false')""" String ak = getS3AK() String sk = getS3SK() diff --git a/regression-test/suites/datatype_p0/nested_types/query/test_nestedtypes_csv_insert_into_with_s3.groovy b/regression-test/suites/datatype_p0/nested_types/query/test_nestedtypes_csv_insert_into_with_s3.groovy index ab9dff9ffb2407..40f76575bcf1fb 100644 --- a/regression-test/suites/datatype_p0/nested_types/query/test_nestedtypes_csv_insert_into_with_s3.groovy +++ b/regression-test/suites/datatype_p0/nested_types/query/test_nestedtypes_csv_insert_into_with_s3.groovy @@ -25,7 +25,6 @@ suite("test_nestedtypes_csv_insert_into_with_s3", "p0") { sql 'set enable_nereids_planner=false' sql 'set max_allowed_packet=4194304' sql 'set topn_opt_limit_threshold=10000' - sql """ADMIN SET FRONTEND CONFIG ('disable_nested_complex_type' = 'false')""" String ak = getS3AK() String sk = getS3SK() diff --git a/regression-test/suites/datatype_p0/nested_types/query/test_nestedtypes_insert_into_select.groovy b/regression-test/suites/datatype_p0/nested_types/query/test_nestedtypes_insert_into_select.groovy index a74687af64e5ea..bc21c18a0a9337 100644 --- a/regression-test/suites/datatype_p0/nested_types/query/test_nestedtypes_insert_into_select.groovy +++ b/regression-test/suites/datatype_p0/nested_types/query/test_nestedtypes_insert_into_select.groovy @@ -22,7 +22,6 @@ import org.codehaus.groovy.runtime.IOGroovyMethods suite("test_nestedtypes_insert_into_select", "p0") { sql "set enable_nereids_planner=false" - sql """ADMIN SET FRONTEND CONFIG ('disable_nested_complex_type' = 'false')""" // create array struct sql "DROP TABLE IF EXISTS ast;" diff --git a/regression-test/suites/datatype_p0/nested_types/query/test_nestedtypes_json_insert_into_with_s3.groovy b/regression-test/suites/datatype_p0/nested_types/query/test_nestedtypes_json_insert_into_with_s3.groovy index fb83de7a340c02..66d5c6bc5eed60 100644 --- a/regression-test/suites/datatype_p0/nested_types/query/test_nestedtypes_json_insert_into_with_s3.groovy +++ b/regression-test/suites/datatype_p0/nested_types/query/test_nestedtypes_json_insert_into_with_s3.groovy @@ -25,7 +25,6 @@ suite("test_nestedtypes_json_insert_into_with_s3", "p0") { sql 'set enable_nereids_planner=false' sql 'set max_allowed_packet=4194304' sql 'set topn_opt_limit_threshold=10000' - sql """ADMIN SET FRONTEND CONFIG ('disable_nested_complex_type' = 'false')""" String ak = getS3AK() String sk = getS3SK() diff --git a/regression-test/suites/external_table_p0/tvf/test_local_tvf_with_complex_type_insertinto_doris.groovy b/regression-test/suites/external_table_p0/tvf/test_local_tvf_with_complex_type_insertinto_doris.groovy index 04feab78ddb80e..c8bd8a8c2d6b4d 100644 --- a/regression-test/suites/external_table_p0/tvf/test_local_tvf_with_complex_type_insertinto_doris.groovy +++ b/regression-test/suites/external_table_p0/tvf/test_local_tvf_with_complex_type_insertinto_doris.groovy @@ -43,7 +43,7 @@ suite("test_local_tvf_with_complex_type_insertinto_doris", "p0") { } } - qt_sql """ADMIN SET FRONTEND CONFIG ('disable_nested_complex_type' = 'false')""" + // create doris table sql """ DROP TABLE IF EXISTS ${table_name} """ diff --git a/regression-test/suites/load_p0/stream_load/test_load_with_map_nested_array.groovy b/regression-test/suites/load_p0/stream_load/test_load_with_map_nested_array.groovy index 7e7e676bf3aae9..312905bc8687d8 100644 --- a/regression-test/suites/load_p0/stream_load/test_load_with_map_nested_array.groovy +++ b/regression-test/suites/load_p0/stream_load/test_load_with_map_nested_array.groovy @@ -17,7 +17,6 @@ suite("test_load_with_map_nested_array", "p0") { def tableName = "test_load_with_map_nested_array" - sql """ ADMIN SET FRONTEND CONFIG ('disable_nested_complex_type' = 'false'); """ sql """ DROP TABLE IF EXISTS ${tableName} """ sql """ diff --git a/regression-test/suites/query_p0/aggregate/map_agg_nested_insert_doris.groovy b/regression-test/suites/query_p0/aggregate/map_agg_nested_insert_doris.groovy index a4b0410cba3a31..8d4ae13b1cc3eb 100644 --- a/regression-test/suites/query_p0/aggregate/map_agg_nested_insert_doris.groovy +++ b/regression-test/suites/query_p0/aggregate/map_agg_nested_insert_doris.groovy @@ -20,7 +20,7 @@ suite("map_agg_nested_insert_doris", "p0") { def tb_doris = "test_map_agg_nested_insert_target" sql "DROP TABLE IF EXISTS `${tb_base}`;" sql "DROP TABLE IF EXISTS `${tb_doris}`;" - sql """ ADMIN SET FRONTEND CONFIG ('disable_nested_complex_type' = 'false'); """ + sql """ CREATE TABLE `${tb_base}` ( diff --git a/regression-test/suites/query_p0/show/test_nested_complex_switch.groovy b/regression-test/suites/query_p0/show/test_nested_complex_switch.groovy index af39d8a9c32957..9b06a00eb4255e 100644 --- a/regression-test/suites/query_p0/show/test_nested_complex_switch.groovy +++ b/regression-test/suites/query_p0/show/test_nested_complex_switch.groovy @@ -137,63 +137,7 @@ suite("test_nested_complex_switch", "query") { sql "DROP TABLE IF EXISTS ${testTable_m}" sql "DROP TABLE IF EXISTS ${testTable_a}" sql "DROP TABLE IF EXISTS ${testTable_s}" - sql "ADMIN SET FRONTEND CONFIG ('disable_nested_complex_type' = 'true')" - // map - test { - sql sql_m_s - exception "java.sql.SQLException: errCode = 2, detailMessage = Unsupported data type: MAP>" - } - - test { - sql sql_m_a - exception "java.sql.SQLException: errCode = 2, detailMessage = Unsupported data type: MAP,TEXT>" - } - - test { - sql sql_m_m - exception "java.sql.SQLException: errCode = 2, detailMessage = Unsupported data type: MAP>" - } - - // array - test { - sql sql_a_s - exception "java.sql.SQLException: errCode = 2, detailMessage = Unsupported data type: ARRAY>" - } - - - test { - sql sql_a_m - exception "java.sql.SQLException: errCode = 2, detailMessage = Unsupported data type: ARRAY>" - } - - // struct - test { - sql sql_s_s - exception "java.sql.SQLException: errCode = 2, detailMessage = Unsupported" - } - - test { - sql sql_s_a - exception "java.sql.SQLException: errCode = 2, detailMessage = Unsupported" - } - - test { - sql sql_s_m - exception "java.sql.SQLException: errCode = 2, detailMessage = Unsupported" - } - - } finally { - try_sql("DROP TABLE IF EXISTS ${testTable_m}") - try_sql("DROP TABLE IF EXISTS ${testTable_a}") - try_sql("DROP TABLE IF EXISTS ${testTable_s}") - } - - try { - sql "DROP TABLE IF EXISTS ${testTable_m}" - sql "DROP TABLE IF EXISTS ${testTable_a}" - sql "DROP TABLE IF EXISTS ${testTable_s}" - sql "ADMIN SET FRONTEND CONFIG ('disable_nested_complex_type' = 'false')" // map From 1f9ae3427f155c9ae487bb2f353492ad9fc0252c Mon Sep 17 00:00:00 2001 From: wuwenchi Date: Wed, 17 Jan 2024 17:43:05 +0800 Subject: [PATCH 065/200] [feature](statistics)support statistics for iceberg/paimon/hudi table (#29868) --- .../docker-compose/iceberg/iceberg.env | 2 +- .../docker-compose/iceberg/iceberg.yaml.tpl | 2 + .../external/IcebergExternalTable.java | 9 + .../catalog/external/PaimonExternalTable.java | 9 + .../doris/common/proc/CatalogsProcDir.java | 10 +- .../statistics/ExternalAnalysisTask.java | 263 ++++++++++++++++++ .../doris/statistics/HMSAnalysisTask.java | 252 ++--------------- .../doris/statistics/util/StatisticsUtil.java | 10 +- .../doris/statistics/HMSAnalysisTaskTest.java | 4 +- regression-test/conf/regression-conf.groovy | 1 + .../iceberg/test_iceberg_statistics.out | 39 +++ .../paimon/test_paimon_statistics.out | 21 ++ .../hive/test_hive_hudi_statistics.out | 16 ++ .../pipeline/p0/conf/regression-conf.groovy | 1 + .../tpch-sf100/conf/regression-conf.groovy | 1 + .../plugins/plugins_get_ids_from_proc.groovy | 62 +++++ .../iceberg/test_iceberg_statistics.groovy | 57 ++++ .../paimon/test_paimon_catalog.groovy | 3 + .../paimon/test_paimon_statistics.groovy | 47 ++++ .../external_table_p0/test_catalog_ddl.groovy | 2 + .../hive/test_hive_hudi_statistics.groovy | 47 ++++ 21 files changed, 621 insertions(+), 237 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/statistics/ExternalAnalysisTask.java create mode 100644 regression-test/data/external_table_p0/iceberg/test_iceberg_statistics.out create mode 100644 regression-test/data/external_table_p0/paimon/test_paimon_statistics.out create mode 100644 regression-test/data/external_table_p2/hive/test_hive_hudi_statistics.out create mode 100644 regression-test/plugins/plugins_get_ids_from_proc.groovy create mode 100644 regression-test/suites/external_table_p0/iceberg/test_iceberg_statistics.groovy create mode 100644 regression-test/suites/external_table_p0/paimon/test_paimon_statistics.groovy create mode 100644 regression-test/suites/external_table_p2/hive/test_hive_hudi_statistics.groovy diff --git a/docker/thirdparties/docker-compose/iceberg/iceberg.env b/docker/thirdparties/docker-compose/iceberg/iceberg.env index 4cc8b42eaf92be..6bebd49f437d80 100644 --- a/docker/thirdparties/docker-compose/iceberg/iceberg.env +++ b/docker/thirdparties/docker-compose/iceberg/iceberg.env @@ -21,4 +21,4 @@ SPARK_DRIVER_UI_PORT=8080 SPARK_HISTORY_UI_PORT=10000 REST_CATALOG_PORT=18181 MINIO_UI_PORT=9000 -MINIO_API_PORT=9001 +MINIO_API_PORT=19001 diff --git a/docker/thirdparties/docker-compose/iceberg/iceberg.yaml.tpl b/docker/thirdparties/docker-compose/iceberg/iceberg.yaml.tpl index d7220f2437619b..bc217c1dd6e8f4 100644 --- a/docker/thirdparties/docker-compose/iceberg/iceberg.yaml.tpl +++ b/docker/thirdparties/docker-compose/iceberg/iceberg.yaml.tpl @@ -58,6 +58,8 @@ services: minio: image: minio/minio container_name: doris--minio + ports: + - ${MINIO_API_PORT}:9000 environment: - MINIO_ROOT_USER=admin - MINIO_ROOT_PASSWORD=password diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/IcebergExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/IcebergExternalTable.java index 7398ff19c9eed5..be99e26de62e70 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/IcebergExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/IcebergExternalTable.java @@ -23,7 +23,10 @@ import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.Type; import org.apache.doris.datasource.iceberg.IcebergExternalCatalog; +import org.apache.doris.statistics.AnalysisInfo; +import org.apache.doris.statistics.BaseAnalysisTask; import org.apache.doris.statistics.ColumnStatistic; +import org.apache.doris.statistics.ExternalAnalysisTask; import org.apache.doris.statistics.util.StatisticsUtil; import org.apache.doris.thrift.THiveTable; import org.apache.doris.thrift.TIcebergTable; @@ -149,4 +152,10 @@ public Optional getColumnStatistic(String colName) { () -> StatisticsUtil.getIcebergColumnStats(colName, ((IcebergExternalCatalog) catalog).getIcebergTable(dbName, name))); } + + @Override + public BaseAnalysisTask createAnalysisTask(AnalysisInfo info) { + makeSureInitialized(); + return new ExternalAnalysisTask(info); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/PaimonExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/PaimonExternalTable.java index c8ea253671d2a2..b517265df6a82d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/PaimonExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/PaimonExternalTable.java @@ -21,6 +21,9 @@ import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.Type; import org.apache.doris.datasource.paimon.PaimonExternalCatalog; +import org.apache.doris.statistics.AnalysisInfo; +import org.apache.doris.statistics.BaseAnalysisTask; +import org.apache.doris.statistics.ExternalAnalysisTask; import org.apache.doris.thrift.THiveTable; import org.apache.doris.thrift.TTableDescriptor; import org.apache.doris.thrift.TTableType; @@ -154,4 +157,10 @@ public TTableDescriptor toThrift() { + getPaimonCatalogType()); } } + + @Override + public BaseAnalysisTask createAnalysisTask(AnalysisInfo info) { + makeSureInitialized(); + return new ExternalAnalysisTask(info); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/CatalogsProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/CatalogsProcDir.java index 854b4dddc796df..e6163645c28242 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/CatalogsProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/CatalogsProcDir.java @@ -27,6 +27,7 @@ import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; +import org.apache.log4j.Logger; import java.util.ArrayList; import java.util.Collections; @@ -37,6 +38,7 @@ * show all catalogs' info */ public class CatalogsProcDir implements ProcDirInterface { + private static final Logger LOG = Logger.getLogger(CatalogsProcDir.class); public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() .add("CatalogIds").add("CatalogName").add("DatabaseNum").add("LastUpdateTime") .build(); @@ -90,7 +92,13 @@ public ProcResult fetchResult() throws AnalysisException { List catalogInfo = Lists.newArrayList(); catalogInfo.add(catalog.getId()); catalogInfo.add(catalog.getName()); - catalogInfo.add(catalog.getDbNames().size()); + int size = -1; + try { + size = catalog.getDbNames().size(); + } catch (Exception e) { + LOG.warn("failed to get database: ", e); + } + catalogInfo.add(size); catalogInfo.add(TimeUtils.longToTimeString(catalog.getLastUpdateTime())); catalogInfos.add(catalogInfo); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/ExternalAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/ExternalAnalysisTask.java new file mode 100644 index 00000000000000..15848c013d68c8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/ExternalAnalysisTask.java @@ -0,0 +1,263 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.statistics; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.external.ExternalTable; +import org.apache.doris.common.FeConstants; +import org.apache.doris.common.Pair; +import org.apache.doris.statistics.util.StatisticsUtil; + +import org.apache.commons.text.StringSubstitutor; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; + +public class ExternalAnalysisTask extends BaseAnalysisTask { + private static final Logger LOG = LogManager.getLogger(ExternalAnalysisTask.class); + + private static final String ANALYZE_TABLE_COUNT_TEMPLATE = "SELECT ROUND(COUNT(1) * ${scaleFactor}) as rowCount " + + "FROM `${catalogName}`.`${dbName}`.`${tblName}` ${sampleHints}"; + private boolean isTableLevelTask; + private boolean isPartitionOnly; + private ExternalTable table; + + // For test + public ExternalAnalysisTask() { + } + + public ExternalAnalysisTask(AnalysisInfo info) { + super(info); + isTableLevelTask = info.externalTableLevelTask; + isPartitionOnly = info.partitionOnly; + table = (ExternalTable) tbl; + } + + public void doExecute() throws Exception { + if (isTableLevelTask) { + getTableStats(); + } else { + getOrdinaryColumnStats(); + } + } + + // For test + protected void setTable(ExternalTable table) { + this.table = table; + } + + /** + * Get table row count + */ + private void getTableStats() { + Map params = buildStatsParams(null); + List columnResult = + StatisticsUtil.execStatisticQuery(new StringSubstitutor(params) + .replace(ANALYZE_TABLE_COUNT_TEMPLATE)); + String rowCount = columnResult.get(0).get(0); + Env.getCurrentEnv().getAnalysisManager() + .updateTableStatsStatus( + new TableStatsMeta(Long.parseLong(rowCount), info, tbl)); + job.rowCountDone(this); + } + + // Get ordinary column stats + protected void getOrdinaryColumnStats() throws Exception { + StringBuilder sb = new StringBuilder(); + Map params = buildStatsParams("NULL"); + params.put("min", getMinFunction()); + params.put("max", getMaxFunction()); + params.put("dataSizeFunction", getDataSizeFunction(col, false)); + Pair sampleInfo = getSampleInfo(); + params.put("scaleFactor", String.valueOf(sampleInfo.first)); + StringSubstitutor stringSubstitutor; + if (tableSample == null) { + // Do full analyze + LOG.debug("Will do full collection for column {}", col.getName()); + sb.append(COLLECT_COL_STATISTICS); + } else { + // Do sample analyze + LOG.debug("Will do sample collection for column {}", col.getName()); + boolean limitFlag = false; + boolean bucketFlag = false; + // If sample size is too large, use limit to control the sample size. + if (needLimit(sampleInfo.second, sampleInfo.first)) { + limitFlag = true; + long columnSize = 0; + for (Column column : table.getFullSchema()) { + columnSize += column.getDataType().getSlotSize(); + } + double targetRows = (double) sampleInfo.second / columnSize; + // Estimate the new scaleFactor based on the schema. + if (targetRows > StatisticsUtil.getHugeTableSampleRows()) { + params.put("limit", "limit " + StatisticsUtil.getHugeTableSampleRows()); + params.put("scaleFactor", + String.valueOf(sampleInfo.first * targetRows / StatisticsUtil.getHugeTableSampleRows())); + } + } + // Single distribution column is not fit for DUJ1 estimator, use linear estimator. + Set distributionColumns = tbl.getDistributionColumnNames(); + if (distributionColumns.size() == 1 && distributionColumns.contains(col.getName().toLowerCase())) { + bucketFlag = true; + sb.append(LINEAR_ANALYZE_TEMPLATE); + params.put("ndvFunction", "ROUND(NDV(`${colName}`) * ${scaleFactor})"); + params.put("rowCount", "ROUND(count(1) * ${scaleFactor})"); + } else { + sb.append(DUJ1_ANALYZE_TEMPLATE); + params.put("dataSizeFunction", getDataSizeFunction(col, true)); + params.put("ndvFunction", getNdvFunction("ROUND(SUM(t1.count) * ${scaleFactor})")); + params.put("rowCount", "ROUND(SUM(t1.count) * ${scaleFactor})"); + } + LOG.info("Sample for column [{}]. Scale factor [{}], " + + "limited [{}], is distribute column [{}]", + col.getName(), params.get("scaleFactor"), limitFlag, bucketFlag); + } + stringSubstitutor = new StringSubstitutor(params); + String sql = stringSubstitutor.replace(sb.toString()); + runQuery(sql); + } + + protected Map buildStatsParams(String partId) { + Map commonParams = new HashMap<>(); + String id = StatisticsUtil.constructId(tbl.getId(), -1); + if (partId == null) { + commonParams.put("partId", "NULL"); + } else { + id = StatisticsUtil.constructId(id, partId); + commonParams.put("partId", "\'" + partId + "\'"); + } + commonParams.put("internalDB", FeConstants.INTERNAL_DB_NAME); + commonParams.put("columnStatTbl", StatisticConstants.STATISTIC_TBL_NAME); + commonParams.put("id", id); + commonParams.put("catalogId", String.valueOf(catalog.getId())); + commonParams.put("dbId", String.valueOf(db.getId())); + commonParams.put("tblId", String.valueOf(tbl.getId())); + commonParams.put("indexId", "-1"); + commonParams.put("idxId", "-1"); + commonParams.put("colName", info.colName); + commonParams.put("colId", info.colName); + commonParams.put("catalogName", catalog.getName()); + commonParams.put("dbName", db.getFullName()); + commonParams.put("tblName", tbl.getName()); + commonParams.put("sampleHints", getSampleHint()); + commonParams.put("limit", ""); + commonParams.put("scaleFactor", "1"); + if (col != null) { + commonParams.put("type", col.getType().toString()); + } + commonParams.put("lastAnalyzeTimeInMs", String.valueOf(System.currentTimeMillis())); + return commonParams; + } + + protected String getSampleHint() { + if (tableSample == null) { + return ""; + } + if (tableSample.isPercent()) { + return String.format("TABLESAMPLE(%d PERCENT)", tableSample.getSampleValue()); + } else { + return String.format("TABLESAMPLE(%d ROWS)", tableSample.getSampleValue()); + } + } + + /** + * Get the pair of sample scale factor and the file size going to sample. + * While analyzing, the result of count, null count and data size need to + * multiply this scale factor to get more accurate result. + * @return Pair of sample scale factor and the file size going to sample. + */ + protected Pair getSampleInfo() { + if (tableSample == null) { + return Pair.of(1.0, 0L); + } + long target; + // Get list of all files' size in this HMS table. + List chunkSizes = table.getChunkSizes(); + Collections.shuffle(chunkSizes, new Random(tableSample.getSeek())); + long total = 0; + // Calculate the total size of this HMS table. + for (long size : chunkSizes) { + total += size; + } + if (total == 0) { + return Pair.of(1.0, 0L); + } + // Calculate the sample target size for percent and rows sample. + if (tableSample.isPercent()) { + target = total * tableSample.getSampleValue() / 100; + } else { + int columnSize = 0; + for (Column column : table.getFullSchema()) { + columnSize += column.getDataType().getSlotSize(); + } + target = columnSize * tableSample.getSampleValue(); + } + // Calculate the actual sample size (cumulate). + long cumulate = 0; + for (long size : chunkSizes) { + cumulate += size; + if (cumulate >= target) { + break; + } + } + return Pair.of(Math.max(((double) total) / cumulate, 1), cumulate); + } + + @Override + protected void afterExecution() { + // Table level task doesn't need to sync any value to sync stats, it stores the value in metadata. + // Partition only task doesn't need to refresh cached. + if (isTableLevelTask || isPartitionOnly) { + return; + } + Env.getCurrentEnv().getStatisticsCache().syncLoadColStats(tbl.getId(), -1, col.getName()); + } + + /** + * If the size to sample is larger than LIMIT_SIZE (1GB) + * and is much larger (1.2*) than the size user want to sample, + * use limit to control the total sample size. + * @param sizeToRead The file size to sample. + * @param factor sizeToRead * factor = Table total size. + * @return True if need to limit. + */ + protected boolean needLimit(long sizeToRead, double factor) { + long total = (long) (sizeToRead * factor); + long target; + if (tableSample.isPercent()) { + target = total * tableSample.getSampleValue() / 100; + } else { + int columnSize = 0; + for (Column column : table.getFullSchema()) { + columnSize += column.getDataType().getSlotSize(); + } + target = columnSize * tableSample.getSampleValue(); + } + if (sizeToRead > LIMIT_SIZE && sizeToRead > target * LIMIT_FACTOR) { + return true; + } + return false; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/HMSAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/HMSAnalysisTask.java index fd0a4c8253821a..9e8be62282468e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/HMSAnalysisTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/HMSAnalysisTask.java @@ -17,12 +17,10 @@ package org.apache.doris.statistics; -import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.external.ExternalTable; import org.apache.doris.catalog.external.HMSExternalTable; import org.apache.doris.common.AnalysisException; -import org.apache.doris.common.FeConstants; -import org.apache.doris.common.Pair; import org.apache.doris.datasource.hive.HiveMetaStoreCache; import org.apache.doris.external.hive.util.HiveUtil; import org.apache.doris.statistics.util.StatisticsUtil; @@ -32,64 +30,36 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Map; -import java.util.Random; import java.util.Set; -public class HMSAnalysisTask extends BaseAnalysisTask { +public class HMSAnalysisTask extends ExternalAnalysisTask { private static final Logger LOG = LogManager.getLogger(HMSAnalysisTask.class); + private HMSExternalTable hmsExternalTable; - private static final String ANALYZE_TABLE_COUNT_TEMPLATE = "SELECT ROUND(COUNT(1) * ${scaleFactor}) as rowCount " - + "FROM `${catalogName}`.`${dbName}`.`${tblName}` ${sampleHints}"; - private boolean isTableLevelTask; - private boolean isPartitionOnly; - private HMSExternalTable table; - + // for test public HMSAnalysisTask() { } public HMSAnalysisTask(AnalysisInfo info) { super(info); - isTableLevelTask = info.externalTableLevelTask; - isPartitionOnly = info.partitionOnly; - table = (HMSExternalTable) tbl; + hmsExternalTable = (HMSExternalTable) tbl; } - public void doExecute() throws Exception { - if (isTableLevelTask) { - getTableStats(); - } else { - getTableColumnStats(); - } + private boolean isPartitionColumn() { + return hmsExternalTable.getPartitionColumns().stream().anyMatch(c -> c.getName().equals(col.getName())); } // For test protected void setTable(HMSExternalTable table) { - this.table = table; + setTable((ExternalTable) table); + this.hmsExternalTable = table; } - /** - * Get table row count - */ - private void getTableStats() throws Exception { - Map params = buildStatsParams(null); - List columnResult = - StatisticsUtil.execStatisticQuery(new StringSubstitutor(params) - .replace(ANALYZE_TABLE_COUNT_TEMPLATE)); - String rowCount = columnResult.get(0).get(0); - Env.getCurrentEnv().getAnalysisManager() - .updateTableStatsStatus( - new TableStatsMeta(Long.parseLong(rowCount), info, tbl)); - job.rowCountDone(this); - } - /** - * Get column statistics and insert the result to __internal_schema.column_statistics - */ - protected void getTableColumnStats() throws Exception { + @Override + protected void getOrdinaryColumnStats() throws Exception { if (!info.usingSqlForPartitionColumn) { try { if (isPartitionColumn()) { @@ -102,77 +72,17 @@ protected void getTableColumnStats() throws Exception { + "fallback to normal collection", isPartitionColumn() ? "partition " : "", col.getName(), e); /* retry using sql way! */ - getOrdinaryColumnStats(); + super.getOrdinaryColumnStats(); } } else { - getOrdinaryColumnStats(); + super.getOrdinaryColumnStats(); } } - private boolean isPartitionColumn() { - return table.getPartitionColumns().stream().anyMatch(c -> c.getName().equals(col.getName())); - } - - // Get ordinary column stats. Ordinary column means not partition column. - private void getOrdinaryColumnStats() throws Exception { - StringBuilder sb = new StringBuilder(); - Map params = buildStatsParams("NULL"); - params.put("min", getMinFunction()); - params.put("max", getMaxFunction()); - params.put("dataSizeFunction", getDataSizeFunction(col, false)); - Pair sampleInfo = getSampleInfo(); - params.put("scaleFactor", String.valueOf(sampleInfo.first)); - StringSubstitutor stringSubstitutor; - if (tableSample == null) { - // Do full analyze - LOG.debug("Will do full collection for column {}", col.getName()); - sb.append(COLLECT_COL_STATISTICS); - } else { - // Do sample analyze - LOG.debug("Will do sample collection for column {}", col.getName()); - boolean limitFlag = false; - boolean bucketFlag = false; - // If sample size is too large, use limit to control the sample size. - if (needLimit(sampleInfo.second, sampleInfo.first)) { - limitFlag = true; - long columnSize = 0; - for (Column column : table.getFullSchema()) { - columnSize += column.getDataType().getSlotSize(); - } - double targetRows = (double) sampleInfo.second / columnSize; - // Estimate the new scaleFactor based on the schema. - if (targetRows > StatisticsUtil.getHugeTableSampleRows()) { - params.put("limit", "limit " + StatisticsUtil.getHugeTableSampleRows()); - params.put("scaleFactor", - String.valueOf(sampleInfo.first * targetRows / StatisticsUtil.getHugeTableSampleRows())); - } - } - // Single distribution column is not fit for DUJ1 estimator, use linear estimator. - Set distributionColumns = tbl.getDistributionColumnNames(); - if (distributionColumns.size() == 1 && distributionColumns.contains(col.getName().toLowerCase())) { - bucketFlag = true; - sb.append(LINEAR_ANALYZE_TEMPLATE); - params.put("ndvFunction", "ROUND(NDV(`${colName}`) * ${scaleFactor})"); - params.put("rowCount", "ROUND(count(1) * ${scaleFactor})"); - } else { - sb.append(DUJ1_ANALYZE_TEMPLATE); - params.put("dataSizeFunction", getDataSizeFunction(col, true)); - params.put("ndvFunction", getNdvFunction("ROUND(SUM(t1.count) * ${scaleFactor})")); - params.put("rowCount", "ROUND(SUM(t1.count) * ${scaleFactor})"); - } - LOG.info("Sample for column [{}]. Scale factor [{}], " - + "limited [{}], is distribute column [{}]", - col.getName(), params.get("scaleFactor"), limitFlag, bucketFlag); - } - stringSubstitutor = new StringSubstitutor(params); - String sql = stringSubstitutor.replace(sb.toString()); - runQuery(sql); - } - // Collect the partition column stats through HMS metadata. // Get all the partition values and calculate the stats based on the values. private void getPartitionColumnStats() throws Exception { - Set partitionNames = table.getPartitionNames(); + Set partitionNames = hmsExternalTable.getPartitionNames(); Set ndvPartValues = Sets.newHashSet(); long numNulls = 0; long dataSize = 0; @@ -198,8 +108,9 @@ private void getPartitionColumnStats() throws Exception { } } // Estimate the row count. This value is inaccurate if the table stats is empty. - TableStatsMeta tableStatsStatus = Env.getCurrentEnv().getAnalysisManager().findTableStatsStatus(table.getId()); - long count = tableStatsStatus == null ? table.estimatedRowCount() : tableStatsStatus.rowCount; + TableStatsMeta tableStatsStatus = Env.getCurrentEnv().getAnalysisManager() + .findTableStatsStatus(hmsExternalTable.getId()); + long count = tableStatsStatus == null ? hmsExternalTable.estimatedRowCount() : tableStatsStatus.rowCount; dataSize = dataSize * count / partitionNames.size(); numNulls = numNulls * count / partitionNames.size(); int ndv = ndvPartValues.size(); @@ -218,8 +129,9 @@ private void getPartitionColumnStats() throws Exception { // Collect the spark analyzed column stats through HMS metadata. private void getHmsColumnStats() throws Exception { - TableStatsMeta tableStatsStatus = Env.getCurrentEnv().getAnalysisManager().findTableStatsStatus(table.getId()); - long count = tableStatsStatus == null ? table.estimatedRowCount() : tableStatsStatus.rowCount; + TableStatsMeta tableStatsStatus = Env.getCurrentEnv().getAnalysisManager() + .findTableStatsStatus(hmsExternalTable.getId()); + long count = tableStatsStatus == null ? hmsExternalTable.estimatedRowCount() : tableStatsStatus.rowCount; Map params = buildStatsParams("NULL"); Map statsParams = new HashMap<>(); @@ -229,7 +141,7 @@ private void getHmsColumnStats() throws Exception { statsParams.put(StatsType.MAX_VALUE, "max"); statsParams.put(StatsType.AVG_SIZE, "avg_len"); - if (table.fillColumnStatistics(info.colName, statsParams, params)) { + if (hmsExternalTable.fillColumnStatistics(info.colName, statsParams, params)) { throw new AnalysisException("some column stats not available"); } @@ -283,126 +195,4 @@ private String updateMaxValue(String currentMax, String value) { } return value.compareTo(currentMax) > 0 ? value : currentMax; } - - private Map buildStatsParams(String partId) { - Map commonParams = new HashMap<>(); - String id = StatisticsUtil.constructId(tbl.getId(), -1); - if (partId == null) { - commonParams.put("partId", "NULL"); - } else { - id = StatisticsUtil.constructId(id, partId); - commonParams.put("partId", "\'" + partId + "\'"); - } - commonParams.put("internalDB", FeConstants.INTERNAL_DB_NAME); - commonParams.put("columnStatTbl", StatisticConstants.STATISTIC_TBL_NAME); - commonParams.put("id", id); - commonParams.put("catalogId", String.valueOf(catalog.getId())); - commonParams.put("dbId", String.valueOf(db.getId())); - commonParams.put("tblId", String.valueOf(tbl.getId())); - commonParams.put("indexId", "-1"); - commonParams.put("idxId", "-1"); - commonParams.put("colName", info.colName); - commonParams.put("colId", info.colName); - commonParams.put("catalogName", catalog.getName()); - commonParams.put("dbName", db.getFullName()); - commonParams.put("tblName", tbl.getName()); - commonParams.put("sampleHints", getSampleHint()); - commonParams.put("limit", ""); - commonParams.put("scaleFactor", "1"); - if (col != null) { - commonParams.put("type", col.getType().toString()); - } - commonParams.put("lastAnalyzeTimeInMs", String.valueOf(System.currentTimeMillis())); - return commonParams; - } - - protected String getSampleHint() { - if (tableSample == null) { - return ""; - } - if (tableSample.isPercent()) { - return String.format("TABLESAMPLE(%d PERCENT)", tableSample.getSampleValue()); - } else { - return String.format("TABLESAMPLE(%d ROWS)", tableSample.getSampleValue()); - } - } - - /** - * Get the pair of sample scale factor and the file size going to sample. - * While analyzing, the result of count, null count and data size need to - * multiply this scale factor to get more accurate result. - * @return Pair of sample scale factor and the file size going to sample. - */ - protected Pair getSampleInfo() { - if (tableSample == null) { - return Pair.of(1.0, 0L); - } - long target; - // Get list of all files' size in this HMS table. - List chunkSizes = table.getChunkSizes(); - Collections.shuffle(chunkSizes, new Random(tableSample.getSeek())); - long total = 0; - // Calculate the total size of this HMS table. - for (long size : chunkSizes) { - total += size; - } - if (total == 0) { - return Pair.of(1.0, 0L); - } - // Calculate the sample target size for percent and rows sample. - if (tableSample.isPercent()) { - target = total * tableSample.getSampleValue() / 100; - } else { - int columnSize = 0; - for (Column column : table.getFullSchema()) { - columnSize += column.getDataType().getSlotSize(); - } - target = columnSize * tableSample.getSampleValue(); - } - // Calculate the actual sample size (cumulate). - long cumulate = 0; - for (long size : chunkSizes) { - cumulate += size; - if (cumulate >= target) { - break; - } - } - return Pair.of(Math.max(((double) total) / cumulate, 1), cumulate); - } - - @Override - protected void afterExecution() { - // Table level task doesn't need to sync any value to sync stats, it stores the value in metadata. - // Partition only task doesn't need to refresh cached. - if (isTableLevelTask || isPartitionOnly) { - return; - } - Env.getCurrentEnv().getStatisticsCache().syncLoadColStats(tbl.getId(), -1, col.getName()); - } - - /** - * If the size to sample is larger than LIMIT_SIZE (1GB) - * and is much larger (1.2*) than the size user want to sample, - * use limit to control the total sample size. - * @param sizeToRead The file size to sample. - * @param factor sizeToRead * factor = Table total size. - * @return True if need to limit. - */ - protected boolean needLimit(long sizeToRead, double factor) { - long total = (long) (sizeToRead * factor); - long target; - if (tableSample.isPercent()) { - target = total * tableSample.getSampleValue() / 100; - } else { - int columnSize = 0; - for (Column column : table.getFullSchema()) { - columnSize += column.getDataType().getSlotSize(); - } - target = columnSize * tableSample.getSampleValue(); - } - if (sizeToRead > LIMIT_SIZE && sizeToRead > target * LIMIT_FACTOR) { - return true; - } - return false; - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java index 5c8aec3fbf6eed..6176ec13bd62c2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java @@ -84,10 +84,12 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Table; import org.apache.iceberg.TableScan; +import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.types.Types; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.io.IOException; import java.net.InetSocketAddress; import java.nio.charset.StandardCharsets; import java.text.SimpleDateFormat; @@ -735,8 +737,12 @@ public static Optional getIcebergColumnStats(String colName, or columnStatisticBuilder.setDataSize(0); columnStatisticBuilder.setAvgSizeByte(0); columnStatisticBuilder.setNumNulls(0); - for (FileScanTask task : tableScan.planFiles()) { - processDataFile(task.file(), task.spec(), colName, columnStatisticBuilder); + try (CloseableIterable fileScanTasks = tableScan.planFiles()) { + for (FileScanTask task : fileScanTasks) { + processDataFile(task.file(), task.spec(), colName, columnStatisticBuilder); + } + } catch (IOException e) { + LOG.warn("Error to close FileScanTask.", e); } if (columnStatisticBuilder.getCount() > 0) { columnStatisticBuilder.setAvgSizeByte(columnStatisticBuilder.getDataSize() diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/HMSAnalysisTaskTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/HMSAnalysisTaskTest.java index fb0a3b3c2ca91f..e1016864525bc2 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/HMSAnalysisTaskTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/HMSAnalysisTaskTest.java @@ -252,7 +252,7 @@ public void runQuery(String sql) { analysisInfoBuilder.setUsingSqlForPartitionColumn(true); task.info = analysisInfoBuilder.build(); - task.getTableColumnStats(); + task.getOrdinaryColumnStats(); } @@ -309,6 +309,6 @@ public void runQuery(String sql) { analysisInfoBuilder.setUsingSqlForPartitionColumn(false); task.info = analysisInfoBuilder.build(); - task.getTableColumnStats(); + task.getOrdinaryColumnStats(); } } diff --git a/regression-test/conf/regression-conf.groovy b/regression-test/conf/regression-conf.groovy index 39c41f7c11a8ae..6d17bd032fdaf4 100644 --- a/regression-test/conf/regression-conf.groovy +++ b/regression-test/conf/regression-conf.groovy @@ -191,6 +191,7 @@ extArrowFlightSqlPassword= "" // iceberg rest catalog config iceberg_rest_uri_port=18181 +iceberg_minio_port=19001 // If the failure suite num exceeds this config // all following suite will be skipped to fast quit the run. diff --git a/regression-test/data/external_table_p0/iceberg/test_iceberg_statistics.out b/regression-test/data/external_table_p0/iceberg/test_iceberg_statistics.out new file mode 100644 index 00000000000000..c094d1714799f5 --- /dev/null +++ b/regression-test/data/external_table_p0/iceberg/test_iceberg_statistics.out @@ -0,0 +1,39 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !s1 -- +city 1000 4 0 Beijing Shanghai 6973 +col_binary 1000 867 0 0 1111101100100001001 15356 +col_boolean 1000 2 0 0 1 1000 +col_byte 1000 251 0 -128 127 4000 +col_char 1000 963 0 ! zy@notj#fkedb($ 9348 +col_date 1000 3 0 1969-09-21 2969-02-03 4000 +col_decimal 1000 1006 0 4.028284 9999.512216 8000 +col_double 1000 990 0 0.005217837593576302 9.996285421163707 8000 +col_float 1000 995 0 0.013126845 9.99709 4000 +col_integer 1000 999 0 -21468189 2108484 4000 +col_long 1000 996 0 -92193877774291102 92127291905311066 8000 +col_short 1000 985 0 -32554 32525 4000 +col_string 1000 992 0 0 zx70Jyeb6TfQ1YUaIGC 10714 +col_timestamp 1000 4 0 1970-01-01 08:00:01.000001 1970-01-04 08:00:01.000001 8000 +col_timestamp_ntz 1000 4 0 2017-12-01 10:12:55.038194 2017-12-04 10:12:55.038194 8000 +col_varchar 1000 988 0 0 zvnZ6bBxh 10764 +id 1000 1001 0 -99567408 99854631 8000 + +-- !s2 -- +city 1000 4 0 Beijing Shanghai 6973 +col_binary 1000 867 0 0 1111101100100001001 15356 +col_boolean 1000 2 0 0 1 1000 +col_byte 1000 251 0 -128 127 4000 +col_char 1000 973 0 ! zy@notj#fkedb($ 9324 +col_date 1000 3 0 1969-09-21 2969-02-03 4000 +col_decimal 1000 1006 0 4.028284 9999.512216 8000 +col_double 1000 990 0 0.005217837593576302 9.996285421163707 8000 +col_float 1000 995 0 0.013126845 9.99709 4000 +col_integer 1000 999 0 -21468189 2108484 4000 +col_long 1000 996 0 -92193877774291102 92127291905311066 8000 +col_short 1000 985 0 -32554 32525 4000 +col_string 1000 992 0 0 zx70Jyeb6TfQ1YUaIGC 10714 +col_timestamp 1000 4 0 1970-01-01 08:00:01.000001 1970-01-04 08:00:01.000001 8000 +col_timestamp_ntz 1000 4 0 2017-12-01 10:12:55.038194 2017-12-04 10:12:55.038194 8000 +col_varchar 1000 988 0 0 zvnZ6bBxh 10764 +id 1000 1001 0 -99567408 99854631 8000 + diff --git a/regression-test/data/external_table_p0/paimon/test_paimon_statistics.out b/regression-test/data/external_table_p0/paimon/test_paimon_statistics.out new file mode 100644 index 00000000000000..0f9f20d47820cf --- /dev/null +++ b/regression-test/data/external_table_p0/paimon/test_paimon_statistics.out @@ -0,0 +1,21 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !s1 -- +c1 2 2 0 1 10 2 +c10 2 2 0 10.1 100.1 16 +c11 2 2 0 11.10 110.10 16 +c12 2 2 0 2020-02-02 2020-03-02 8 +c13 2 2 0 130str 13str 11 +c14 2 2 0 140varchar 14varchar 19 +c15 2 2 0 a b 2 +c16 2 2 0 0 1 2 +c17 2 2 0 aaaa bbbb 8 +c18 2 2 0 2023-08-13 09:32:38.530000 2023-08-14 08:32:52.821000 16 +c2 2 2 0 2 20 2 +c3 2 2 0 3 30 4 +c4 2 2 0 4 40 4 +c5 2 2 0 5 50 8 +c6 2 2 0 6 60 8 +c7 2 2 0 7 70 16 +c8 2 2 0 8 80 16 +c9 2 2 0 9.1 90.1 8 + diff --git a/regression-test/data/external_table_p2/hive/test_hive_hudi_statistics.out b/regression-test/data/external_table_p2/hive/test_hive_hudi_statistics.out new file mode 100644 index 00000000000000..66a36a81afc68a --- /dev/null +++ b/regression-test/data/external_table_p2/hive/test_hive_hudi_statistics.out @@ -0,0 +1,16 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !s1 -- +_hoodie_commit_seqno 4 4 0 20230605145009209_0_1 20230801201335031_1_1 84 +_hoodie_commit_time 4 3 0 20230605145009209 20230801201335031 68 +_hoodie_file_name 4 4 0 65ffc5d9-397a-456e-a735-30f3ad37466f-0 e33d645c-6e2f-41f3-b8d6-f658771bd460-0_1-83-220_20230605145403388.parquet 221 +_hoodie_partition_path 4 3 0 partitionId=2011-11-11/versionId=v_1 partitionId=2021-02-01/versionId=v_4 144 +_hoodie_record_key 4 3 0 rowId:row_1 rowId:row_4 44 +inttolong 4 2 0 0 1 16 +longtoint 4 3 0 1000000 1000004 32 +name 4 3 0 ashin john 15 +partitionid 4 3 0 2011-11-11 2021-02-01 40 +precomb 4 3 0 0 4 32 +rowid 4 3 0 row_1 row_4 20 +tobedeletedstr 4 3 0 toBeDel0 toBeDel4 32 +versionid 4 3 0 v_0 v_4 12 + diff --git a/regression-test/pipeline/p0/conf/regression-conf.groovy b/regression-test/pipeline/p0/conf/regression-conf.groovy index be70ab0f730b4f..b99e21c4e7ccbd 100644 --- a/regression-test/pipeline/p0/conf/regression-conf.groovy +++ b/regression-test/pipeline/p0/conf/regression-conf.groovy @@ -96,6 +96,7 @@ kafka_port=19193 // iceberg test config iceberg_rest_uri_port=18181 +iceberg_minio_port=19001 enableEsTest=false es_6_port=19200 diff --git a/regression-test/pipeline/tpch/tpch-sf100/conf/regression-conf.groovy b/regression-test/pipeline/tpch/tpch-sf100/conf/regression-conf.groovy index 364a7103fe8612..5234ccc42417ff 100644 --- a/regression-test/pipeline/tpch/tpch-sf100/conf/regression-conf.groovy +++ b/regression-test/pipeline/tpch/tpch-sf100/conf/regression-conf.groovy @@ -94,6 +94,7 @@ kafka_port=19193 // iceberg test config iceberg_rest_uri_port=18181 +iceberg_minio_port=19001 enableEsTest=false es_6_port=19200 diff --git a/regression-test/plugins/plugins_get_ids_from_proc.groovy b/regression-test/plugins/plugins_get_ids_from_proc.groovy new file mode 100644 index 00000000000000..74a4d4d2010c48 --- /dev/null +++ b/regression-test/plugins/plugins_get_ids_from_proc.groovy @@ -0,0 +1,62 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +import org.apache.doris.regression.suite.Suite + +Suite.metaClass.get_catalog_id = {String catalog_name /* param */ -> + String catalog_id; + def catalogs = sql """show proc '/catalogs'""" + for (catalog in catalogs) { + if (catalog[1].equals(catalog_name)) { + catalog_id = catalog[0] + break + } + } + log.info("get catalogid: " + catalog_id) + return catalog_id +} + + +Suite.metaClass.get_database_id = {String catalog_name, String db_name /* param */ -> + String database_id; + def catalog_id = get_catalog_id(catalog_name) + def dbs = sql """show proc '/catalogs/${catalog_id}'""" + for (db in dbs) { + if (db[1].equals(db_name)) { + database_id = db[0] + break + } + } + log.info("get database_id: " + database_id) + return database_id +} + + +Suite.metaClass.get_table_id = {String catalog_name, String db_name, String tb_name /* param */ -> + String table_id; + def catalog_id = get_catalog_id(catalog_name) + def database_id = get_database_id(catalog_name, db_name) + def tbs = sql """show proc '/catalogs/${catalog_id}/${database_id}'""" + for (tb in tbs) { + if (tb[1].equals(tb_name)) { + table_id = tb[0] + break + } + } + log.info("get table_id: " + table_id) + return table_id +} diff --git a/regression-test/suites/external_table_p0/iceberg/test_iceberg_statistics.groovy b/regression-test/suites/external_table_p0/iceberg/test_iceberg_statistics.groovy new file mode 100644 index 00000000000000..24b27eb70b44f2 --- /dev/null +++ b/regression-test/suites/external_table_p0/iceberg/test_iceberg_statistics.groovy @@ -0,0 +1,57 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +suite("test_iceberg_statistics", "p0,external,doris,external_docker,external_docker_doris") { + String enabled = context.config.otherConfigs.get("enableIcebergTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + try { + String rest_port = context.config.otherConfigs.get("iceberg_rest_uri_port") + String minio_port = context.config.otherConfigs.get("iceberg_minio_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String catalog_name = "test_iceberg_rest_catalog" + String db_name = "format_v2" + + sql """drop catalog if exists ${catalog_name}""" + sql """CREATE CATALOG ${catalog_name} PROPERTIES ( + 'type'='iceberg', + 'iceberg.catalog.type'='rest', + 'uri' = 'http://${externalEnvIp}:${rest_port}', + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "http://${externalEnvIp}:${minio_port}", + "s3.region" = "us-east-1" + );""" + + def table_id_mor = get_table_id(catalog_name, db_name, "sample_mor_parquet") + def table_id_cow = get_table_id(catalog_name, db_name, "sample_cow_parquet") + + // analyze + sql """use `${catalog_name}`.`${db_name}`""" + sql """analyze table sample_mor_parquet with sync""" + sql """analyze table sample_cow_parquet with sync""" + + // select + def s1 = """select col_id,count,ndv,null_count,min,max,data_size_in_bytes from internal.__internal_schema.column_statistics where tbl_id = ${table_id_mor} order by id;""" + def s2 = """select col_id,count,ndv,null_count,min,max,data_size_in_bytes from internal.__internal_schema.column_statistics where tbl_id = ${table_id_cow} order by id;""" + + qt_s1 s1 + qt_s2 s2 + } finally { + } + } +} + diff --git a/regression-test/suites/external_table_p0/paimon/test_paimon_catalog.groovy b/regression-test/suites/external_table_p0/paimon/test_paimon_catalog.groovy index 87ea14ad2fdb02..ce8c9b5e849ac9 100644 --- a/regression-test/suites/external_table_p0/paimon/test_paimon_catalog.groovy +++ b/regression-test/suites/external_table_p0/paimon/test_paimon_catalog.groovy @@ -52,6 +52,9 @@ suite("test_paimon_catalog", "p0,external,doris,external_docker,external_docker_ ); """ + sql """drop catalog ${file_ctl_name}"""; + sql """drop catalog ${hms_ctl_name}"""; + String enabled = context.config.otherConfigs.get("enablePaimonTest") if (enabled != null && enabled.equalsIgnoreCase("true")) { def all = """select * from all_table order by c1;""" diff --git a/regression-test/suites/external_table_p0/paimon/test_paimon_statistics.groovy b/regression-test/suites/external_table_p0/paimon/test_paimon_statistics.groovy new file mode 100644 index 00000000000000..c75e7b797d9381 --- /dev/null +++ b/regression-test/suites/external_table_p0/paimon/test_paimon_statistics.groovy @@ -0,0 +1,47 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +suite("test_paimon_statistics", "p0,external,doris,external_docker,external_docker_doris") { + String enabled = context.config.otherConfigs.get("enablePaimonTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + try { + String hdfs_port = context.config.otherConfigs.get("hdfs_port") + String catalog_name = "paimon1" + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + + sql """drop catalog if exists ${catalog_name}""" + sql """create catalog if not exists ${catalog_name} properties ( + "type" = "paimon", + "paimon.catalog.type"="filesystem", + "warehouse" = "hdfs://${externalEnvIp}:${hdfs_port}/user/doris/paimon1" + );""" + + def table_id = get_table_id(catalog_name, "db1", "all_table") + + // analyze + sql """use `${catalog_name}`.`db1`""" + sql """analyze table all_table with sync""" + + // select + def s1 = """select col_id,count,ndv,null_count,min,max,data_size_in_bytes from internal.__internal_schema.column_statistics where tbl_id = ${table_id} order by id;""" + + qt_s1 s1 + } finally { + } + } +} + diff --git a/regression-test/suites/external_table_p0/test_catalog_ddl.groovy b/regression-test/suites/external_table_p0/test_catalog_ddl.groovy index b236567e8bdc77..a9a67f51853fbd 100644 --- a/regression-test/suites/external_table_p0/test_catalog_ddl.groovy +++ b/regression-test/suites/external_table_p0/test_catalog_ddl.groovy @@ -44,4 +44,6 @@ suite("test_catalog_ddl", "p0,external,external_docker") { result = sql """show create catalog ${catalog1};""" assertEquals(result.size(), 1) assertTrue(result[0][1].contains("COMMENT \"alter_comment\"")) + + sql """drop catalog ${catalog1}""" } diff --git a/regression-test/suites/external_table_p2/hive/test_hive_hudi_statistics.groovy b/regression-test/suites/external_table_p2/hive/test_hive_hudi_statistics.groovy new file mode 100644 index 00000000000000..55e5037de458c7 --- /dev/null +++ b/regression-test/suites/external_table_p2/hive/test_hive_hudi_statistics.groovy @@ -0,0 +1,47 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +suite("test_hive_hudi_statistics", "p2,external,hive,hudi") { + String enabled = context.config.otherConfigs.get("enableExternalHiveTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String extHiveHmsHost = context.config.otherConfigs.get("extHiveHmsHost") + String extHiveHmsPort = context.config.otherConfigs.get("extHiveHmsPort") + String catalog_name = "test_hive_hudi_statistics" + String db_name = "hudi_catalog" + + sql """drop catalog if exists ${catalog_name};""" + sql """ + create catalog if not exists ${catalog_name} properties ( + 'hadoop.username'='hadoop', + 'type'='hms', + 'hive.metastore.uris' = 'thrift://${extHiveHmsHost}:${extHiveHmsPort}' + ); + """ + + def table_id = get_table_id(catalog_name, db_name, "partitioned_mor_rt") + + // analyze + sql """use `${catalog_name}`.`${db_name}`""" + sql """analyze table partitioned_mor_rt with sync""" + + // select + def s1 = """select col_id,count,ndv,null_count,min,max,data_size_in_bytes from internal.__internal_schema.column_statistics where tbl_id = ${table_id} order by id;""" + + qt_s1 s1 + + } +} From df9f2bc15d0b7a060d253a47fc94ed95efadf639 Mon Sep 17 00:00:00 2001 From: Pxl Date: Wed, 17 Jan 2024 17:49:59 +0800 Subject: [PATCH 066/200] [Chore](config) remove some unused config (#29983) remove some unused config --- be/src/common/config.cpp | 8 -------- be/src/common/config.h | 8 -------- docs/en/docs/admin-manual/config/be-config.md | 18 ------------------ .../docs/admin-manual/config/be-config.md | 18 ------------------ .../suites/update/test_update_configs.groovy | 15 ++------------- 5 files changed, 2 insertions(+), 65 deletions(-) diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 14efa8a4b9da6b..8b7edb860655c6 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -265,8 +265,6 @@ DEFINE_mInt32(doris_max_scan_key_num, "48"); // the max number of push down values of a single column. // if exceed, no conditions will be pushed down for that column. DEFINE_mInt32(max_pushdown_conditions_per_column, "1024"); -// return_row / total_row -DEFINE_mInt32(doris_max_pushdown_conjuncts_return_rate, "90"); // (Advanced) Maximum size of per-query receive-side buffer DEFINE_mInt32(exchg_node_buffer_size_bytes, "20485760"); @@ -542,10 +540,6 @@ DEFINE_Int32(min_buffer_size, "1024"); // 1024, The minimum read buffer size (in // With 1024B through 8MB buffers, this is up to ~2GB of buffers. DEFINE_Int32(max_free_io_buffers, "128"); -// The probing algorithm of partitioned hash table. -// Enable quadratic probing hash table -DEFINE_Bool(enable_quadratic_probing, "false"); - // for pprof DEFINE_String(pprof_profile_dir, "${DORIS_HOME}/log"); // for jeprofile in jemalloc @@ -558,8 +552,6 @@ DEFINE_mBool(enable_token_check, "true"); // to open/close system metrics DEFINE_Bool(enable_system_metrics, "true"); -DEFINE_mBool(enable_prefetch, "true"); - // Number of cores Doris will used, this will effect only when it's greater than 0. // Otherwise, Doris will use all cores returned from "/proc/cpuinfo". DEFINE_Int32(num_cores, "0"); diff --git a/be/src/common/config.h b/be/src/common/config.h index 9c0b766f4090be..6d5b54a270a32b 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -307,8 +307,6 @@ DECLARE_mInt32(doris_max_scan_key_num); // the max number of push down values of a single column. // if exceed, no conditions will be pushed down for that column. DECLARE_mInt32(max_pushdown_conditions_per_column); -// return_row / total_row -DECLARE_mInt32(doris_max_pushdown_conjuncts_return_rate); // (Advanced) Maximum size of per-query receive-side buffer DECLARE_mInt32(exchg_node_buffer_size_bytes); @@ -592,10 +590,6 @@ DECLARE_Int32(min_buffer_size); // 1024, The minimum read buffer size (in bytes) // With 1024B through 8MB buffers, this is up to ~2GB of buffers. DECLARE_Int32(max_free_io_buffers); -// The probing algorithm of partitioned hash table. -// Enable quadratic probing hash table -DECLARE_Bool(enable_quadratic_probing); - // for pprof DECLARE_String(pprof_profile_dir); // for jeprofile in jemalloc @@ -609,8 +603,6 @@ DECLARE_mBool(enable_token_check); // to open/close system metrics DECLARE_Bool(enable_system_metrics); -DECLARE_mBool(enable_prefetch); - // Number of cores Doris will used, this will effect only when it's greater than 0. // Otherwise, Doris will use all cores returned from "/proc/cpuinfo". DECLARE_Int32(num_cores); diff --git a/docs/en/docs/admin-manual/config/be-config.md b/docs/en/docs/admin-manual/config/be-config.md index 3f00e62d38bf26..82f54818f6e63a 100644 --- a/docs/en/docs/admin-manual/config/be-config.md +++ b/docs/en/docs/admin-manual/config/be-config.md @@ -343,12 +343,6 @@ There are two ways to configure BE configuration items: * Description: Follow up query requests create threads dynamically, with a maximum of 512 threads created. * Default value: 2048 -#### `doris_max_pushdown_conjuncts_return_rate` - -* Type: int32 -* Description: When BE performs HashJoin, it will adopt a dynamic partitioning method to push the join condition to OlapScanner. When the data scanned by OlapScanner is larger than 32768 rows, BE will check the filter condition. If the filter rate of the filter condition is lower than this configuration, Doris will stop using the dynamic partition clipping condition for data filtering. -* Default value: 90 - #### `doris_max_scan_key_num` * Type: int @@ -397,18 +391,6 @@ There are two ways to configure BE configuration items: * Description: Max thread number of Remote scanner thread pool. Remote scanner thread pool is used for scan task of all external data sources. * Default: 512 -#### `enable_prefetch` - -* Type: bool -* Description: When using PartitionedHashTable for aggregation and join calculations, whether to perform HashBucket prefetch. Recommended to be set to true -* Default value: true - -#### `enable_quadratic_probing` - -* Type: bool -* Description: When a Hash conflict occurs when using PartitionedHashTable, enable to use the square detection method to resolve the Hash conflict. If the value is false, linear detection is used to resolve the Hash conflict. For the square detection method, please refer to: [quadratic_probing](https://en.wikipedia.org/wiki/Quadratic_probing) -* Default value: true - #### `exchg_node_buffer_size_bytes` * Type: int32 diff --git a/docs/zh-CN/docs/admin-manual/config/be-config.md b/docs/zh-CN/docs/admin-manual/config/be-config.md index dc8418d548bb2b..bda5dce7e50683 100644 --- a/docs/zh-CN/docs/admin-manual/config/be-config.md +++ b/docs/zh-CN/docs/admin-manual/config/be-config.md @@ -354,12 +354,6 @@ BE 重启后该配置将失效。如果想持久化修改结果,使用如下 * 描述:后续查询请求动态创建线程,最大创建512个线程。 * 默认值:2048 -#### `doris_max_pushdown_conjuncts_return_rate` - -* 类型:int32 -* 描述:BE在进行HashJoin时,会采取动态分区裁剪的方式将join条件下推到OlapScanner上。当OlapScanner扫描的数据大于32768行时,BE会进行过滤条件检查,如果该过滤条件的过滤率低于该配置,则Doris会停止使用动态分区裁剪的条件进行数据过滤。 -* 默认值:90 - #### `doris_max_scan_key_num` * 类型:int @@ -408,18 +402,6 @@ BE 重启后该配置将失效。如果想持久化修改结果,使用如下 * 描述:Remote scanner thread pool 的最大线程数。Remote scanner thread pool 用于除内表外的所有 scan 任务的执行。 * 默认值:512 -#### `enable_prefetch` - -* 类型:bool -* 描述:当使用PartitionedHashTable进行聚合和join计算时,是否进行 HashBucket 的预取,推荐设置为true。 -* 默认值:true - -#### `enable_quadratic_probing` - -* 类型:bool -* 描述:当使用PartitionedHashTable时发生Hash冲突时,是否采用平方探测法来解决Hash冲突。该值为false的话,则选用线性探测发来解决Hash冲突。关于平方探测法可参考:[quadratic_probing](https://en.wikipedia.org/wiki/Quadratic_probing) -* 默认值:true - #### `exchg_node_buffer_size_bytes` * 类型:int32 diff --git a/regression-test/suites/update/test_update_configs.groovy b/regression-test/suites/update/test_update_configs.groovy index f2206e61c3b1b5..cbce43293c1c9a 100644 --- a/regression-test/suites/update/test_update_configs.groovy +++ b/regression-test/suites/update/test_update_configs.groovy @@ -38,16 +38,12 @@ suite("test_update_configs", "p0") { if (((List) ele)[0] == "disable_auto_compaction") { disableAutoCompaction = Boolean.parseBoolean(((List) ele)[2]) } - if (((List) ele)[0] == "enable_prefetch") { - enablePrefetch = Boolean.parseBoolean(((List) ele)[2]) - } if (((List) ele)[0] == "enable_segcompaction") { enableSegcompaction = Boolean.parseBoolean(((List) ele)[2]) } } - logger.info("disable_auto_compaction:${disableAutoCompaction}, enable_prefetch:${enablePrefetch}, enable_segcompaction:${enableSegcompaction}") - curl("POST", String.format("http://%s:%s/api/update_config?%s=%s&%s=%s&%s=%s", beIp, bePort, "disable_auto_compaction", String.valueOf(!disableAutoCompaction), "enable_prefetch", String.valueOf(!enablePrefetch), "enable_segcompaction", String.valueOf(!enableSegcompaction))) + curl("POST", String.format("http://%s:%s/api/update_config?%s=%s&%s=%s", beIp, bePort, "disable_auto_compaction", String.valueOf(!disableAutoCompaction), "enable_segcompaction", String.valueOf(!enableSegcompaction))) (code, out, err) = show_be_config(beIp, bePort) @@ -61,10 +57,6 @@ suite("test_update_configs", "p0") { logger.info("disable_auto_compaction: ${((List) ele)[2]}") assertEquals(((List) ele)[2], String.valueOf(!disableAutoCompaction)) } - if (((List) ele)[0] == "enable_prefetch") { - logger.info("enable_prefetch: ${((List) ele)[3]}") - assertEquals(((List) ele)[2], String.valueOf(!enablePrefetch)) - } if (((List) ele)[0] == "enable_segcompaction") { // enable_segcompaction is not mutable logger.info("enable_segcompaction: ${((List) ele)[3]}") @@ -72,7 +64,7 @@ suite("test_update_configs", "p0") { } } - curl("POST", String.format("http://%s:%s/api/update_config?%s=%s&%s=%s", beIp, bePort, "disable_auto_compaction", String.valueOf(disableAutoCompaction), "enable_prefetch", String.valueOf(enablePrefetch))) + curl("POST", String.format("http://%s:%s/api/update_config?%s=%s", beIp, bePort, "disable_auto_compaction", String.valueOf(disableAutoCompaction))) (code, out, err) = show_be_config(beIp, bePort) assertEquals(code, 0) @@ -83,8 +75,5 @@ suite("test_update_configs", "p0") { if (((List) ele)[0] == "disable_auto_compaction") { assertEquals(((List) ele)[2], String.valueOf(disableAutoCompaction)) } - if (((List) ele)[0] == "enable_prefetch") { - assertEquals(((List) ele)[2], String.valueOf(enablePrefetch)) - } } } From 5bd03fdc6696fff811b88c094e566ae6ef7dfecb Mon Sep 17 00:00:00 2001 From: jakevin Date: Wed, 17 Jan 2024 19:20:19 +0800 Subject: [PATCH 067/200] [minor](Nereids): remove useless method isMysqlCompatibleDatabase (#30064) --- .../java/org/apache/doris/analysis/DropDbStmt.java | 4 ++-- .../java/org/apache/doris/catalog/Database.java | 13 +++---------- .../java/org/apache/doris/catalog/DatabaseIf.java | 2 +- .../doris/catalog/MysqlCompatibleDatabase.java | 5 ----- .../java/org/apache/doris/clone/TabletChecker.java | 3 ++- .../java/org/apache/doris/datasource/CatalogIf.java | 6 +++--- .../apache/doris/datasource/InternalCatalog.java | 10 +++++----- .../org/apache/doris/httpv2/rest/ShowAction.java | 5 +++-- .../master/PartitionInMemoryInfoCollector.java | 3 ++- .../transaction/DbUsedDataQuotaInfoCollector.java | 3 ++- .../org/apache/doris/clone/RebalancerTestUtil.java | 3 ++- 11 files changed, 25 insertions(+), 32 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java index 9b4f58efbf0f6a..08a1e630eb52cf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java @@ -17,9 +17,9 @@ package org.apache.doris.analysis; -import org.apache.doris.catalog.Database; import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MysqlCompatibleDatabase; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.UserException; @@ -62,7 +62,7 @@ public void analyze(Analyzer analyzer) throws UserException { InternalDatabaseUtil.checkDatabase(dbName, ConnectContext.get()); // Don't allow to drop mysql compatible databases DatabaseIf db = Env.getCurrentInternalCatalog().getDbNullable(dbName); - if (db != null && (db instanceof Database) && ((Database) db).isMysqlCompatibleDatabase()) { + if (db != null && db instanceof MysqlCompatibleDatabase) { ErrorReport.reportAnalysisException(ErrorCode.ERR_DBACCESS_DENIED_ERROR, analyzer.getQualifiedUser(), dbName); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java index 71e61d5791ca2e..96db31af430fbb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java @@ -84,14 +84,14 @@ public class Database extends MetaObject implements Writable, DatabaseIf private long id; @SerializedName(value = "fullQualifiedName") private volatile String fullQualifiedName; - private ReentrantReadWriteLock rwLock; + private final ReentrantReadWriteLock rwLock; // table family group map - private Map idToTable; + private final Map idToTable; @SerializedName(value = "nameToTable") private Map nameToTable; // table name lower cast -> table name - private Map lowerCaseToTableName; + private final Map lowerCaseToTableName; // user define function @SerializedName(value = "name2Function") @@ -893,11 +893,4 @@ public String toJson() { public String toString() { return toJson(); } - - // Return ture if database is created for mysql compatibility. - // Currently, we have two dbs that are created for this purpose, InformationSchemaDb and MysqlDb, - public boolean isMysqlCompatibleDatabase() { - return false; - } - } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseIf.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseIf.java index 3ba295733740a7..e80f335be17805 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseIf.java @@ -273,5 +273,5 @@ default long getLastUpdateTime() { return -1L; } - public Map getIdToTable(); + Map getIdToTable(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MysqlCompatibleDatabase.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MysqlCompatibleDatabase.java index 1d4aa62dc9dd91..12d915321ede50 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MysqlCompatibleDatabase.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MysqlCompatibleDatabase.java @@ -77,11 +77,6 @@ public void readFields(DataInput in) throws IOException { throw new IOException("Not support."); } - @Override - public boolean isMysqlCompatibleDatabase() { - return true; - } - /** * This method must be re-implemented since {@link Env#createView(CreateViewStmt)} * will call this method. And create view should not succeed under this database. diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletChecker.java b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletChecker.java index c6f6be3d460c45..66497e7d18e435 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletChecker.java +++ b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletChecker.java @@ -23,6 +23,7 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.MaterializedIndex; import org.apache.doris.catalog.MaterializedIndex.IndexExtState; +import org.apache.doris.catalog.MysqlCompatibleDatabase; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Partition.PartitionState; @@ -280,7 +281,7 @@ private void checkTablets() { continue; } - if (db.isMysqlCompatibleDatabase()) { + if (db instanceof MysqlCompatibleDatabase) { continue; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogIf.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogIf.java index c8f3033c6688ab..7fd240aa8d07d2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogIf.java @@ -172,9 +172,9 @@ default CatalogLog constructEditLog() { } // Return a copy of all db collection. - public Collection> getAllDbs(); + Collection> getAllDbs(); - public boolean enableAutoAnalyze(); + boolean enableAutoAnalyze(); - public ConcurrentHashMap getIdToDb(); + ConcurrentHashMap getIdToDb(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java index 791fcd007d8cfe..ae32e45aa81c37 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java @@ -841,8 +841,8 @@ public void dropTable(DropTableStmt stmt) throws DdlException { LOG.info("begin to drop table: {} from db: {}, is force: {}", tableName, dbName, stmt.isForceDrop()); // check database - Database db = (Database) getDbOrDdlException(dbName); - if (db.isMysqlCompatibleDatabase()) { + Database db = getDbOrDdlException(dbName); + if (db instanceof MysqlCompatibleDatabase) { throw new DdlException("Drop table from this database is not allowed."); } @@ -1100,7 +1100,7 @@ public void createTable(CreateTableStmt stmt) throws UserException { // check if db exists Database db = getDbOrDdlException(dbName); // InfoSchemaDb and MysqlDb can not create table manually - if (db.isMysqlCompatibleDatabase()) { + if (db instanceof MysqlCompatibleDatabase) { ErrorReport.reportDdlException(ErrorCode.ERR_CANT_CREATE_TABLE, tableName, ErrorCode.ERR_CANT_CREATE_TABLE.getCode(), "not supported create table in this database"); } @@ -3326,7 +3326,7 @@ public long saveDb(CountingDataOutputStream dos, long checksum) throws IOExcepti for (Map.Entry entry : idToDb.entrySet()) { Database db = entry.getValue(); // Don't write internal database meta. - if (!db.isMysqlCompatibleDatabase()) { + if (!(db instanceof MysqlCompatibleDatabase)) { checksum ^= entry.getKey(); db.write(dos); } @@ -3345,7 +3345,7 @@ public long loadDb(DataInputStream dis, long checksum) throws IOException, DdlEx Database dbPrev = fullNameToDb.get(db.getFullName()); if (dbPrev != null) { String errMsg; - if (dbPrev.isMysqlCompatibleDatabase() || db.isMysqlCompatibleDatabase()) { + if (dbPrev instanceof MysqlCompatibleDatabase || db instanceof MysqlCompatibleDatabase) { errMsg = String.format( "Mysql compatibility problem, previous checkpoint already has a database with full name " + "%s. If its name is mysql, try to add mysqldb_replace_name=\"mysql_comp\" in fe.conf.", diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/ShowAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/ShowAction.java index 6db2a7e7b51d3f..0aa5dd54d78b46 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/ShowAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/ShowAction.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.Database; import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MysqlCompatibleDatabase; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Table; import org.apache.doris.catalog.TableIf.TableType; @@ -199,7 +200,7 @@ public Object show_data(HttpServletRequest request, HttpServletResponse response } else { for (long dbId : Env.getCurrentInternalCatalog().getDbIds()) { DatabaseIf db = Env.getCurrentInternalCatalog().getDbNullable(dbId); - if (db == null || !(db instanceof Database) || ((Database) db).isMysqlCompatibleDatabase()) { + if (db == null || !(db instanceof Database) || db instanceof MysqlCompatibleDatabase) { continue; } totalSize += getDataSizeOfDatabase(db); @@ -232,7 +233,7 @@ public Object show_table_data(HttpServletRequest request, HttpServletResponse re } else { for (long dbId : Env.getCurrentInternalCatalog().getDbIds()) { DatabaseIf db = Env.getCurrentInternalCatalog().getDbNullable(dbId); - if (db == null || !(db instanceof Database) || ((Database) db).isMysqlCompatibleDatabase()) { + if (db == null || !(db instanceof Database) || ((Database) db) instanceof MysqlCompatibleDatabase) { continue; } Map tablesEntry = getDataSizeOfTables(db, tableName, singleReplicaBool); diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/PartitionInMemoryInfoCollector.java b/fe/fe-core/src/main/java/org/apache/doris/master/PartitionInMemoryInfoCollector.java index 22f2d41cfad3c8..77ed5829799363 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/PartitionInMemoryInfoCollector.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/PartitionInMemoryInfoCollector.java @@ -19,6 +19,7 @@ import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MysqlCompatibleDatabase; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Table; @@ -56,7 +57,7 @@ private void updatePartitionInMemoryInfo() { LOG.warn("Database [" + dbId + "] does not exist, skip to update database used data quota"); continue; } - if (db.isMysqlCompatibleDatabase()) { + if (db instanceof MysqlCompatibleDatabase) { continue; } try { diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/DbUsedDataQuotaInfoCollector.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/DbUsedDataQuotaInfoCollector.java index 87d509bbb4d35d..3a713bdc63f190 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/DbUsedDataQuotaInfoCollector.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/DbUsedDataQuotaInfoCollector.java @@ -19,6 +19,7 @@ import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MysqlCompatibleDatabase; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; import org.apache.doris.common.util.MasterDaemon; @@ -50,7 +51,7 @@ private void updateAllDatabaseUsedDataQuota() { LOG.warn("Database [" + dbId + "] does not exist, skip to update database used data quota"); continue; } - if (db.isMysqlCompatibleDatabase()) { + if (db instanceof MysqlCompatibleDatabase) { continue; } try { diff --git a/fe/fe-core/src/test/java/org/apache/doris/clone/RebalancerTestUtil.java b/fe/fe-core/src/test/java/org/apache/doris/clone/RebalancerTestUtil.java index da03d42a644cba..063faf2d3b2ea6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/clone/RebalancerTestUtil.java +++ b/fe/fe-core/src/test/java/org/apache/doris/clone/RebalancerTestUtil.java @@ -21,6 +21,7 @@ import org.apache.doris.catalog.DiskInfo; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MysqlCompatibleDatabase; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Replica; @@ -145,7 +146,7 @@ public static void updateReplicaDataSize(long minReplicaSize, int tableSkew, in continue; } - if (db.isMysqlCompatibleDatabase()) { + if (db instanceof MysqlCompatibleDatabase) { continue; } From 809e8cb5475c58174cbbfeeaa7348e6687e03f55 Mon Sep 17 00:00:00 2001 From: HHoflittlefish777 <77738092+HHoflittlefish777@users.noreply.github.com> Date: Wed, 17 Jan 2024 19:59:14 +0800 Subject: [PATCH 068/200] [cleanup](insert-into) clean up some insert into log (#30063) --- be/src/pipeline/exec/union_source_operator.cpp | 6 ------ be/src/runtime/fragment_mgr.cpp | 5 ----- be/src/vec/exec/vunion_node.cpp | 6 ------ .../main/java/org/apache/doris/qe/Coordinator.java | 14 +++----------- regression-test/pipeline/p0/conf/be.conf | 2 +- 5 files changed, 4 insertions(+), 29 deletions(-) diff --git a/be/src/pipeline/exec/union_source_operator.cpp b/be/src/pipeline/exec/union_source_operator.cpp index 31806d7242cd54..e8ef1ba7207653 100644 --- a/be/src/pipeline/exec/union_source_operator.cpp +++ b/be/src/pipeline/exec/union_source_operator.cpp @@ -223,9 +223,6 @@ Status UnionSourceOperatorX::get_next_const(RuntimeState* state, vectorized::Blo &result_list[i])); } tmp_block.erase_not_in(result_list); - VLOG_ROW << "query id: " << print_id(state->query_id()) - << ", instance id: " << print_id(state->fragment_instance_id()) - << ", tmp_block rows: " << tmp_block.rows(); if (tmp_block.rows() > 0) { RETURN_IF_ERROR(mblock.merge(tmp_block)); tmp_block.clear(); @@ -235,9 +232,6 @@ Status UnionSourceOperatorX::get_next_const(RuntimeState* state, vectorized::Blo // some insert query like "insert into string_test select 1, repeat('a', 1024 * 1024);" // the const expr will be in output expr cause the union node return a empty block. so here we // need add one row to make sure the union node exec const expr return at least one row - VLOG_ROW << "query id: " << print_id(state->query_id()) - << ", instance id: " << print_id(state->fragment_instance_id()) - << ", tmp_block rows: " << block->rows(); if (block->rows() == 0) { block->insert({vectorized::ColumnUInt8::create(1), std::make_shared(), ""}); diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index 7d3dba810205d9..0fbaacb4fa22c8 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -322,11 +322,6 @@ void FragmentMgr::coordinator_callback(const ReportStatusRequest& req) { params.load_counters.emplace(s_dpp_normal_all, std::to_string(num_rows_load_success)); params.load_counters.emplace(s_dpp_abnormal_all, std::to_string(num_rows_load_filtered)); params.load_counters.emplace(s_unselected_rows, std::to_string(num_rows_load_unselected)); - LOG(INFO) << "execute coordinator callback, query id: " << print_id(req.query_id) - << ", instance id: " << print_id(req.fragment_instance_id) - << ", num_rows_load_success: " << num_rows_load_success - << ", num_rows_load_filtered: " << num_rows_load_filtered - << ", num_rows_load_unselected: " << num_rows_load_unselected; if (!req.runtime_state->get_error_log_file_path().empty()) { params.__set_tracking_url( diff --git a/be/src/vec/exec/vunion_node.cpp b/be/src/vec/exec/vunion_node.cpp index 77f9c0a40f28fb..8ee258ca05678e 100644 --- a/be/src/vec/exec/vunion_node.cpp +++ b/be/src/vec/exec/vunion_node.cpp @@ -229,18 +229,12 @@ Status VUnionNode::get_next_const(RuntimeState* state, Block* block) { &result_list[i])); } tmp_block.erase_not_in(result_list); - VLOG_ROW << "query id: " << print_id(state->query_id()) - << ", instance id: " << print_id(state->fragment_instance_id()) - << ", tmp_block rows: " << tmp_block.rows(); if (tmp_block.rows() > 0) { RETURN_IF_ERROR(mblock.merge(tmp_block)); tmp_block.clear(); } } block->set_columns(std::move(mblock.mutable_columns())); - VLOG_ROW << "query id: " << print_id(state->query_id()) - << ", instance id: " << print_id(state->fragment_instance_id()) - << ", block rows: " << block->rows(); // some insert query like "insert into string_test select 1, repeat('a', 1024 * 1024);" // the const expr will be in output expr cause the union node return a empty block. so here we diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index 165583b9e686bc..2f58e560740ffc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -688,12 +688,6 @@ private void execInternal() throws Exception { } else { executionProfile.markInstances(instanceIds); } - StringBuilder ids = new StringBuilder(); - for (TUniqueId instanceId : instanceIds) { - ids.append(DebugUtil.printId(instanceId)); - ids.append(", "); - } - LOG.info("mark instances: {}", ids.toString()); if (enablePipelineEngine) { sendPipelineCtx(); @@ -2494,7 +2488,7 @@ public void updateFragmentExecStatus(TReportExecStatusParams params) { Preconditions.checkArgument(params.isSetDetailedReport()); if (ctx.done) { - LOG.info("Query {} fragment {} is marked done", + LOG.debug("Query {} fragment {} is marked done", DebugUtil.printId(queryId), ctx.profileFragmentId); executionProfile.markOneFragmentDone(ctx.profileFragmentId); } @@ -2552,11 +2546,11 @@ public void updateFragmentExecStatus(TReportExecStatusParams params) { if (params.isSetErrorTabletInfos()) { updateErrorTabletInfos(params.getErrorTabletInfos()); } - LOG.info("Query {} instance {} is marked done", + LOG.debug("Query {} instance {} is marked done", DebugUtil.printId(queryId), DebugUtil.printId(params.getFragmentInstanceId())); executionProfile.markOneInstanceDone(params.getFragmentInstanceId()); } else { - LOG.info("Query {} instance {} is not marked done", + LOG.debug("Query {} instance {} is not marked done", DebugUtil.printId(queryId), DebugUtil.printId(params.getFragmentInstanceId())); } } else { @@ -2623,8 +2617,6 @@ public void updateFragmentExecStatus(TReportExecStatusParams params) { if (params.isSetErrorTabletInfos()) { updateErrorTabletInfos(params.getErrorTabletInfos()); } - LOG.info("Query {} instance {} is marked done", - DebugUtil.printId(queryId), DebugUtil.printId(params.getFragmentInstanceId())); executionProfile.markOneInstanceDone(params.getFragmentInstanceId()); } } diff --git a/regression-test/pipeline/p0/conf/be.conf b/regression-test/pipeline/p0/conf/be.conf index c2c09ec89dd868..d562e864a845be 100644 --- a/regression-test/pipeline/p0/conf/be.conf +++ b/regression-test/pipeline/p0/conf/be.conf @@ -59,7 +59,7 @@ chunk_reserved_bytes_limit = 134217728 # sys_log_dir = ${DORIS_HOME}/log # sys_log_roll_mode = SIZE-MB-1024 # sys_log_roll_num = 10 -sys_log_verbose_modules = fragment_mgr,vunion_node,union_source_operator +# sys_log_verbose_modules = * log_buffer_level = -1 enable_stream_load_record = true # palo_cgroups From 24a009bb3df7ce899efc9b6c243a042115b0f739 Mon Sep 17 00:00:00 2001 From: abmdocrt Date: Wed, 17 Jan 2024 20:37:29 +0800 Subject: [PATCH 069/200] [Fix](typo) Fix group commit regression test typo (#30057) --- .../test_low_wal_disk_space_fault_injection.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/regression-test/suites/fault_injection_p0/test_low_wal_disk_space_fault_injection.groovy b/regression-test/suites/fault_injection_p0/test_low_wal_disk_space_fault_injection.groovy index 815bafe3d4b095..0b7cab481f70ce 100644 --- a/regression-test/suites/fault_injection_p0/test_low_wal_disk_space_fault_injection.groovy +++ b/regression-test/suites/fault_injection_p0/test_low_wal_disk_space_fault_injection.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_wal_mem_back_pressure_time_out_fault_injection","nonConcurrent") { +suite("test_low_wal_disk_space_fault_injection","nonConcurrent") { def tableName = "wal_test" From 747ac1cf1f561f784acbe4d3938c8afe7c394166 Mon Sep 17 00:00:00 2001 From: HappenLee Date: Wed, 17 Jan 2024 20:53:42 +0800 Subject: [PATCH 070/200] [Performance](Join) Support all match one logic (#30019) Support all match one logic --- .../vec/exec/join/process_hash_table_probe.h | 4 ++-- .../exec/join/process_hash_table_probe_impl.h | 24 ++++++++++++------- 2 files changed, 18 insertions(+), 10 deletions(-) diff --git a/be/src/vec/exec/join/process_hash_table_probe.h b/be/src/vec/exec/join/process_hash_table_probe.h index 295317517d6703..4b3140b7b594e4 100644 --- a/be/src/vec/exec/join/process_hash_table_probe.h +++ b/be/src/vec/exec/join/process_hash_table_probe.h @@ -47,8 +47,8 @@ struct ProcessHashTableProbe { int size, bool have_other_join_conjunct); void probe_side_output_column(MutableColumns& mcol, const std::vector& output_slot_flags, - int size, int last_probe_index, size_t probe_size, - bool all_match_one, bool have_other_join_conjunct); + int size, int last_probe_index, bool all_match_one, + bool have_other_join_conjunct); template Status process(HashTableType& hash_table_ctx, ConstNullMapPtr null_map, diff --git a/be/src/vec/exec/join/process_hash_table_probe_impl.h b/be/src/vec/exec/join/process_hash_table_probe_impl.h index a8ef9c06484ada..4f950f876cf0b5 100644 --- a/be/src/vec/exec/join/process_hash_table_probe_impl.h +++ b/be/src/vec/exec/join/process_hash_table_probe_impl.h @@ -97,15 +97,14 @@ void ProcessHashTableProbe::build_side_output_column( template void ProcessHashTableProbe::probe_side_output_column( MutableColumns& mcol, const std::vector& output_slot_flags, int size, - int last_probe_index, size_t probe_size, bool all_match_one, - bool have_other_join_conjunct) { + int last_probe_index, bool all_match_one, bool have_other_join_conjunct) { SCOPED_TIMER(_probe_side_output_timer); auto& probe_block = _parent->_probe_block; for (int i = 0; i < output_slot_flags.size(); ++i) { if (output_slot_flags[i]) { auto& column = probe_block.get_by_position(i).column; if (all_match_one) { - mcol[i]->insert_range_from(*column, last_probe_index, probe_size); + mcol[i]->insert_range_from(*column, last_probe_index, size); } else { column->replicate(_probe_indexs.data(), size, *mcol[i]); } @@ -168,8 +167,6 @@ Status ProcessHashTableProbe::do_process(HashTableType& hash auto& mcol = mutable_block.mutable_columns(); int current_offset = 0; - bool all_match_one = false; - size_t probe_size = 0; std::unique_ptr mark_column; if (is_mark_join) { @@ -188,16 +185,27 @@ Status ProcessHashTableProbe::do_process(HashTableType& hash probe_index = new_probe_idx; build_index = new_build_idx; current_offset = new_current_offset; - probe_size = probe_index - last_probe_index; } build_side_output_column(mcol, *_right_output_slot_flags, current_offset, with_other_conjuncts); if constexpr (with_other_conjuncts || (JoinOpType != TJoinOp::RIGHT_SEMI_JOIN && JoinOpType != TJoinOp::RIGHT_ANTI_JOIN)) { + auto check_all_match_one = [](const std::vector& vecs, uint32_t probe_idx, + int size) { + if (size < 1 || vecs[0] != probe_idx) return false; + for (int i = 1; i < size; i++) { + if (vecs[i] - vecs[i - 1] != 1) { + return false; + } + } + return true; + }; + RETURN_IF_CATCH_EXCEPTION(probe_side_output_column( - mcol, *_left_output_slot_flags, current_offset, last_probe_index, probe_size, - all_match_one, with_other_conjuncts)); + mcol, *_left_output_slot_flags, current_offset, last_probe_index, + check_all_match_one(_probe_indexs, last_probe_index, current_offset), + with_other_conjuncts)); } output_block->swap(mutable_block.to_block()); From 66c2c9680705888a38956c731278c8dfeb84ba3f Mon Sep 17 00:00:00 2001 From: Jibing-Li <64681310+Jibing-Li@users.noreply.github.com> Date: Wed, 17 Jan 2024 22:41:44 +0800 Subject: [PATCH 071/200] [feature](statistics, metadata)Meta data place holder for statistics (#29867) Meta data place holder for statistics in version 2.1.x. Users could upgrade to this version, but doesn't support rollback. After this change, statistics related functions doesn't need to change meta data any more in the 2.1 series. --- .../apache/doris/common/FeMetaVersion.java | 4 +- .../org/apache/doris/persist/EditLog.java | 22 +++++-- .../apache/doris/persist/OperationType.java | 6 ++ .../doris/statistics/AnalysisJobInfo.java | 54 +++++++++++++++ .../doris/statistics/AnalysisManager.java | 46 +++---------- .../doris/statistics/AnalysisTaskInfo.java | 58 +++++++++++++++++ .../statistics/NewPartitionLoadedEvent.java | 57 ++++++++++++++++ .../doris/statistics/UpdateRowsEvent.java | 61 +++++++++++++++++ .../doris/statistics/util/SimpleQueue.java | 65 ------------------- .../doris/statistics/AnalysisManagerTest.java | 24 ------- 10 files changed, 264 insertions(+), 133 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisJobInfo.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisTaskInfo.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/statistics/NewPartitionLoadedEvent.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/statistics/UpdateRowsEvent.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/statistics/util/SimpleQueue.java diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java b/fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java index 387ce91e2c68e2..9f5474fa01c76d 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java @@ -74,9 +74,11 @@ public final class FeMetaVersion { public static final int VERSION_126 = 126; // For constraints public static final int VERSION_127 = 127; + // For statistics. Update rows, new partition loaded, AnalysisJobInfo and AnalysisTaskInfo + public static final int VERSION_128 = 128; // note: when increment meta version, should assign the latest version to VERSION_CURRENT - public static final int VERSION_CURRENT = VERSION_127; + public static final int VERSION_CURRENT = VERSION_128; // all logs meta version should >= the minimum version, so that we could remove many if clause, for example // if (FE_METAVERSION < VERSION_94) ... diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index 5b2ff25f54e479..8c2424d48373ea 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -81,7 +81,9 @@ import org.apache.doris.resource.workloadgroup.WorkloadGroup; import org.apache.doris.resource.workloadschedpolicy.WorkloadSchedPolicy; import org.apache.doris.statistics.AnalysisInfo; +import org.apache.doris.statistics.AnalysisJobInfo; import org.apache.doris.statistics.AnalysisManager; +import org.apache.doris.statistics.AnalysisTaskInfo; import org.apache.doris.statistics.TableStatsMeta; import org.apache.doris.system.Backend; import org.apache.doris.system.Frontend; @@ -1080,6 +1082,10 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { break; } case OperationType.OP_CREATE_ANALYSIS_JOB: { + if (journal.getData() instanceof AnalysisJobInfo) { + // For rollback compatible. + break; + } AnalysisInfo info = (AnalysisInfo) journal.getData(); if (AnalysisManager.needAbandon(info)) { break; @@ -1088,6 +1094,10 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { break; } case OperationType.OP_CREATE_ANALYSIS_TASK: { + if (journal.getData() instanceof AnalysisTaskInfo) { + // For rollback compatible. + break; + } AnalysisInfo info = (AnalysisInfo) journal.getData(); if (AnalysisManager.needAbandon(info)) { break; @@ -1131,7 +1141,7 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { break; } case OperationType.OP_PERSIST_AUTO_JOB: { - env.getAnalysisManager().replayPersistSysJob((AnalysisInfo) journal.getData()); + // Do nothing break; } case OperationType.OP_DELETE_TABLE_STATS: { @@ -1155,6 +1165,12 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { env.getBackupHandler().getRepoMgr().alterRepo(repository, true); break; } + case OperationType.OP_LOG_UPDATE_ROWS: + case OperationType.OP_LOG_NEW_PARTITION_LOADED: + case OperationType.OP_LOG_ALTER_COLUMN_STATS: { + // TODO: implement this while statistics finished related work. + break; + } default: { IOException e = new IOException(); LOG.error("UNKNOWN Operation Type {}", opCode, e); @@ -1968,10 +1984,6 @@ public void logCreateTableStats(TableStatsMeta tableStats) { logEdit(OperationType.OP_UPDATE_TABLE_STATS, tableStats); } - public void logAutoJob(AnalysisInfo analysisInfo) { - logEdit(OperationType.OP_PERSIST_AUTO_JOB, analysisInfo); - } - public void logDeleteTableStats(TableStatsDeletionLog log) { logEdit(OperationType.OP_DELETE_TABLE_STATS, log); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java index 42312297b917d7..0945dc0f1510e3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java @@ -345,6 +345,7 @@ public class OperationType { public static final short OP_UPDATE_TABLE_STATS = 455; + @Deprecated public static final short OP_PERSIST_AUTO_JOB = 456; public static final short OP_DELETE_TABLE_STATS = 457; @@ -357,6 +358,11 @@ public class OperationType { public static final short OP_INSERT_OVERWRITE = 461; + public static final short OP_LOG_UPDATE_ROWS = 462; + + public static final short OP_LOG_NEW_PARTITION_LOADED = 463; + + public static final short OP_LOG_ALTER_COLUMN_STATS = 464; /** * Get opcode name by op code. diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisJobInfo.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisJobInfo.java new file mode 100644 index 00000000000000..57ae461fe13693 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisJobInfo.java @@ -0,0 +1,54 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.statistics; + +import org.apache.doris.common.io.Text; +import org.apache.doris.common.io.Writable; +import org.apache.doris.persist.gson.GsonUtils; + +import com.google.gson.annotations.SerializedName; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +public class AnalysisJobInfo implements Writable { + + private static final Logger LOG = LogManager.getLogger(AnalysisJobInfo.class); + + @SerializedName("jobId") + public final long jobId; + + public AnalysisJobInfo(long jobId) { + this.jobId = jobId; + } + + @Override + public void write(DataOutput out) throws IOException { + String json = GsonUtils.GSON.toJson(this); + Text.writeString(out, json); + } + + public static AnalysisJobInfo read(DataInput dataInput) throws IOException { + String json = Text.readString(dataInput); + AnalysisJobInfo analysisJobInfo = GsonUtils.GSON.fromJson(json, AnalysisJobInfo.class); + return analysisJobInfo; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java index 00e858c15369fb..2e12b4433900d2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java @@ -39,6 +39,7 @@ import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; import org.apache.doris.common.FeConstants; +import org.apache.doris.common.FeMetaVersion; import org.apache.doris.common.ThreadPoolManager.BlockedPolicy; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; @@ -56,7 +57,6 @@ import org.apache.doris.statistics.AnalysisInfo.JobType; import org.apache.doris.statistics.AnalysisInfo.ScheduleType; import org.apache.doris.statistics.util.DBObjects; -import org.apache.doris.statistics.util.SimpleQueue; import org.apache.doris.statistics.util.StatisticsUtil; import com.google.common.annotations.VisibleForTesting; @@ -122,9 +122,6 @@ public class AnalysisManager implements Writable { private final Map idToAnalysisJob = new ConcurrentHashMap<>(); - // To be deprecated, keep it for meta compatibility now, will remove later. - protected SimpleQueue autoJobs = createSimpleQueue(null, this); - private final String progressDisplayTemplate = "%d Finished | %d Failed | %d In Progress | %d Total"; public AnalysisManager() { @@ -862,13 +859,16 @@ public static AnalysisManager readFields(DataInput in) throws IOException { readAnalysisInfo(in, analysisManager.analysisJobInfoMap, true); readAnalysisInfo(in, analysisManager.analysisTaskInfoMap, false); readIdToTblStats(in, analysisManager.idToTblStats); - readAutoJobs(in, analysisManager); + if (Env.getCurrentEnvJournalVersion() < FeMetaVersion.VERSION_128) { + readAutoJobs(in, analysisManager); + } return analysisManager; } private static void readAnalysisInfo(DataInput in, Map map, boolean job) throws IOException { int size = in.readInt(); for (int i = 0; i < size; i++) { + // AnalysisInfo is compatible with AnalysisJobInfo and AnalysisTaskInfo. AnalysisInfo analysisInfo = AnalysisInfo.read(in); // Unfinished manual once job/tasks doesn't need to keep in memory anymore. if (needAbandon(analysisInfo)) { @@ -884,6 +884,9 @@ public static boolean needAbandon(AnalysisInfo analysisInfo) { if (analysisInfo == null) { return true; } + if (analysisInfo.scheduleType == null || analysisInfo.scheduleType == null || analysisInfo.jobType == null) { + return true; + } if ((AnalysisState.PENDING.equals(analysisInfo.state) || AnalysisState.RUNNING.equals(analysisInfo.state)) && ScheduleType.ONCE.equals(analysisInfo.scheduleType) && JobType.MANUAL.equals(analysisInfo.jobType)) { @@ -904,7 +907,6 @@ private static void readIdToTblStats(DataInput in, Map map private static void readAutoJobs(DataInput in, AnalysisManager analysisManager) throws IOException { Type type = new TypeToken>() {}.getType(); GsonUtils.GSON.fromJson(Text.readString(in), type); - analysisManager.autoJobs = analysisManager.createSimpleQueue(null, null); } @Override @@ -912,7 +914,6 @@ public void write(DataOutput out) throws IOException { writeJobInfo(out, analysisJobInfoMap); writeJobInfo(out, analysisTaskInfoMap); writeTableStats(out); - writeAutoJobsStatus(out); } private void writeJobInfo(DataOutput out, Map infoMap) throws IOException { @@ -929,12 +930,6 @@ private void writeTableStats(DataOutput out) throws IOException { } } - private void writeAutoJobsStatus(DataOutput output) throws IOException { - Type type = new TypeToken>() {}.getType(); - String autoJobs = GsonUtils.GSON.toJson(this.autoJobs, type); - Text.writeString(output, autoJobs); - } - // For unit test use only. public void addToJobIdTasksMap(long jobId, Map tasks) { analysisJobIdToTaskMap.put(jobId, tasks); @@ -980,31 +975,6 @@ public void registerSysJob(AnalysisInfo jobInfo, Map tas analysisJobIdToTaskMap.put(jobInfo.jobId, taskInfos); } - protected void logAutoJob(AnalysisInfo autoJob) { - Env.getCurrentEnv().getEditLog().logAutoJob(autoJob); - } - - public void replayPersistSysJob(AnalysisInfo analysisInfo) { - autoJobs.offer(analysisInfo); - } - - protected SimpleQueue createSimpleQueue(Collection collection, - AnalysisManager analysisManager) { - return new SimpleQueue<>(Config.analyze_record_limit, - a -> { - // FE is not ready when replaying log and operations triggered by replaying - // shouldn't be logged again. - if (Env.getCurrentEnv().isReady() && Env.getCurrentEnv().isMaster() && !Env.isCheckpointThread()) { - analysisManager.logAutoJob(a); - } - return null; - }, - a -> { - // DO NOTHING - return null; - }, collection); - } - // Remove col stats status from TableStats if failed load some col stats after analyze corresponding column so that // we could make sure it would be analyzed again soon if user or system submit job for that column again. public void removeColStatsStatus(long tblId, String colName) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisTaskInfo.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisTaskInfo.java new file mode 100644 index 00000000000000..fb7a5dcbd58cce --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisTaskInfo.java @@ -0,0 +1,58 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.statistics; + +import org.apache.doris.common.io.Text; +import org.apache.doris.common.io.Writable; +import org.apache.doris.persist.gson.GsonUtils; + +import com.google.gson.annotations.SerializedName; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +public class AnalysisTaskInfo implements Writable { + + private static final Logger LOG = LogManager.getLogger(AnalysisTaskInfo.class); + + @SerializedName("jobId") + public final long jobId; + + @SerializedName("taskId") + public final long taskId; + + public AnalysisTaskInfo(long jobId, long taskId) { + this.jobId = jobId; + this.taskId = taskId; + } + + @Override + public void write(DataOutput out) throws IOException { + String json = GsonUtils.GSON.toJson(this); + Text.writeString(out, json); + } + + public static AnalysisTaskInfo read(DataInput dataInput) throws IOException { + String json = Text.readString(dataInput); + AnalysisTaskInfo analysisTaskInfo = GsonUtils.GSON.fromJson(json, AnalysisTaskInfo.class); + return analysisTaskInfo; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/NewPartitionLoadedEvent.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/NewPartitionLoadedEvent.java new file mode 100644 index 00000000000000..d09cb2df6c423a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/NewPartitionLoadedEvent.java @@ -0,0 +1,57 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.statistics; + +import org.apache.doris.common.io.Text; +import org.apache.doris.common.io.Writable; +import org.apache.doris.persist.gson.GsonUtils; + +import com.google.common.annotations.VisibleForTesting; +import com.google.gson.annotations.SerializedName; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class NewPartitionLoadedEvent implements Writable { + + @SerializedName("partitionIdToTableId") + public final Map partitionIdToTableId = new HashMap<>(); + + @VisibleForTesting + public NewPartitionLoadedEvent() {} + + // No need to be thread safe, only publish thread will call this. + public void addPartition(long tableId, long partitionId) { + partitionIdToTableId.put(tableId, partitionId); + } + + @Override + public void write(DataOutput out) throws IOException { + String json = GsonUtils.GSON.toJson(this); + Text.writeString(out, json); + } + + public static NewPartitionLoadedEvent read(DataInput dataInput) throws IOException { + String json = Text.readString(dataInput); + NewPartitionLoadedEvent newPartitionLoadedEvent = GsonUtils.GSON.fromJson(json, NewPartitionLoadedEvent.class); + return newPartitionLoadedEvent; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/UpdateRowsEvent.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/UpdateRowsEvent.java new file mode 100644 index 00000000000000..04f185c8b73fc0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/UpdateRowsEvent.java @@ -0,0 +1,61 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.statistics; + +import org.apache.doris.common.io.Text; +import org.apache.doris.common.io.Writable; +import org.apache.doris.persist.gson.GsonUtils; + +import com.google.common.annotations.VisibleForTesting; +import com.google.gson.annotations.SerializedName; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class UpdateRowsEvent implements Writable { + + @SerializedName("tableIdToUpdateRows") + public final Map tableIdToUpdateRows = new HashMap<>(); + + @VisibleForTesting + public UpdateRowsEvent() {} + + // No need to be thread safe, only publish thread will call this. + public void addUpdateRows(long tableId, long rows) { + if (tableIdToUpdateRows.containsKey(tableId)) { + tableIdToUpdateRows.put(tableId, tableIdToUpdateRows.get(tableId) + rows); + } else { + tableIdToUpdateRows.put(tableId, rows); + } + } + + @Override + public void write(DataOutput out) throws IOException { + String json = GsonUtils.GSON.toJson(this); + Text.writeString(out, json); + } + + public static UpdateRowsEvent read(DataInput dataInput) throws IOException { + String json = Text.readString(dataInput); + UpdateRowsEvent updateRowsEvent = GsonUtils.GSON.fromJson(json, UpdateRowsEvent.class); + return updateRowsEvent; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/util/SimpleQueue.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/util/SimpleQueue.java deleted file mode 100644 index 5740c4e30885a3..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/util/SimpleQueue.java +++ /dev/null @@ -1,65 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you 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 org.apache.doris.statistics.util; - -import java.util.Collection; -import java.util.LinkedList; -import java.util.function.Function; - -// Any operation on this structure should be thread-safe -public class SimpleQueue extends LinkedList { - - private final long limit; - - private final Function offerFunc; - - private final Function evictFunc; - - - public SimpleQueue(long limit, Function offerFunc, Function evictFunc) { - this.limit = limit; - this.offerFunc = offerFunc; - this.evictFunc = evictFunc; - } - - @Override - public synchronized boolean offer(T analysisInfo) { - while (size() >= limit) { - remove(); - } - super.offer(analysisInfo); - offerFunc.apply(analysisInfo); - return true; - } - - @Override - public synchronized T remove() { - T analysisInfo = super.remove(); - evictFunc.apply(analysisInfo); - return analysisInfo; - } - - public SimpleQueue(long limit, Function offerFunc, Function evictFunc, Collection collection) { - this(limit, offerFunc, evictFunc); - if (collection != null) { - for (T e : collection) { - offer(e); - } - } - } -} diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisManagerTest.java index 69f6dc0dd5dc4e..2fc6d24e305b30 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisManagerTest.java @@ -30,7 +30,6 @@ import org.apache.doris.statistics.AnalysisInfo.AnalysisType; import org.apache.doris.statistics.AnalysisInfo.JobType; import org.apache.doris.statistics.AnalysisInfo.ScheduleType; -import org.apache.doris.statistics.util.SimpleQueue; import org.apache.doris.statistics.util.StatisticsUtil; import com.google.common.annotations.VisibleForTesting; @@ -331,28 +330,6 @@ public void testRecordLimit2() { Assertions.assertTrue(analysisManager.analysisTaskInfoMap.containsKey(3L)); } - @Test - public void testRecordLimit3() { - Config.analyze_record_limit = 2; - AnalysisManager analysisManager = new AnalysisManager(); - analysisManager.autoJobs.offer(new AnalysisInfoBuilder().setJobId(1).build()); - analysisManager.autoJobs.offer(new AnalysisInfoBuilder().setJobId(2).build()); - analysisManager.autoJobs.offer(new AnalysisInfoBuilder().setJobId(3).build()); - Assertions.assertEquals(2, analysisManager.autoJobs.size()); - } - - @Test - public void testCreateSimpleQueue() { - AnalysisManager analysisManager = new AnalysisManager(); - ArrayList jobs = Lists.newArrayList(); - jobs.add(new AnalysisInfoBuilder().setJobId(1).build()); - jobs.add(new AnalysisInfoBuilder().setJobId(2).build()); - SimpleQueue simpleQueue = analysisManager.createSimpleQueue(jobs, analysisManager); - Assertions.assertEquals(2, simpleQueue.size()); - simpleQueue = analysisManager.createSimpleQueue(null, analysisManager); - Assertions.assertEquals(0, simpleQueue.size()); - } - @Test public void testShowAutoJobs(@Injectable ShowAnalyzeStmt stmt) { new MockUp() { @@ -404,5 +381,4 @@ public void testShowAutoTasks(@Injectable ShowAnalyzeStmt stmt) { Assertions.assertEquals(AnalysisState.FINISHED, analysisInfos.get(1).getState()); Assertions.assertEquals(AnalysisState.FAILED, analysisInfos.get(2).getState()); } - } From 6d1bb7028b1592af6d470b8f5b67a82c032b93b3 Mon Sep 17 00:00:00 2001 From: Nitin-Kashyap <66766227+Nitin-Kashyap@users.noreply.github.com> Date: Wed, 17 Jan 2024 21:07:25 +0530 Subject: [PATCH 072/200] [fix](ut) fixed test cases failure for showCreateFunction (#28593) 2 tests cases (ShowCreateFunctionTest and ShowFunctionTest) have function name clash; because of which one of the test fails to complete setup() and eventually fail for not able to create funciton. Changed the name of the test function in 1 of the file. --- .../doris/analysis/ShowCreateFunctionTest.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateFunctionTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateFunctionTest.java index f89f6fdbf35bcf..30256d394deb47 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateFunctionTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateFunctionTest.java @@ -34,28 +34,28 @@ protected void runBeforeAll() throws Exception { createDatabase(dbName); useDatabase(dbName); createFunction( - "CREATE ALIAS FUNCTION id_masking(bigint) WITH PARAMETER(id) AS CONCAT(LEFT(id,3),'****',RIGHT(id,4));"); + "CREATE ALIAS FUNCTION id_masking_create(bigint) WITH PARAMETER(id) AS CONCAT(LEFT(id,3),'****',RIGHT(id,4));"); createFunction( - "CREATE GLOBAL ALIAS FUNCTION id_masking_global(bigint) WITH PARAMETER(id) AS CONCAT(LEFT(id,3),'****',RIGHT(id,4));"); + "CREATE GLOBAL ALIAS FUNCTION id_masking_global_create(bigint) WITH PARAMETER(id) AS CONCAT(LEFT(id,3),'****',RIGHT(id,4));"); } @Test public void testNormal() throws Exception { - String sql = "SHOW CREATE FUNCTION id_masking(bigint)"; + String sql = "SHOW CREATE FUNCTION id_masking_create(bigint)"; ShowResultSet showResultSet = showCreateFunction(sql); String showSql = showResultSet.getResultRows().get(0).get(1); - Assertions.assertTrue(showSql.contains("CREATE ALIAS FUNCTION id_masking(BIGINT) WITH PARAMETER(id)")); + Assertions.assertTrue(showSql.contains("CREATE ALIAS FUNCTION id_masking_create(BIGINT) WITH PARAMETER(id)")); } @Test public void testShowCreateGlobalFunction() throws Exception { - String sql = "SHOW CREATE GLOBAL FUNCTION id_masking_global(bigint)"; + String sql = "SHOW CREATE GLOBAL FUNCTION id_masking_global_create(bigint)"; ShowResultSet showResultSet = showCreateFunction(sql); String showSql = showResultSet.getResultRows().get(0).get(1); Assertions.assertTrue( - showSql.contains("CREATE GLOBAL ALIAS FUNCTION id_masking_global(BIGINT) WITH PARAMETER(id)")); + showSql.contains("CREATE GLOBAL ALIAS FUNCTION id_masking_global_create(BIGINT) WITH PARAMETER(id)")); } } From 5a330a179d2fbd644de9c0ab9f2448f078f17fdc Mon Sep 17 00:00:00 2001 From: Xin Liao Date: Thu, 18 Jan 2024 08:12:01 +0800 Subject: [PATCH 073/200] [fix](regression) fault injection may cause fd to be closed twice (#30070) --- be/src/io/fs/local_file_writer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/be/src/io/fs/local_file_writer.cpp b/be/src/io/fs/local_file_writer.cpp index 4c35101584c560..78ea87cf3f27cf 100644 --- a/be/src/io/fs/local_file_writer.cpp +++ b/be/src/io/fs/local_file_writer.cpp @@ -200,6 +200,7 @@ Status LocalFileWriter::_close(bool sync) { if (0 != ::close(_fd)) { return localfs_error(errno, fmt::format("failed to close {}", _path.native())); } + _closed = true; DBUG_EXECUTE_IF("LocalFileWriter.close.failed", { // spare '.testfile' to make bad disk checker happy @@ -208,7 +209,6 @@ Status LocalFileWriter::_close(bool sync) { } }); - _closed = true; return Status::OK(); } From 183ca1f60ba311a943c945ca730b2624746c3a40 Mon Sep 17 00:00:00 2001 From: Mingyu Chen Date: Thu, 18 Jan 2024 08:54:22 +0800 Subject: [PATCH 074/200] [fix](compatibility) remove cluster prefix in user property info (#30078) --- .../org/apache/doris/mysql/privilege/UserPropertyInfo.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/UserPropertyInfo.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/UserPropertyInfo.java index 647ac40fcef3bf..e7ee45174833b4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/UserPropertyInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/UserPropertyInfo.java @@ -17,6 +17,7 @@ package org.apache.doris.mysql.privilege; +import org.apache.doris.cluster.ClusterNamespace; import org.apache.doris.common.Pair; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; @@ -67,7 +68,7 @@ public void write(DataOutput out) throws IOException { } public void readFields(DataInput in) throws IOException { - user = Text.readString(in); + user = ClusterNamespace.getNameFromFullName(Text.readString(in)); int size = in.readInt(); for (int i = 0; i < size; i++) { String key = Text.readString(in); From 74ce980987df0b3c15305ff057b9c7354f0ab2df Mon Sep 17 00:00:00 2001 From: yiguolei <676222867@qq.com> Date: Thu, 18 Jan 2024 10:00:53 +0800 Subject: [PATCH 075/200] [improvement](fe) skip attach profile during send fragment (#30085) Sometime attach profile may cost a lot of time in large machine with many instances. --------- Co-authored-by: yiguolei --- .../main/java/org/apache/doris/qe/Coordinator.java | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index 2f58e560740ffc..586a27ee1033d1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -233,7 +233,6 @@ public class Coordinator implements CoordInterface { // Input parameter private long jobId = -1; // job which this task belongs to private TUniqueId queryId; - private final boolean needReport; // parallel execute private final TUniqueId nextInstanceId; @@ -324,7 +323,6 @@ public Coordinator(ConnectContext context, Analyzer analyzer, Planner planner) { } else { this.queryGlobals.setTimeZone(context.getSessionVariable().getTimeZone()); } - this.needReport = context.getSessionVariable().enableProfile(); this.nextInstanceId = new TUniqueId(); nextInstanceId.setHi(queryId.hi); nextInstanceId.setLo(queryId.lo + 1); @@ -348,7 +346,6 @@ public Coordinator(Long jobId, TUniqueId queryId, DescriptorTable descTable, Lis this.queryGlobals.setTimeZone(timezone); this.queryGlobals.setLoadZeroTolerance(loadZeroTolerance); this.queryOptions.setBeExecVersion(Config.be_exec_version); - this.needReport = true; this.nextInstanceId = new TUniqueId(); nextInstanceId.setHi(queryId.hi); nextInstanceId.setLo(queryId.lo + 1); @@ -845,8 +842,9 @@ private void sendFragment() throws TException, RpcException, UserException { } waitRpc(futures, this.timeoutDeadline - System.currentTimeMillis(), "send execution start"); } - - attachInstanceProfileToFragmentProfile(); + if (context != null && context.getSessionVariable().enableProfile()) { + attachInstanceProfileToFragmentProfile(); + } } finally { unlock(); } @@ -988,8 +986,9 @@ private void sendPipelineCtx() throws TException, RpcException, UserException { } waitPipelineRpc(futures, this.timeoutDeadline - System.currentTimeMillis(), "send execution start"); } - - attachInstanceProfileToFragmentProfile(); + if (context != null && context.getSessionVariable().enableProfile()) { + attachInstanceProfileToFragmentProfile(); + } } finally { unlock(); } From 901c863623ed995ac277df90d20f8483c0a2165f Mon Sep 17 00:00:00 2001 From: Luwei <814383175@qq.com> Date: Thu, 18 Jan 2024 10:04:55 +0800 Subject: [PATCH 076/200] [Fix](schema change) fix tablet meta write race (#30081) --- .../doris/service/FrontendServiceImpl.java | 24 +++++++++++++------ 1 file changed, 17 insertions(+), 7 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index 280cf5ebf77c93..a9ae485d8f9902 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -2002,14 +2002,24 @@ private TExecPlanFragmentParams streamLoadPutImpl(TStreamLoadPutRequest request, } long timeoutMs = request.isSetThriftRpcTimeoutMs() ? request.getThriftRpcTimeoutMs() : 5000; Table table = db.getTableOrMetaException(request.getTbl(), TableType.OLAP); - if (!((OlapTable) table).getTableProperty().getUseSchemaLightChange() && (request.getGroupCommitMode() != null - && !request.getGroupCommitMode().equals("off_mode"))) { - throw new UserException("table light_schema_change is false, can't do stream load with group commit mode"); + if (!table.tryReadLock(timeoutMs, TimeUnit.MILLISECONDS)) { + throw new UserException( + "get table read lock timeout, database=" + fullDbName + ",table=" + table.getName()); + } + try { + if (!((OlapTable) table).getTableProperty().getUseSchemaLightChange() + && (request.getGroupCommitMode() != null + && !request.getGroupCommitMode().equals("off_mode"))) { + throw new UserException( + "table light_schema_change is false, can't do stream load with group commit mode"); + } + result.setDbId(db.getId()); + result.setTableId(table.getId()); + result.setBaseSchemaVersion(((OlapTable) table).getBaseSchemaVersion()); + return generatePlanFragmentParams(request, db, fullDbName, (OlapTable) table, timeoutMs); + } finally { + table.readUnlock(); } - result.setDbId(db.getId()); - result.setTableId(table.getId()); - result.setBaseSchemaVersion(((OlapTable) table).getBaseSchemaVersion()); - return generatePlanFragmentParams(request, db, fullDbName, (OlapTable) table, timeoutMs); } private TExecPlanFragmentParams generatePlanFragmentParams(TStreamLoadPutRequest request, Database db, From 4f74b2aac86d80dc22df56d55a643a9e12a278d1 Mon Sep 17 00:00:00 2001 From: zy-kkk Date: Thu, 18 Jan 2024 10:11:38 +0800 Subject: [PATCH 077/200] [Enhancement](Jdbc Catalog) Map Jdbc Catalog JSON Type to String for Improved Performance and Compatibility (#30035) This PR proposes mapping external catalog JSON types to String instead of JsonB in Apache Doris. This change is motivated by the realization that JDBC retrieves JSON data as a String JSON string, regardless of its storage format (Json(String) or Json(Binary)). Mapping to String streamlines data retrieval, simplifies write-backs, and ensures compatibility with all JSON(String) and JSON(Binary) functions, despite potentially misleading displays of JSON data as Strings in Doris. This approach avoids the performance overhead and complexity of converting each row of data from JsonB to String, making the process more efficient and elegant. About Upgrade To ensure query compatibility with existing Catalogs in the upgraded version,we currently still retain the capability to query external JSON types as JSONB. However, once you upgrade to the new version and either refresh the Catalog or create a new one, all external JSON types will be treated as Strings. To ensure consistent behavior,and possible future removal of support for JSON as JSONB query code, it is highly recommended that you manually refresh your Catalog as soon as possible after upgrading to the new version. --- be/src/vec/exec/vjdbc_connector.cpp | 1 + docs/en/docs/lakehouse/multi-catalog/jdbc.md | 110 +- .../docs/lakehouse/multi-catalog/jdbc.md | 110 +- .../org/apache/doris/jdbc/JdbcExecutor.java | 1 + .../jdbc/client/JdbcMySQLClient.java | 4 +- .../jdbc/client/JdbcPostgreSQLClient.java | 3 +- .../jdbc/test_doris_jdbc_catalog.out | 2 +- .../jdbc/test_mysql_jdbc_catalog.out | 14 +- .../jdbc/test_pg_jdbc_catalog.out | 2122 ++++++++--------- 9 files changed, 1183 insertions(+), 1184 deletions(-) diff --git a/be/src/vec/exec/vjdbc_connector.cpp b/be/src/vec/exec/vjdbc_connector.cpp index c80d84f98be8b6..f20df0af98f8ec 100644 --- a/be/src/vec/exec/vjdbc_connector.cpp +++ b/be/src/vec/exec/vjdbc_connector.cpp @@ -706,6 +706,7 @@ Status JdbcConnector::_cast_string_to_bitmap(const SlotDescriptor* slot_desc, Bl return Status::OK(); } +// Deprecated, this code is retained only for compatibility with query problems that may be encountered when upgrading the version that maps JSON to JSONB to this version, and will be deleted in subsequent versions. Status JdbcConnector::_cast_string_to_json(const SlotDescriptor* slot_desc, Block* block, int column_index, int rows) { DataTypePtr _target_data_type = slot_desc->get_data_type_ptr(); diff --git a/docs/en/docs/lakehouse/multi-catalog/jdbc.md b/docs/en/docs/lakehouse/multi-catalog/jdbc.md index 1a401146df9034..dc45c247d54ab0 100644 --- a/docs/en/docs/lakehouse/multi-catalog/jdbc.md +++ b/docs/en/docs/lakehouse/multi-catalog/jdbc.md @@ -303,37 +303,37 @@ CREATE CATALOG jdbc_mysql PROPERTIES ( #### Type Mapping -| MYSQL Type | Doris Type | Comment | -|-------------------------------------------|----------------|-------------------------------------------------------------------------------| -| BOOLEAN | TINYINT | | -| TINYINT | TINYINT | | -| SMALLINT | SMALLINT | | -| MEDIUMINT | INT | | -| INT | INT | | -| BIGINT | BIGINT | | -| UNSIGNED TINYINT | SMALLINT | Doris does not have an UNSIGNED data type, so expand by an order of magnitude | -| UNSIGNED MEDIUMINT | INT | Doris does not have an UNSIGNED data type, so expand by an order of magnitude | -| UNSIGNED INT | BIGINT | Doris does not have an UNSIGNED data type, so expand by an order of magnitude | -| UNSIGNED BIGINT | LARGEINT | | -| FLOAT | FLOAT | | -| DOUBLE | DOUBLE | | -| DECIMAL | DECIMAL | | -| UNSIGNED DECIMAL(p,s) | DECIMAL(p+1,s) / STRING | If p+1>38, the Doris STRING type will be used. | -| DATE | DATE | | -| TIMESTAMP | DATETIME | | -| DATETIME | DATETIME | | -| YEAR | SMALLINT | | -| TIME | STRING | | -| CHAR | CHAR | | -| VARCHAR | VARCHAR | | -| JSON | JSON | | -| SET | STRING | | -| BIT | BOOLEAN/STRING | BIT(1) will be mapped to BOOLEAN, and other BITs will be mapped to STRING | -| TINYTEXT、TEXT、MEDIUMTEXT、LONGTEXT | STRING | | -| BLOB、MEDIUMBLOB、LONGBLOB、TINYBLOB | STRING | | -| TINYSTRING、STRING、MEDIUMSTRING、LONGSTRING | STRING | | -| BINARY、VARBINARY | STRING | | -| Other | UNSUPPORTED | | +| MYSQL Type | Doris Type | Comment | +|-------------------------------------------|-------------------------|----------------------------------------------------------------------------------------| +| BOOLEAN | TINYINT | | +| TINYINT | TINYINT | | +| SMALLINT | SMALLINT | | +| MEDIUMINT | INT | | +| INT | INT | | +| BIGINT | BIGINT | | +| UNSIGNED TINYINT | SMALLINT | Doris does not have an UNSIGNED data type, so expand by an order of magnitude | +| UNSIGNED MEDIUMINT | INT | Doris does not have an UNSIGNED data type, so expand by an order of magnitude | +| UNSIGNED INT | BIGINT | Doris does not have an UNSIGNED data type, so expand by an order of magnitude | +| UNSIGNED BIGINT | LARGEINT | | +| FLOAT | FLOAT | | +| DOUBLE | DOUBLE | | +| DECIMAL | DECIMAL | | +| UNSIGNED DECIMAL(p,s) | DECIMAL(p+1,s) / STRING | If p+1>38, the Doris STRING type will be used. | +| DATE | DATE | | +| TIMESTAMP | DATETIME | | +| DATETIME | DATETIME | | +| YEAR | SMALLINT | | +| TIME | STRING | | +| CHAR | CHAR | | +| VARCHAR | VARCHAR | | +| JSON | STRING | For better performance, map JSON from external data sources to STRING instead of JSONB | +| SET | STRING | | +| BIT | BOOLEAN/STRING | BIT(1) will be mapped to BOOLEAN, and other BITs will be mapped to STRING | +| TINYTEXT、TEXT、MEDIUMTEXT、LONGTEXT | STRING | | +| BLOB、MEDIUMBLOB、LONGBLOB、TINYBLOB | STRING | | +| TINYSTRING、STRING、MEDIUMSTRING、LONGSTRING | STRING | | +| BINARY、VARBINARY | STRING | | +| Other | UNSUPPORTED | | ### PostgreSQL @@ -366,30 +366,30 @@ Doris obtains all schemas that PG user can access through the SQL statement: `se #### Type Mapping - | POSTGRESQL Type | Doris Type | Comment | - |-----------------------------------------|----------------|-------------------------------------------| - | boolean | BOOLEAN | | - | smallint/int2 | SMALLINT | | - | integer/int4 | INT | | - | bigint/int8 | BIGINT | | - | decimal/numeric | DECIMAL | | - | real/float4 | FLOAT | | - | double precision | DOUBLE | | - | smallserial | SMALLINT | | - | serial | INT | | - | bigserial | BIGINT | | - | char | CHAR | | - | varchar/text | STRING | | - | timestamp | DATETIME | | - | date | DATE | | - | json/josnb | JSON | | - | time | STRING | | - | interval | STRING | | - | point/line/lseg/box/path/polygon/circle | STRING | | - | cidr/inet/macaddr | STRING | | - | bit | BOOLEAN/STRING | bit(1) will be mapped to BOOLEAN, and other bits will be mapped to STRING | - | uuid | STRING | | - | Other | UNSUPPORTED | | + | POSTGRESQL Type | Doris Type | Comment | + |-----------------------------------------|----------------|----------------------------------------------------------------------------------------| + | boolean | BOOLEAN | | + | smallint/int2 | SMALLINT | | + | integer/int4 | INT | | + | bigint/int8 | BIGINT | | + | decimal/numeric | DECIMAL | | + | real/float4 | FLOAT | | + | double precision | DOUBLE | | + | smallserial | SMALLINT | | + | serial | INT | | + | bigserial | BIGINT | | + | char | CHAR | | + | varchar/text | STRING | | + | timestamp | DATETIME | | + | date | DATE | | + | json/jsonb | STRING | For better performance, map JSON from external data sources to STRING instead of JSONB | + | time | STRING | | + | interval | STRING | | + | point/line/lseg/box/path/polygon/circle | STRING | | + | cidr/inet/macaddr | STRING | | + | bit | BOOLEAN/STRING | bit(1) will be mapped to BOOLEAN, and other bits will be mapped to STRING | + | uuid | STRING | | + | Other | UNSUPPORTED | | ### Oracle diff --git a/docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md b/docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md index a86ab26c33dd90..52aece0db2b5c5 100644 --- a/docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md +++ b/docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md @@ -303,37 +303,37 @@ CALL EXECUTE_STMT(jdbc_catalog", "create table dbl1.tbl2 (k1 int)"); #### 类型映射 -| MYSQL Type | Doris Type | Comment | -|-------------------------------------------|----------------|-------------------------------------------------| -| BOOLEAN | TINYINT | | -| TINYINT | TINYINT | | -| SMALLINT | SMALLINT | | -| MEDIUMINT | INT | | -| INT | INT | | -| BIGINT | BIGINT | | -| UNSIGNED TINYINT | SMALLINT | Doris 没有 UNSIGNED 数据类型,所以扩大一个数量级 | -| UNSIGNED MEDIUMINT | INT | Doris 没有 UNSIGNED 数据类型,所以扩大一个数量级 | -| UNSIGNED INT | BIGINT | Doris 没有 UNSIGNED 数据类型,所以扩大一个数量级 | -| UNSIGNED BIGINT | LARGEINT | | -| FLOAT | FLOAT | | -| DOUBLE | DOUBLE | | -| DECIMAL | DECIMAL | | -| UNSIGNED DECIMAL(p,s) | DECIMAL(p+1,s) / STRING | 如果p+1>38, 将使用Doris STRING类型 | -| DATE | DATE | | -| TIMESTAMP | DATETIME | | -| DATETIME | DATETIME | | -| YEAR | SMALLINT | | -| TIME | STRING | | -| CHAR | CHAR | | -| VARCHAR | VARCHAR | | -| JSON | JSON | | -| SET | STRING | | -| BIT | BOOLEAN/STRING | BIT(1) 会映射为 BOOLEAN,其他 BIT 映射为 STRING | -| TINYTEXT、TEXT、MEDIUMTEXT、LONGTEXT | STRING | | -| BLOB、MEDIUMBLOB、LONGBLOB、TINYBLOB | STRING | | -| TINYSTRING、STRING、MEDIUMSTRING、LONGSTRING | STRING | | -| BINARY、VARBINARY | STRING | | -| Other | UNSUPPORTED | | +| MYSQL Type | Doris Type | Comment | +|-------------------------------------------|-------------------------|-------------------------------------------------------| +| BOOLEAN | TINYINT | | +| TINYINT | TINYINT | | +| SMALLINT | SMALLINT | | +| MEDIUMINT | INT | | +| INT | INT | | +| BIGINT | BIGINT | | +| UNSIGNED TINYINT | SMALLINT | Doris 没有 UNSIGNED 数据类型,所以扩大一个数量级 | +| UNSIGNED MEDIUMINT | INT | Doris 没有 UNSIGNED 数据类型,所以扩大一个数量级 | +| UNSIGNED INT | BIGINT | Doris 没有 UNSIGNED 数据类型,所以扩大一个数量级 | +| UNSIGNED BIGINT | LARGEINT | | +| FLOAT | FLOAT | | +| DOUBLE | DOUBLE | | +| DECIMAL | DECIMAL | | +| UNSIGNED DECIMAL(p,s) | DECIMAL(p+1,s) / STRING | 如果p+1>38, 将使用Doris STRING类型 | +| DATE | DATE | | +| TIMESTAMP | DATETIME | | +| DATETIME | DATETIME | | +| YEAR | SMALLINT | | +| TIME | STRING | | +| CHAR | CHAR | | +| VARCHAR | VARCHAR | | +| JSON | STRING | 为了更好的性能,将外部数据源的 JSON 映射为 STRING 而不是JSONB | +| SET | STRING | | +| BIT | BOOLEAN/STRING | BIT(1) 会映射为 BOOLEAN,其他 BIT 映射为 STRING | +| TINYTEXT、TEXT、MEDIUMTEXT、LONGTEXT | STRING | | +| BLOB、MEDIUMBLOB、LONGBLOB、TINYBLOB | STRING | | +| TINYSTRING、STRING、MEDIUMSTRING、LONGSTRING | STRING | | +| BINARY、VARBINARY | STRING | | +| Other | UNSUPPORTED | | ### PostgreSQL @@ -366,30 +366,30 @@ Doris 通过sql 语句 `select nspname from pg_namespace where has_schema_privil #### 类型映射 - | POSTGRESQL Type | Doris Type | Comment | - |-----------------------------------------|----------------|-----------------------------------------------| - | boolean | BOOLEAN | | - | smallint/int2 | SMALLINT | | - | integer/int4 | INT | | - | bigint/int8 | BIGINT | | - | decimal/numeric | DECIMAL | | - | real/float4 | FLOAT | | - | double precision | DOUBLE | | - | smallserial | SMALLINT | | - | serial | INT | | - | bigserial | BIGINT | | - | char | CHAR | | - | varchar/text | STRING | | - | timestamp | DATETIME | | - | date | DATE | | - | json/josnb | JSON | | - | time | STRING | | - | interval | STRING | | - | point/line/lseg/box/path/polygon/circle | STRING | | - | cidr/inet/macaddr | STRING | | - | bit | BOOLEAN/STRING | bit(1)会映射为 BOOLEAN,其他 bit 映射为 STRING | - | uuid | STRING | | - | Other | UNSUPPORTED | | + | POSTGRESQL Type | Doris Type | Comment | + |-----------------------------------------|----------------|-----------------------------------------------------| + | boolean | BOOLEAN | | + | smallint/int2 | SMALLINT | | + | integer/int4 | INT | | + | bigint/int8 | BIGINT | | + | decimal/numeric | DECIMAL | | + | real/float4 | FLOAT | | + | double precision | DOUBLE | | + | smallserial | SMALLINT | | + | serial | INT | | + | bigserial | BIGINT | | + | char | CHAR | | + | varchar/text | STRING | | + | timestamp | DATETIME | | + | date | DATE | | + | json/jsonb | STRING | 为了更好的性能,将外部数据源的 JSON 映射为 STRING 而不是JSONB| + | time | STRING | | + | interval | STRING | | + | point/line/lseg/box/path/polygon/circle | STRING | | + | cidr/inet/macaddr | STRING | | + | bit | BOOLEAN/STRING | bit(1)会映射为 BOOLEAN,其他 bit 映射为 STRING | + | uuid | STRING | | + | Other | UNSUPPORTED | | ### Oracle diff --git a/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/JdbcExecutor.java b/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/JdbcExecutor.java index ad34dd0035fad9..1129be2783321c 100644 --- a/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/JdbcExecutor.java +++ b/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/JdbcExecutor.java @@ -128,6 +128,7 @@ public void close() throws Exception { resultSet.close(); } if (stmt != null) { + stmt.cancel(); stmt.close(); } if (conn != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcMySQLClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcMySQLClient.java index 231af739e37a7c..c86d3aa2e65ea8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcMySQLClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcMySQLClient.java @@ -312,7 +312,6 @@ protected Type jdbcTypeToDoris(JdbcFieldSchema fieldSchema) { return ScalarType.createStringType(); } case "JSON": - return ScalarType.createJsonbType(); case "TIME": case "TINYTEXT": case "TEXT": @@ -430,9 +429,8 @@ private Type dorisTypeToDoris(String type) { } case "STRING": case "TEXT": - return ScalarType.createStringType(); case "JSON": - return ScalarType.createJsonbType(); + return ScalarType.createStringType(); case "HLL": return ScalarType.createHllType(); case "BITMAP": diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcPostgreSQLClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcPostgreSQLClient.java index 7fe1803ad564e1..c8f2751d7a1c41 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcPostgreSQLClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcPostgreSQLClient.java @@ -101,10 +101,9 @@ protected Type jdbcTypeToDoris(JdbcFieldSchema fieldSchema) { case "varbit": case "uuid": case "bytea": - return ScalarType.createStringType(); case "json": case "jsonb": - return ScalarType.createJsonbType(); + return ScalarType.createStringType(); default: return Type.UNSUPPORTED; } diff --git a/regression-test/data/external_table_p0/jdbc/test_doris_jdbc_catalog.out b/regression-test/data/external_table_p0/jdbc/test_doris_jdbc_catalog.out index dc85e705e53159..25793a482d59a0 100644 --- a/regression-test/data/external_table_p0/jdbc/test_doris_jdbc_catalog.out +++ b/regression-test/data/external_table_p0/jdbc/test_doris_jdbc_catalog.out @@ -92,7 +92,7 @@ date_col DATE Yes false \N NONE datetime_col DATETIME(3) Yes false \N NONE char_col CHAR(10) Yes false \N NONE varchar_col VARCHAR(10) Yes false \N NONE -json_col JSON Yes false \N NONE +json_col TEXT Yes false \N NONE -- !desc_ctas_arr -- int_col INT Yes true \N diff --git a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out index 96710f09e36aa7..bff117916a4cca 100644 --- a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out +++ b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out @@ -292,9 +292,9 @@ information_schema -- !mysql_all_types -- \N 302 \N 502 602 4.14159 \N 6.14159 \N -124 -302 2013 -402 -502 -602 \N 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 \N -7.1400 row2 \N 09:11:09.567 text2 0xE86F6C6C6F20576F726C67 \N \N 0x2F \N 0x88656C6C9F Value3 -201 301 401 501 601 3.14159 4.1415926 5.14159 1 -123 -301 2012 -401 -501 -601 2012-10-30 2012-10-25T12:05:36.345700 2012-10-25T08:08:08 -4.14145 -5.1400000001 -6.1400 row1 line1 09:09:09.567 text1 0x48656C6C6F20576F726C64 {"age":30,"city":"London","name":"Alice"} Option1,Option3 0x2A 0x48656C6C6F00000000000000 0x48656C6C6F Value2 -202 302 402 502 602 4.14159 5.1415926 6.14159 0 -124 -302 2013 -402 -502 -602 2012-11-01 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 -6.1400000001 -7.1400 row2 line2 09:11:09.567 text2 0xE86F6C6C6F20576F726C67 {"age":18,"city":"ChongQing","name":"Gaoxin"} Option1,Option2 0x2F 0x58676C6C6F00000000000000 0x88656C6C9F Value3 -203 303 403 503 603 7.14159 8.1415926 9.14159 0 \N -402 2017 -602 -902 -1102 2012-11-02 \N 2013-10-27T08:11:18 -5.14145 -6.1400000000001 -7.1400 row3 line3 09:11:09.567 text3 0xE86F6C6C6F20576F726C67 {"age":24,"city":"ChongQing","name":"ChenQi"} Option2 0x2F 0x58676C6C6F00000000000000 \N Value1 +201 301 401 501 601 3.14159 4.1415926 5.14159 1 -123 -301 2012 -401 -501 -601 2012-10-30 2012-10-25T12:05:36.345700 2012-10-25T08:08:08 -4.14145 -5.1400000001 -6.1400 row1 line1 09:09:09.567 text1 0x48656C6C6F20576F726C64 {"age": 30, "city": "London", "name": "Alice"} Option1,Option3 0x2A 0x48656C6C6F00000000000000 0x48656C6C6F Value2 +202 302 402 502 602 4.14159 5.1415926 6.14159 0 -124 -302 2013 -402 -502 -602 2012-11-01 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 -6.1400000001 -7.1400 row2 line2 09:11:09.567 text2 0xE86F6C6C6F20576F726C67 {"age": 18, "city": "ChongQing", "name": "Gaoxin"} Option1,Option2 0x2F 0x58676C6C6F00000000000000 0x88656C6C9F Value3 +203 303 403 503 603 7.14159 8.1415926 9.14159 0 \N -402 2017 -602 -902 -1102 2012-11-02 \N 2013-10-27T08:11:18 -5.14145 -6.1400000000001 -7.1400 row3 line3 09:11:09.567 text3 0xE86F6C6C6F20576F726C67 {"age": 24, "city": "ChongQing", "name": "ChenQi"} Option2 0x2F 0x58676C6C6F00000000000000 \N Value1 -- !select_insert_all_types -- \N 302 \N 502 602 4.14159 \N 6.14159 \N -124 -302 2013 -402 -502 -602 \N 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 \N -7.1400 row2 \N 09:11:09.567 text2 0xE86F6C6C6F20576F726C67 \N \N 0x2F \N 0x88656C6C9F Value3 @@ -304,9 +304,9 @@ information_schema -- !ctas -- \N 302 \N 502 602 4.14159 \N 6.14159 \N -124 -302 2013 -402 -502 -602 \N 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 \N -7.1400 row2 \N 09:11:09.567 text2 0xE86F6C6C6F20576F726C67 \N \N 0x2F \N 0x88656C6C9F Value3 -201 301 401 501 601 3.14159 4.1415926 5.14159 1 -123 -301 2012 -401 -501 -601 2012-10-30 2012-10-25T12:05:36.345700 2012-10-25T08:08:08 -4.14145 -5.1400000001 -6.1400 row1 line1 09:09:09.567 text1 0x48656C6C6F20576F726C64 {"age":30,"city":"London","name":"Alice"} Option1,Option3 0x2A 0x48656C6C6F00000000000000 0x48656C6C6F Value2 -202 302 402 502 602 4.14159 5.1415926 6.14159 0 -124 -302 2013 -402 -502 -602 2012-11-01 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 -6.1400000001 -7.1400 row2 line2 09:11:09.567 text2 0xE86F6C6C6F20576F726C67 {"age":18,"city":"ChongQing","name":"Gaoxin"} Option1,Option2 0x2F 0x58676C6C6F00000000000000 0x88656C6C9F Value3 -203 303 403 503 603 7.14159 8.1415926 9.14159 0 \N -402 2017 -602 -902 -1102 2012-11-02 \N 2013-10-27T08:11:18 -5.14145 -6.1400000000001 -7.1400 row3 line3 09:11:09.567 text3 0xE86F6C6C6F20576F726C67 {"age":24,"city":"ChongQing","name":"ChenQi"} Option2 0x2F 0x58676C6C6F00000000000000 \N Value1 +201 301 401 501 601 3.14159 4.1415926 5.14159 1 -123 -301 2012 -401 -501 -601 2012-10-30 2012-10-25T12:05:36.345700 2012-10-25T08:08:08 -4.14145 -5.1400000001 -6.1400 row1 line1 09:09:09.567 text1 0x48656C6C6F20576F726C64 {"age": 30, "city": "London", "name": "Alice"} Option1,Option3 0x2A 0x48656C6C6F00000000000000 0x48656C6C6F Value2 +202 302 402 502 602 4.14159 5.1415926 6.14159 0 -124 -302 2013 -402 -502 -602 2012-11-01 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 -6.1400000001 -7.1400 row2 line2 09:11:09.567 text2 0xE86F6C6C6F20576F726C67 {"age": 18, "city": "ChongQing", "name": "Gaoxin"} Option1,Option2 0x2F 0x58676C6C6F00000000000000 0x88656C6C9F Value3 +203 303 403 503 603 7.14159 8.1415926 9.14159 0 \N -402 2017 -602 -902 -1102 2012-11-02 \N 2013-10-27T08:11:18 -5.14145 -6.1400000000001 -7.1400 row3 line3 09:11:09.567 text3 0xE86F6C6C6F20576F726C67 {"age": 24, "city": "ChongQing", "name": "ChenQi"} Option2 0x2F 0x58676C6C6F00000000000000 \N Value1 -- !ctas_desc -- bigint BIGINT Yes false \N NONE @@ -327,7 +327,7 @@ float FLOAT Yes false \N NONE float_u FLOAT Yes false \N NONE int INT Yes false \N NONE int_u BIGINT Yes false \N NONE -json JSON Yes false \N NONE +json TEXT Yes false \N NONE mediumint INT Yes false \N NONE mediumint_u INT Yes true \N set TEXT Yes false \N NONE diff --git a/regression-test/data/external_table_p0/jdbc/test_pg_jdbc_catalog.out b/regression-test/data/external_table_p0/jdbc/test_pg_jdbc_catalog.out index 124393ea0e4e98..ce9e24dd17764e 100644 --- a/regression-test/data/external_table_p0/jdbc/test_pg_jdbc_catalog.out +++ b/regression-test/data/external_table_p0/jdbc/test_pg_jdbc_catalog.out @@ -1059,1060 +1059,1060 @@ true abc def 2022-10-11 1.234 1 2 99 2022-10-22T10:59:59 34.123 234 -- !test7 -- -1 {"id":1,"test0":"测试","test4":"测试","test8":"测试","tes736":"测试","test12":"测试","test16":"测试","test20":"测试","test24":"测试","test28":"测试","test32":"测试","test36":"测试","test40":"测试","test44":"测试","test48":"测试","test52":"测试","test56":"测试","test60":"测试","test64":"测试","test68":"测试","test72":"测试","test76":"测试","test80":"测试","test84":"测试","test88":"测试","test92":"测试","test96":"测试","test100":"测试","test104":"测试","test108":"测试","test112":"测试","test116":"测试","test120":"测试","test124":"测试","test128":"测试","test132":"测试","test136":"测试","test140":"测试","test144":"测试","test148":"测试","test152":"测试","test156":"测试","test160":"测试","test164":"测试","test168":"测试","test172":"测试","test176":"测试","test180":"测试","test184":"测试","test188":"测试","test192":"测试","test196":"测试","test200":"测试","test204":"测试","test208":"测试","test212":"测试","test216":"测试","test220":"测试","test224":"测试","test228":"测试","test232":"测试","test236":"测试","test240":"测试","test244":"测试","test248":"测试","test252":"测试","test256":"测试","test260":"测试","test264":"测试","test268":"测试","test272":"测试","test276":"测试","test280":"测试","test284":"测试","test288":"测试","test292":"测试","test296":"测试","test300":"测试","test304":"测试","test308":"测试","test312":"测试","test316":"测试","test320":"测试","test324":"测试","test328":"测试","test332":"测试","test336":"测试","test340":"测试","test344":"测试","test348":"测试","test352":"测试","test356":"测试","test360":"测试","test364":"测试","test368":"测试","test372":"测试","test376":"测试","test380":"测试","test384":"测试","test388":"测试","test392":"测试","test396":"测试","test400":"测试","test404":"测试","test408":"测试","test412":"测试","test416":"测试","test420":"测试","test424":"测试","test428":"测试","test432":"测试","test436":"测试","test440":"测试","test444":"测试","test448":"测试","test452":"测试","test456":"测试","test460":"测试","test464":"测试","test468":"测试","test472":"测试","test476":"测试","test480":"测试","test484":"测试","test488":"测试","test492":"测试","test496":"测试","test500":"测试","test504":"测试","test508":"测试","test512":"测试","test516":"测试","test520":"测试","test524":"测试","test528":"测试","test532":"测试","test536":"测试","test540":"测试","test544":"测试","test548":"测试","test552":"测试","test556":"测试","test560":"测试","test564":"测试","test568":"测试","test572":"测试","test576":"测试","test580":"测试","test584":"测试","test588":"测试","test592":"测试","test596":"测试","test600":"测试","test604":"测试","test608":"测试","test612":"测试","test616":"测试","test620":"测试","test624":"测试","test628":"测试","test632":"测试","test636":"测试","test640":"测试","test644":"测试","test648":"测试","test652":"测试","test656":"测试","test660":"测试","test664":"测试","test668":"测试","test672":"测试","test676":"测试","test680":"测试","test684":"测试","test688":"测试","test692":"测试","test696":"测试","test700":"测试","test704":"测试","test708":"测试","test712":"测试","test716":"测试","test720":"测试","test724":"测试","test728":"测试","test732":"测试","test740":"测试","test744":"测试","test748":"测试","test752":"测试","test756":"测试","test760":"测试","test764":"测试","test768":"测试","test772":"测试","test776":"测试","test780":"测试","test784":"测试","test788":"测试","test792":"测试","test796":"测试","test800":"测试","test804":"测试","test808":"测试","test812":"测试","test816":"测试","test820":"测试","test824":"测试","test828":"测试","test832":"测试","test836":"测试","test840":"测试","test844":"测试","test848":"测试","test852":"测试","test856":"测试","test860":"测试","test864":"测试","test868":"测试","test872":"测试","test876":"测试","test880":"测试","test884":"测试","test888":"测试","test892":"测试","test896":"测试","test900":"测试","test904":"测试","test908":"测试","test912":"测试","test916":"测试","test920":"测试","test924":"测试","test928":"测试","test932":"测试","test936":"测试","test940":"测试","test944":"测试","test948":"测试","test952":"测试","test956":"测试","test960":"测试","test964":"测试","test968":"测试","test972":"测试","test976":"测试","test980":"测试","test984":"测试","test988":"测试","test992":"测试","test996":"测试","test1000":"测试","test1004":"测试","test1008":"测试","test1012":"测试","test1016":"测试","test1020":"测试","test1024":"测试","test1028":"测试","test1032":"测试","test1036":"测试","test1040":"测试","test1044":"测试","test1048":"测试","test1052":"测试","test1056":"测试","test1060":"测试","test1064":"测试","test1068":"测试","test1072":"测试","test1076":"测试","test1080":"测试","test1084":"测试","test1088":"测试","test1092":"测试","test1096":"测试","test1100":"测试","test1104":"测试","test1108":"测试","test1110":"测试","test1112":"测试","test1116":"测试","test1120":"测试","test1124":"测试","test1128":"测试","test1132":"测试","test1136":"测试","test1140":"测试","test1144":"测试","test1148":"测试","test1152":"测试","test1156":"测试","test1160":"测试","test1164":"测试","test1168":"测试","test1172":"测试","test1176":"测试","test1180":"测试","test1184":"测试","test1188":"测试","test1192":"测试","test1196":"测试","test1200":"测试","test1204":"测试","test1208":"测试","test1212":"测试","test1216":"测试","test1220":"测试","test1224":"测试","test1228":"测试","test1232":"测试","test1236":"测试","test1240":"测试","test1244":"测试","test1248":"测试","test1252":"测试","test1256":"测试","test1260":"测试","test1264":"测试","test1268":"测试","test1272":"测试","test1276":"测试","test1280":"测试","test1284":"测试","test1288":"测试","test1292":"测试","test1296":"测试","test1300":"测试","test1304":"测试","test1308":"测试","test1312":"测试","test1316":"测试","test1320":"测试","test1324":"测试","test1328":"测试","test1332":"测试","test1336":"测试","test1340":"测试","test1344":"测试","test1348":"测试","test1352":"测试","test1356":"测试","test1360":"测试","test1364":"测试","test1368":"测试","test1372":"测试","test1376":"测试","test1380":"测试","test1384":"测试","test1388":"测试","test1392":"测试","test1396":"测试","test1400":"测试","test1404":"测试","test1408":"测试","test1412":"测试","test1416":"测试","test1420":"测试","test1424":"测试","test1428":"测试","test1432":"测试","test1436":"测试","test1440":"测试","test1444":"测试","test1448":"测试","test1452":"测试","test1456":"测试","test1460":"测试","test1464":"测试","test1468":"测试","test1472":"测试","test1476":"测试","test1480":"测试","test1484":"测试","test1488":"测试","test1492":"测试","test1496":"测试","test1500":"测试","test1504":"测试","test1508":"测试","test1512":"测试","test1516":"测试","test1520":"测试","test1524":"测试","test1528":"测试","test1532":"测试","test1536":"测试","test1540":"测试","test1544":"测试","test1548":"测试","test1552":"测试","test1556":"测试","test1560":"测试","test1564":"测试","test1568":"测试","test1572":"测试","test1576":"测试","test1580":"测试","test1584":"测试","test1588":"测试","test1592":"测试","test1596":"测试","test1600":"测试","test1604":"测试","test1608":"测试","test1612":"测试","test1616":"测试","test1620":"测试","test1624":"测试","test1628":"测试","test1632":"测试","test1636":"测试","test1640":"测试","test1644":"测试","test1648":"测试","test1652":"测试","test1656":"测试","test1660":"测试","test1664":"测试","test1668":"测试","test1672":"测试","test1676":"测试","test1680":"测试","test1684":"测试","test1688":"测试","test1692":"测试","test1696":"测试","test1700":"测试","test1704":"测试","test1708":"测试","test1712":"测试","test1716":"测试","test1720":"测试","test1724":"测试","test1728":"测试","test1732":"测试","test1740":"测试","test1744":"测试","test1748":"测试","test1752":"测试","test1756":"测试","test1760":"测试","test1764":"测试","test1768":"测试","test1772":"测试","test1776":"测试","test1780":"测试","test1784":"测试","test1788":"测试","test1792":"测试","test1796":"测试","test1800":"测试","test1804":"测试","test1808":"测试","test1812":"测试","test1816":"测试","test1820":"测试","test1824":"测试","test1828":"测试","test1832":"测试","test1836":"测试","test1840":"测试","test1844":"测试","test1848":"测试","test1852":"测试","test1856":"测试","test1860":"测试","test1864":"测试","test1868":"测试","test1872":"测试","test1876":"测试","test1880":"测试","test1884":"测试","test1888":"测试","test1892":"测试","test1896":"测试","test1900":"测试","test1904":"测试","test1908":"测试","test1912":"测试","test1916":"测试","test1920":"测试","test1924":"测试","test1928":"测试","test1932":"测试","test1936":"测试","test1940":"测试","test1944":"测试","test1948":"测试","test1952":"测试","test1956":"测试","test1960":"测试","test1964":"测试","test1968":"测试","test1972":"测试","test1976":"测试","test1980":"测试","test1984":"测试","test1988":"测试","test1992":"测试","test1996":"测试","test2000":"测试","test2004":"测试","test2008":"测试","test2012":"测试","test2016":"测试","test2020":"测试","test2024":"测试","test2028":"测试","test2032":"测试","test2036":"测试","test2040":"测试","test2044":"测试","test2048":"测试","test2052":"测试","test2056":"测试","test2060":"测试","test2064":"测试","test2068":"测试","test2072":"测试","test2076":"测试","test2080":"测试","test2084":"测试","test2088":"测试","test2092":"测试","test2096":"测试","test2100":"测试","test2104":"测试","test2108":"测试","test2110":"测试","test2112":"测试","test2116":"测试","test2120":"测试","test2124":"测试","test2128":"测试","test2132":"测试","test2136":"测试","test2140":"测试","test2144":"测试","test2148":"测试","test2152":"测试","test2156":"测试","test2160":"测试","test2164":"测试","test2168":"测试","test2172":"测试","test2176":"测试","test2180":"测试","test2184":"测试","test2188":"测试","test2192":"测试","test2196":"测试","test2200":"测试","test2204":"测试","test2208":"测试","test2212":"测试","test2216":"测试","test2220":"测试","test2224":"测试","test2228":"测试","test2232":"测试","test2236":"测试","test2240":"测试","test2244":"测试","test2248":"测试","test2252":"测试","test2256":"测试","test2260":"测试","test2264":"测试","test2268":"测试","test2272":"测试","test2276":"测试","test2280":"测试","test2284":"测试","test2288":"测试","test2292":"测试","test2296":"测试","test2300":"测试","test2304":"测试","test2308":"测试","test2312":"测试","test2316":"测试","test2320":"测试","test2324":"测试","test2328":"测试","test2332":"测试","test2336":"测试","test2340":"测试","test2344":"测试","test2348":"测试","test2352":"测试","test2356":"测试","test2360":"测试","test2364":"测试","test2368":"测试","test2372":"测试","test2376":"测试","test2380":"测试","test2384":"测试","test2388":"测试","test2392":"测试","test2396":"测试","test2400":"测试","test2404":"测试","test2408":"测试","test2412":"测试","test2416":"测试","test2420":"测试","test2424":"测试","test2428":"测试","test2432":"测试","test2436":"测试","test2440":"测试","test2444":"测试","test2448":"测试","test2452":"测试","test2456":"测试","test2460":"测试","test2464":"测试","test2468":"测试","test2472":"测试","test2476":"测试","test2480":"测试","test2484":"测试","test2488":"测试","test2492":"测试","test2496":"测试","test2500":"测试","test2504":"测试","test2508":"测试","test2512":"测试","test2516":"测试","test2520":"测试","test2524":"测试","test2528":"测试","test2532":"测试","test2536":"测试","test2540":"测试","test2544":"测试","test2548":"测试","test2552":"测试","test2556":"测试","test2560":"测试","test2564":"测试","test2568":"测试","test2572":"测试","test2576":"测试","test2580":"测试","test2584":"测试","test2588":"测试","test2592":"测试","test2596":"测试","test2600":"测试","test2604":"测试","test2608":"测试","test2612":"测试","test2616":"测试","test2620":"测试","test2624":"测试","test2628":"测试","test2632":"测试","test2636":"测试","test2640":"测试","test2644":"测试","test2648":"测试","test2652":"测试","test2656":"测试","test2660":"测试","test2664":"测试","test2668":"测试","test2672":"测试","test2676":"测试","test2680":"测试","test2684":"测试","test2688":"测试","test2692":"测试","test2696":"测试","test2700":"测试","test2704":"测试","test2708":"测试","test2712":"测试","test2716":"测试","test2720":"测试","test2724":"测试","test2728":"测试","test2732":"测试","test2740":"测试","test2744":"测试","test2748":"测试","test2752":"测试","test2756":"测试","test2760":"测试","test2764":"测试","test2768":"测试","test2772":"测试","test2776":"测试","test2780":"测试","test2784":"测试","test2788":"测试","test2792":"测试","test2796":"测试","test2800":"测试","test2804":"测试","test2808":"测试","test2812":"测试","test2816":"测试","test2820":"测试","test2824":"测试","test2828":"测试","test2832":"测试","test2836":"测试","test2840":"测试","test2844":"测试","test2848":"测试","test2852":"测试","test2856":"测试","test2860":"测试","test2864":"测试","test2868":"测试","test2872":"测试","test2876":"测试","test2880":"测试","test2884":"测试","test2888":"测试","test2892":"测试","test2896":"测试","test2900":"测试","test2904":"测试","test2908":"测试","test2912":"测试","test2916":"测试","test2920":"测试","test2924":"测试","test2928":"测试","test2932":"测试","test2936":"测试","test2940":"测试","test2944":"测试","test2948":"测试","test2952":"测试","test2956":"测试","test2960":"测试","test2964":"测试","test2968":"测试","test2972":"测试","test2976":"测试","test2980":"测试","test2984":"测试","test2988":"测试","test2992":"测试","test2996":"测试","test3000":"测试","test3004":"测试","test3008":"测试","test3012":"测试","test3016":"测试","test3020":"测试","test3024":"测试","test3028":"测试","test3032":"测试","test3036":"测试","test3040":"测试","test3044":"测试","test3048":"测试","test3052":"测试","test3056":"测试","test3060":"测试","test3064":"测试","test3068":"测试","test3072":"测试","test3076":"测试","test3080":"测试","test3084":"测试","test3088":"测试","test3092":"测试","test3096":"测试","test3100":"测试","test3104":"测试","test3108":"测试","test3110":"测试","test3112":"测试","test3116":"测试","test3120":"测试","test3124":"测试","test3128":"测试","test3132":"测试","test3136":"测试","test3140":"测试","test3144":"测试","test3148":"测试","test3152":"测试","test3156":"测试","test3160":"测试","test3164":"测试","test3168":"测试","test3172":"测试","test3176":"测试","test3180":"测试","test3184":"测试","test3188":"测试","test3192":"测试","test3196":"测试","test3200":"测试","test3204":"测试","test3208":"测试","test3212":"测试","test3216":"测试","test3220":"测试","test3224":"测试","test3228":"测试","test3232":"测试","test3236":"测试","test3240":"测试","test3244":"测试","test3248":"测试","test3252":"测试","test3256":"测试","test3260":"测试","test3264":"测试","test3268":"测试","test3272":"测试","test3276":"测试","test3280":"测试","test3284":"测试","test3288":"测试","test3292":"测试","test3296":"测试","test3300":"测试","test3304":"测试","test3308":"测试","test3312":"测试","test3316":"测试","test3320":"测试","test3324":"测试","test3328":"测试","test3332":"测试","test3336":"测试","test3340":"测试","test3344":"测试","test3348":"测试","test3352":"测试","test3356":"测试","test3360":"测试","test3364":"测试","test3368":"测试","test3372":"测试","test3376":"测试","test3380":"测试","test3384":"测试","test3388":"测试","test3392":"测试","test3396":"测试","test3400":"测试","test3404":"测试","test3408":"测试","test3412":"测试","test3416":"测试","test3420":"测试","test3424":"测试","test3428":"测试","test3432":"测试","test3436":"测试","test3440":"测试","test3444":"测试","test3448":"测试","test3452":"测试","test3456":"测试","test3460":"测试","test3464":"测试","test3468":"测试","test3472":"测试","test3476":"测试","test3480":"测试","test3484":"测试","test3488":"测试","test3492":"测试","test3496":"测试","test3500":"测试","test3504":"测试","test3508":"测试","test3512":"测试","test3516":"测试","test3520":"测试","test3524":"测试","test3528":"测试","test3532":"测试","test3536":"测试","test3540":"测试","test3544":"测试","test3548":"测试","test3552":"测试","test3556":"测c试","test3560":"测试","test3564":"测试","test3568":"测试","test3572":"测试","test3576":"测试","test3580":"测试","test3584":"测试","test3588":"测试","test3592":"测试","test3596":"测试","test3600":"测试","test3604":"测试","test3608":"测试","test3612":"测试","test3616":"测试","test3620":"测试","test3624":"测试","test3628":"测试","test3632":"测试","test3636":"测试","test3640":"测试","test3644":"测试","test3648":"测试","test3652":"测试","test3656":"测试","test3660":"测试","test3664":"测试","test3668":"测试","test3672":"测试","test3676":"测试","test3680":"测试","test3684":"测试","test3688":"测试","test3692":"测试","test3696":"测试","test3700":"测试","test3704":"测试","test3708":"测试","test3712":"测试","test3716":"测试","test3720":"测试","test3724":"测试","test3728":"测试","test3732":"测试","test3740":"测试","test3744":"测试","test3748":"测试","test3752":"测试","test3756":"测试","test3760":"测试","test3764":"测试","test3768":"测试","test3772":"测试","test3776":"测试","test3780":"测试","test3784":"测试","test3788":"测试","test3792":"测试","test3796":"测试","test3800":"测试","test3804":"测试","test3808":"测试","test3812":"测试","test3816":"测试","test3820":"测试","test3824":"测试","test3828":"测试","test3832":"测试","test3836":"测试","test3840":"测试","test3844":"测试","test3848":"测试","test3852":"测试","test3856":"测试","test3860":"测试","test3864":"测试","test3868":"测试","test3872":"测试","test3876":"测试","test3880":"测试","test3884":"测试","test3888":"测试","test3892":"测试","test3896":"测试","test3900":"测试","test3904":"测试","test3908":"测试","test3912":"测试","test3916":"测试","test3920":"测试","test3924":"测试","test3928":"测试","test3932":"测试","test3936":"测试","test3940":"测试","test3944":"测试","test3948":"测试","test3952":"测试","test3956":"测试","test3960":"测试","test3964":"测试","test3968":"测试","test3972":"测试","test3976":"测试","test3980":"测试","test3984":"测试","test3992":"测试","test3996":"测试","test4000":"测试","test4004":"测试","test4008":"测试","test4012":"测试","test4016":"测试","test4020":"测试","test4024":"测试","test4028":"测试","test4032":"测试","test4036":"测试","test4040":"测试","test4044":"测试","test4048":"测试","test4052":"测试","test4056":"测试","test4060":"测试","test4064":"测试","test4068":"测试","test4072":"测试","test4076":"测试","test4080":"测试","test4084":"测试","test4088":"测试","test4092":"测试","test4096":"测试","test4100":"测试","test4104":"测试","test4108":"测试","test4110":"测试","test4112":"测试","test4116":"测试","test4120":"测试","test4124":"测试","test4128":"测试","test4132":"测试","test4136":"测试","test4140":"测试","test4144":"测试","test4148":"测试","test4152":"测试","test4156":"测试","test4160":"测试","test4164":"测试","test4168":"测试","test4172":"测试","test4176":"测试","test4180":"测试","test4184":"测试","test4188":"测试","test4192":"测试","test4196":"测试","test4200":"测试","test4204":"测试","test4208":"测试","test4212":"测试","test4216":"测试","test4220":"测试","test4224":"测试","test4228":"测试","test4232":"测试","test4236":"测试","test4240":"测试","test4244":"测试","test4248":"测试","test4252":"测试","test4256":"测试","test4260":"测试","test4264":"测试","test4268":"测试","test4272":"测试","test4276":"测试","test4280":"测试","test4284":"测试","test4288":"测试","test4292":"测试","test4296":"测试","test4300":"测试","test4304":"测试","test4308":"测试","test4312":"测试","test4316":"测试","test4320":"测试","test4324":"测试","test4328":"测试","test4332":"测试","test4336":"测试","test4340":"测试","test4344":"测试","test4348":"测试","test4352":"测试","test4356":"测试","test4360":"测试","test4364":"测试","test4368":"测试","test4372":"测试","test4376":"测试","test4380":"测试","test4384":"测试","test4388":"测试","test4392":"测试","test4396":"测试","test4400":"测试","test4404":"测试","test4408":"测试","test4412":"测试","test4416":"测试","test4420":"测试","test4424":"测试","test4428":"测试","test4432":"测试","test4436":"测试","test4440":"测试","test4444":"测试","test4448":"测试","test4452":"测试","test4456":"测试","test4460":"测试","test4464":"测试","test4468":"测试","test4472":"测试","test4476":"测试","test4480":"测试","test4484":"测试","test4488":"测试","test4492":"测试","test4496":"测试","test4500":"测试","test4504":"测试","test4508":"测试","test4512":"测试","test4516":"测试","test4520":"测试","test4524":"测试","test4528":"测试","test4532":"测试","test4536":"测试","test4540":"测试","test4544":"测试","test4548":"测试","test4552":"测试","test4556":"测试","test4560":"测试","test4564":"测试","test4568":"测试","test4572":"测试","test4576":"测试","test4580":"测试","test4584":"测试","test4588":"测试","test4592":"测试","test4596":"测试","test4600":"测试","test4604":"测试","test4608":"测试","test4612":"测试","test4616":"测试","test4620":"测试","test4624":"测试","test4628":"测试","test4632":"测试","test4636":"测试","test4640":"测试","test4644":"测试","test4648":"测试","test4652":"测试","test4656":"测试","test4660":"测试","test4664":"测试","test4668":"测试","test4672":"测试","test4676":"测试","test4680":"测试","test4684":"测试","test4688":"测试","test4692":"测试","test4696":"测试","test4700":"测试","test4704":"测试","test4708":"测试","test4712":"测试","test4716":"测试","test4720":"测试","test4724":"测试","test4728":"测试","test4732":"测试","test4740":"测试","test4744":"测试","test4748":"测试","test4752":"测试","test4756":"测试","test4760":"测试","test4764":"测试","test4768":"测试","test4772":"测试","test4776":"测试","test4780":"测试","test4784":"测试","test4788":"测试","test4792":"测试","test4796":"测试","test4800":"测试","test4804":"测试","test4808":"测试","test4812":"测试","test4816":"测试","test4820":"测试","test4824":"测试","test4828":"测试","test4832":"测试","test4836":"测试","test4840":"测试","test4844":"测试","test4848":"测试","test4852":"测试","test4856":"测试","test4860":"测试","test4864":"测试","test4868":"测试","test4872":"测试","test4876":"测试","test4880":"测试","test4884":"测试","test4888":"测试","test4892":"测试","test4896":"测试","test4900":"测试","test4904":"测试","test4908":"测试","test4912":"测试","test4916":"测试","test4920":"测试","test4924":"测试","test4928":"测试","test4932":"测试","test4936":"测试","test4940":"测试","test4944":"测试","test4948":"测试","test4952":"测试","test4956":"测试","test4960":"测试","test4964":"测试","test4968":"测试","test4972":"测试","test4976":"测试","test4980":"测试","test4984":"测试","test4988":"测试","test4992":"测试","test4996":"测试","test5000":"测试","test5004":"测试","test5008":"测试","test5012":"测试","test5016":"测试","test5020":"测试","test5024":"测试","test5028":"测试","test5032":"测试","test5036":"测试","test5040":"测试","test5044":"测试","test5048":"测试","test5052":"测试","test5056":"测试","test5060":"测试","test5064":"测试","test5068":"测试","test5072":"测试","test5076":"测试","test5080":"测试","test5084":"测试","test5088":"测试","test5092":"测试","test5096":"测试","test5100":"测试","test5104":"测试","test5108":"测试","test5110":"测试","test5112":"测试","test5116":"测试","test5120":"测试","test5124":"测试","test5128":"测试","test5132":"测试","test5136":"测试","test5140":"测试","test5144":"测试","test5148":"测试","test5152":"测试","test5156":"测试","test5160":"测试","test5164":"测试","test5168":"测试","test5172":"测试","test5176":"测试","test5180":"测试","test5184":"测试","test5188":"测试","test5192":"测试","test5196":"测试","test5200":"测试","test5204":"测试","test5208":"测试","test5212":"测试","test5216":"测试","test5220":"测试","test5224":"测试","test5228":"测试","test5232":"测试","test5236":"测试","test5240":"测试","test5244":"测试","test5248":"测试","test5252":"测试","test5256":"测试","test5260":"测试","test5264":"测试","test5268":"测试","test5272":"测试","test5276":"测试","test5280":"测试","test5284":"测试","test5288":"测试","test5292":"测试","test5296":"测试","test5300":"测试","test5304":"测试","test5308":"测试","test5312":"测试","test5316":"测试","test5320":"测试","test5324":"测试","test5328":"测试","test5332":"测试","test5336":"测试","test5340":"测试","test5344":"测试","test5348":"测试","test5352":"测试","test5356":"测试","test5360":"测试","test5364":"测试","test5368":"测试","test5372":"测试","test5376":"测试","test5380":"测试","test5384":"测试","test5388":"测试","test5392":"测试","test5396":"测试","test5400":"测试","test5404":"测试","test5408":"测试","test5412":"测试","test5416":"测试","test5420":"测试","test5424":"测试","test5428":"测试","test5432":"测试","test5436":"测试","test5440":"测试","test5444":"测试","test5448":"测试","test5452":"测试","test5456":"测试","test5460":"测试","test5464":"测试","test5468":"测试","test5472":"测试","test5476":"测试","test5480":"测试","test5484":"测试","test5488":"测试","test5492":"测试","test5496":"测试","test5500":"测试","test5504":"测试","test5508":"测试","test5512":"测试","test5516":"测试","test5520":"测试","test5524":"测试","test5528":"测试","test5532":"测试","test5536":"测试","test5540":"测试","test5544":"测试","test5548":"测试","test5552":"测试","test5556":"测试","test5560":"测试","test5564":"测试","test5568":"测试","test5572":"测试","test5576":"测试","test5580":"测试","test5584":"测试","test5588":"测试","test5592":"测试","test5596":"测试","test5600":"测试","test5604":"测试","test5608":"测试","test5612":"测试","test5616":"测试","test5620":"测试","test5624":"测试","test5628":"测试","test5632":"测试","test5636":"测试","test5640":"测试","test5644":"测试","test5648":"测试","test5652":"测试","test5656":"测试","test5660":"测试","test5664":"测试","test5668":"测试","test5672":"测试","test5676":"测试","test5680":"测试","test5684":"测试","test5688":"测试","test5692":"测试","test5696":"测试","test5700":"测试","test5704":"测试","test5708":"测试","test5712":"测试","test5716":"测试","test5720":"测试","test5724":"测试","test5728":"测试","test5732":"测试","test5740":"测试","test5744":"测试","test5748":"测试","test5752":"测试","test5756":"测试","test5760":"测试","test5764":"测试","test5768":"测试","test5772":"测试","test5776":"测试","test5780":"测试","test5784":"测试","test5788":"测试","test5792":"测试","test5796":"测试","test5800":"测试","test5804":"测试","test5808":"测试","test5812":"测试","test5816":"测试","test5820":"测试","test5824":"测试","test5828":"测试","test5832":"测试","test5836":"测试","test5840":"测试","test5844":"测试","test5848":"测试","test5852":"测试","test5856":"测试","test5860":"测试","test5864":"测试","test5868":"测试","test5872":"测试","test5876":"测试","test5880":"测试","test5884":"测试","test5888":"测试","test5892":"测试","test5896":"测试","test5900":"测试","test5904":"测试","test5908":"测试","test5912":"测试","test5916":"测试","test5920":"测试","test5924":"测试","test5928":"测试","test5932":"测试","test5936":"测试","test5940":"测试","test5944":"测试","test5948":"测试","test5952":"测试","test5956":"测试","test5960":"测试","test5964":"测试","test5968":"测试","test5972":"测试","test5976":"测试","test5980":"测试","test5984":"测试","test5988":"测试","test5992":"测试","test5996":"测试","test6000":"测试","test6004":"测试","test6008":"测试","test6012":"测试","test6016":"测试","test6020":"测试","test6024":"测试","test6028":"测试","test6032":"测试","test6036":"测试","test6040":"测试","test6044":"测试","test6048":"测试","test6052":"测试","test6056":"测试","test6060":"测试","test6064":"测试","test6068":"测试","test6072":"测试","test6076":"测试","test6080":"测试","test6084":"测试","test6088":"测试","test6092":"测试","test6096":"测试","test6100":"测试","test6104":"测试","test6108":"测试","test6110":"测试","test6112":"测试","test6116":"测试","test6120":"测试","test6124":"测试","test6128":"测试","test6132":"测试","test6136":"测试","test6140":"测试","test6144":"测试","test6148":"测试","test6152":"测试","test6156":"测试","test6160":"测试","test6164":"测试","test6168":"测试","test6172":"测试","test6176":"测试","test6180":"测试","test6184":"测试","test6188":"测试","test6192":"测试","test6196":"测试","test6200":"测试","test6204":"测试","test6208":"测试","test6212":"测试","test6216":"测试","test6220":"测试","test6224":"测试","test6228":"测试","test6232":"测试","test6236":"测试","test6240":"测试","test6244":"测试","test6248":"测试","test6252":"测试","test6256":"测试","test6260":"测试","test6264":"测试","test6268":"测试","test6272":"测试","test6276":"测试","test6280":"测试","test6284":"测试","test6288":"测试","test6292":"测试","test6296":"测试","test6300":"测试","test6304":"测试","test6308":"测试","test6312":"测试","test6316":"测试","test6320":"测试","test6324":"测试","test6328":"测试","test6332":"测试","test6336":"测试","test6340":"测试","test6344":"测试","test6348":"测试","test6352":"测试","test6356":"测试","test6360":"测试","test6364":"测试","test6368":"测试","test6372":"测试","test6376":"测试","test6380":"测试","test6384":"测试","test6388":"测试","test6392":"测试","test6396":"测试","test6400":"测试","test6404":"测试","test6408":"测试","test6412":"测试","test6416":"测试","test6420":"测试","test6424":"测试","test6428":"测试","test6432":"测试","test6436":"测试","test6440":"测试","test6444":"测试","test6448":"测试","test6452":"测试","test6456":"测试","test6460":"测试","test6464":"测试","test6468":"测试","test6472":"测试","test6476":"测试","test6480":"测试","test6484":"测试","test6488":"测试","test6492":"测试","test6496":"测试","test6500":"测试","test6504":"测试","test6508":"测试","test6512":"测试","test6516":"测试","test6520":"测试","test6524":"测试","test6528":"测试","test6532":"测试","test6536":"测试","test6540":"测试","test6544":"测试","test6548":"测试","test6552":"测试","test6556":"测试","test6560":"测试","test6564":"测试","test6568":"测试","test6572":"测试","test6576":"测试","test6580":"测试","test6584":"测试","test6588":"测试","test6592":"测试","test6596":"测试","test6600":"测试","test6604":"测试","test6608":"测试","test6612":"测试","test6616":"测试","test6620":"测试","test6624":"测试","test6628":"测试","test6632":"测试","test6636":"测试","test6640":"测试","test6644":"测试","test6648":"测试","test6652":"测试","test6656":"测试","test6660":"测试","test6664":"测试","test6668":"测试","test6672":"测试","test6676":"测试","test6680":"测试","test6684":"测试","test6688":"测试","test6692":"测试","test6696":"测试","test6700":"测试","test6704":"测试","test6708":"测试","test6712":"测试","test6716":"测试","test6720":"测试","test6724":"测试","test6728":"测试","test6732":"测试","test6740":"测试","test6744":"测试","test6748":"测试","test6752":"测试","test6756":"测试","test6760":"测试","test6764":"测试","test6768":"测试","test6772":"测试","test6776":"测试","test6780":"测试","test6784":"测试","test6788":"测试","test6792":"测试","test6796":"测试","test6800":"测试","test6804":"测试","test6808":"测试","test6812":"测试","test6816":"测试","test6820":"测试","test6824":"测试","test6828":"测试","test6832":"测试","test6836":"测试","test6840":"测试","test6844":"测试","test6848":"测试","test6852":"测试","test6856":"测试","test6860":"测试","test6864":"测试","test6868":"测试","test6872":"测试","test6876":"测试","test6880":"测试","test6884":"测试","test6888":"测试","test6892":"测试","test6896":"测试","test6900":"测试","test6904":"测试","test6908":"测试","test6912":"测试","test6916":"测试","test6920":"测试","test6924":"测试","test6928":"测试","test6932":"测试","test6936":"测试","test6940":"测试","test6944":"测试","test6948":"测试","test6952":"测试","test6956":"测试","test6960":"测试","test6964":"测试","test6968":"测试","test6972":"测试","test6976":"测试","test6980":"测试","test6984":"测试","test6988":"测试","test6992":"测试","test6996":"测试","test7000":"测试","test7004":"测试","test7008":"测试","test7012":"测试","test7016":"测试","test7020":"测试","test7024":"测试","test7028":"测试","test7032":"测试","test7036":"测试","test7040":"测试","test7044":"测试","test7048":"测试","test7052":"测试","test7056":"测试","test7060":"测试","test7064":"测试","test7068":"测试","test7072":"测试","test7076":"测试","test7080":"测试","test7084":"测试","test7088":"测试","test7092":"测试","test7096":"测试","test7100":"测试","test7104":"测试","test7108":"测试","test7112":"测试","test7116":"测试","test7120":"测试","test7124":"测试","test7128":"测试","test7132":"测试","test7136":"测试","test7140":"测试","test7144":"测试","test7148":"测试","test7152":"测试","test7156":"测试","test7160":"测试","test7164":"测试","test7168":"测试","test7172":"测试","test7176":"测试","test7180":"测试","test7184":"测试","test7188":"测试","test7192":"测试","test7196":"测试","test7200":"测试","test7204":"测试","test7208":"测试","test7212":"测试","test7216":"测试","test7220":"测试","test7224":"测试","test7228":"测试","test7232":"测试","test7236":"测试","test7240":"测试","test7244":"测试","test7248":"测试","test7252":"测试","test7256":"测试","test7260":"测试","test7264":"测试","test7268":"测试","test7272":"测试","test7276":"测试","test7280":"测试","test7284":"测试","test7288":"测试","test7292":"测试","test7296":"测试","test7300":"测试","test7304":"测试","test7308":"测试","test7312":"测试","test7316":"测试","test7320":"测试","test7324":"测试","test7328":"测试","test7332":"测试","test7336":"测试","test7340":"测试","test7344":"测试","test7348":"测试","test7352":"测试","test7356":"测试","test7372":"测试","test7376":"测试","test7380":"测试","test7384":"测试","test7388":"测试","test7392":"测试","test7396":"测试","test7400":"测试","test7404":"测试","test7408":"测试","test7412":"测试","test7416":"测试","test7420":"测试","test7424":"测试","test7428":"测试","test7432":"测试","test7436":"测试","test7440":"测试","test7444":"测试","test7448":"测试","test7452":"测试","test7456":"测试","test7460":"测试","test7464":"测试","test7468":"测试","test7472":"测试","test7476":"测试","test7480":"测试","test7484":"测试","test7488":"测试","test7492":"测试","test7496":"测试","test7500":"测试","test7504":"测试","test7508":"测试","test7512":"测试","test7516":"测试","test7520":"测试","test7524":"测试","test7528":"测试","test7532":"测试","test7536":"测试","test7540":"测试","test7544":"测试","test7548":"测试","test7552":"测试","test7556":"测试","test7560":"测试","test7564":"测试","test7568":"测试","test7572":"测试","test7576":"测试","test7580":"测试","test7584":"测试","test7588":"测试","test7592":"测试","test7596":"测试"} -2 {"fuel_type":"测试","structure":"测试","plate_type":"01","violations":[{"act":"测试测试测试度、测试、测试测试测试的","date":"2016-12-27 14:50:00"},{"act":"测试测试测试测试测试测试的","date":"2016-11-16 09:27:00"},{"act":"测试测试测试测试测试测试测试测试测试测试的","date":"2016-10-30 09:21:00"},{"act":"测试测试测试测试测试测试测试测试测试的","date":"2015-02-14 14:48:00"},{"act":"测试测试测试测试测试测试测试测试测试的","date":"2015-01-16 12:28:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2015-01-15 10:16:00"},{"act":"测试测试测试测试测试测试测试","date":"2015-01-09 16:32:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2015-01-09 16:28:00"},{"act":"测试测试测试测试测试测试测试","date":"2014-12-24 09:42:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2014-12-24 09:41:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2014-12-10 09:18:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2014-12-03 15:12:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2014-11-24 10:35:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2014-11-06 10:47:00"},{"act":"测试测试测试测试测试测试测试测试测试测试,测试测试测试测试测试测试的","date":"2014-11-06 10:46:00"},{"act":"测试测试测试测试放、测试测试测试测试测试测试测试测试测试,测试测试测试、测试测试的","date":"2014-11-03 09:44:00"},{"act":"测试测试测试测试放、测试测试测试测试测试测试测试测试测试,测试测试测试、测试测试的","date":"2014-10-21 09:22:00"},{"act":"测试测试测试测试放、测试测试测试测试测试测试测试测试测试,测试测试测试、测试测试的","date":"2014-10-16 14:54:00"},{"act":"测试测试测试测试的","date":"2014-09-28 19:06:00"},{"act":"测试测试测试测试放、测试测试测试测试测试测试测试测试测试,测试测试测试、测试测试的","date":"2014-09-16 09:54:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2014-08-30 09:30:00"},{"act":"测试门、测试测试测试测试测试","date":"2014-08-30 09:27:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2014-08-27 14:02:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2014-08-11 10:42:00"},{"act":"测试测试测试测试测试测试测试达30%的","date":"2014-07-30 14:38:00"},{"act":"测试门、测试测试测试测试测试","date":"2014-07-10 07:31:00"},{"act":"测试测试测试测试放、测试测试测试测试测试测试测试测试测试,测试测试测试、测试测试的","date":"2014-07-08 17:33:00"},{"act":"测试测试测试测试放、测试测试测试测试测试测试测试测试测试,测试测试测试、测试测试的","date":"2014-07-08 10:42:00"},{"act":"测试测试测试测试放、测试测试测试测试测试测试测试测试测试,测试测试测试、测试测试的","date":"2014-06-30 10:15:00"},{"act":"测试门、测试测试测试测试测试","date":"2014-06-29 08:51:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2014-06-24 09:32:00"},{"act":"测试测试测试测试放、测试测试测试测试测试测试测试测试测试,测试测试测试、测试测试的","date":"2014-06-21 10:41:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2014-06-20 14:45:00"},{"act":"测试测试测试测试放、测试测试测试测试测试测试,测试测试测试、测试测试的","date":"2014-06-14 11:45:00"},{"act":"测试测试测试测试放、测试测试测试测试测试测试测试测试测试,测试测试测试、测试测试的","date":"2014-06-12 10:41:00"},{"act":"测试测试测试测试放、测试测试测试测试测试测试测试测试测试,测试测试测试、测试测试的","date":"2014-05-10 09:56:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2014-01-15 10:01:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2014-01-15 10:00:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2014-01-06 12:45:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-12-29 08:18:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-12-29 08:17:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-12-28 15:46:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2013-12-27 08:58:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-12-27 08:56:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-12-26 08:43:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-12-19 09:58:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2013-12-19 07:52:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-12-19 07:50:00"},{"act":"测试测试测试测试测试测试,测试测试测试测试测试测试的","date":"2013-12-18 10:14:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-12-18 08:49:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2013-12-18 07:17:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-12-18 07:16:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-12-17 07:49:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-12-16 07:39:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-12-16 07:37:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-12-06 15:43:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-12-03 14:47:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2013-11-29 16:00:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-11-29 15:59:00"},{"act":"测试门、测试测试测试测试测试","date":"2013-11-27 11:18:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-11-22 21:37:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-11-22 21:36:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-11-16 12:23:00"},{"act":"测试测试测试、测试测试机、测试测试测试,测试测试测试测试测试测试的","date":"2013-11-16 09:14:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-11-15 10:44:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2013-11-15 10:42:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-11-14 08:58:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-11-08 10:05:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-11-08 10:04:00"},{"act":"测试测试测试测试测试测试测试测试测试的","date":"2013-11-08 10:01:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-11-06 10:30:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2013-11-04 08:00:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-11-04 07:59:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-11-03 10:35:00"},{"act":"测试门、测试测试测试测试测试","date":"2013-11-03 09:07:00"},{"act":"测试测试测试测试测试测试测试","date":"2013-11-02 14:18:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-11-02 09:29:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-11-02 08:40:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-10-30 09:35:00"},{"act":"测试测试测试测试测试测试测试测试的","date":"2013-10-25 09:15:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-10-24 09:21:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-10-24 09:06:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-10-23 13:05:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-10-23 09:04:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-10-23 08:42:00"},{"act":"测试测试测试测试测试测试测试测试的","date":"2013-10-19 09:33:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-10-18 17:16:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-10-18 09:59:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2013-10-18 09:41:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-10-18 06:25:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-10-18 06:23:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-10-17 14:59:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-10-17 14:58:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-10-13 09:29:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-10-12 10:00:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-10-12 10:00:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-10-12 10:00:00"},{"act":"测试测试测试测试测试测试测试测试测试测试,测试测试测试测试测试测试的","date":"2013-10-12 09:47:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-10-11 13:49:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-10-10 11:05:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-10-10 11:04:00"},{"act":"测试门、测试测试测试测试测试","date":"2013-10-10 09:52:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-10-09 16:12:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-10-09 10:47:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-10-09 10:46:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2013-09-30 13:52:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-09-30 13:50:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-09-28 15:32:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-09-28 12:41:00"},{"act":"测试测试测试测试测试测试测试测试的","date":"2013-09-28 12:40:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-09-26 15:01:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-09-19 09:10:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-09-18 09:14:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-09-18 07:22:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-09-18 07:21:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-09-17 15:34:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-09-17 15:33:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2013-09-17 10:05:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-09-17 10:04:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-09-16 14:16:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-09-16 14:16:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-09-15 15:18:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-09-12 15:45:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2013-09-12 09:21:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-09-12 09:19:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-09-11 08:23:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2013-09-11 08:23:00"},{"act":"测试测试测试测试测试测试的","date":"2013-09-05 09:07:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-09-05 09:06:00"},{"act":"测试测试测试测试测试测试测试测试的","date":"2013-09-05 09:04:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-09-05 09:00:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-08-31 12:02:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-08-30 10:24:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-08-30 10:02:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-08-29 12:11:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-08-29 09:42:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-08-27 15:22:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-08-24 10:59:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-08-23 13:31:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-08-22 07:57:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-08-22 07:56:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-08-21 13:01:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-08-20 08:39:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-08-20 08:36:00"},{"act":"测试测试测试测试测试测试测试测试测试的","date":"2013-08-16 13:09:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-08-16 09:08:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-08-16 08:20:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-08-15 09:34:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-08-15 09:14:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-08-15 07:35:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-08-14 12:21:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-08-14 12:20:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-08-07 10:10:00"},{"act":"测试测试测试测试测试测试测试测试的","date":"2013-08-07 09:21:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-08-07 08:11:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-08-06 09:27:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-08-06 08:50:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-08-06 08:49:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-08-05 13:08:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-08-05 10:08:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-08-05 08:53:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-08-05 07:56:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-08-02 16:09:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-08-02 15:09:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-08-01 10:42:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-08-01 08:28:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-08-01 08:26:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-07-30 08:50:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-07-29 19:00:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-07-29 18:49:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-07-29 18:46:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-07-28 15:16:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-07-27 08:33:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-07-26 08:55:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-07-25 11:15:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-07-25 11:13:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-07-23 09:12:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-07-23 09:11:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-07-22 10:35:00"},{"act":"测试测试测试测试测试测试的","date":"2013-07-22 10:25:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-07-22 09:02:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-07-22 08:36:00"},{"act":"测试测试测试、测试测试机、测试测试测试,测试测试测试测试测试测试的","date":"2013-07-13 09:23:00"},{"act":"测试测试测试测试测试测试测试测试的","date":"2013-07-08 08:55:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-07-08 08:55:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2013-07-08 07:42:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-07-08 07:40:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-07-05 15:19:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2013-07-05 08:35:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-07-05 08:33:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-07-04 16:05:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-07-04 09:03:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2013-07-04 07:30:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-07-04 07:28:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-07-03 09:52:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-07-03 07:35:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-07-03 07:35:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-07-03 07:30:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-07-02 09:43:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-07-02 09:41:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-07-02 09:03:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2013-07-02 07:04:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-07-02 07:02:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-07-01 15:41:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-07-01 15:38:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-07-01 14:44:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-06-28 09:42:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-06-28 08:02:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-26 15:24:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-06-26 10:15:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-25 12:12:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-24 12:04:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-24 08:37:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-24 08:20:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-24 08:18:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-24 07:28:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-06-24 07:26:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-22 08:56:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-22 08:32:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-06-22 08:30:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-21 09:18:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-06-21 09:17:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-21 08:53:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-06-20 16:55:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-20 13:03:00"},{"act":"测试门、测试测试测试测试测试","date":"2013-06-20 09:25:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-06-19 16:02:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-19 16:01:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-19 09:54:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-18 08:46:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-18 08:23:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-17 11:54:00"},{"act":"测试测试测试、测试测试机、测试测试测试,测试测试测试测试测试测试的","date":"2013-06-17 10:58:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-15 15:51:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-15 10:18:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-06-15 10:15:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-06-14 08:33:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-14 08:31:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-06-14 06:06:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-06-13 16:48:00"},{"act":"测试门、测试测试测试测试测试","date":"2013-06-13 16:46:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-13 14:09:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-07 10:12:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-06 11:45:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-05 10:25:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-06-05 09:46:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-06-04 09:57:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-04 07:50:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-06-03 12:13:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-06-03 07:50:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-06-03 07:45:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-05-29 14:28:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-05-29 13:50:00"},{"act":"测试门、测试测试测试测试测试","date":"2013-05-29 10:47:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-05-28 16:25:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-05-28 08:40:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-05-26 14:24:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-05-25 12:51:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-05-25 12:50:00"},{"act":"测试门、测试测试测试测试测试","date":"2013-05-25 08:36:00"},{"act":"测试测试测试测试测试测试测试测试测试测试的","date":"2013-05-24 15:16:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-05-24 12:28:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-05-23 15:42:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-05-23 09:07:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-05-23 09:05:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-05-22 16:04:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-05-22 10:35:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-05-22 10:34:00"},{"act":"测试测试测试、测试测试机、测试测试测试,测试测试测试测试测试测试的","date":"2013-05-16 11:05:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-05-16 09:00:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-05-15 07:56:00"},{"act":"测试测试测试测试测试测试测试","date":"2013-05-15 07:55:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-05-14 08:30:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-05-14 08:30:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-05-08 15:02:00"},{"act":"测试测试测试测试测试测试测试","date":"2013-05-08 10:22:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-05-07 10:10:00"},{"act":"测试门、测试测试测试测试测试","date":"2013-05-07 08:35:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-05-03 08:55:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-05-03 08:54:00"},{"act":"测试测试测试测试测试测试测试测试的","date":"2013-04-28 17:06:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-04-28 08:58:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-04-28 08:57:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-04-27 10:18:00"},{"act":"测试测试测试测试测试测试测试","date":"2013-04-27 10:17:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-04-26 13:45:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-04-26 13:44:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-04-25 09:45:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-04-23 13:41:00"},{"act":"测试测试测试测试测试测试测试测试的","date":"2013-04-23 08:54:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-04-23 08:53:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-04-18 15:20:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-04-18 08:15:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-04-18 08:14:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-04-17 16:10:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-04-17 08:27:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-04-17 08:25:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-04-16 09:24:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-04-16 09:23:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-04-09 09:49:00"},{"act":"测试门、测试测试测试测试测试","date":"2013-04-08 13:48:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-04-07 08:34:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-04-07 08:33:00"},{"act":"测试测试测试、测试测试机、测试测试测试,测试测试测试测试测试测试的","date":"2013-04-03 14:52:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-04-03 09:30:00"},{"act":"测试门、测试测试测试测试测试","date":"2013-04-03 09:28:00"},{"act":"测试测试测试测试测试测试,测试测试测试测试测试测试","date":"2013-04-02 10:18:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2013-04-02 09:13:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-04-02 09:12:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-03-27 10:58:00"},{"act":"测试门、测试测试测试测试测试","date":"2013-03-26 10:19:00"},{"act":"测试测试测试测试测试测试测试测试的","date":"2013-03-25 14:31:00"},{"act":"测试门、测试测试测试测试测试","date":"2013-03-23 11:36:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-03-23 09:50:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-03-22 09:36:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-03-21 17:35:00"},{"act":"测试测试测试测试测试测试测试测试的","date":"2013-03-21 16:24:00"},{"act":"测试测试测试测试测试测试,测试测试测试测试测试测试的","date":"2013-03-21 16:22:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-03-15 15:03:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-03-15 07:52:00"},{"act":"测试测试测试测试测试测试的","date":"2013-03-15 07:51:00"},{"act":"测试门、测试测试测试测试测试","date":"2013-03-13 10:27:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-03-13 10:09:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-03-12 07:58:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-03-12 07:57:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-03-11 09:47:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-03-11 09:46:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-03-08 16:16:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-03-08 16:15:00"},{"act":"测试测试测试测试测试测试测试达30%的","date":"2013-03-07 17:01:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2013-03-05 08:05:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-03-05 08:04:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-03-05 08:01:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-03-04 18:07:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2013-03-04 16:11:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-03-04 16:09:00"},{"act":"测试门、测试测试测试测试测试","date":"2013-02-28 10:12:00"},{"act":"测试测试测试、测试测试机、测试测试测试,测试测试测试测试测试测试的","date":"2013-02-25 11:11:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-02-23 10:12:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-02-22 17:00:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-02-22 16:59:00"},{"act":"测试门、测试测试测试测试测试","date":"2013-02-22 10:21:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-02-21 09:57:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2013-02-21 08:16:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-02-21 08:14:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-02-21 08:11:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-02-17 08:40:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2013-02-17 08:40:00"},{"act":"测试测试测试测试测试测试的","date":"2013-02-17 08:39:00"},{"act":"测试门、测试测试测试测试测试","date":"2013-02-08 09:39:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-02-07 14:07:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-02-07 14:06:00"},{"act":"测试测试测试测试测试测试测试测试的","date":"2013-02-04 09:16:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-02-04 09:15:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-01-26 12:43:00"},{"act":"测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的","date":"2013-01-25 16:00:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-01-25 11:56:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-01-25 11:55:00"},{"act":"测试测试测试测试测试测试测试测试测试测试","date":"2013-01-25 07:28:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-01-24 08:54:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-01-23 10:56:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-01-23 10:54:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-01-22 10:54:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-01-21 09:44:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-01-17 10:57:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-01-16 11:06:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-01-09 09:34:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-01-09 09:33:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2013-01-06 09:17:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2013-01-06 09:15:00"},{"act":"测试测试测试测试测试测试的","date":"2013-01-05 09:43:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2013-01-04 10:57:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-12-26 15:29:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-12-26 15:28:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-12-25 11:40:00"},{"act":"测试测试测试测试测试测试","date":"2012-12-25 10:01:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-12-24 09:52:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-12-20 08:48:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-12-20 08:46:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-12-19 11:08:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-12-19 09:24:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-12-19 09:23:00"},{"act":"测试测试测试测试测试测试测试","date":"2012-12-18 09:47:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-12-18 09:45:00"},{"act":"测试测试测试测试测试测试","date":"2012-12-17 16:07:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-12-17 10:01:00"},{"act":"测试测试测试测试测试测试测试测试测试的","date":"2012-12-14 20:11:00"},{"act":"测试测试测试测试测试测试","date":"2012-12-10 08:15:00"},{"act":"测试测试测试测试测试测试测试的","date":"2012-12-10 08:14:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-12-10 08:13:00"},{"act":"测试测试测试测试测试测试测试测试测试测试,测试测试测试测试测试测试的","date":"2012-12-08 16:34:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-12-08 16:32:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-12-06 09:41:00"},{"act":"测试测试测试测试测试测试测试的","date":"2012-12-06 09:40:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-12-05 09:11:00"},{"act":"测试测试测试测试测试测试","date":"2012-12-05 09:09:00"},{"act":"测试测试测试测试测试测试","date":"2012-12-04 16:01:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-12-04 09:40:00"},{"act":"测试测试测试测试测试测试的","date":"2012-12-01 14:30:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-12-01 14:29:00"},{"act":"测试测试测试测试测试测试","date":"2012-11-28 16:19:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-11-28 09:35:00"},{"act":"测试测试测试测试测试测试","date":"2012-11-27 14:40:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-11-27 09:53:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2012-11-23 15:20:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-11-23 15:19:00"},{"act":"测试测试测试测试测试测试","date":"2012-11-22 08:09:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-11-21 13:45:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-11-20 10:44:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-11-20 10:00:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-11-19 08:02:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-11-19 08:01:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-11-16 08:20:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-11-14 08:15:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-11-14 08:14:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-11-09 15:16:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-11-09 15:15:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-11-08 08:47:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-11-08 08:46:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-11-06 09:48:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-11-05 09:26:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-11-05 09:25:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-11-04 09:30:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-11-03 09:50:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-11-02 09:35:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-11-02 08:36:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-11-01 15:24:00"},{"act":"测试测试测试测试测试测试的","date":"2012-11-01 10:06:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-10-31 09:29:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2012-10-31 09:27:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-10-27 08:27:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-10-25 16:57:00"},{"act":"测试测试测试测试测试测试测试的","date":"2012-10-25 16:56:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-10-25 09:48:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-10-24 11:37:00"},{"act":"测试测试测试测试测试测试","date":"2012-10-24 09:14:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-10-23 08:56:00"},{"act":"测试测试测试测试测试测试测试测试的","date":"2012-10-23 08:30:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-10-23 08:28:00"},{"act":"测试测试测试测试测试测试测试测试测试测试,测试测试测试测试测试测试的","date":"2012-10-22 16:49:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-10-22 16:48:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-10-22 16:46:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-10-18 10:02:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-10-17 08:08:00"},{"act":"测试测试测试测试测试测试测试测试的","date":"2012-10-16 13:41:00"},{"act":"测试测试测试测试测试测试","date":"2012-10-16 13:38:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-10-12 10:14:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-10-11 12:12:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-10-09 08:45:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-10-09 08:43:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-10-08 14:45:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-09-25 15:11:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-09-25 15:09:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-09-24 16:05:00"},{"act":"测试测试测试测试测试测试","date":"2012-09-22 15:33:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-09-22 15:08:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-09-21 09:35:00"},{"act":"测试测试测试测试测试测试","date":"2012-09-19 10:06:00"},{"act":"测试测试测试测试测试测试测试的","date":"2012-09-19 09:15:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-09-19 09:14:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-09-19 08:22:00"},{"act":"测试测试测试测试测试测试测试的","date":"2012-09-19 08:21:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-09-15 15:21:00"},{"act":"测试测试测试测试测试测试测试测试测试测试,测试测试测试测试测试测试的","date":"2012-09-15 09:51:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-09-10 11:11:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-09-07 14:50:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-09-07 08:39:00"},{"act":"测试测试测试测试测试测试","date":"2012-09-05 16:30:00"},{"act":"测试测试测试测试测试测试测试30%的","date":"2012-09-05 16:23:00"},{"act":"测试测试测试测试测试测试测试测试测试测试,测试测试测试测试测试测试的","date":"2012-09-05 08:22:00"},{"act":"测试测试涂、测试测试测试测试测试,测试测试测试的","date":"2012-09-05 08:20:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-09-05 08:17:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-09-04 09:40:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-09-04 08:08:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-09-03 08:33:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-09-03 08:05:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-09-01 09:10:00"},{"act":"测试测试测试测试测试测试","date":"2012-08-31 14:28:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-08-27 16:16:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-08-27 14:20:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-08-25 14:36:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-08-24 09:11:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-08-23 10:13:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-08-22 12:26:00"},{"act":"测试测试测试测试测试测试测试测试的","date":"2012-08-22 12:25:00"},{"act":"测试测试测试测试测试测试","date":"2012-08-16 10:09:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-08-15 09:16:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-08-10 09:32:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-08-03 15:26:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-08-02 09:37:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-08-01 09:42:00"},{"act":"测试测试测试测试测试测试","date":"2012-07-31 10:35:00"},{"act":"测试测试测试测试测试测试测试30%的","date":"2012-07-30 14:56:00"},{"act":"测试测试测试测试测试测试","date":"2012-07-27 10:10:00"},{"act":"测试测试测试测试测试测试测试测试的","date":"2012-07-26 16:36:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-07-26 15:10:00"},{"act":"测试测试测试测试测试测试","date":"2012-07-20 07:58:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-07-18 13:55:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-07-16 15:32:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-07-05 14:30:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-07-05 07:39:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-06-25 13:25:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-06-21 09:11:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-06-21 09:09:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-06-20 08:24:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-06-19 10:00:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-06-18 14:27:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-06-16 09:30:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-06-14 07:54:00"},{"act":"测试测试测试测试测试测试的","date":"2012-06-13 09:55:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-06-06 08:56:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-06-05 07:33:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-06-04 10:16:00"},{"act":"测试测试测试测试测试测试测试测试测试测试,测试测试测试测试测试测试的","date":"2012-06-01 12:02:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-06-01 08:58:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-05-30 14:00:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-05-28 08:25:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-05-26 09:40:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-05-25 10:02:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-05-25 10:01:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-05-24 08:19:00"},{"act":"测试测试测试测试测试测试测试测试测试测试,测试测试测试测试测试测试的","date":"2012-05-23 12:34:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-05-23 12:33:00"},{"act":"测试测试测试度、测试、测试测试测试的","date":"2012-05-21 10:34:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-05-21 09:50:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-05-21 09:48:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-05-17 11:04:00"},{"act":"测试测试测试测试测试测试测试测试测试测试的","date":"2012-05-16 16:11:00"},{"act":"测试测试测试测试测试测试","date":"2012-05-16 07:57:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-05-15 14:54:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-05-15 14:53:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-05-15 08:59:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-05-15 08:57:00"},{"act":"测试测试测试测试测试测试的","date":"2012-05-15 07:33:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-05-14 10:16:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-05-14 10:14:00"},{"act":"测试测试测试测试测试测试测试测试测试测试的","date":"2012-05-14 09:52:00"},{"act":"测试测试测试度、测试、测试测试测试的","date":"2012-05-11 11:09:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-05-08 10:53:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-05-08 10:51:00"},{"act":"测试测试测试度、测试、测试测试测试的","date":"2012-05-08 08:53:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-05-07 07:55:00"},{"act":"测试测试测试测试测试测试","date":"2012-05-07 07:49:00"},{"act":"测试测试测试度、测试、测试测试测试的","date":"2012-05-04 12:47:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-05-03 08:13:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-04-26 14:50:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-04-21 09:24:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-04-18 09:12:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-04-13 07:57:00"},{"act":"测试测试测试测试测试测试","date":"2012-04-12 07:35:00"},{"act":"测试测试测试测试测试测试的","date":"2012-04-12 07:35:00"},{"act":"测试测试测试测试测试测试测试测试测试测试测试的","date":"2012-04-11 14:55:00"},{"act":"测试测试测试测试测试测试的","date":"2012-04-01 15:14:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-03-27 10:40:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-03-26 10:26:00"},{"act":"测试测试测试测试测试测试、测试测试测试测试测试测试上,测试测试测试测试的","date":"2012-03-26 10:24:00"},{"act":"测试测试测试测试测试测试","date":"2012-03-21 13:50:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-03-17 10:20:00"},{"act":"测试测试测试测试测试测试的","date":"2012-03-16 15:13:00"},{"act":"测试测试测试测试测试测试的","date":"2012-03-14 12:02:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2012-03-13 09:40:00"},{"act":"测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试","date":"2012-03-13 09:36:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-03-10 10:53:00"},{"act":"测试测试测试测试测试测试","date":"2012-02-27 10:33:00"},{"act":"测试测试测试测试测试测试的","date":"2012-02-23 15:32:00"},{"act":"测试测试测试测试测试测试测试30%的","date":"2012-02-20 16:55:00"},{"act":"测试测试测试测试测试测试","date":"2012-02-06 09:30:00"},{"act":"测试测试测试测试测试测试的","date":"2012-02-03 10:32:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-02-03 10:32:00"},{"act":"测试测试测试测试测试测试的","date":"2012-02-01 09:40:00"},{"act":"测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的","date":"2012-01-17 20:22:14"},{"act":"测试测试测试测试测试测试的","date":"2012-01-13 14:20:00"},{"act":"测试测试测试测试测试测试的","date":"2012-01-12 08:20:00"},{"act":"测试测试测试测试测试测试测试测试","date":"2012-01-09 14:50:00"},{"act":"测试门、测试测试测试测试测试","date":"2012-01-06 09:20:00"}],"rated_power":208.875,"vehicle_age":8,"vehicle_type":"测试车","vehicle_class":"测试","vehicle_type_4":"测试、测试","plate_type_name":"测试测试","manufacture_date":"2011-03-17 00:00:00","pol_ind_combined":"测试户","vehicle_capacity":33.1,"vehicle_seat_number":2} +1 {"id": 1, "test0": "测试", "test4": "测试", "test8": "测试", "tes736": "测试", "test12": "测试", "test16": "测试", "test20": "测试", "test24": "测试", "test28": "测试", "test32": "测试", "test36": "测试", "test40": "测试", "test44": "测试", "test48": "测试", "test52": "测试", "test56": "测试", "test60": "测试", "test64": "测试", "test68": "测试", "test72": "测试", "test76": "测试", "test80": "测试", "test84": "测试", "test88": "测试", "test92": "测试", "test96": "测试", "test100": "测试", "test104": "测试", "test108": "测试", "test112": "测试", "test116": "测试", "test120": "测试", "test124": "测试", "test128": "测试", "test132": "测试", "test136": "测试", "test140": "测试", "test144": "测试", "test148": "测试", "test152": "测试", "test156": "测试", "test160": "测试", "test164": "测试", "test168": "测试", "test172": "测试", "test176": "测试", "test180": "测试", "test184": "测试", "test188": "测试", "test192": "测试", "test196": "测试", "test200": "测试", "test204": "测试", "test208": "测试", "test212": "测试", "test216": "测试", "test220": "测试", "test224": "测试", "test228": "测试", "test232": "测试", "test236": "测试", "test240": "测试", "test244": "测试", "test248": "测试", "test252": "测试", "test256": "测试", "test260": "测试", "test264": "测试", "test268": "测试", "test272": "测试", "test276": "测试", "test280": "测试", "test284": "测试", "test288": "测试", "test292": "测试", "test296": "测试", "test300": "测试", "test304": "测试", "test308": "测试", "test312": "测试", "test316": "测试", "test320": "测试", "test324": "测试", "test328": "测试", "test332": "测试", "test336": "测试", "test340": "测试", "test344": "测试", "test348": "测试", "test352": "测试", "test356": "测试", "test360": "测试", "test364": "测试", "test368": "测试", "test372": "测试", "test376": "测试", "test380": "测试", "test384": "测试", "test388": "测试", "test392": "测试", "test396": "测试", "test400": "测试", "test404": "测试", "test408": "测试", "test412": "测试", "test416": "测试", "test420": "测试", "test424": "测试", "test428": "测试", "test432": "测试", "test436": "测试", "test440": "测试", "test444": "测试", "test448": "测试", "test452": "测试", "test456": "测试", "test460": "测试", "test464": "测试", "test468": "测试", "test472": "测试", "test476": "测试", "test480": "测试", "test484": "测试", "test488": "测试", "test492": "测试", "test496": "测试", "test500": "测试", "test504": "测试", "test508": "测试", "test512": "测试", "test516": "测试", "test520": "测试", "test524": "测试", "test528": "测试", "test532": "测试", "test536": "测试", "test540": "测试", "test544": "测试", "test548": "测试", "test552": "测试", "test556": "测试", "test560": "测试", "test564": "测试", "test568": "测试", "test572": "测试", "test576": "测试", "test580": "测试", "test584": "测试", "test588": "测试", "test592": "测试", "test596": "测试", "test600": "测试", "test604": "测试", "test608": "测试", "test612": "测试", "test616": "测试", "test620": "测试", "test624": "测试", "test628": "测试", "test632": "测试", "test636": "测试", "test640": "测试", "test644": "测试", "test648": "测试", "test652": "测试", "test656": "测试", "test660": "测试", "test664": "测试", "test668": "测试", "test672": "测试", "test676": "测试", "test680": "测试", "test684": "测试", "test688": "测试", "test692": "测试", "test696": "测试", "test700": "测试", "test704": "测试", "test708": "测试", "test712": "测试", "test716": "测试", "test720": "测试", "test724": "测试", "test728": "测试", "test732": "测试", "test740": "测试", "test744": "测试", "test748": "测试", "test752": "测试", "test756": "测试", "test760": "测试", "test764": "测试", "test768": "测试", "test772": "测试", "test776": "测试", "test780": "测试", "test784": "测试", "test788": "测试", "test792": "测试", "test796": "测试", "test800": "测试", "test804": "测试", "test808": "测试", "test812": "测试", "test816": "测试", "test820": "测试", "test824": "测试", "test828": "测试", "test832": "测试", "test836": "测试", "test840": "测试", "test844": "测试", "test848": "测试", "test852": "测试", "test856": "测试", "test860": "测试", "test864": "测试", "test868": "测试", "test872": "测试", "test876": "测试", "test880": "测试", "test884": "测试", "test888": "测试", "test892": "测试", "test896": "测试", "test900": "测试", "test904": "测试", "test908": "测试", "test912": "测试", "test916": "测试", "test920": "测试", "test924": "测试", "test928": "测试", "test932": "测试", "test936": "测试", "test940": "测试", "test944": "测试", "test948": "测试", "test952": "测试", "test956": "测试", "test960": "测试", "test964": "测试", "test968": "测试", "test972": "测试", "test976": "测试", "test980": "测试", "test984": "测试", "test988": "测试", "test992": "测试", "test996": "测试", "test1000": "测试", "test1004": "测试", "test1008": "测试", "test1012": "测试", "test1016": "测试", "test1020": "测试", "test1024": "测试", "test1028": "测试", "test1032": "测试", "test1036": "测试", "test1040": "测试", "test1044": "测试", "test1048": "测试", "test1052": "测试", "test1056": "测试", "test1060": "测试", "test1064": "测试", "test1068": "测试", "test1072": "测试", "test1076": "测试", "test1080": "测试", "test1084": "测试", "test1088": "测试", "test1092": "测试", "test1096": "测试", "test1100": "测试", "test1104": "测试", "test1108": "测试", "test1110": "测试", "test1112": "测试", "test1116": "测试", "test1120": "测试", "test1124": "测试", "test1128": "测试", "test1132": "测试", "test1136": "测试", "test1140": "测试", "test1144": "测试", "test1148": "测试", "test1152": "测试", "test1156": "测试", "test1160": "测试", "test1164": "测试", "test1168": "测试", "test1172": "测试", "test1176": "测试", "test1180": "测试", "test1184": "测试", "test1188": "测试", "test1192": "测试", "test1196": "测试", "test1200": "测试", "test1204": "测试", "test1208": "测试", "test1212": "测试", "test1216": "测试", "test1220": "测试", "test1224": "测试", "test1228": "测试", "test1232": "测试", "test1236": "测试", "test1240": "测试", "test1244": "测试", "test1248": "测试", "test1252": "测试", "test1256": "测试", "test1260": "测试", "test1264": "测试", "test1268": "测试", "test1272": "测试", "test1276": "测试", "test1280": "测试", "test1284": "测试", "test1288": "测试", "test1292": "测试", "test1296": "测试", "test1300": "测试", "test1304": "测试", "test1308": "测试", "test1312": "测试", "test1316": "测试", "test1320": "测试", "test1324": "测试", "test1328": "测试", "test1332": "测试", "test1336": "测试", "test1340": "测试", "test1344": "测试", "test1348": "测试", "test1352": "测试", "test1356": "测试", "test1360": "测试", "test1364": "测试", "test1368": "测试", "test1372": "测试", "test1376": "测试", "test1380": "测试", "test1384": "测试", "test1388": "测试", "test1392": "测试", "test1396": "测试", "test1400": "测试", "test1404": "测试", "test1408": "测试", "test1412": "测试", "test1416": "测试", "test1420": "测试", "test1424": "测试", "test1428": "测试", "test1432": "测试", "test1436": "测试", "test1440": "测试", "test1444": "测试", "test1448": "测试", "test1452": "测试", "test1456": "测试", "test1460": "测试", "test1464": "测试", "test1468": "测试", "test1472": "测试", "test1476": "测试", "test1480": "测试", "test1484": "测试", "test1488": "测试", "test1492": "测试", "test1496": "测试", "test1500": "测试", "test1504": "测试", "test1508": "测试", "test1512": "测试", "test1516": "测试", "test1520": "测试", "test1524": "测试", "test1528": "测试", "test1532": "测试", "test1536": "测试", "test1540": "测试", "test1544": "测试", "test1548": "测试", "test1552": "测试", "test1556": "测试", "test1560": "测试", "test1564": "测试", "test1568": "测试", "test1572": "测试", "test1576": "测试", "test1580": "测试", "test1584": "测试", "test1588": "测试", "test1592": "测试", "test1596": "测试", "test1600": "测试", "test1604": "测试", "test1608": "测试", "test1612": "测试", "test1616": "测试", "test1620": "测试", "test1624": "测试", "test1628": "测试", "test1632": "测试", "test1636": "测试", "test1640": "测试", "test1644": "测试", "test1648": "测试", "test1652": "测试", "test1656": "测试", "test1660": "测试", "test1664": "测试", "test1668": "测试", "test1672": "测试", "test1676": "测试", "test1680": "测试", "test1684": "测试", "test1688": "测试", "test1692": "测试", "test1696": "测试", "test1700": "测试", "test1704": "测试", "test1708": "测试", "test1712": "测试", "test1716": "测试", "test1720": "测试", "test1724": "测试", "test1728": "测试", "test1732": "测试", "test1740": "测试", "test1744": "测试", "test1748": "测试", "test1752": "测试", "test1756": "测试", "test1760": "测试", "test1764": "测试", "test1768": "测试", "test1772": "测试", "test1776": "测试", "test1780": "测试", "test1784": "测试", "test1788": "测试", "test1792": "测试", "test1796": "测试", "test1800": "测试", "test1804": "测试", "test1808": "测试", "test1812": "测试", "test1816": "测试", "test1820": "测试", "test1824": "测试", "test1828": "测试", "test1832": "测试", "test1836": "测试", "test1840": "测试", "test1844": "测试", "test1848": "测试", "test1852": "测试", "test1856": "测试", "test1860": "测试", "test1864": "测试", "test1868": "测试", "test1872": "测试", "test1876": "测试", "test1880": "测试", "test1884": "测试", "test1888": "测试", "test1892": "测试", "test1896": "测试", "test1900": "测试", "test1904": "测试", "test1908": "测试", "test1912": "测试", "test1916": "测试", "test1920": "测试", "test1924": "测试", "test1928": "测试", "test1932": "测试", "test1936": "测试", "test1940": "测试", "test1944": "测试", "test1948": "测试", "test1952": "测试", "test1956": "测试", "test1960": "测试", "test1964": "测试", "test1968": "测试", "test1972": "测试", "test1976": "测试", "test1980": "测试", "test1984": "测试", "test1988": "测试", "test1992": "测试", "test1996": "测试", "test2000": "测试", "test2004": "测试", "test2008": "测试", "test2012": "测试", "test2016": "测试", "test2020": "测试", "test2024": "测试", "test2028": "测试", "test2032": "测试", "test2036": "测试", "test2040": "测试", "test2044": "测试", "test2048": "测试", "test2052": "测试", "test2056": "测试", "test2060": "测试", "test2064": "测试", "test2068": "测试", "test2072": "测试", "test2076": "测试", "test2080": "测试", "test2084": "测试", "test2088": "测试", "test2092": "测试", "test2096": "测试", "test2100": "测试", "test2104": "测试", "test2108": "测试", "test2110": "测试", "test2112": "测试", "test2116": "测试", "test2120": "测试", "test2124": "测试", "test2128": "测试", "test2132": "测试", "test2136": "测试", "test2140": "测试", "test2144": "测试", "test2148": "测试", "test2152": "测试", "test2156": "测试", "test2160": "测试", "test2164": "测试", "test2168": "测试", "test2172": "测试", "test2176": "测试", "test2180": "测试", "test2184": "测试", "test2188": "测试", "test2192": "测试", "test2196": "测试", "test2200": "测试", "test2204": "测试", "test2208": "测试", "test2212": "测试", "test2216": "测试", "test2220": "测试", "test2224": "测试", "test2228": "测试", "test2232": "测试", "test2236": "测试", "test2240": "测试", "test2244": "测试", "test2248": "测试", "test2252": "测试", "test2256": "测试", "test2260": "测试", "test2264": "测试", "test2268": "测试", "test2272": "测试", "test2276": "测试", "test2280": "测试", "test2284": "测试", "test2288": "测试", "test2292": "测试", "test2296": "测试", "test2300": "测试", "test2304": "测试", "test2308": "测试", "test2312": "测试", "test2316": "测试", "test2320": "测试", "test2324": "测试", "test2328": "测试", "test2332": "测试", "test2336": "测试", "test2340": "测试", "test2344": "测试", "test2348": "测试", "test2352": "测试", "test2356": "测试", "test2360": "测试", "test2364": "测试", "test2368": "测试", "test2372": "测试", "test2376": "测试", "test2380": "测试", "test2384": "测试", "test2388": "测试", "test2392": "测试", "test2396": "测试", "test2400": "测试", "test2404": "测试", "test2408": "测试", "test2412": "测试", "test2416": "测试", "test2420": "测试", "test2424": "测试", "test2428": "测试", "test2432": "测试", "test2436": "测试", "test2440": "测试", "test2444": "测试", "test2448": "测试", "test2452": "测试", "test2456": "测试", "test2460": "测试", "test2464": "测试", "test2468": "测试", "test2472": "测试", "test2476": "测试", "test2480": "测试", "test2484": "测试", "test2488": "测试", "test2492": "测试", "test2496": "测试", "test2500": "测试", "test2504": "测试", "test2508": "测试", "test2512": "测试", "test2516": "测试", "test2520": "测试", "test2524": "测试", "test2528": "测试", "test2532": "测试", "test2536": "测试", "test2540": "测试", "test2544": "测试", "test2548": "测试", "test2552": "测试", "test2556": "测试", "test2560": "测试", "test2564": "测试", "test2568": "测试", "test2572": "测试", "test2576": "测试", "test2580": "测试", "test2584": "测试", "test2588": "测试", "test2592": "测试", "test2596": "测试", "test2600": "测试", "test2604": "测试", "test2608": "测试", "test2612": "测试", "test2616": "测试", "test2620": "测试", "test2624": "测试", "test2628": "测试", "test2632": "测试", "test2636": "测试", "test2640": "测试", "test2644": "测试", "test2648": "测试", "test2652": "测试", "test2656": "测试", "test2660": "测试", "test2664": "测试", "test2668": "测试", "test2672": "测试", "test2676": "测试", "test2680": "测试", "test2684": "测试", "test2688": "测试", "test2692": "测试", "test2696": "测试", "test2700": "测试", "test2704": "测试", "test2708": "测试", "test2712": "测试", "test2716": "测试", "test2720": "测试", "test2724": "测试", "test2728": "测试", "test2732": "测试", "test2740": "测试", "test2744": "测试", "test2748": "测试", "test2752": "测试", "test2756": "测试", "test2760": "测试", "test2764": "测试", "test2768": "测试", "test2772": "测试", "test2776": "测试", "test2780": "测试", "test2784": "测试", "test2788": "测试", "test2792": "测试", "test2796": "测试", "test2800": "测试", "test2804": "测试", "test2808": "测试", "test2812": "测试", "test2816": "测试", "test2820": "测试", "test2824": "测试", "test2828": "测试", "test2832": "测试", "test2836": "测试", "test2840": "测试", "test2844": "测试", "test2848": "测试", "test2852": "测试", "test2856": "测试", "test2860": "测试", "test2864": "测试", "test2868": "测试", "test2872": "测试", "test2876": "测试", "test2880": "测试", "test2884": "测试", "test2888": "测试", "test2892": "测试", "test2896": "测试", "test2900": "测试", "test2904": "测试", "test2908": "测试", "test2912": "测试", "test2916": "测试", "test2920": "测试", "test2924": "测试", "test2928": "测试", "test2932": "测试", "test2936": "测试", "test2940": "测试", "test2944": "测试", "test2948": "测试", "test2952": "测试", "test2956": "测试", "test2960": "测试", "test2964": "测试", "test2968": "测试", "test2972": "测试", "test2976": "测试", "test2980": "测试", "test2984": "测试", "test2988": "测试", "test2992": "测试", "test2996": "测试", "test3000": "测试", "test3004": "测试", "test3008": "测试", "test3012": "测试", "test3016": "测试", "test3020": "测试", "test3024": "测试", "test3028": "测试", "test3032": "测试", "test3036": "测试", "test3040": "测试", "test3044": "测试", "test3048": "测试", "test3052": "测试", "test3056": "测试", "test3060": "测试", "test3064": "测试", "test3068": "测试", "test3072": "测试", "test3076": "测试", "test3080": "测试", "test3084": "测试", "test3088": "测试", "test3092": "测试", "test3096": "测试", "test3100": "测试", "test3104": "测试", "test3108": "测试", "test3110": "测试", "test3112": "测试", "test3116": "测试", "test3120": "测试", "test3124": "测试", "test3128": "测试", "test3132": "测试", "test3136": "测试", "test3140": "测试", "test3144": "测试", "test3148": "测试", "test3152": "测试", "test3156": "测试", "test3160": "测试", "test3164": "测试", "test3168": "测试", "test3172": "测试", "test3176": "测试", "test3180": "测试", "test3184": "测试", "test3188": "测试", "test3192": "测试", "test3196": "测试", "test3200": "测试", "test3204": "测试", "test3208": "测试", "test3212": "测试", "test3216": "测试", "test3220": "测试", "test3224": "测试", "test3228": "测试", "test3232": "测试", "test3236": "测试", "test3240": "测试", "test3244": "测试", "test3248": "测试", "test3252": "测试", "test3256": "测试", "test3260": "测试", "test3264": "测试", "test3268": "测试", "test3272": "测试", "test3276": "测试", "test3280": "测试", "test3284": "测试", "test3288": "测试", "test3292": "测试", "test3296": "测试", "test3300": "测试", "test3304": "测试", "test3308": "测试", "test3312": "测试", "test3316": "测试", "test3320": "测试", "test3324": "测试", "test3328": "测试", "test3332": "测试", "test3336": "测试", "test3340": "测试", "test3344": "测试", "test3348": "测试", "test3352": "测试", "test3356": "测试", "test3360": "测试", "test3364": "测试", "test3368": "测试", "test3372": "测试", "test3376": "测试", "test3380": "测试", "test3384": "测试", "test3388": "测试", "test3392": "测试", "test3396": "测试", "test3400": "测试", "test3404": "测试", "test3408": "测试", "test3412": "测试", "test3416": "测试", "test3420": "测试", "test3424": "测试", "test3428": "测试", "test3432": "测试", "test3436": "测试", "test3440": "测试", "test3444": "测试", "test3448": "测试", "test3452": "测试", "test3456": "测试", "test3460": "测试", "test3464": "测试", "test3468": "测试", "test3472": "测试", "test3476": "测试", "test3480": "测试", "test3484": "测试", "test3488": "测试", "test3492": "测试", "test3496": "测试", "test3500": "测试", "test3504": "测试", "test3508": "测试", "test3512": "测试", "test3516": "测试", "test3520": "测试", "test3524": "测试", "test3528": "测试", "test3532": "测试", "test3536": "测试", "test3540": "测试", "test3544": "测试", "test3548": "测试", "test3552": "测试", "test3556": "测c试", "test3560": "测试", "test3564": "测试", "test3568": "测试", "test3572": "测试", "test3576": "测试", "test3580": "测试", "test3584": "测试", "test3588": "测试", "test3592": "测试", "test3596": "测试", "test3600": "测试", "test3604": "测试", "test3608": "测试", "test3612": "测试", "test3616": "测试", "test3620": "测试", "test3624": "测试", "test3628": "测试", "test3632": "测试", "test3636": "测试", "test3640": "测试", "test3644": "测试", "test3648": "测试", "test3652": "测试", "test3656": "测试", "test3660": "测试", "test3664": "测试", "test3668": "测试", "test3672": "测试", "test3676": "测试", "test3680": "测试", "test3684": "测试", "test3688": "测试", "test3692": "测试", "test3696": "测试", "test3700": "测试", "test3704": "测试", "test3708": "测试", "test3712": "测试", "test3716": "测试", "test3720": "测试", "test3724": "测试", "test3728": "测试", "test3732": "测试", "test3740": "测试", "test3744": "测试", "test3748": "测试", "test3752": "测试", "test3756": "测试", "test3760": "测试", "test3764": "测试", "test3768": "测试", "test3772": "测试", "test3776": "测试", "test3780": "测试", "test3784": "测试", "test3788": "测试", "test3792": "测试", "test3796": "测试", "test3800": "测试", "test3804": "测试", "test3808": "测试", "test3812": "测试", "test3816": "测试", "test3820": "测试", "test3824": "测试", "test3828": "测试", "test3832": "测试", "test3836": "测试", "test3840": "测试", "test3844": "测试", "test3848": "测试", "test3852": "测试", "test3856": "测试", "test3860": "测试", "test3864": "测试", "test3868": "测试", "test3872": "测试", "test3876": "测试", "test3880": "测试", "test3884": "测试", "test3888": "测试", "test3892": "测试", "test3896": "测试", "test3900": "测试", "test3904": "测试", "test3908": "测试", "test3912": "测试", "test3916": "测试", "test3920": "测试", "test3924": "测试", "test3928": "测试", "test3932": "测试", "test3936": "测试", "test3940": "测试", "test3944": "测试", "test3948": "测试", "test3952": "测试", "test3956": "测试", "test3960": "测试", "test3964": "测试", "test3968": "测试", "test3972": "测试", "test3976": "测试", "test3980": "测试", "test3984": "测试", "test3992": "测试", "test3996": "测试", "test4000": "测试", "test4004": "测试", "test4008": "测试", "test4012": "测试", "test4016": "测试", "test4020": "测试", "test4024": "测试", "test4028": "测试", "test4032": "测试", "test4036": "测试", "test4040": "测试", "test4044": "测试", "test4048": "测试", "test4052": "测试", "test4056": "测试", "test4060": "测试", "test4064": "测试", "test4068": "测试", "test4072": "测试", "test4076": "测试", "test4080": "测试", "test4084": "测试", "test4088": "测试", "test4092": "测试", "test4096": "测试", "test4100": "测试", "test4104": "测试", "test4108": "测试", "test4110": "测试", "test4112": "测试", "test4116": "测试", "test4120": "测试", "test4124": "测试", "test4128": "测试", "test4132": "测试", "test4136": "测试", "test4140": "测试", "test4144": "测试", "test4148": "测试", "test4152": "测试", "test4156": "测试", "test4160": "测试", "test4164": "测试", "test4168": "测试", "test4172": "测试", "test4176": "测试", "test4180": "测试", "test4184": "测试", "test4188": "测试", "test4192": "测试", "test4196": "测试", "test4200": "测试", "test4204": "测试", "test4208": "测试", "test4212": "测试", "test4216": "测试", "test4220": "测试", "test4224": "测试", "test4228": "测试", "test4232": "测试", "test4236": "测试", "test4240": "测试", "test4244": "测试", "test4248": "测试", "test4252": "测试", "test4256": "测试", "test4260": "测试", "test4264": "测试", "test4268": "测试", "test4272": "测试", "test4276": "测试", "test4280": "测试", "test4284": "测试", "test4288": "测试", "test4292": "测试", "test4296": "测试", "test4300": "测试", "test4304": "测试", "test4308": "测试", "test4312": "测试", "test4316": "测试", "test4320": "测试", "test4324": "测试", "test4328": "测试", "test4332": "测试", "test4336": "测试", "test4340": "测试", "test4344": "测试", "test4348": "测试", "test4352": "测试", "test4356": "测试", "test4360": "测试", "test4364": "测试", "test4368": "测试", "test4372": "测试", "test4376": "测试", "test4380": "测试", "test4384": "测试", "test4388": "测试", "test4392": "测试", "test4396": "测试", "test4400": "测试", "test4404": "测试", "test4408": "测试", "test4412": "测试", "test4416": "测试", "test4420": "测试", "test4424": "测试", "test4428": "测试", "test4432": "测试", "test4436": "测试", "test4440": "测试", "test4444": "测试", "test4448": "测试", "test4452": "测试", "test4456": "测试", "test4460": "测试", "test4464": "测试", "test4468": "测试", "test4472": "测试", "test4476": "测试", "test4480": "测试", "test4484": "测试", "test4488": "测试", "test4492": "测试", "test4496": "测试", "test4500": "测试", "test4504": "测试", "test4508": "测试", "test4512": "测试", "test4516": "测试", "test4520": "测试", "test4524": "测试", "test4528": "测试", "test4532": "测试", "test4536": "测试", "test4540": "测试", "test4544": "测试", "test4548": "测试", "test4552": "测试", "test4556": "测试", "test4560": "测试", "test4564": "测试", "test4568": "测试", "test4572": "测试", "test4576": "测试", "test4580": "测试", "test4584": "测试", "test4588": "测试", "test4592": "测试", "test4596": "测试", "test4600": "测试", "test4604": "测试", "test4608": "测试", "test4612": "测试", "test4616": "测试", "test4620": "测试", "test4624": "测试", "test4628": "测试", "test4632": "测试", "test4636": "测试", "test4640": "测试", "test4644": "测试", "test4648": "测试", "test4652": "测试", "test4656": "测试", "test4660": "测试", "test4664": "测试", "test4668": "测试", "test4672": "测试", "test4676": "测试", "test4680": "测试", "test4684": "测试", "test4688": "测试", "test4692": "测试", "test4696": "测试", "test4700": "测试", "test4704": "测试", "test4708": "测试", "test4712": "测试", "test4716": "测试", "test4720": "测试", "test4724": "测试", "test4728": "测试", "test4732": "测试", "test4740": "测试", "test4744": "测试", "test4748": "测试", "test4752": "测试", "test4756": "测试", "test4760": "测试", "test4764": "测试", "test4768": "测试", "test4772": "测试", "test4776": "测试", "test4780": "测试", "test4784": "测试", "test4788": "测试", "test4792": "测试", "test4796": "测试", "test4800": "测试", "test4804": "测试", "test4808": "测试", "test4812": "测试", "test4816": "测试", "test4820": "测试", "test4824": "测试", "test4828": "测试", "test4832": "测试", "test4836": "测试", "test4840": "测试", "test4844": "测试", "test4848": "测试", "test4852": "测试", "test4856": "测试", "test4860": "测试", "test4864": "测试", "test4868": "测试", "test4872": "测试", "test4876": "测试", "test4880": "测试", "test4884": "测试", "test4888": "测试", "test4892": "测试", "test4896": "测试", "test4900": "测试", "test4904": "测试", "test4908": "测试", "test4912": "测试", "test4916": "测试", "test4920": "测试", "test4924": "测试", "test4928": "测试", "test4932": "测试", "test4936": "测试", "test4940": "测试", "test4944": "测试", "test4948": "测试", "test4952": "测试", "test4956": "测试", "test4960": "测试", "test4964": "测试", "test4968": "测试", "test4972": "测试", "test4976": "测试", "test4980": "测试", "test4984": "测试", "test4988": "测试", "test4992": "测试", "test4996": "测试", "test5000": "测试", "test5004": "测试", "test5008": "测试", "test5012": "测试", "test5016": "测试", "test5020": "测试", "test5024": "测试", "test5028": "测试", "test5032": "测试", "test5036": "测试", "test5040": "测试", "test5044": "测试", "test5048": "测试", "test5052": "测试", "test5056": "测试", "test5060": "测试", "test5064": "测试", "test5068": "测试", "test5072": "测试", "test5076": "测试", "test5080": "测试", "test5084": "测试", "test5088": "测试", "test5092": "测试", "test5096": "测试", "test5100": "测试", "test5104": "测试", "test5108": "测试", "test5110": "测试", "test5112": "测试", "test5116": "测试", "test5120": "测试", "test5124": "测试", "test5128": "测试", "test5132": "测试", "test5136": "测试", "test5140": "测试", "test5144": "测试", "test5148": "测试", "test5152": "测试", "test5156": "测试", "test5160": "测试", "test5164": "测试", "test5168": "测试", "test5172": "测试", "test5176": "测试", "test5180": "测试", "test5184": "测试", "test5188": "测试", "test5192": "测试", "test5196": "测试", "test5200": "测试", "test5204": "测试", "test5208": "测试", "test5212": "测试", "test5216": "测试", "test5220": "测试", "test5224": "测试", "test5228": "测试", "test5232": "测试", "test5236": "测试", "test5240": "测试", "test5244": "测试", "test5248": "测试", "test5252": "测试", "test5256": "测试", "test5260": "测试", "test5264": "测试", "test5268": "测试", "test5272": "测试", "test5276": "测试", "test5280": "测试", "test5284": "测试", "test5288": "测试", "test5292": "测试", "test5296": "测试", "test5300": "测试", "test5304": "测试", "test5308": "测试", "test5312": "测试", "test5316": "测试", "test5320": "测试", "test5324": "测试", "test5328": "测试", "test5332": "测试", "test5336": "测试", "test5340": "测试", "test5344": "测试", "test5348": "测试", "test5352": "测试", "test5356": "测试", "test5360": "测试", "test5364": "测试", "test5368": "测试", "test5372": "测试", "test5376": "测试", "test5380": "测试", "test5384": "测试", "test5388": "测试", "test5392": "测试", "test5396": "测试", "test5400": "测试", "test5404": "测试", "test5408": "测试", "test5412": "测试", "test5416": "测试", "test5420": "测试", "test5424": "测试", "test5428": "测试", "test5432": "测试", "test5436": "测试", "test5440": "测试", "test5444": "测试", "test5448": "测试", "test5452": "测试", "test5456": "测试", "test5460": "测试", "test5464": "测试", "test5468": "测试", "test5472": "测试", "test5476": "测试", "test5480": "测试", "test5484": "测试", "test5488": "测试", "test5492": "测试", "test5496": "测试", "test5500": "测试", "test5504": "测试", "test5508": "测试", "test5512": "测试", "test5516": "测试", "test5520": "测试", "test5524": "测试", "test5528": "测试", "test5532": "测试", "test5536": "测试", "test5540": "测试", "test5544": "测试", "test5548": "测试", "test5552": "测试", "test5556": "测试", "test5560": "测试", "test5564": "测试", "test5568": "测试", "test5572": "测试", "test5576": "测试", "test5580": "测试", "test5584": "测试", "test5588": "测试", "test5592": "测试", "test5596": "测试", "test5600": "测试", "test5604": "测试", "test5608": "测试", "test5612": "测试", "test5616": "测试", "test5620": "测试", "test5624": "测试", "test5628": "测试", "test5632": "测试", "test5636": "测试", "test5640": "测试", "test5644": "测试", "test5648": "测试", "test5652": "测试", "test5656": "测试", "test5660": "测试", "test5664": "测试", "test5668": "测试", "test5672": "测试", "test5676": "测试", "test5680": "测试", "test5684": "测试", "test5688": "测试", "test5692": "测试", "test5696": "测试", "test5700": "测试", "test5704": "测试", "test5708": "测试", "test5712": "测试", "test5716": "测试", "test5720": "测试", "test5724": "测试", "test5728": "测试", "test5732": "测试", "test5740": "测试", "test5744": "测试", "test5748": "测试", "test5752": "测试", "test5756": "测试", "test5760": "测试", "test5764": "测试", "test5768": "测试", "test5772": "测试", "test5776": "测试", "test5780": "测试", "test5784": "测试", "test5788": "测试", "test5792": "测试", "test5796": "测试", "test5800": "测试", "test5804": "测试", "test5808": "测试", "test5812": "测试", "test5816": "测试", "test5820": "测试", "test5824": "测试", "test5828": "测试", "test5832": "测试", "test5836": "测试", "test5840": "测试", "test5844": "测试", "test5848": "测试", "test5852": "测试", "test5856": "测试", "test5860": "测试", "test5864": "测试", "test5868": "测试", "test5872": "测试", "test5876": "测试", "test5880": "测试", "test5884": "测试", "test5888": "测试", "test5892": "测试", "test5896": "测试", "test5900": "测试", "test5904": "测试", "test5908": "测试", "test5912": "测试", "test5916": "测试", "test5920": "测试", "test5924": "测试", "test5928": "测试", "test5932": "测试", "test5936": "测试", "test5940": "测试", "test5944": "测试", "test5948": "测试", "test5952": "测试", "test5956": "测试", "test5960": "测试", "test5964": "测试", "test5968": "测试", "test5972": "测试", "test5976": "测试", "test5980": "测试", "test5984": "测试", "test5988": "测试", "test5992": "测试", "test5996": "测试", "test6000": "测试", "test6004": "测试", "test6008": "测试", "test6012": "测试", "test6016": "测试", "test6020": "测试", "test6024": "测试", "test6028": "测试", "test6032": "测试", "test6036": "测试", "test6040": "测试", "test6044": "测试", "test6048": "测试", "test6052": "测试", "test6056": "测试", "test6060": "测试", "test6064": "测试", "test6068": "测试", "test6072": "测试", "test6076": "测试", "test6080": "测试", "test6084": "测试", "test6088": "测试", "test6092": "测试", "test6096": "测试", "test6100": "测试", "test6104": "测试", "test6108": "测试", "test6110": "测试", "test6112": "测试", "test6116": "测试", "test6120": "测试", "test6124": "测试", "test6128": "测试", "test6132": "测试", "test6136": "测试", "test6140": "测试", "test6144": "测试", "test6148": "测试", "test6152": "测试", "test6156": "测试", "test6160": "测试", "test6164": "测试", "test6168": "测试", "test6172": "测试", "test6176": "测试", "test6180": "测试", "test6184": "测试", "test6188": "测试", "test6192": "测试", "test6196": "测试", "test6200": "测试", "test6204": "测试", "test6208": "测试", "test6212": "测试", "test6216": "测试", "test6220": "测试", "test6224": "测试", "test6228": "测试", "test6232": "测试", "test6236": "测试", "test6240": "测试", "test6244": "测试", "test6248": "测试", "test6252": "测试", "test6256": "测试", "test6260": "测试", "test6264": "测试", "test6268": "测试", "test6272": "测试", "test6276": "测试", "test6280": "测试", "test6284": "测试", "test6288": "测试", "test6292": "测试", "test6296": "测试", "test6300": "测试", "test6304": "测试", "test6308": "测试", "test6312": "测试", "test6316": "测试", "test6320": "测试", "test6324": "测试", "test6328": "测试", "test6332": "测试", "test6336": "测试", "test6340": "测试", "test6344": "测试", "test6348": "测试", "test6352": "测试", "test6356": "测试", "test6360": "测试", "test6364": "测试", "test6368": "测试", "test6372": "测试", "test6376": "测试", "test6380": "测试", "test6384": "测试", "test6388": "测试", "test6392": "测试", "test6396": "测试", "test6400": "测试", "test6404": "测试", "test6408": "测试", "test6412": "测试", "test6416": "测试", "test6420": "测试", "test6424": "测试", "test6428": "测试", "test6432": "测试", "test6436": "测试", "test6440": "测试", "test6444": "测试", "test6448": "测试", "test6452": "测试", "test6456": "测试", "test6460": "测试", "test6464": "测试", "test6468": "测试", "test6472": "测试", "test6476": "测试", "test6480": "测试", "test6484": "测试", "test6488": "测试", "test6492": "测试", "test6496": "测试", "test6500": "测试", "test6504": "测试", "test6508": "测试", "test6512": "测试", "test6516": "测试", "test6520": "测试", "test6524": "测试", "test6528": "测试", "test6532": "测试", "test6536": "测试", "test6540": "测试", "test6544": "测试", "test6548": "测试", "test6552": "测试", "test6556": "测试", "test6560": "测试", "test6564": "测试", "test6568": "测试", "test6572": "测试", "test6576": "测试", "test6580": "测试", "test6584": "测试", "test6588": "测试", "test6592": "测试", "test6596": "测试", "test6600": "测试", "test6604": "测试", "test6608": "测试", "test6612": "测试", "test6616": "测试", "test6620": "测试", "test6624": "测试", "test6628": "测试", "test6632": "测试", "test6636": "测试", "test6640": "测试", "test6644": "测试", "test6648": "测试", "test6652": "测试", "test6656": "测试", "test6660": "测试", "test6664": "测试", "test6668": "测试", "test6672": "测试", "test6676": "测试", "test6680": "测试", "test6684": "测试", "test6688": "测试", "test6692": "测试", "test6696": "测试", "test6700": "测试", "test6704": "测试", "test6708": "测试", "test6712": "测试", "test6716": "测试", "test6720": "测试", "test6724": "测试", "test6728": "测试", "test6732": "测试", "test6740": "测试", "test6744": "测试", "test6748": "测试", "test6752": "测试", "test6756": "测试", "test6760": "测试", "test6764": "测试", "test6768": "测试", "test6772": "测试", "test6776": "测试", "test6780": "测试", "test6784": "测试", "test6788": "测试", "test6792": "测试", "test6796": "测试", "test6800": "测试", "test6804": "测试", "test6808": "测试", "test6812": "测试", "test6816": "测试", "test6820": "测试", "test6824": "测试", "test6828": "测试", "test6832": "测试", "test6836": "测试", "test6840": "测试", "test6844": "测试", "test6848": "测试", "test6852": "测试", "test6856": "测试", "test6860": "测试", "test6864": "测试", "test6868": "测试", "test6872": "测试", "test6876": "测试", "test6880": "测试", "test6884": "测试", "test6888": "测试", "test6892": "测试", "test6896": "测试", "test6900": "测试", "test6904": "测试", "test6908": "测试", "test6912": "测试", "test6916": "测试", "test6920": "测试", "test6924": "测试", "test6928": "测试", "test6932": "测试", "test6936": "测试", "test6940": "测试", "test6944": "测试", "test6948": "测试", "test6952": "测试", "test6956": "测试", "test6960": "测试", "test6964": "测试", "test6968": "测试", "test6972": "测试", "test6976": "测试", "test6980": "测试", "test6984": "测试", "test6988": "测试", "test6992": "测试", "test6996": "测试", "test7000": "测试", "test7004": "测试", "test7008": "测试", "test7012": "测试", "test7016": "测试", "test7020": "测试", "test7024": "测试", "test7028": "测试", "test7032": "测试", "test7036": "测试", "test7040": "测试", "test7044": "测试", "test7048": "测试", "test7052": "测试", "test7056": "测试", "test7060": "测试", "test7064": "测试", "test7068": "测试", "test7072": "测试", "test7076": "测试", "test7080": "测试", "test7084": "测试", "test7088": "测试", "test7092": "测试", "test7096": "测试", "test7100": "测试", "test7104": "测试", "test7108": "测试", "test7112": "测试", "test7116": "测试", "test7120": "测试", "test7124": "测试", "test7128": "测试", "test7132": "测试", "test7136": "测试", "test7140": "测试", "test7144": "测试", "test7148": "测试", "test7152": "测试", "test7156": "测试", "test7160": "测试", "test7164": "测试", "test7168": "测试", "test7172": "测试", "test7176": "测试", "test7180": "测试", "test7184": "测试", "test7188": "测试", "test7192": "测试", "test7196": "测试", "test7200": "测试", "test7204": "测试", "test7208": "测试", "test7212": "测试", "test7216": "测试", "test7220": "测试", "test7224": "测试", "test7228": "测试", "test7232": "测试", "test7236": "测试", "test7240": "测试", "test7244": "测试", "test7248": "测试", "test7252": "测试", "test7256": "测试", "test7260": "测试", "test7264": "测试", "test7268": "测试", "test7272": "测试", "test7276": "测试", "test7280": "测试", "test7284": "测试", "test7288": "测试", "test7292": "测试", "test7296": "测试", "test7300": "测试", "test7304": "测试", "test7308": "测试", "test7312": "测试", "test7316": "测试", "test7320": "测试", "test7324": "测试", "test7328": "测试", "test7332": "测试", "test7336": "测试", "test7340": "测试", "test7344": "测试", "test7348": "测试", "test7352": "测试", "test7356": "测试", "test7372": "测试", "test7376": "测试", "test7380": "测试", "test7384": "测试", "test7388": "测试", "test7392": "测试", "test7396": "测试", "test7400": "测试", "test7404": "测试", "test7408": "测试", "test7412": "测试", "test7416": "测试", "test7420": "测试", "test7424": "测试", "test7428": "测试", "test7432": "测试", "test7436": "测试", "test7440": "测试", "test7444": "测试", "test7448": "测试", "test7452": "测试", "test7456": "测试", "test7460": "测试", "test7464": "测试", "test7468": "测试", "test7472": "测试", "test7476": "测试", "test7480": "测试", "test7484": "测试", "test7488": "测试", "test7492": "测试", "test7496": "测试", "test7500": "测试", "test7504": "测试", "test7508": "测试", "test7512": "测试", "test7516": "测试", "test7520": "测试", "test7524": "测试", "test7528": "测试", "test7532": "测试", "test7536": "测试", "test7540": "测试", "test7544": "测试", "test7548": "测试", "test7552": "测试", "test7556": "测试", "test7560": "测试", "test7564": "测试", "test7568": "测试", "test7572": "测试", "test7576": "测试", "test7580": "测试", "test7584": "测试", "test7588": "测试", "test7592": "测试", "test7596": "测试"} +2 {"fuel_type": "测试", "structure": "测试", "plate_type": "01", "violations": [{"act": "测试测试测试度、测试、测试测试测试的", "date": "2016-12-27 14:50:00"}, {"act": "测试测试测试测试测试测试的", "date": "2016-11-16 09:27:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试的", "date": "2016-10-30 09:21:00"}, {"act": "测试测试测试测试测试测试测试测试测试的", "date": "2015-02-14 14:48:00"}, {"act": "测试测试测试测试测试测试测试测试测试的", "date": "2015-01-16 12:28:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2015-01-15 10:16:00"}, {"act": "测试测试测试测试测试测试测试", "date": "2015-01-09 16:32:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2015-01-09 16:28:00"}, {"act": "测试测试测试测试测试测试测试", "date": "2014-12-24 09:42:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2014-12-24 09:41:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2014-12-10 09:18:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2014-12-03 15:12:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2014-11-24 10:35:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2014-11-06 10:47:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试,测试测试测试测试测试测试的", "date": "2014-11-06 10:46:00"}, {"act": "测试测试测试测试放、测试测试测试测试测试测试测试测试测试,测试测试测试、测试测试的", "date": "2014-11-03 09:44:00"}, {"act": "测试测试测试测试放、测试测试测试测试测试测试测试测试测试,测试测试测试、测试测试的", "date": "2014-10-21 09:22:00"}, {"act": "测试测试测试测试放、测试测试测试测试测试测试测试测试测试,测试测试测试、测试测试的", "date": "2014-10-16 14:54:00"}, {"act": "测试测试测试测试的", "date": "2014-09-28 19:06:00"}, {"act": "测试测试测试测试放、测试测试测试测试测试测试测试测试测试,测试测试测试、测试测试的", "date": "2014-09-16 09:54:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2014-08-30 09:30:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2014-08-30 09:27:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2014-08-27 14:02:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2014-08-11 10:42:00"}, {"act": "测试测试测试测试测试测试测试达30%的", "date": "2014-07-30 14:38:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2014-07-10 07:31:00"}, {"act": "测试测试测试测试放、测试测试测试测试测试测试测试测试测试,测试测试测试、测试测试的", "date": "2014-07-08 17:33:00"}, {"act": "测试测试测试测试放、测试测试测试测试测试测试测试测试测试,测试测试测试、测试测试的", "date": "2014-07-08 10:42:00"}, {"act": "测试测试测试测试放、测试测试测试测试测试测试测试测试测试,测试测试测试、测试测试的", "date": "2014-06-30 10:15:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2014-06-29 08:51:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2014-06-24 09:32:00"}, {"act": "测试测试测试测试放、测试测试测试测试测试测试测试测试测试,测试测试测试、测试测试的", "date": "2014-06-21 10:41:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2014-06-20 14:45:00"}, {"act": "测试测试测试测试放、测试测试测试测试测试测试,测试测试测试、测试测试的", "date": "2014-06-14 11:45:00"}, {"act": "测试测试测试测试放、测试测试测试测试测试测试测试测试测试,测试测试测试、测试测试的", "date": "2014-06-12 10:41:00"}, {"act": "测试测试测试测试放、测试测试测试测试测试测试测试测试测试,测试测试测试、测试测试的", "date": "2014-05-10 09:56:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2014-01-15 10:01:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2014-01-15 10:00:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2014-01-06 12:45:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-12-29 08:18:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-12-29 08:17:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-12-28 15:46:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2013-12-27 08:58:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-12-27 08:56:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-12-26 08:43:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-12-19 09:58:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2013-12-19 07:52:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-12-19 07:50:00"}, {"act": "测试测试测试测试测试测试,测试测试测试测试测试测试的", "date": "2013-12-18 10:14:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-12-18 08:49:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2013-12-18 07:17:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-12-18 07:16:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-12-17 07:49:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-12-16 07:39:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-12-16 07:37:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-12-06 15:43:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-12-03 14:47:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2013-11-29 16:00:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-11-29 15:59:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2013-11-27 11:18:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-11-22 21:37:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-11-22 21:36:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-11-16 12:23:00"}, {"act": "测试测试测试、测试测试机、测试测试测试,测试测试测试测试测试测试的", "date": "2013-11-16 09:14:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-11-15 10:44:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2013-11-15 10:42:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-11-14 08:58:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-11-08 10:05:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-11-08 10:04:00"}, {"act": "测试测试测试测试测试测试测试测试测试的", "date": "2013-11-08 10:01:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-11-06 10:30:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2013-11-04 08:00:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-11-04 07:59:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-11-03 10:35:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2013-11-03 09:07:00"}, {"act": "测试测试测试测试测试测试测试", "date": "2013-11-02 14:18:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-11-02 09:29:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-11-02 08:40:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-10-30 09:35:00"}, {"act": "测试测试测试测试测试测试测试测试的", "date": "2013-10-25 09:15:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-10-24 09:21:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-10-24 09:06:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-10-23 13:05:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-10-23 09:04:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-10-23 08:42:00"}, {"act": "测试测试测试测试测试测试测试测试的", "date": "2013-10-19 09:33:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-10-18 17:16:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-10-18 09:59:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2013-10-18 09:41:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-10-18 06:25:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-10-18 06:23:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-10-17 14:59:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-10-17 14:58:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-10-13 09:29:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-10-12 10:00:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-10-12 10:00:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-10-12 10:00:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试,测试测试测试测试测试测试的", "date": "2013-10-12 09:47:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-10-11 13:49:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-10-10 11:05:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-10-10 11:04:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2013-10-10 09:52:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-10-09 16:12:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-10-09 10:47:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-10-09 10:46:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2013-09-30 13:52:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-09-30 13:50:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-09-28 15:32:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-09-28 12:41:00"}, {"act": "测试测试测试测试测试测试测试测试的", "date": "2013-09-28 12:40:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-09-26 15:01:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-09-19 09:10:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-09-18 09:14:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-09-18 07:22:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-09-18 07:21:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-09-17 15:34:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-09-17 15:33:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2013-09-17 10:05:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-09-17 10:04:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-09-16 14:16:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-09-16 14:16:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-09-15 15:18:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-09-12 15:45:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2013-09-12 09:21:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-09-12 09:19:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-09-11 08:23:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2013-09-11 08:23:00"}, {"act": "测试测试测试测试测试测试的", "date": "2013-09-05 09:07:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-09-05 09:06:00"}, {"act": "测试测试测试测试测试测试测试测试的", "date": "2013-09-05 09:04:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-09-05 09:00:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-08-31 12:02:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-08-30 10:24:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-08-30 10:02:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-08-29 12:11:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-08-29 09:42:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-08-27 15:22:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-08-24 10:59:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-08-23 13:31:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-08-22 07:57:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-08-22 07:56:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-08-21 13:01:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-08-20 08:39:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-08-20 08:36:00"}, {"act": "测试测试测试测试测试测试测试测试测试的", "date": "2013-08-16 13:09:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-08-16 09:08:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-08-16 08:20:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-08-15 09:34:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-08-15 09:14:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-08-15 07:35:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-08-14 12:21:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-08-14 12:20:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-08-07 10:10:00"}, {"act": "测试测试测试测试测试测试测试测试的", "date": "2013-08-07 09:21:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-08-07 08:11:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-08-06 09:27:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-08-06 08:50:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-08-06 08:49:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-08-05 13:08:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-08-05 10:08:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-08-05 08:53:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-08-05 07:56:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-08-02 16:09:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-08-02 15:09:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-08-01 10:42:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-08-01 08:28:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-08-01 08:26:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-07-30 08:50:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-07-29 19:00:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-07-29 18:49:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-07-29 18:46:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-07-28 15:16:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-07-27 08:33:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-07-26 08:55:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-07-25 11:15:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-07-25 11:13:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-07-23 09:12:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-07-23 09:11:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-07-22 10:35:00"}, {"act": "测试测试测试测试测试测试的", "date": "2013-07-22 10:25:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-07-22 09:02:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-07-22 08:36:00"}, {"act": "测试测试测试、测试测试机、测试测试测试,测试测试测试测试测试测试的", "date": "2013-07-13 09:23:00"}, {"act": "测试测试测试测试测试测试测试测试的", "date": "2013-07-08 08:55:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-07-08 08:55:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2013-07-08 07:42:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-07-08 07:40:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-07-05 15:19:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2013-07-05 08:35:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-07-05 08:33:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-07-04 16:05:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-07-04 09:03:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2013-07-04 07:30:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-07-04 07:28:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-07-03 09:52:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-07-03 07:35:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-07-03 07:35:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-07-03 07:30:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-07-02 09:43:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-07-02 09:41:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-07-02 09:03:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2013-07-02 07:04:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-07-02 07:02:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-07-01 15:41:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-07-01 15:38:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-07-01 14:44:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-06-28 09:42:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-06-28 08:02:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-26 15:24:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-06-26 10:15:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-25 12:12:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-24 12:04:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-24 08:37:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-24 08:20:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-24 08:18:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-24 07:28:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-06-24 07:26:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-22 08:56:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-22 08:32:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-06-22 08:30:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-21 09:18:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-06-21 09:17:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-21 08:53:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-06-20 16:55:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-20 13:03:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2013-06-20 09:25:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-06-19 16:02:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-19 16:01:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-19 09:54:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-18 08:46:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-18 08:23:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-17 11:54:00"}, {"act": "测试测试测试、测试测试机、测试测试测试,测试测试测试测试测试测试的", "date": "2013-06-17 10:58:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-15 15:51:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-15 10:18:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-06-15 10:15:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-06-14 08:33:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-14 08:31:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-06-14 06:06:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-06-13 16:48:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2013-06-13 16:46:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-13 14:09:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-07 10:12:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-06 11:45:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-05 10:25:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-06-05 09:46:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-06-04 09:57:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-04 07:50:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-06-03 12:13:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-06-03 07:50:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-06-03 07:45:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-05-29 14:28:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-05-29 13:50:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2013-05-29 10:47:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-05-28 16:25:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-05-28 08:40:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-05-26 14:24:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-05-25 12:51:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-05-25 12:50:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2013-05-25 08:36:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试的", "date": "2013-05-24 15:16:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-05-24 12:28:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-05-23 15:42:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-05-23 09:07:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-05-23 09:05:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-05-22 16:04:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-05-22 10:35:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-05-22 10:34:00"}, {"act": "测试测试测试、测试测试机、测试测试测试,测试测试测试测试测试测试的", "date": "2013-05-16 11:05:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-05-16 09:00:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-05-15 07:56:00"}, {"act": "测试测试测试测试测试测试测试", "date": "2013-05-15 07:55:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-05-14 08:30:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-05-14 08:30:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-05-08 15:02:00"}, {"act": "测试测试测试测试测试测试测试", "date": "2013-05-08 10:22:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-05-07 10:10:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2013-05-07 08:35:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-05-03 08:55:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-05-03 08:54:00"}, {"act": "测试测试测试测试测试测试测试测试的", "date": "2013-04-28 17:06:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-04-28 08:58:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-04-28 08:57:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-04-27 10:18:00"}, {"act": "测试测试测试测试测试测试测试", "date": "2013-04-27 10:17:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-04-26 13:45:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-04-26 13:44:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-04-25 09:45:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-04-23 13:41:00"}, {"act": "测试测试测试测试测试测试测试测试的", "date": "2013-04-23 08:54:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-04-23 08:53:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-04-18 15:20:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-04-18 08:15:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-04-18 08:14:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-04-17 16:10:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-04-17 08:27:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-04-17 08:25:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-04-16 09:24:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-04-16 09:23:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-04-09 09:49:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2013-04-08 13:48:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-04-07 08:34:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-04-07 08:33:00"}, {"act": "测试测试测试、测试测试机、测试测试测试,测试测试测试测试测试测试的", "date": "2013-04-03 14:52:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-04-03 09:30:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2013-04-03 09:28:00"}, {"act": "测试测试测试测试测试测试,测试测试测试测试测试测试", "date": "2013-04-02 10:18:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2013-04-02 09:13:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-04-02 09:12:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-03-27 10:58:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2013-03-26 10:19:00"}, {"act": "测试测试测试测试测试测试测试测试的", "date": "2013-03-25 14:31:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2013-03-23 11:36:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-03-23 09:50:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-03-22 09:36:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-03-21 17:35:00"}, {"act": "测试测试测试测试测试测试测试测试的", "date": "2013-03-21 16:24:00"}, {"act": "测试测试测试测试测试测试,测试测试测试测试测试测试的", "date": "2013-03-21 16:22:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-03-15 15:03:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-03-15 07:52:00"}, {"act": "测试测试测试测试测试测试的", "date": "2013-03-15 07:51:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2013-03-13 10:27:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-03-13 10:09:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-03-12 07:58:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-03-12 07:57:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-03-11 09:47:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-03-11 09:46:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-03-08 16:16:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-03-08 16:15:00"}, {"act": "测试测试测试测试测试测试测试达30%的", "date": "2013-03-07 17:01:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2013-03-05 08:05:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-03-05 08:04:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-03-05 08:01:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-03-04 18:07:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2013-03-04 16:11:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-03-04 16:09:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2013-02-28 10:12:00"}, {"act": "测试测试测试、测试测试机、测试测试测试,测试测试测试测试测试测试的", "date": "2013-02-25 11:11:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-02-23 10:12:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-02-22 17:00:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-02-22 16:59:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2013-02-22 10:21:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-02-21 09:57:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2013-02-21 08:16:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-02-21 08:14:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-02-21 08:11:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-02-17 08:40:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2013-02-17 08:40:00"}, {"act": "测试测试测试测试测试测试的", "date": "2013-02-17 08:39:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2013-02-08 09:39:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-02-07 14:07:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-02-07 14:06:00"}, {"act": "测试测试测试测试测试测试测试测试的", "date": "2013-02-04 09:16:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-02-04 09:15:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-01-26 12:43:00"}, {"act": "测试测试、测试测试测试测试测试测试测试测试测试置、测试测试测试测试的", "date": "2013-01-25 16:00:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-01-25 11:56:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-01-25 11:55:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试", "date": "2013-01-25 07:28:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-01-24 08:54:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-01-23 10:56:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-01-23 10:54:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-01-22 10:54:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-01-21 09:44:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-01-17 10:57:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-01-16 11:06:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-01-09 09:34:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-01-09 09:33:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2013-01-06 09:17:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2013-01-06 09:15:00"}, {"act": "测试测试测试测试测试测试的", "date": "2013-01-05 09:43:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2013-01-04 10:57:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-12-26 15:29:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-12-26 15:28:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-12-25 11:40:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-12-25 10:01:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-12-24 09:52:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-12-20 08:48:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-12-20 08:46:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-12-19 11:08:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-12-19 09:24:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-12-19 09:23:00"}, {"act": "测试测试测试测试测试测试测试", "date": "2012-12-18 09:47:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-12-18 09:45:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-12-17 16:07:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-12-17 10:01:00"}, {"act": "测试测试测试测试测试测试测试测试测试的", "date": "2012-12-14 20:11:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-12-10 08:15:00"}, {"act": "测试测试测试测试测试测试测试的", "date": "2012-12-10 08:14:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-12-10 08:13:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试,测试测试测试测试测试测试的", "date": "2012-12-08 16:34:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-12-08 16:32:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-12-06 09:41:00"}, {"act": "测试测试测试测试测试测试测试的", "date": "2012-12-06 09:40:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-12-05 09:11:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-12-05 09:09:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-12-04 16:01:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-12-04 09:40:00"}, {"act": "测试测试测试测试测试测试的", "date": "2012-12-01 14:30:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-12-01 14:29:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-11-28 16:19:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-11-28 09:35:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-11-27 14:40:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-11-27 09:53:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2012-11-23 15:20:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-11-23 15:19:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-11-22 08:09:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-11-21 13:45:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-11-20 10:44:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-11-20 10:00:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-11-19 08:02:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-11-19 08:01:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-11-16 08:20:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-11-14 08:15:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-11-14 08:14:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-11-09 15:16:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-11-09 15:15:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-11-08 08:47:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-11-08 08:46:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-11-06 09:48:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-11-05 09:26:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-11-05 09:25:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-11-04 09:30:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-11-03 09:50:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-11-02 09:35:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-11-02 08:36:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-11-01 15:24:00"}, {"act": "测试测试测试测试测试测试的", "date": "2012-11-01 10:06:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-10-31 09:29:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2012-10-31 09:27:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-10-27 08:27:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-10-25 16:57:00"}, {"act": "测试测试测试测试测试测试测试的", "date": "2012-10-25 16:56:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-10-25 09:48:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-10-24 11:37:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-10-24 09:14:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-10-23 08:56:00"}, {"act": "测试测试测试测试测试测试测试测试的", "date": "2012-10-23 08:30:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-10-23 08:28:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试,测试测试测试测试测试测试的", "date": "2012-10-22 16:49:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-10-22 16:48:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-10-22 16:46:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-10-18 10:02:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-10-17 08:08:00"}, {"act": "测试测试测试测试测试测试测试测试的", "date": "2012-10-16 13:41:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-10-16 13:38:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-10-12 10:14:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-10-11 12:12:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-10-09 08:45:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-10-09 08:43:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-10-08 14:45:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-09-25 15:11:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-09-25 15:09:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-09-24 16:05:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-09-22 15:33:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-09-22 15:08:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-09-21 09:35:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-09-19 10:06:00"}, {"act": "测试测试测试测试测试测试测试的", "date": "2012-09-19 09:15:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-09-19 09:14:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-09-19 08:22:00"}, {"act": "测试测试测试测试测试测试测试的", "date": "2012-09-19 08:21:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-09-15 15:21:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试,测试测试测试测试测试测试的", "date": "2012-09-15 09:51:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-09-10 11:11:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-09-07 14:50:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-09-07 08:39:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-09-05 16:30:00"}, {"act": "测试测试测试测试测试测试测试30%的", "date": "2012-09-05 16:23:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试,测试测试测试测试测试测试的", "date": "2012-09-05 08:22:00"}, {"act": "测试测试涂、测试测试测试测试测试,测试测试测试的", "date": "2012-09-05 08:20:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-09-05 08:17:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-09-04 09:40:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-09-04 08:08:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-09-03 08:33:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-09-03 08:05:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-09-01 09:10:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-08-31 14:28:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-08-27 16:16:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-08-27 14:20:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-08-25 14:36:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-08-24 09:11:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-08-23 10:13:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-08-22 12:26:00"}, {"act": "测试测试测试测试测试测试测试测试的", "date": "2012-08-22 12:25:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-08-16 10:09:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-08-15 09:16:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-08-10 09:32:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-08-03 15:26:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-08-02 09:37:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-08-01 09:42:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-07-31 10:35:00"}, {"act": "测试测试测试测试测试测试测试30%的", "date": "2012-07-30 14:56:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-07-27 10:10:00"}, {"act": "测试测试测试测试测试测试测试测试的", "date": "2012-07-26 16:36:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-07-26 15:10:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-07-20 07:58:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-07-18 13:55:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-07-16 15:32:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-07-05 14:30:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-07-05 07:39:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-06-25 13:25:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-06-21 09:11:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-06-21 09:09:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-06-20 08:24:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-06-19 10:00:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-06-18 14:27:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-06-16 09:30:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-06-14 07:54:00"}, {"act": "测试测试测试测试测试测试的", "date": "2012-06-13 09:55:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-06-06 08:56:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-06-05 07:33:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-06-04 10:16:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试,测试测试测试测试测试测试的", "date": "2012-06-01 12:02:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-06-01 08:58:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-05-30 14:00:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-05-28 08:25:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-05-26 09:40:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-05-25 10:02:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-05-25 10:01:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-05-24 08:19:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试,测试测试测试测试测试测试的", "date": "2012-05-23 12:34:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-05-23 12:33:00"}, {"act": "测试测试测试度、测试、测试测试测试的", "date": "2012-05-21 10:34:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-05-21 09:50:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-05-21 09:48:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-05-17 11:04:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试的", "date": "2012-05-16 16:11:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-05-16 07:57:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-05-15 14:54:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-05-15 14:53:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-05-15 08:59:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-05-15 08:57:00"}, {"act": "测试测试测试测试测试测试的", "date": "2012-05-15 07:33:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-05-14 10:16:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-05-14 10:14:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试的", "date": "2012-05-14 09:52:00"}, {"act": "测试测试测试度、测试、测试测试测试的", "date": "2012-05-11 11:09:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-05-08 10:53:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-05-08 10:51:00"}, {"act": "测试测试测试度、测试、测试测试测试的", "date": "2012-05-08 08:53:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-05-07 07:55:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-05-07 07:49:00"}, {"act": "测试测试测试度、测试、测试测试测试的", "date": "2012-05-04 12:47:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-05-03 08:13:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-04-26 14:50:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-04-21 09:24:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-04-18 09:12:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-04-13 07:57:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-04-12 07:35:00"}, {"act": "测试测试测试测试测试测试的", "date": "2012-04-12 07:35:00"}, {"act": "测试测试测试测试测试测试测试测试测试测试测试的", "date": "2012-04-11 14:55:00"}, {"act": "测试测试测试测试测试测试的", "date": "2012-04-01 15:14:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-03-27 10:40:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-03-26 10:26:00"}, {"act": "测试测试测试测试测试测试、测试测试测试测试测试测试上,测试测试测试测试的", "date": "2012-03-26 10:24:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-03-21 13:50:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-03-17 10:20:00"}, {"act": "测试测试测试测试测试测试的", "date": "2012-03-16 15:13:00"}, {"act": "测试测试测试测试测试测试的", "date": "2012-03-14 12:02:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2012-03-13 09:40:00"}, {"act": "测试、测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试测试", "date": "2012-03-13 09:36:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-03-10 10:53:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-02-27 10:33:00"}, {"act": "测试测试测试测试测试测试的", "date": "2012-02-23 15:32:00"}, {"act": "测试测试测试测试测试测试测试30%的", "date": "2012-02-20 16:55:00"}, {"act": "测试测试测试测试测试测试", "date": "2012-02-06 09:30:00"}, {"act": "测试测试测试测试测试测试的", "date": "2012-02-03 10:32:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-02-03 10:32:00"}, {"act": "测试测试测试测试测试测试的", "date": "2012-02-01 09:40:00"}, {"act": "测试测试测试测试测试测试测试测试、测试测试测试测试测试测试的", "date": "2012-01-17 20:22:14"}, {"act": "测试测试测试测试测试测试的", "date": "2012-01-13 14:20:00"}, {"act": "测试测试测试测试测试测试的", "date": "2012-01-12 08:20:00"}, {"act": "测试测试测试测试测试测试测试测试", "date": "2012-01-09 14:50:00"}, {"act": "测试门、测试测试测试测试测试", "date": "2012-01-06 09:20:00"}], "rated_power": 208.875, "vehicle_age": 8, "vehicle_type": "测试车", "vehicle_class": "测试", "vehicle_type_4": "测试、测试", "plate_type_name": "测试测试", "manufacture_date": "2011-03-17 00:00:00", "pol_ind_combined": "测试户", "vehicle_capacity": 33.1, "vehicle_seat_number": 2} -- !test8 -- -1 {"id":1} abcdefg -2 {"id":2} abcdefg +1 {"id": 1} abcdefg +2 {"id": 2} abcdefg -- !test9 -- -1 {"id":1} -2 {"id":2} +1 {"id": 1} +2 {"id": 2} -- !test10 -- -110169402202689536 10000 1993-07-24T10:34:22.851 10000 1993-07-24T10:34:22.851 201 123321 false 1993-07-24T10:34:22 xyz {"wTS":"abc58062","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc58062-xyz -110175095697739776 10000 1993-07-24T10:57:00.286 10000 1993-07-24T10:57:00.286 201 123321 false 1993-07-24T10:56:59 xyz {"wTS":"abc59419","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc59419-xyz -110176899974074368 10000 1993-07-24T11:04:10.459 10000 1993-07-24T11:04:10.459 201 123321 false 1993-07-24T11:04:09 xyz {"wTS":"abc59849","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc59849-xyz -110178694460903424 10000 1993-07-24T11:11:18.298 10000 1993-07-24T11:11:18.298 201 123321 false 1993-07-24T11:11:17 xyz {"wTS":"abc60277","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 12332112345 -110178823263784960 10000 1993-07-24T11:11:49.007 10000 1993-07-24T11:11:49.007 201 123321 false 1993-07-24T11:11:48 xyz {"wTS":"abc60308","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60308-xyz -110178991782531072 10000 1993-07-24T11:12:29.185 10000 1993-07-24T11:12:29.186 201 123321 false 1993-07-24T11:12:28 xyz {"wTS":"abc60348","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60348-xyz -110179118664421376 10000 1993-07-24T11:12:59.436 10000 1993-07-24T11:12:59.436 201 123321 false 1993-07-24T11:12:58 xyz {"wTS":"abc60378","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60378-xyz -110179245076549632 10000 1993-07-24T11:13:29.575 10000 1993-07-24T11:13:29.575 201 123321 false 1993-07-24T11:13:28 xyz {"wTS":"abc60408","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60408-xyz -110179371597729792 10000 1993-07-24T11:13:59.740 10000 1993-07-24T11:13:59.740 201 123321 false 1993-07-24T11:13:59 xyz {"wTS":"abc60439","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60439-xyz -110179498144075776 10000 1993-07-24T11:14:29.911 10000 1993-07-24T11:14:29.911 201 123321 false 1993-07-24T11:14:29 xyz {"wTS":"abc60469","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60469-xyz -110179625034354688 10000 1993-07-24T11:15:00.164 10000 1993-07-24T11:15:00.164 201 123321 false 1993-07-24T11:14:59 xyz {"wTS":"abc60499","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60499-xyz -110179751375179776 10000 1993-07-24T11:15:30.286 10000 1993-07-24T11:15:30.286 201 123321 false 1993-07-24T11:15:29 xyz {"wTS":"abc60529","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60529-xyz -110179877997023232 10000 1993-07-24T11:16:00.475 10000 1993-07-24T11:16:00.475 201 123321 false 1993-07-24T11:15:59 xyz {"wTS":"abc60559","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60559-xyz -110180004425928704 10000 1993-07-24T11:16:30.618 10000 1993-07-24T11:16:30.618 201 123321 false 1993-07-24T11:16:30 xyz {"wTS":"abc60590","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60590-xyz -110180131114881024 10000 1993-07-24T11:17:00.823 10000 1993-07-24T11:17:00.823 201 123321 false 1993-07-24T11:17 xyz {"wTS":"abc60620","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60620-xyz -110180257979994112 10000 1993-07-24T11:17:31.070 10000 1993-07-24T11:17:31.070 201 123321 false 1993-07-24T11:17:30 xyz {"wTS":"abc60650","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60650-xyz -110180395045654528 10000 1993-07-24T11:18:03.749 10000 1993-07-24T11:18:03.749 201 123321 false 1993-07-24T11:18 xyz {"wTS":"abc60680","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60680-xyz -110180510913302528 10000 1993-07-24T11:18:31.374 10000 1993-07-24T11:18:31.374 201 123321 false 1993-07-24T11:18:30 xyz {"wTS":"abc60710","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60710-xyz -110180637086355456 10000 1993-07-24T11:19:01.456 10000 1993-07-24T11:19:01.456 201 123321 false 1993-07-24T11:19 xyz {"wTS":"abc60740","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60740-xyz -110180763712393216 10000 1993-07-24T11:19:31.646 10000 1993-07-24T11:19:31.646 201 123321 false 1993-07-24T11:19:31 xyz {"wTS":"abc60771","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60771-xyz -110180890539757568 10000 1993-07-24T11:20:01.884 10000 1993-07-24T11:20:01.884 201 123321 false 1993-07-24T11:20:01 xyz {"wTS":"abc60801","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60801-xyz -110181018361171968 10000 1993-07-24T11:20:32.359 10000 1993-07-24T11:20:32.359 201 123321 false 1993-07-24T11:20:31 xyz {"wTS":"abc60831","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60831-xyz -110181143443705856 10000 1993-07-24T11:21:02.181 10000 1993-07-24T11:21:02.181 201 123321 false 1993-07-24T11:21:01 xyz {"wTS":"abc60861","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60861-xyz -110181270325596160 10000 1993-07-24T11:21:32.432 10000 1993-07-24T11:21:32.432 201 123321 false 1993-07-24T11:21:31 xyz {"wTS":"abc60891","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60891-xyz -110181396674809856 10000 1993-07-24T11:22:02.556 10000 1993-07-24T11:22:02.556 201 123321 false 1993-07-24T11:22:01 xyz {"wTS":"abc60921","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60921-xyz -110181522885611520 10000 1993-07-24T11:22:32.647 10000 1993-07-24T11:22:32.647 201 123321 false 1993-07-24T11:22:32 xyz {"wTS":"abc60952","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60952-xyz -110181649587146752 10000 1993-07-24T11:23:02.855 10000 1993-07-24T11:23:02.855 201 123321 false 1993-07-24T11:23:02 xyz {"wTS":"abc60982","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc60982-xyz -110181776418705408 10000 1993-07-24T11:23:33.094 10000 1993-07-24T11:23:33.094 201 123321 false 1993-07-24T11:23:32 xyz {"wTS":"abc61012","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc61012-xyz -110181902755336192 10000 1993-07-24T11:24:03.215 10000 1993-07-24T11:24:03.215 201 123321 false 1993-07-24T11:24:02 xyz {"wTS":"abc61042","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc61042-xyz -110182029335236608 10000 1993-07-24T11:24:33.394 10000 1993-07-24T11:24:33.394 201 123321 false 1993-07-24T11:24:32 xyz {"wTS":"abc61072","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc61072-xyz -110182155852222464 10000 1993-07-24T11:25:03.558 10000 1993-07-24T11:25:03.558 201 123321 false 1993-07-24T11:25:02 xyz {"wTS":"abc61102","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc61102-xyz -110182282490843136 10000 1993-07-24T11:25:33.751 10000 1993-07-24T11:25:33.751 201 123321 false 1993-07-24T11:25:33 xyz {"wTS":"abc61133","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc61133-xyz -110182409053966336 10000 1993-07-24T11:26:03.926 10000 1993-07-24T11:26:03.926 201 123321 false 1993-07-24T11:26:03 xyz {"wTS":"abc61163","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc61163-xyz -110182535369625600 10000 1993-07-24T11:26:34.042 10000 1993-07-24T11:26:34.042 201 123321 false 1993-07-24T11:26:33 xyz {"wTS":"abc61193","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc61193-xyz -110182662033412096 10000 1993-07-24T11:27:04.241 10000 1993-07-24T11:27:04.241 201 123321 false 1993-07-24T11:27:03 xyz {"wTS":"abc61223","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc61223-xyz -110182790806933504 10000 1993-07-24T11:27:34.943 10000 1993-07-24T11:27:34.943 201 123321 false 1993-07-24T11:27:33 xyz {"wTS":"abc61253","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc61253-xyz -110182915377762304 10000 1993-07-24T11:28:04.643 10000 1993-07-24T11:28:04.643 201 123321 false 1993-07-24T11:28:03 xyz {"wTS":"abc61283","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc61283-xyz -110183041982828544 10000 1993-07-24T11:28:34.828 10000 1993-07-24T11:28:34.828 201 123321 false 1993-07-24T11:28:34 xyz {"wTS":"abc61314","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc61314-xyz -110188354488274944 10000 1993-07-24T11:49:41.428 10000 1993-07-24T11:49:41.428 201 123321 false 1993-07-24T11:49:40 xyz {"wTS":"abc62580","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc62580-xyz -110188483035303936 10000 1993-07-24T11:50:12.076 10000 1993-07-24T11:50:12.076 201 123321 false 1993-07-24T11:50:11 xyz {"wTS":"abc62611","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc62611-xyz -110188609283854336 10000 1993-07-24T11:50:42.176 10000 1993-07-24T11:50:42.176 201 123321 false 1993-07-24T11:50:41 xyz {"wTS":"abc62641","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc62641-xyz -110188860698824704 10000 1993-07-24T11:51:42.118 10000 1993-07-24T11:51:42.118 201 123321 false 1993-07-24T11:51:41 xyz {"wTS":"abc62701","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc62701-xyz -110188994555842560 10000 1993-07-24T11:52:14.032 10000 1993-07-24T11:52:14.032 201 123321 false 1993-07-24T11:52:11 xyz {"wTS":"abc62731","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc62731-xyz -110189113330143232 10000 1993-07-24T11:52:42.350 10000 1993-07-24T11:52:42.350 201 123321 false 1993-07-24T11:52:41 xyz {"wTS":"abc62761","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc62761-xyz -110189239381561344 10000 1993-07-24T11:53:12.403 10000 1993-07-24T11:53:12.403 201 123321 false 1993-07-24T11:53:11 xyz {"wTS":"abc62791","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc62791-xyz -110189365340704768 10000 1993-07-24T11:53:42.434 10000 1993-07-24T11:53:42.434 201 123321 false 1993-07-24T11:53:41 xyz {"wTS":"abc62821","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc62821-xyz -110189491983519744 10000 1993-07-24T11:54:12.629 10000 1993-07-24T11:54:12.629 201 123321 false 1993-07-24T11:54:11 xyz {"wTS":"abc62851","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc62851-xyz -110189618773135360 10000 1993-07-24T11:54:42.857 10000 1993-07-24T11:54:42.857 201 123321 false 1993-07-24T11:54:41 xyz {"wTS":"abc62881","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc62881-xyz -110189794522861568 10000 1993-07-24T11:55:24.759 10000 1993-07-24T11:55:24.759 201 123321 false 1993-07-24T11:55:11 xyz {"wTS":"abc62911","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc62911-xyz -110189870402015232 10000 1993-07-24T11:55:42.850 10000 1993-07-24T11:55:42.850 201 123321 false 1993-07-24T11:55:41 xyz {"wTS":"abc62941","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc62941-xyz -110189996034002944 10000 1993-07-24T11:56:12.803 10000 1993-07-24T11:56:12.803 201 123321 false 1993-07-24T11:56:12 xyz {"wTS":"abc62972","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc62972-xyz -110190121573715968 10000 1993-07-24T11:56:42.734 10000 1993-07-24T11:56:42.734 201 123321 false 1993-07-24T11:56:42 xyz {"wTS":"abc63002","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63002-xyz -110190247864209408 10000 1993-07-24T11:57:12.844 10000 1993-07-24T11:57:12.844 201 123321 false 1993-07-24T11:57:12 xyz {"wTS":"abc63032","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63032-xyz -110190373731078144 10000 1993-07-24T11:57:42.853 10000 1993-07-24T11:57:42.853 201 123321 false 1993-07-24T11:57:42 xyz {"wTS":"abc63062","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63062-xyz -110190500046737408 10000 1993-07-24T11:58:12.969 10000 1993-07-24T11:58:12.969 201 123321 false 1993-07-24T11:58:12 xyz {"wTS":"abc63092","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63092-xyz -110190626324647936 10000 1993-07-24T11:58:43.076 10000 1993-07-24T11:58:43.076 201 123321 false 1993-07-24T11:58:42 xyz {"wTS":"abc63122","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63122-xyz -110190752267014144 10000 1993-07-24T11:59:13.103 10000 1993-07-24T11:59:13.103 201 123321 false 1993-07-24T11:59:12 xyz {"wTS":"abc63152","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63152-xyz -110190878452649984 10000 1993-07-24T11:59:43.188 10000 1993-07-24T11:59:43.188 201 123321 false 1993-07-24T11:59:42 xyz {"wTS":"abc63182","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63182-xyz -110191004369850368 10000 1993-07-24T12:00:13.209 10000 1993-07-24T12:00:13.209 201 123321 false 1993-07-24T12:00:12 xyz {"wTS":"abc63212","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63212-xyz -110191130433851392 10000 1993-07-24T12:00:43.265 10000 1993-07-24T12:00:43.265 201 123321 false 1993-07-24T12:00:42 xyz {"wTS":"abc63242","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63242-xyz -110191256502046720 10000 1993-07-24T12:01:13.323 10000 1993-07-24T12:01:13.323 201 123321 false 1993-07-24T12:01:12 xyz {"wTS":"abc63272","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63272-xyz -110191382591213568 10000 1993-07-24T12:01:43.384 10000 1993-07-24T12:01:43.384 201 123321 false 1993-07-24T12:01:42 xyz {"wTS":"abc63302","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63302-xyz -110191508500025344 10000 1993-07-24T12:02:13.403 10000 1993-07-24T12:02:13.403 201 123321 false 1993-07-24T12:02:12 xyz {"wTS":"abc63332","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63332-xyz -110191634446585856 10000 1993-07-24T12:02:43.431 10000 1993-07-24T12:02:43.431 201 123321 false 1993-07-24T12:02:42 xyz {"wTS":"abc63362","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63362-xyz -110191760862908416 10000 1993-07-24T12:03:13.571 10000 1993-07-24T12:03:13.571 201 123321 false 1993-07-24T12:03:12 xyz {"wTS":"abc63392","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63392-xyz -110191886729777152 10000 1993-07-24T12:03:43.580 10000 1993-07-24T12:03:43.580 201 123321 false 1993-07-24T12:03:42 xyz {"wTS":"abc63422","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63422-xyz -110192013620056064 10000 1993-07-24T12:04:13.833 10000 1993-07-24T12:04:13.833 201 123321 false 1993-07-24T12:04:12 xyz {"wTS":"abc63452","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63452-xyz -110192140048961536 10000 1993-07-24T12:04:43.976 10000 1993-07-24T12:04:43.976 201 123321 false 1993-07-24T12:04:42 xyz {"wTS":"abc63482","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63482-xyz -110192390960615424 10000 1993-07-24T12:05:43.798 10000 1993-07-24T12:05:43.798 201 123321 false 1993-07-24T12:05:43 xyz {"wTS":"abc63543","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63543-xyz -110192642979565568 10000 1993-07-24T12:06:43.884 10000 1993-07-24T12:06:43.884 201 123321 false 1993-07-24T12:06:43 xyz {"wTS":"abc63603","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63603-xyz -110192769030983680 10000 1993-07-24T12:07:13.937 10000 1993-07-24T12:07:13.937 201 123321 false 1993-07-24T12:07:13 xyz {"wTS":"abc63633","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63633-xyz -110192895040458752 10000 1993-07-24T12:07:43.980 10000 1993-07-24T12:07:43.980 201 123321 false 1993-07-24T12:07:43 xyz {"wTS":"abc63663","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63663-xyz -110193021058322432 10000 1993-07-24T12:08:14.025 10000 1993-07-24T12:08:14.025 201 123321 false 1993-07-24T12:08:13 xyz {"wTS":"abc63693","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63693-xyz -110193147101351936 10000 1993-07-24T12:08:44.076 10000 1993-07-24T12:08:44.076 201 123321 false 1993-07-24T12:08:43 xyz {"wTS":"abc63723","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63723-xyz -110193273085661184 10000 1993-07-24T12:09:14.113 10000 1993-07-24T12:09:14.113 201 123321 false 1993-07-24T12:09:13 xyz {"wTS":"abc63753","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63753-xyz -110193399204188160 10000 1993-07-24T12:09:44.182 10000 1993-07-24T12:09:44.182 201 123321 false 1993-07-24T12:09:43 xyz {"wTS":"abc63783","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63783-xyz -110193525255606272 10000 1993-07-24T12:10:14.235 10000 1993-07-24T12:10:14.235 201 123321 false 1993-07-24T12:10:13 xyz {"wTS":"abc63813","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63813-xyz -110193651541905408 10000 1993-07-24T12:10:44.344 10000 1993-07-24T12:10:44.344 201 123321 false 1993-07-24T12:10:43 xyz {"wTS":"abc63843","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63843-xyz -110193777673015296 10000 1993-07-24T12:11:14.416 10000 1993-07-24T12:11:14.416 201 123321 false 1993-07-24T12:11:13 xyz {"wTS":"abc63873","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63873-xyz -110193903774765056 10000 1993-07-24T12:11:44.481 10000 1993-07-24T12:11:44.481 201 123321 false 1993-07-24T12:11:43 xyz {"wTS":"abc63903","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63903-xyz -110194030237224960 10000 1993-07-24T12:12:14.633 10000 1993-07-24T12:12:14.633 201 123321 false 1993-07-24T12:12:13 xyz {"wTS":"abc63933","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63933-xyz -110194156341103328 10000 1993-07-24T12:12:44.699 10000 1993-07-24T12:12:44.699 201 123321 false 1993-07-24T12:12:43 xyz {"wTS":"abc63963","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63963-xyz -110194282004516864 10000 1993-07-24T12:13:14.658 10000 1993-07-24T12:13:14.658 201 123321 false 1993-07-24T12:13:14 xyz {"wTS":"abc63994","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc63994-xyz -110194408274038784 10000 1993-07-24T12:13:44.763 10000 1993-07-24T12:13:44.763 201 123321 false 1993-07-24T12:13:44 xyz {"wTS":"abc64024","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64024-xyz -110194534354817024 10000 1993-07-24T12:14:14.823 10000 1993-07-24T12:14:14.823 201 123321 false 1993-07-24T12:14:14 xyz {"wTS":"abc64054","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64054-xyz -110194660330737664 10000 1993-07-24T12:14:44.858 10000 1993-07-24T12:14:44.858 201 123321 false 1993-07-24T12:14:44 xyz {"wTS":"abc64084","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64084-xyz -110194786499596288 10000 1993-07-24T12:15:14.939 10000 1993-07-24T12:15:14.939 201 123321 false 1993-07-24T12:15:14 xyz {"wTS":"abc64114","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64114-xyz -110194912936890368 10000 1993-07-24T12:15:45.084 10000 1993-07-24T12:15:45.084 201 123321 false 1993-07-24T12:15:44 xyz {"wTS":"abc64144","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64144-xyz -110195039147692032 10000 1993-07-24T12:16:15.175 10000 1993-07-24T12:16:15.175 201 123321 false 1993-07-24T12:16:14 xyz {"wTS":"abc64174","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64174-xyz -110195290638159872 10000 1993-07-24T12:17:15.135 10000 1993-07-24T12:17:15.135 201 123321 false 1993-07-24T12:17:14 xyz {"wTS":"abc64234","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64234-xyz -110195416685383680 10000 1993-07-24T12:17:45.187 10000 1993-07-24T12:17:45.187 201 123321 false 1993-07-24T12:17:44 xyz {"wTS":"abc64264","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64264-xyz -110195542942322688 10000 1993-07-24T12:18:15.289 10000 1993-07-24T12:18:15.289 201 123321 false 1993-07-24T12:18:14 xyz {"wTS":"abc64294","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64294-xyz -110195668876300288 10000 1993-07-24T12:18:45.314 10000 1993-07-24T12:18:45.314 201 123321 false 1993-07-24T12:18:44 xyz {"wTS":"abc64324","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64324-xyz -110195795045158912 10000 1993-07-24T12:19:15.395 10000 1993-07-24T12:19:15.395 201 123321 false 1993-07-24T12:19:14 xyz {"wTS":"abc64354","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64354-xyz -110195920932999168 10000 1993-07-24T12:19:45.409 10000 1993-07-24T12:19:45.409 201 123321 false 1993-07-24T12:19:44 xyz {"wTS":"abc64384","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64384-xyz -110196047487733760 10000 1993-07-24T12:20:15.582 10000 1993-07-24T12:20:15.582 201 123321 false 1993-07-24T12:20:14 xyz {"wTS":"abc64414","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64414-xyz -110196173232967680 10000 1993-07-24T12:20:45.562 10000 1993-07-24T12:20:45.562 201 123321 false 1993-07-24T12:20:44 xyz {"wTS":"abc64444","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64444-xyz -110196299162750976 10000 1993-07-24T12:21:15.586 10000 1993-07-24T12:21:15.586 201 123321 false 1993-07-24T12:21:14 xyz {"wTS":"abc64474","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64474-xyz -110196425423884288 10000 1993-07-24T12:21:45.689 10000 1993-07-24T12:21:45.689 201 123321 false 1993-07-24T12:21:45 xyz {"wTS":"abc64505","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64505-xyz -110196551647268864 10000 1993-07-24T12:22:15.783 10000 1993-07-24T12:22:15.783 201 123321 false 1993-07-24T12:22:15 xyz {"wTS":"abc64535","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64535-xyz -110196677765795840 10000 1993-07-24T12:22:45.852 10000 1993-07-24T12:22:45.852 201 123321 false 1993-07-24T12:22:45 xyz {"wTS":"abc64565","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64565-xyz -110196803972403200 10000 1993-07-24T12:23:15.943 10000 1993-07-24T12:23:15.943 201 123321 false 1993-07-24T12:23:15 xyz {"wTS":"abc64595","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64595-xyz -110196930053181440 10000 1993-07-24T12:23:46.002 10000 1993-07-24T12:23:46.002 201 123321 false 1993-07-24T12:23:45 xyz {"wTS":"abc64625","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64625-xyz -110197056020713472 10000 1993-07-24T12:24:16.035 10000 1993-07-24T12:24:16.035 201 123321 false 1993-07-24T12:24:15 xyz {"wTS":"abc64655","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc64655-xyz -110217731217330176 10000 1993-07-24T13:46:25.386 10000 1993-07-24T13:46:25.386 201 123321 false 1993-07-24T13:46:05 xyz {"wTS":"abc69565","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc69565-xyz -110217731531902976 10000 1993-07-24T13:46:25.461 10000 1993-07-24T13:46:25.461 201 123321 false 1993-07-24T13:45:35 xyz {"wTS":"abc69535","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc69535-xyz -110217731724840960 10000 1993-07-24T13:46:25.507 10000 1993-07-24T13:46:25.507 201 123321 false 1993-07-24T13:44:34 xyz {"wTS":"abc69474","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc69474-xyz -110217731951333376 10000 1993-07-24T13:46:25.561 10000 1993-07-24T13:46:25.561 201 123321 false 1993-07-24T13:45:04 xyz {"wTS":"abc69504","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc69504-xyz -110217898096103424 10000 1993-07-24T13:47:05.173 10000 1993-07-24T13:47:05.173 201 123321 false 1993-07-24T13:47:05 xyz {"wTS":"abc69625","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc69625-xyz -110218024139132928 10000 1993-07-24T13:47:35.224 10000 1993-07-24T13:47:35.224 201 123321 false 1993-07-24T13:47:35 xyz {"wTS":"abc69655","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc69655-xyz -110218150177968128 10000 1993-07-24T13:48:05.274 10000 1993-07-24T13:48:05.274 201 123321 false 1993-07-24T13:48:05 xyz {"wTS":"abc69685","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc69685-xyz -110218276145500160 10000 1993-07-24T13:48:35.307 10000 1993-07-24T13:48:35.307 201 123321 false 1993-07-24T13:48:35 xyz {"wTS":"abc69715","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc69715-xyz -110218402381467648 10000 1993-07-24T13:49:05.404 10000 1993-07-24T13:49:05.404 201 123321 false 1993-07-24T13:49:05 xyz {"wTS":"abc69745","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc69745-xyz -110218528369971200 10000 1993-07-24T13:49:35.442 10000 1993-07-24T13:49:35.442 201 123321 false 1993-07-24T13:49:35 xyz {"wTS":"abc69775","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc69775-xyz -110218765297815552 10000 1993-07-24T13:50:31.930 10000 1993-07-24T13:50:31.930 201 123321 false 1993-07-24T13:50:31 xyz {"wTS":"abc69831","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc69831-xyz -110218891194044416 10000 1993-07-24T13:51:01.946 10000 1993-07-24T13:51:01.946 201 123321 false 1993-07-24T13:51:02 xyz {"wTS":"abc69862","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc69862-xyz -110219017400651776 10000 1993-07-24T13:51:32.036 10000 1993-07-24T13:51:32.036 201 123321 false 1993-07-24T13:51:32 xyz {"wTS":"abc69892","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc69892-xyz -110219143410126848 10000 1993-07-24T13:52:02.079 10000 1993-07-24T13:52:02.079 201 123321 false 1993-07-24T13:52:02 xyz {"wTS":"abc69922","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc69922-xyz -110219269524459520 10000 1993-07-24T13:52:32.147 10000 1993-07-24T13:52:32.147 201 123321 false 1993-07-24T13:52:32 xyz {"wTS":"abc69952","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc69952-xyz -110219395626209280 10000 1993-07-24T13:53:02.212 10000 1993-07-24T13:53:02.212 201 123321 false 1993-07-24T13:53:02 xyz {"wTS":"abc69982","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc69982-xyz -110219522440990720 10000 1993-07-24T13:53:32.447 10000 1993-07-24T13:53:32.447 201 123321 false 1993-07-24T13:53:32 xyz {"wTS":"abc70012","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70012-xyz -110219648433688576 10000 1993-07-24T13:54:02.486 10000 1993-07-24T13:54:02.486 201 123321 false 1993-07-24T13:54:02 xyz {"wTS":"abc70042","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70042-xyz -110219773797240832 10000 1993-07-24T13:54:32.375 10000 1993-07-24T13:54:32.375 201 123321 false 1993-07-24T13:54:32 xyz {"wTS":"abc70072","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70072-xyz -110219899630555136 10000 1993-07-24T13:55:02.376 10000 1993-07-24T13:55:02.376 201 123321 false 1993-07-24T13:55:02 xyz {"wTS":"abc70102","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70102-xyz -110220025518395392 10000 1993-07-24T13:55:32.390 10000 1993-07-24T13:55:32.390 201 123321 false 1993-07-24T13:55:32 xyz {"wTS":"abc70132","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70132-xyz -110220151716614144 10000 1993-07-24T13:56:02.478 10000 1993-07-24T13:56:02.478 201 123321 false 1993-07-24T13:56:02 xyz {"wTS":"abc70162","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70162-xyz -110220277596065792 10000 1993-07-24T13:56:32.490 10000 1993-07-24T13:56:32.490 201 123321 false 1993-07-24T13:56:32 xyz {"wTS":"abc70192","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70192-xyz -110220403542626304 10000 1993-07-24T13:57:02.518 10000 1993-07-24T13:57:02.518 201 123321 false 1993-07-24T13:57:02 xyz {"wTS":"abc70222","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70222-xyz -110220529875062784 10000 1993-07-24T13:57:32.638 10000 1993-07-24T13:57:32.638 201 123321 false 1993-07-24T13:57:32 xyz {"wTS":"abc70252","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70252-xyz -110220781780766720 10000 1993-07-24T13:58:32.697 10000 1993-07-24T13:58:32.697 201 123321 false 1993-07-24T13:58:32 xyz {"wTS":"abc70312","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70312-xyz -110220907869933568 10000 1993-07-24T13:59:02.759 10000 1993-07-24T13:59:02.759 201 123321 false 1993-07-24T13:59:02 xyz {"wTS":"abc70342","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70342-xyz -110221033896185856 10000 1993-07-24T13:59:32.806 10000 1993-07-24T13:59:32.806 201 123321 false 1993-07-24T13:59:32 xyz {"wTS":"abc70372","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70372-xyz -110221159939215360 10000 1993-07-24T14:00:02.857 10000 1993-07-24T14:00:02.857 201 123321 false 1993-07-24T14:00:02 xyz {"wTS":"abc70402","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70402-xyz -110221287076958208 10000 1993-07-24T14:00:33.170 10000 1993-07-24T14:00:33.170 201 123321 false 1993-07-24T14:00:32 xyz {"wTS":"abc70432","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70432-xyz -110221412973187072 10000 1993-07-24T14:01:03.185 10000 1993-07-24T14:01:03.185 201 123321 false 1993-07-24T14:01:02 xyz {"wTS":"abc70462","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70462-xyz -110221538944913408 10000 1993-07-24T14:01:33.219 10000 1993-07-24T14:01:33.219 201 123321 false 1993-07-24T14:01:33 xyz {"wTS":"abc70493","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70493-xyz -110221789957230592 10000 1993-07-24T14:02:33.065 10000 1993-07-24T14:02:33.065 201 123321 false 1993-07-24T14:02:33 xyz {"wTS":"abc70553","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70553-xyz -110221916008648704 10000 1993-07-24T14:03:03.118 10000 1993-07-24T14:03:03.118 201 123321 false 1993-07-24T14:03:03 xyz {"wTS":"abc70583","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70583-xyz -110222042202673152 10000 1993-07-24T14:03:33.205 10000 1993-07-24T14:03:33.205 201 123321 false 1993-07-24T14:03:33 xyz {"wTS":"abc70613","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70613-xyz -110222168291840000 10000 1993-07-24T14:04:03.267 10000 1993-07-24T14:04:03.267 201 123321 false 1993-07-24T14:04:03 xyz {"wTS":"abc70643","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70643-xyz -110222361863163904 10000 1993-07-24T14:04:49.418 10000 1993-07-24T14:04:49.418 201 123321 false 1993-07-24T14:04:49 xyz {"wTS":"abc70689","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70689-xyz -110222488090742784 10000 1993-07-24T14:05:19.513 10000 1993-07-24T14:05:19.513 201 123321 false 1993-07-24T14:05:19 xyz {"wTS":"abc70719","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70719-xyz -110222613961805824 10000 1993-07-24T14:05:49.523 10000 1993-07-24T14:05:49.523 201 123321 false 1993-07-24T14:05:49 xyz {"wTS":"abc70749","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70749-xyz -110222740306825216 10000 1993-07-24T14:06:19.647 10000 1993-07-24T14:06:19.647 201 123321 false 1993-07-24T14:06:19 xyz {"wTS":"abc70779","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70779-xyz -110222866127556608 10000 1993-07-24T14:06:49.644 10000 1993-07-24T14:06:49.644 201 123321 false 1993-07-24T14:06:49 xyz {"wTS":"abc70809","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70809-xyz -110222992099282944 10000 1993-07-24T14:07:19.678 10000 1993-07-24T14:07:19.678 201 123321 false 1993-07-24T14:07:19 xyz {"wTS":"abc70839","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70839-xyz -110223118272335872 10000 1993-07-24T14:07:49.760 10000 1993-07-24T14:07:49.760 201 123321 false 1993-07-24T14:07:49 xyz {"wTS":"abc70869","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70869-xyz -110223245355552768 10000 1993-07-24T14:08:20.059 10000 1993-07-24T14:08:20.059 201 123321 false 1993-07-24T14:08:19 xyz {"wTS":"abc70899","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70899-xyz -110223371398582272 10000 1993-07-24T14:08:50.110 10000 1993-07-24T14:08:50.110 201 123321 false 1993-07-24T14:08:49 xyz {"wTS":"abc70929","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70929-xyz -110223496602750976 10000 1993-07-24T14:09:19.961 10000 1993-07-24T14:09:19.961 201 123321 false 1993-07-24T14:09:19 xyz {"wTS":"abc70959","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70959-xyz -110223623874711552 10000 1993-07-24T14:09:50.305 10000 1993-07-24T14:09:50.305 201 123321 false 1993-07-24T14:09:49 xyz {"wTS":"abc70989","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc70989-xyz -110223748852387840 10000 1993-07-24T14:10:20.102 10000 1993-07-24T14:10:20.102 201 123321 false 1993-07-24T14:10:19 xyz {"wTS":"abc71019","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71019-xyz -110223874110033792 10000 1993-07-24T14:10:50.115 10000 1993-07-24T14:10:50.115 201 123321 false 1993-07-24T14:10:49 xyz {"wTS":"abc71049","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71049-xyz -110224000581931008 10000 1993-07-24T14:11:20.119 10000 1993-07-24T14:11:20.119 201 123321 false 1993-07-24T14:11:19 xyz {"wTS":"abc71079","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71079-xyz -110224127031808000 10000 1993-07-24T14:11:50.267 10000 1993-07-24T14:11:50.267 201 123321 false 1993-07-24T14:11:49 xyz {"wTS":"abc71109","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71109-xyz -110224253032894464 10000 1993-07-24T14:12:20.308 10000 1993-07-24T14:12:20.309 201 123321 false 1993-07-24T14:12:19 xyz {"wTS":"abc71139","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71139-xyz -110224379105284096 10000 1993-07-24T14:12:50.366 10000 1993-07-24T14:12:50.366 201 123321 false 1993-07-24T14:12:49 xyz {"wTS":"abc71169","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71169-xyz -110224505295114240 10000 1993-07-24T14:13:20.453 10000 1993-07-24T14:13:20.453 201 123321 false 1993-07-24T14:13:19 xyz {"wTS":"abc71199","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71199-xyz -110224631157788672 10000 1993-07-24T14:13:50.460 10000 1993-07-24T14:13:50.460 201 123321 false 1993-07-24T14:13:49 xyz {"wTS":"abc71229","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71229-xyz -110224757565722624 10000 1993-07-24T14:14:20.598 10000 1993-07-24T14:14:20.598 201 123321 false 1993-07-24T14:14:19 xyz {"wTS":"abc71259","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71259-xyz -110224883621335040 10000 1993-07-24T14:14:50.652 10000 1993-07-24T14:14:50.652 201 123321 false 1993-07-24T14:14:50 xyz {"wTS":"abc71290","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71290-xyz -110225009978937344 10000 1993-07-24T14:15:20.778 10000 1993-07-24T14:15:20.778 201 123321 false 1993-07-24T14:15:20 xyz {"wTS":"abc71320","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71320-xyz -110225135732559872 10000 1993-07-24T14:15:50.760 10000 1993-07-24T14:15:50.760 201 123321 false 1993-07-24T14:15:50 xyz {"wTS":"abc71350","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71350-xyz -110225261687508992 10000 1993-07-24T14:16:20.790 10000 1993-07-24T14:16:20.790 201 123321 false 1993-07-24T14:16:20 xyz {"wTS":"abc71380","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71380-xyz -110225388510679040 10000 1993-07-24T14:16:51.027 10000 1993-07-24T14:16:51.027 201 123321 false 1993-07-24T14:16:50 xyz {"wTS":"abc71410","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71410-xyz -110225514373353472 10000 1993-07-24T14:17:21.035 10000 1993-07-24T14:17:21.035 201 123321 false 1993-07-24T14:17:20 xyz {"wTS":"abc71440","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71440-xyz -110225640051478528 10000 1993-07-24T14:17:50.999 10000 1993-07-24T14:17:50.999 201 123321 false 1993-07-24T14:17:50 xyz {"wTS":"abc71470","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71470-xyz -110225892120760320 10000 1993-07-24T14:18:51.097 10000 1993-07-24T14:18:51.097 201 123321 false 1993-07-24T14:18:50 xyz {"wTS":"abc71530","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71530-xyz -110226018105069568 10000 1993-07-24T14:19:21.134 10000 1993-07-24T14:19:21.134 201 123321 false 1993-07-24T14:19:20 xyz {"wTS":"abc71560","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71560-xyz -110226144315871232 10000 1993-07-24T14:19:51.225 10000 1993-07-24T14:19:51.225 201 123321 false 1993-07-24T14:19:50 xyz {"wTS":"abc71590","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71590-xyz -110226270233071616 10000 1993-07-24T14:20:21.246 10000 1993-07-24T14:20:21.246 201 123321 false 1993-07-24T14:20:20 xyz {"wTS":"abc71620","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71620-xyz -110226396309655552 10000 1993-07-24T14:20:51.305 10000 1993-07-24T14:20:51.305 201 123321 false 1993-07-24T14:20:50 xyz {"wTS":"abc71650","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71650-xyz -110226522470125568 10000 1993-07-24T14:21:21.384 10000 1993-07-24T14:21:21.384 201 123321 false 1993-07-24T14:21:20 xyz {"wTS":"abc71680","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71680-xyz -110226648936779776 10000 1993-07-24T14:21:51.536 10000 1993-07-24T14:21:51.536 201 123321 false 1993-07-24T14:21:50 xyz {"wTS":"abc71710","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71710-xyz -110226774807842816 10000 1993-07-24T14:22:21.546 10000 1993-07-24T14:22:21.546 201 123321 false 1993-07-24T14:22:20 xyz {"wTS":"abc71740","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71740-xyz -110226900670517248 10000 1993-07-24T14:22:51.554 10000 1993-07-24T14:22:51.555 201 123321 false 1993-07-24T14:22:50 xyz {"wTS":"abc71770","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71770-xyz -110227026835181568 10000 1993-07-24T14:23:21.634 10000 1993-07-24T14:23:21.634 201 123321 false 1993-07-24T14:23:20 xyz {"wTS":"abc71800","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71800-xyz -110227153578659840 10000 1993-07-24T14:23:51.852 10000 1993-07-24T14:23:51.852 201 123321 false 1993-07-24T14:23:50 xyz {"wTS":"abc71830","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71830-xyz -110227279382614016 10000 1993-07-24T14:24:21.846 10000 1993-07-24T14:24:21.846 201 123321 false 1993-07-24T14:24:21 xyz {"wTS":"abc71861","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71861-xyz -110227404746166272 10000 1993-07-24T14:24:51.735 10000 1993-07-24T14:24:51.735 201 123321 false 1993-07-24T14:24:51 xyz {"wTS":"abc71891","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71891-xyz -110227530835333120 10000 1993-07-24T14:25:21.797 10000 1993-07-24T14:25:21.797 201 123321 false 1993-07-24T14:25:21 xyz {"wTS":"abc71921","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71921-xyz -110227783114330112 10000 1993-07-24T14:26:21.945 10000 1993-07-24T14:26:21.945 201 123321 false 1993-07-24T14:26:21 xyz {"wTS":"abc71981","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc71981-xyz -110228035099725824 10000 1993-07-24T14:27:22.023 10000 1993-07-24T14:27:22.023 201 123321 false 1993-07-24T14:27:21 xyz {"wTS":"abc72041","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72041-xyz -110228161386024960 10000 1993-07-24T14:27:52.132 10000 1993-07-24T14:27:52.132 201 123321 false 1993-07-24T14:27:51 xyz {"wTS":"abc72071","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72071-xyz -110228287328391168 10000 1993-07-24T14:28:22.159 10000 1993-07-24T14:28:22.159 201 123321 false 1993-07-24T14:28:21 xyz {"wTS":"abc72101","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72101-xyz -110228539603193856 10000 1993-07-24T14:29:22.306 10000 1993-07-24T14:29:22.306 201 123321 false 1993-07-24T14:29:21 xyz {"wTS":"abc72161","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72161-xyz -110228792138043392 10000 1993-07-24T14:30:22.515 10000 1993-07-24T14:30:22.515 201 123321 false 1993-07-24T14:30:21 xyz {"wTS":"abc72221","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72221-xyz -110228917686145024 10000 1993-07-24T14:30:52.448 10000 1993-07-24T14:30:52.448 201 123321 false 1993-07-24T14:30:51 xyz {"wTS":"abc72251","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72251-xyz -110229045104906240 10000 1993-07-24T14:31:22.827 10000 1993-07-24T14:31:22.827 201 123321 false 1993-07-24T14:31:21 xyz {"wTS":"abc72281","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72281-xyz -110229169679929344 10000 1993-07-24T14:31:52.528 10000 1993-07-24T14:31:52.528 201 123321 false 1993-07-24T14:31:51 xyz {"wTS":"abc72311","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72311-xyz -110229295722958848 10000 1993-07-24T14:32:22.579 10000 1993-07-24T14:32:22.579 201 123321 false 1993-07-24T14:32:21 xyz {"wTS":"abc72341","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72341-xyz -110229421824708608 10000 1993-07-24T14:32:52.644 10000 1993-07-24T14:32:52.644 201 123321 false 1993-07-24T14:32:52 xyz {"wTS":"abc72372","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72372-xyz -110229547901292544 10000 1993-07-24T14:33:22.703 10000 1993-07-24T14:33:22.703 201 123321 false 1993-07-24T14:33:22 xyz {"wTS":"abc72402","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72402-xyz -110229674154037248 10000 1993-07-24T14:33:52.804 10000 1993-07-24T14:33:52.804 201 123321 false 1993-07-24T14:33:52 xyz {"wTS":"abc72432","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72432-xyz -110229800067043328 10000 1993-07-24T14:34:22.824 10000 1993-07-24T14:34:22.824 201 123321 false 1993-07-24T14:34:22 xyz {"wTS":"abc72462","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72462-xyz -110229925980049408 10000 1993-07-24T14:34:52.844 10000 1993-07-24T14:34:52.844 201 123321 false 1993-07-24T14:34:52 xyz {"wTS":"abc72492","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72492-xyz -110230052274737152 10000 1993-07-24T14:35:22.955 10000 1993-07-24T14:35:22.955 201 123321 false 1993-07-24T14:35:22 xyz {"wTS":"abc72522","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72522-xyz -110230178028359680 10000 1993-07-24T14:35:52.937 10000 1993-07-24T14:35:52.937 201 123321 false 1993-07-24T14:35:52 xyz {"wTS":"abc72552","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72552-xyz -110230304310464512 10000 1993-07-24T14:36:23.045 10000 1993-07-24T14:36:23.045 201 123321 false 1993-07-24T14:36:22 xyz {"wTS":"abc72582","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72582-xyz -110230430491906048 10000 1993-07-24T14:36:53.129 10000 1993-07-24T14:36:53.129 201 123321 false 1993-07-24T14:36:52 xyz {"wTS":"abc72612","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72612-xyz -110230556434272256 10000 1993-07-24T14:37:23.156 10000 1993-07-24T14:37:23.156 201 123321 false 1993-07-24T14:37:22 xyz {"wTS":"abc72642","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72642-xyz -110230682510856192 10000 1993-07-24T14:37:53.215 10000 1993-07-24T14:37:53.215 201 123321 false 1993-07-24T14:37:52 xyz {"wTS":"abc72672","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72672-xyz -110230808419667968 10000 1993-07-24T14:38:23.234 10000 1993-07-24T14:38:23.234 201 123321 false 1993-07-24T14:38:22 xyz {"wTS":"abc72702","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72702-xyz -110230935968452608 10000 1993-07-24T14:38:53.644 10000 1993-07-24T14:38:53.644 201 123321 false 1993-07-24T14:38:52 xyz {"wTS":"abc72732","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72732-xyz -110231061294256128 10000 1993-07-24T14:39:23.524 10000 1993-07-24T14:39:23.524 201 123321 false 1993-07-24T14:39:22 xyz {"wTS":"abc72762","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72762-xyz -110231186666196992 10000 1993-07-24T14:39:53.415 10000 1993-07-24T14:39:53.415 201 123321 false 1993-07-24T14:39:52 xyz {"wTS":"abc72792","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72792-xyz -110231312654700544 10000 1993-07-24T14:40:23.453 10000 1993-07-24T14:40:23.453 201 123321 false 1993-07-24T14:40:22 xyz {"wTS":"abc72822","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72822-xyz -110231438513180672 10000 1993-07-24T14:40:53.460 10000 1993-07-24T14:40:53.460 201 123321 false 1993-07-24T14:40:52 xyz {"wTS":"abc72852","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72852-xyz -110233741108480768 10000 1993-07-24T14:50:03.934 10000 1993-07-24T14:50:03.934 201 123321 false 1993-07-24T14:45:53 xyz {"wTS":"abc73153","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73153-xyz -110233743476166656 10000 1993-07-24T14:50:03.006 10000 1993-07-24T14:50:03.006 201 123321 false 1993-07-24T14:41:22 xyz {"wTS":"abc72882","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72882-xyz -110233744231141376 10000 1993-07-24T14:50:03.186 10000 1993-07-24T14:50:03.186 201 123321 false 1993-07-24T14:42:23 xyz {"wTS":"abc72943","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72943-xyz -110233744503771136 10000 1993-07-24T14:50:03.251 10000 1993-07-24T14:50:03.251 201 123321 false 1993-07-24T14:41:52 xyz {"wTS":"abc72912","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72912-xyz -110233745111945216 10000 1993-07-24T14:50:03.396 10000 1993-07-24T14:50:03.396 201 123321 false 1993-07-24T14:42:53 xyz {"wTS":"abc72973","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc72973-xyz -110233745283911680 10000 1993-07-24T14:50:03.437 10000 1993-07-24T14:50:03.437 201 123321 false 1993-07-24T14:43:53 xyz {"wTS":"abc73033","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73033-xyz -110233745485238272 10000 1993-07-24T14:50:03.485 10000 1993-07-24T14:50:03.485 201 123321 false 1993-07-24T14:43:23 xyz {"wTS":"abc73003","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73003-xyz -110233745703342080 10000 1993-07-24T14:50:03.537 10000 1993-07-24T14:50:03.537 201 123321 false 1993-07-24T14:44:53 xyz {"wTS":"abc73093","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73093-xyz -110233746064052224 10000 1993-07-24T14:50:03.623 10000 1993-07-24T14:50:03.623 201 123321 false 1993-07-24T14:45:23 xyz {"wTS":"abc73123","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73123-xyz -110233746315710464 10000 1993-07-24T14:50:03.683 10000 1993-07-24T14:50:03.683 201 123321 false 1993-07-24T14:44:23 xyz {"wTS":"abc73063","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73063-xyz -110233746919690240 10000 1993-07-24T14:50:03.827 10000 1993-07-24T14:50:03.827 201 123321 false 1993-07-24T14:46:53 xyz {"wTS":"abc73213","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73213-xyz -110233747469144064 10000 1993-07-24T14:50:03.958 10000 1993-07-24T14:50:03.958 201 123321 false 1993-07-24T14:46:23 xyz {"wTS":"abc73183","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73183-xyz -110233747863408640 10000 1993-07-24T14:50:04.052 10000 1993-07-24T14:50:04.052 201 123321 false 1993-07-24T14:47:53 xyz {"wTS":"abc73273","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73273-xyz -110233748387696640 10000 1993-07-24T14:50:04.177 10000 1993-07-24T14:50:04.177 201 123321 false 1993-07-24T14:48:23 xyz {"wTS":"abc73303","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73303-xyz -110233748760989696 10000 1993-07-24T14:50:04.266 10000 1993-07-24T14:50:04.266 201 123321 false 1993-07-24T14:47:23 xyz {"wTS":"abc73243","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73243-xyz -110233748987482112 10000 1993-07-24T14:50:04.320 10000 1993-07-24T14:50:04.320 201 123321 false 1993-07-24T14:48:53 xyz {"wTS":"abc73333","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73333-xyz -110233749117505536 10000 1993-07-24T14:50:04.351 10000 1993-07-24T14:50:04.351 201 123321 false 1993-07-24T14:49:23 xyz {"wTS":"abc73363","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73363-xyz -110233749348192256 10000 1993-07-24T14:50:04.406 10000 1993-07-24T14:50:04.406 201 123321 false 1993-07-24T14:49:53 xyz {"wTS":"abc73393","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73393-xyz -110233834433843200 10000 1993-07-24T14:50:24.692 10000 1993-07-24T14:50:24.692 201 123321 false 1993-07-24T14:50:23 xyz {"wTS":"abc73423","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73423-xyz -110233960820805632 10000 1993-07-24T14:50:54.825 10000 1993-07-24T14:50:54.825 201 123321 false 1993-07-24T14:50:54 xyz {"wTS":"abc73454","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73454-xyz -110234212168667136 10000 1993-07-24T14:51:54.751 10000 1993-07-24T14:51:54.751 201 123321 false 1993-07-24T14:51:54 xyz {"wTS":"abc73514","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73514-xyz -110234338161364992 10000 1993-07-24T14:52:24.790 10000 1993-07-24T14:52:24.790 201 123321 false 1993-07-24T14:52:24 xyz {"wTS":"abc73544","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73544-xyz -110234590239035392 10000 1993-07-24T14:53:24.890 10000 1993-07-24T14:53:24.890 201 123321 false 1993-07-24T14:53:24 xyz {"wTS":"abc11004","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc11004-xyz -110234716424671232 10000 1993-07-24T14:53:54.975 10000 1993-07-24T14:53:54.975 201 123321 false 1993-07-24T14:53:54 xyz {"wTS":"abc11034","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc11034-xyz -110235560025034752 10000 1993-07-24T14:57:16.105 10000 1993-07-24T14:57:16.106 201 123321 false 1993-07-24T14:57:15 xyz {"wTS":"abc73835","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73835-xyz -110235688232325120 10000 1993-07-24T14:57:46.672 10000 1993-07-24T14:57:46.672 201 123321 false 1993-07-24T14:57:46 xyz {"wTS":"abc73866","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73866-xyz -110235814371823616 10000 1993-07-24T14:58:16.746 10000 1993-07-24T14:58:16.746 201 123321 false 1993-07-24T14:58:16 xyz {"wTS":"abc73896","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73896-xyz -110235940360327168 10000 1993-07-24T14:58:46.784 10000 1993-07-24T14:58:46.784 201 123321 false 1993-07-24T14:58:46 xyz {"wTS":"abc73926","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73926-xyz -110236067707785216 10000 1993-07-24T14:59:17.146 10000 1993-07-24T14:59:17.146 201 123321 false 1993-07-24T14:59:16 xyz {"wTS":"abc73956","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73956-xyz -110236192224088064 10000 1993-07-24T14:59:46.833 10000 1993-07-24T14:59:46.833 201 123321 false 1993-07-24T14:59:46 xyz {"wTS":"abc73986","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc73986-xyz -110236318577496064 10000 1993-07-24T15:00:16.958 10000 1993-07-24T15:00:16.958 201 123321 false 1993-07-24T15:00:16 xyz {"wTS":"abc74016","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74016-xyz -110236570869075968 10000 1993-07-24T15:01:17.109 10000 1993-07-24T15:01:17.109 201 123321 false 1993-07-24T15:01:16 xyz {"wTS":"abc74076","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74076-xyz -110236697297981440 10000 1993-07-24T15:01:47.252 10000 1993-07-24T15:01:47.252 201 123321 false 1993-07-24T15:01:46 xyz {"wTS":"abc74106","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74106-xyz -110236823001272320 10000 1993-07-24T15:02:17.222 10000 1993-07-24T15:02:17.222 201 123321 false 1993-07-24T15:02:16 xyz {"wTS":"abc74136","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74136-xyz -110236948876529664 10000 1993-07-24T15:02:47.233 10000 1993-07-24T15:02:47.233 201 123321 false 1993-07-24T15:02:46 xyz {"wTS":"abc74166","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74166-xyz -110237074726621184 10000 1993-07-24T15:03:17.238 10000 1993-07-24T15:03:17.238 201 123321 false 1993-07-24T15:03:16 xyz {"wTS":"abc74196","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74196-xyz -110238336427790336 10000 1993-07-24T15:08:18.051 10000 1993-07-24T15:08:18.051 201 123321 false 1993-07-24T15:08:17 xyz {"wTS":"abc74497","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74497-xyz -110238461564850176 10000 1993-07-24T15:08:47.886 10000 1993-07-24T15:08:47.886 201 123321 false 1993-07-24T15:08:47 xyz {"wTS":"abc74527","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74527-xyz -110238587549159424 10000 1993-07-24T15:09:17.923 10000 1993-07-24T15:09:17.923 201 123321 false 1993-07-24T15:09:17 xyz {"wTS":"abc74557","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74557-xyz -110238713772544000 10000 1993-07-24T15:09:48.017 10000 1993-07-24T15:09:48.017 201 123321 false 1993-07-24T15:09:47 xyz {"wTS":"abc74587","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74587-xyz -110238839517777920 10000 1993-07-24T15:10:17.997 10000 1993-07-24T15:10:17.997 201 123321 false 1993-07-24T15:10:17 xyz {"wTS":"abc74617","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74617-xyz -110238965250428928 10000 1993-07-24T15:10:47.974 10000 1993-07-24T15:10:47.974 201 123321 false 1993-07-24T15:10:47 xyz {"wTS":"abc74647","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74647-xyz -110239216921251840 10000 1993-07-24T15:11:47.977 10000 1993-07-24T15:11:47.977 201 123321 false 1993-07-24T15:11:46 xyz {"wTS":"abc74706","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74706-xyz -110239342381273088 10000 1993-07-24T15:12:17.889 10000 1993-07-24T15:12:17.889 201 123321 false 1993-07-24T15:12:16 xyz {"wTS":"abc74110","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74110-xyz -110239467153428480 10000 1993-07-24T15:12:47.637 10000 1993-07-24T15:12:47.637 201 123321 false 1993-07-24T15:12:46 xyz {"wTS":"abc74766","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74766-xyz -110239593200652288 10000 1993-07-24T15:13:17.689 10000 1993-07-24T15:13:17.689 201 123321 false 1993-07-24T15:13:17 xyz {"wTS":"abc74797","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74797-xyz -110239719096881152 10000 1993-07-24T15:13:47.705 10000 1993-07-24T15:13:47.705 201 123321 false 1993-07-24T15:13:47 xyz {"wTS":"abc74827","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74827-xyz -110239845207019520 10000 1993-07-24T15:14:17.772 10000 1993-07-24T15:14:17.772 201 123321 false 1993-07-24T15:14:17 xyz {"wTS":"abc74857","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74857-xyz -110239971363295232 10000 1993-07-24T15:14:47.850 10000 1993-07-24T15:14:47.850 201 123321 false 1993-07-24T15:14:47 xyz {"wTS":"abc74887","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74887-xyz -110240098626867200 10000 1993-07-24T15:15:18.192 10000 1993-07-24T15:15:18.192 201 123321 false 1993-07-24T15:15:17 xyz {"wTS":"abc74917","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74917-xyz -110240224049139712 10000 1993-07-24T15:15:48.095 10000 1993-07-24T15:15:48.095 201 123321 false 1993-07-24T15:15:47 xyz {"wTS":"abc74947","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74947-xyz -110240349840510976 10000 1993-07-24T15:16:18.086 10000 1993-07-24T15:16:18.086 201 123321 false 1993-07-24T15:16:17 xyz {"wTS":"abc74977","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc74977-xyz -110240476395245568 10000 1993-07-24T15:16:48.259 10000 1993-07-24T15:16:48.259 201 123321 false 1993-07-24T15:16:47 xyz {"wTS":"abc75007","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75007-xyz -110240601515528192 10000 1993-07-24T15:17:18.090 10000 1993-07-24T15:17:18.090 201 123321 false 1993-07-24T15:17:17 xyz {"wTS":"abc75037","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75037-xyz -110240727730524160 10000 1993-07-24T15:17:48.182 10000 1993-07-24T15:17:48.182 201 123321 false 1993-07-24T15:17:47 xyz {"wTS":"abc75067","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75067-xyz -110240853538672640 10000 1993-07-24T15:18:18.178 10000 1993-07-24T15:18:18.178 201 123321 false 1993-07-24T15:18:17 xyz {"wTS":"abc75097","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75097-xyz -110240979623645184 10000 1993-07-24T15:18:48.238 10000 1993-07-24T15:18:48.238 201 123321 false 1993-07-24T15:18:47 xyz {"wTS":"abc75127","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75127-xyz -110241106841079808 10000 1993-07-24T15:19:18.569 10000 1993-07-24T15:19:18.569 201 123321 false 1993-07-24T15:19:17 xyz {"wTS":"abc75157","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75157-xyz -110241110250920960 10000 1993-07-24T15:21:48.632 10000 1993-07-24T15:21:48.632 201 123321 false 1993-07-24T15:21:47 xyz {"wTS":"abc75307","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75307-xyz -110241231827144704 10000 1993-07-24T15:19:48.368 10000 1993-07-24T15:19:48.368 201 123321 false 1993-07-24T15:19:47 xyz {"wTS":"abc75187","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75187-xyz -110241357886951424 10000 1993-07-24T15:20:18.423 10000 1993-07-24T15:20:18.423 201 123321 false 1993-07-24T15:20:17 xyz {"wTS":"abc75217","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75217-xyz -110241484072587264 10000 1993-07-24T15:20:48.508 10000 1993-07-24T15:20:48.508 201 123321 false 1993-07-24T15:20:47 xyz {"wTS":"abc75247","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75247-xyz -110241610224668672 10000 1993-07-24T15:21:18.585 10000 1993-07-24T15:21:18.585 201 123321 false 1993-07-24T15:21:17 xyz {"wTS":"abc75277","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75277-xyz -110241862428168192 10000 1993-07-24T15:22:18.715 10000 1993-07-24T15:22:18.715 201 123321 false 1993-07-24T15:22:17 xyz {"wTS":"abc75337","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75337-xyz -110241988550889472 10000 1993-07-24T15:22:48.785 10000 1993-07-24T15:22:48.785 201 123321 false 1993-07-24T15:22:48 xyz {"wTS":"abc75368","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75368-xyz -110242114287734784 10000 1993-07-24T15:23:18.763 10000 1993-07-24T15:23:18.763 201 123321 false 1993-07-24T15:23:18 xyz {"wTS":"abc75398","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75398-xyz -110242240129437696 10000 1993-07-24T15:23:48.766 10000 1993-07-24T15:23:48.766 201 123321 false 1993-07-24T15:23:48 xyz {"wTS":"abc75428","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75428-xyz -110242366264741888 10000 1993-07-24T15:24:18.839 10000 1993-07-24T15:24:18.839 201 123321 false 1993-07-24T15:24:18 xyz {"wTS":"abc75458","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75458-xyz -110242492534263808 10000 1993-07-24T15:24:48.945 10000 1993-07-24T15:24:48.945 201 123321 false 1993-07-24T15:24:48 xyz {"wTS":"abc75488","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75488-xyz -110242618149474304 10000 1993-07-24T15:25:18.893 10000 1993-07-24T15:25:18.893 201 123321 false 1993-07-24T15:25:18 xyz {"wTS":"abc75518","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75518-xyz -110242744481910784 10000 1993-07-24T15:25:49.013 10000 1993-07-24T15:25:49.013 201 123321 false 1993-07-24T15:25:48 xyz {"wTS":"abc75548","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75548-xyz -110242870382333952 10000 1993-07-24T15:26:19.030 10000 1993-07-24T15:26:19.030 201 123321 false 1993-07-24T15:26:18 xyz {"wTS":"abc75578","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75578-xyz -110242996479889408 10000 1993-07-24T15:26:49.094 10000 1993-07-24T15:26:49.094 201 123321 false 1993-07-24T15:26:48 xyz {"wTS":"abc75608","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75608-xyz -110243122464198656 10000 1993-07-24T15:27:19.131 10000 1993-07-24T15:27:19.131 201 123321 false 1993-07-24T15:27:18 xyz {"wTS":"abc75638","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75638-xyz -110243248985378816 10000 1993-07-24T15:27:49.296 10000 1993-07-24T15:27:49.296 201 123321 false 1993-07-24T15:27:48 xyz {"wTS":"abc75668","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75668-xyz -110243374441205760 10000 1993-07-24T15:28:19.207 10000 1993-07-24T15:28:19.207 201 123321 false 1993-07-24T15:28:18 xyz {"wTS":"abc75698","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75698-xyz -110243500798808064 10000 1993-07-24T15:28:49.333 10000 1993-07-24T15:28:49.333 201 123321 false 1993-07-24T15:28:48 xyz {"wTS":"abc75728","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75728-xyz -110243627026386944 10000 1993-07-24T15:29:19.428 10000 1993-07-24T15:29:19.428 201 123321 false 1993-07-24T15:29:18 xyz {"wTS":"abc75758","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75758-xyz -110243754067660800 10000 1993-07-24T15:29:49.717 10000 1993-07-24T15:29:49.717 201 123321 false 1993-07-24T15:29:48 xyz {"wTS":"abc75788","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75788-xyz -110243878957256704 10000 1993-07-24T15:30:19.493 10000 1993-07-24T15:30:19.493 201 123321 false 1993-07-24T15:30:18 xyz {"wTS":"abc75818","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75818-xyz -110244004870262784 10000 1993-07-24T15:30:49.513 10000 1993-07-24T15:30:49.513 201 123321 false 1993-07-24T15:30:48 xyz {"wTS":"abc75848","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75848-xyz -110244130875543552 10000 1993-07-24T15:31:19.555 10000 1993-07-24T15:31:19.555 201 123321 false 1993-07-24T15:31:18 xyz {"wTS":"abc75878","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75878-xyz -110244256989876224 10000 1993-07-24T15:31:49.623 10000 1993-07-24T15:31:49.624 201 123321 false 1993-07-24T15:31:49 xyz {"wTS":"abc75909","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75909-xyz -110244383129374720 10000 1993-07-24T15:32:19.697 10000 1993-07-24T15:32:19.697 201 123321 false 1993-07-24T15:32:19 xyz {"wTS":"abc75939","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75939-xyz -110244509516337152 10000 1993-07-24T15:32:49.830 10000 1993-07-24T15:32:49.830 201 123321 false 1993-07-24T15:32:49 xyz {"wTS":"abc75969","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75969-xyz -110244635370622976 10000 1993-07-24T15:33:19.836 10000 1993-07-24T15:33:19.836 201 123321 false 1993-07-24T15:33:19 xyz {"wTS":"abc75999","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc75999-xyz -110244761581424640 10000 1993-07-24T15:33:49.927 10000 1993-07-24T15:33:49.927 201 123321 false 1993-07-24T15:33:49 xyz {"wTS":"abc76029","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76029-xyz -110244887590899712 10000 1993-07-24T15:34:19.970 10000 1993-07-24T15:34:19.970 201 123321 false 1993-07-24T15:34:19 xyz {"wTS":"abc76059","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76059-xyz -110245013550043136 10000 1993-07-24T15:34:50.001 10000 1993-07-24T15:34:50.002 201 123321 false 1993-07-24T15:34:49 xyz {"wTS":"abc76089","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76089-xyz -110245139765039104 10000 1993-07-24T15:35:20.093 10000 1993-07-24T15:35:20.093 201 123321 false 1993-07-24T15:35:19 xyz {"wTS":"abc76119","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76119-xyz -110245265950674944 10000 1993-07-24T15:35:50.178 10000 1993-07-24T15:35:50.178 201 123321 false 1993-07-24T15:35:49 xyz {"wTS":"abc76149","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76149-xyz -110245392366997504 10000 1993-07-24T15:36:20.318 10000 1993-07-24T15:36:20.318 201 123321 false 1993-07-24T15:36:19 xyz {"wTS":"abc76179","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76179-xyz -110245517818630144 10000 1993-07-24T15:36:50.228 10000 1993-07-24T15:36:50.228 201 123321 false 1993-07-24T15:36:49 xyz {"wTS":"abc76209","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76209-xyz -110245644071374848 10000 1993-07-24T15:37:20.329 10000 1993-07-24T15:37:20.329 201 123321 false 1993-07-24T15:37:19 xyz {"wTS":"abc76239","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76239-xyz -110245770043101184 10000 1993-07-24T15:37:50.363 10000 1993-07-24T15:37:50.363 201 123321 false 1993-07-24T15:37:49 xyz {"wTS":"abc76269","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76269-xyz -110245896203571200 10000 1993-07-24T15:38:20.442 10000 1993-07-24T15:38:20.442 201 123321 false 1993-07-24T15:38:19 xyz {"wTS":"abc76299","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76299-xyz -110246022175297536 10000 1993-07-24T15:38:50.476 10000 1993-07-24T15:38:50.476 201 123321 false 1993-07-24T15:38:49 xyz {"wTS":"abc76329","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76329-xyz -110246147996028928 10000 1993-07-24T15:39:20.474 10000 1993-07-24T15:39:20.474 201 123321 false 1993-07-24T15:39:19 xyz {"wTS":"abc76359","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76359-xyz -110246274026475520 10000 1993-07-24T15:39:50.522 10000 1993-07-24T15:39:50.522 201 123321 false 1993-07-24T15:39:49 xyz {"wTS":"abc76389","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76389-xyz -110246400031756288 10000 1993-07-24T15:40:20.564 10000 1993-07-24T15:40:20.565 201 123321 false 1993-07-24T15:40:19 xyz {"wTS":"abc76419","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76419-xyz -110246526204809216 10000 1993-07-24T15:40:50.646 10000 1993-07-24T15:40:50.646 201 123321 false 1993-07-24T15:40:49 xyz {"wTS":"abc76449","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76449-xyz -110246651966820352 10000 1993-07-24T15:41:20.630 10000 1993-07-24T15:41:20.630 201 123321 false 1993-07-24T15:41:20 xyz {"wTS":"abc76480","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76480-xyz -110246778110513152 10000 1993-07-24T15:41:50.705 10000 1993-07-24T15:41:50.705 201 123321 false 1993-07-24T15:41:50 xyz {"wTS":"abc76510","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76510-xyz -110246904396812288 10000 1993-07-24T15:42:20.814 10000 1993-07-24T15:42:20.814 201 123321 false 1993-07-24T15:42:20 xyz {"wTS":"abc76540","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76540-xyz -110247030221737984 10000 1993-07-24T15:42:50.813 10000 1993-07-24T15:42:50.813 201 123321 false 1993-07-24T15:42:50 xyz {"wTS":"abc76570","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76570-xyz -110247157414006784 10000 1993-07-24T15:43:21.138 10000 1993-07-24T15:43:21.138 201 123321 false 1993-07-24T15:43:20 xyz {"wTS":"abc76600","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76600-xyz -110247282832084992 10000 1993-07-24T15:43:51.040 10000 1993-07-24T15:43:51.040 201 123321 false 1993-07-24T15:43:50 xyz {"wTS":"abc76630","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76630-xyz -110247408644427776 10000 1993-07-24T15:44:21.036 10000 1993-07-24T15:44:21.036 201 123321 false 1993-07-24T15:44:20 xyz {"wTS":"abc76660","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76660-xyz -110247534695845888 10000 1993-07-24T15:44:51.089 10000 1993-07-24T15:44:51.089 201 123321 false 1993-07-24T15:44:50 xyz {"wTS":"abc76690","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76690-xyz -110247660474634240 10000 1993-07-24T15:45:21.077 10000 1993-07-24T15:45:21.077 201 123321 false 1993-07-24T15:45:20 xyz {"wTS":"abc76720","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76720-xyz -110247786630909952 10000 1993-07-24T15:45:51.155 10000 1993-07-24T15:45:51.155 201 123321 false 1993-07-24T15:45:50 xyz {"wTS":"abc76750","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76750-xyz -110247912862683136 10000 1993-07-24T15:46:21.251 10000 1993-07-24T15:46:21.251 201 123321 false 1993-07-24T15:46:20 xyz {"wTS":"abc76780","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76780-xyz -110248038553391104 10000 1993-07-24T15:46:51.218 10000 1993-07-24T15:46:51.218 201 123321 false 1993-07-24T15:46:50 xyz {"wTS":"abc76810","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76810-xyz -110248164768387072 10000 1993-07-24T15:47:21.310 10000 1993-07-24T15:47:21.310 201 123321 false 1993-07-24T15:47:20 xyz {"wTS":"abc76840","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76840-xyz -110248290807222272 10000 1993-07-24T15:47:51.360 10000 1993-07-24T15:47:51.360 201 123321 false 1993-07-24T15:47:50 xyz {"wTS":"abc76870","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76870-xyz -110248417680723968 10000 1993-07-24T15:48:21.609 10000 1993-07-24T15:48:21.609 201 123321 false 1993-07-24T15:48:20 xyz {"wTS":"abc76900","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76900-xyz -110248543044276224 10000 1993-07-24T15:48:51.498 10000 1993-07-24T15:48:51.498 201 123321 false 1993-07-24T15:48:50 xyz {"wTS":"abc76930","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76930-xyz -110248668856619008 10000 1993-07-24T15:49:21.494 10000 1993-07-24T15:49:21.494 201 123321 false 1993-07-24T15:49:20 xyz {"wTS":"abc76960","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76960-xyz -110248794903842816 10000 1993-07-24T15:49:51.546 10000 1993-07-24T15:49:51.546 201 123321 false 1993-07-24T15:49:50 xyz {"wTS":"abc76990","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc76990-xyz -110248922033197056 10000 1993-07-24T15:50:21.856 10000 1993-07-24T15:50:21.856 201 123321 false 1993-07-24T15:50:20 xyz {"wTS":"abc77020","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc77020-xyz -110249047547744256 10000 1993-07-24T15:50:51.781 10000 1993-07-24T15:50:51.781 201 123321 false 1993-07-24T15:50:51 xyz {"wTS":"abc77051","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc77051-xyz -110249173741768704 10000 1993-07-24T15:51:21.868 10000 1993-07-24T15:51:21.868 201 123321 false 1993-07-24T15:51:21 xyz {"wTS":"abc77081","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc77081-xyz -110249299239538688 10000 1993-07-24T15:51:51.789 10000 1993-07-24T15:51:51.789 201 123321 false 1993-07-24T15:51:51 xyz {"wTS":"abc77111","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc77111-xyz -110249425433563136 10000 1993-07-24T15:52:21.876 10000 1993-07-24T15:52:21.876 201 123321 false 1993-07-24T15:52:21 xyz {"wTS":"abc77141","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc77141-xyz -110249551396900864 10000 1993-07-24T15:52:51.908 10000 1993-07-24T15:52:51.908 201 123321 false 1993-07-24T15:52:51 xyz {"wTS":"abc77171","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc77171-xyz -110249677473484800 10000 1993-07-24T15:53:21.967 10000 1993-07-24T15:53:21.967 201 123321 false 1993-07-24T15:53:21 xyz {"wTS":"abc77201","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc77201-xyz -110249803487154176 10000 1993-07-24T15:53:52.011 10000 1993-07-24T15:53:52.011 201 123321 false 1993-07-24T15:53:51 xyz {"wTS":"abc77231","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc77231-xyz -110249929664401408 10000 1993-07-24T15:54:22.094 10000 1993-07-24T15:54:22.094 201 123321 false 1993-07-24T15:54:21 xyz {"wTS":"abc77261","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc77261-xyz -110250055740985344 10000 1993-07-24T15:54:52.153 10000 1993-07-24T15:54:52.153 201 123321 false 1993-07-24T15:54:51 xyz {"wTS":"abc77291","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc77291-xyz -110250181804986368 10000 1993-07-24T15:55:22.209 10000 1993-07-24T15:55:22.209 201 123321 false 1993-07-24T15:55:21 xyz {"wTS":"abc77321","rfId":"xyz","remark":"数据","weight":100} 1233211234567891011 abc77321-xyz -110708314073595904 10000 1993-07-25T22:15:49.453 10000 1993-07-25T22:15:49.453 201 123321 false 1993-07-25T22:15:48 0410-137ijk {"wTS":"1595686548","rfId":"0410-137ijk","remark":"数据","weight":0.9} 1233211234567891011 1595686548-0410-137ijk -110708469292204032 10000 1993-07-25T22:16:26.460 10000 1993-07-25T22:16:26.461 201 123321 false 1993-07-25T22:16:25 0410-137ijk {"wTS":"1595686585","rfId":"0410-137ijk","remark":"数据","weight":0.9} 1233211234567891011 1595686585-0410-137ijk -135164473750999040 10000 1993-07-21T16:01:09.237 10000 1993-07-21T16:01:09.237 201 123321 false 0410-137ijk {"wTS":"1595318468052","rfId":"0410-137ijk","remark":"数据","weight":0.2} 1233211234567891011 1595318468052-0410-137ijk -135165117484388352 10000 1993-07-21T16:03:42.715 10000 1993-07-21T16:03:42.715 201 123321 false 1993-07-21T16:03:41 0410-137ijk {"wTS":"1595318621","rfId":"0410-137ijk","remark":"数据","weight":0.2} 1233211234567891011 1595318621-0410-137ijk -135167443414700032 10000 1993-07-21T16:12:57.260 10000 1993-07-21T16:12:57.260 201 123321 false 1993-07-21T16:12:56 0410-137ijk {"wTS":"1595319176","rfId":"0410-137ijk","remark":"数据","weight":0.2} 1233211234567891011 1595319176-0410-137ijk -135167805882257408 10000 1993-07-21T16:14:23.679 10000 1993-07-21T16:14:23.679 201 123321 false 1993-07-21T16:14:22 0410-137ijk {"wTS":"1595319262","rfId":"0410-137ijk","remark":"数据","weight":2} 1233211234567891011 1595319262-0410-137ijk -135167875595784192 10000 1993-07-21T16:14:40.300 10000 1993-07-21T16:14:40.300 201 123321 false 1993-07-21T16:14:39 0410-137ijk {"wTS":"1595319279","rfId":"0410-137ijk","remark":"数据","weight":2} 1233211234567891011 1595319279-0410-137ijk -138903405079396352 10000 1993-07-31T23:38:19.932 10000 1993-07-31T23:38:19.932 201 123321 false 1993-07-31T23:38:18 0410-137110005 {"wTS":"1596209898","rfId":"0410-137110005","remark":"数据","weight":30} 1233211234567891011 1596209898-0410-137110005 -138905785832038400 10000 1993-07-31T23:47:47.547 10000 1993-07-31T23:47:47.547 201 123321 false 1993-07-31T23:47:45 0410-137110005 {"wTS":"1596210465","rfId":"0410-137110005","remark":"数据","weight":500} 1233211234567891011 1596210465-0410-137110005 -138905859697926144 10000 1993-07-31T23:48:05.158 10000 1993-07-31T23:48:05.159 201 123321 false 1993-07-31T23:48:03 0410-137110005 {"wTS":"1596210483","rfId":"0410-137110005","remark":"数据","weight":500.001} 1233211234567891011 1596210483-0410-137110005 -138906169749266432 10000 1993-07-31T23:49:19.080 10000 1993-07-31T23:49:19.080 201 123321 false 1993-07-31T23:49:17 0410-137110005 {"wTS":"1596210557","rfId":"0410-137110005","remark":"数据","weight":0.5} 1233211234567891011 1596210557-0410-137110005 -13978385225 10000 1993-08-03T15:28:11.301 10000 1993-08-03T15:28:11.301 201 456678 false 1993-08-03T15:28:10 010-18741038 {"wTS":"1596439690","rfId":"010-18741038","remark":"数据","weight":70.404} 710078936338096128 1596439690-010-18741038 -139866186189201408 10000 1993-08-03T15:24:04.836 10000 1993-08-03T15:24:04.836 201 456678 false 1993-08-03T15:24:04 010-18741039 {"wTS":"1596439444","rfId":"010-18741039","remark":"数据","weight":100} 710078936338096128 1596439444-010-18741039 -139866186201784320 10000 1993-08-03T15:24:04.839 10000 1993-08-03T15:24:04.839 201 456678 false 1993-08-03T15:24:04 010-18741035 {"wTS":"1596439444","rfId":"010-18741035","remark":"数据","weight":100} 710078936338096128 1596439444-010-18741035 -139866186575077376 10000 1993-08-03T15:24:04.928 10000 1993-08-03T15:24:04.928 201 456678 false 1993-08-03T15:24:04 010-18741036 {"wTS":"1596439444","rfId":"010-18741036","remark":"数据","weight":100} 710078936338096128 1596439444-010-18741036 -139866187183251456 10000 1993-08-03T15:24:05.073 10000 1993-08-03T15:24:05.073 201 456678 false 1993-08-03T15:24:04 010-18741040 {"wTS":"1596439444","rfId":"010-18741040","remark":"数据","weight":100} 710078936338096128 1596439444-010-18741040 -139866187455881216 10000 1993-08-03T15:24:05.138 10000 1993-08-03T15:24:05.138 201 456678 false 1993-08-03T15:24:04 010-18741033 {"wTS":"1596439444","rfId":"010-18741033","remark":"数据","weight":100} 710078936338096128 1596439444-010-18741033 -139866187862728704 10000 1993-08-03T15:24:05.235 10000 1993-08-03T15:24:05.235 201 456678 false 1993-08-03T15:24:04 010-18741034 {"wTS":"1596439444","rfId":"010-18741034","remark":"数据","weight":100} 710078936338096128 1596439444-010-18741034 -139866188911304704 10000 1993-08-03T15:24:05.486 10000 1993-08-03T15:24:05.486 201 456678 false 1993-08-03T15:24:04 010-18741038 {"wTS":"1596439444","rfId":"010-18741038","remark":"数据","weight":100} 710078936338096128 1596439444-010-18741038 -139866189112631296 10000 1993-08-03T15:24:05.533 10000 1993-08-03T15:24:05.533 201 456678 false 1993-08-03T15:24:04 010-18741032 {"wTS":"1596439444","rfId":"010-18741032","remark":"数据","weight":100} 710078936338096128 1596439444-010-18741032 -139866189481730048 10000 1993-08-03T15:24:05.621 10000 1993-08-03T15:24:05.621 201 456678 false 1993-08-03T15:24:04 010-18741031 {"wTS":"1596439444","rfId":"010-18741031","remark":"数据","weight":100} 710078936338096128 1596439444-010-18741031 -139866190014406656 10000 1993-08-03T15:24:05.748 10000 1993-08-03T15:24:05.748 201 456678 false 1993-08-03T15:24:04 010-18741037 {"wTS":"1596439444","rfId":"010-18741037","remark":"数据","weight":100} 710078936338096128 1596439444-010-18741037 -139866315625422848 10000 1993-08-03T15:24:35.696 10000 1993-08-03T15:24:35.696 201 456678 false 1993-08-03T15:24:35 010-18741039 {"wTS":"1596439475","rfId":"010-18741039","remark":"数据","weight":100} 710078936338096128 1596439475-010-18741039 -139866315784806400 10000 1993-08-03T15:24:35.734 10000 1993-08-03T15:24:35.734 201 456678 false 1993-08-03T15:24:35 010-18741040 {"wTS":"1596439475","rfId":"010-18741040","remark":"数据","weight":100} 710078936338096128 1596439475-010-18741040 -139866316082601984 10000 1993-08-03T15:24:35.805 10000 1993-08-03T15:24:35.805 201 456678 false 1993-08-03T15:24:35 010-18741037 {"wTS":"1596439475","rfId":"010-18741037","remark":"数据","weight":100} 710078936338096128 1596439475-010-18741037 -139866316590112768 10000 1993-08-03T15:24:35.926 10000 1993-08-03T15:24:35.926 201 456678 false 1993-08-03T15:24:35 010-18741034 {"wTS":"1596439475","rfId":"010-18741034","remark":"数据","weight":100} 710078936338096128 1596439475-010-18741034 -139866317424779264 10000 1993-08-03T15:24:36.125 10000 1993-08-03T15:24:36.125 201 456678 false 1993-08-03T15:24:35 010-18741036 {"wTS":"1596439475","rfId":"010-18741036","remark":"数据","weight":100} 710078936338096128 1596439475-010-18741036 -139866317953261568 10000 1993-08-03T15:24:36.251 10000 1993-08-03T15:24:36.251 201 456678 false 1993-08-03T15:24:35 010-18741033 {"wTS":"1596439475","rfId":"010-18741033","remark":"数据","weight":100} 710078936338096128 1596439475-010-18741033 -139866318209114112 10000 1993-08-03T15:24:36.312 10000 1993-08-03T15:24:36.312 201 456678 false 1993-08-03T15:24:35 010-18741032 {"wTS":"1596439475","rfId":"010-18741032","remark":"数据","weight":100} 710078936338096128 1596439475-010-18741032 -139866318750179328 10000 1993-08-03T15:24:36.441 10000 1993-08-03T15:24:36.441 201 456678 false 1993-08-03T15:24:35 010-18741035 {"wTS":"1596439475","rfId":"010-18741035","remark":"数据","weight":100} 710078936338096128 1596439475-010-18741035 -139866319706480640 10000 1993-08-03T15:24:36.669 10000 1993-08-03T15:24:36.669 201 456678 false 1993-08-03T15:24:35 010-18741031 {"wTS":"1596439475","rfId":"010-18741031","remark":"数据","weight":100} 710078936338096128 1596439475-010-18741031 -139866320658587648 10000 1993-08-03T15:24:36.896 10000 1993-08-03T15:24:36.896 201 456678 false 1993-08-03T15:24:35 010-18741038 {"wTS":"1596439475","rfId":"010-18741038","remark":"数据","weight":100} 710078936338096128 1596439475-010-18741038 -139866709332156416 10000 1993-08-03T15:26:09.563 10000 1993-08-03T15:26:09.563 201 456678 false 1993-08-03T15:26:09 010-18741039 {"wTS":"1596439569","rfId":"010-18741039","remark":"数据","weight":166.655} 710078936338096128 1596439569-010-18741039 -139866711446085632 10000 1993-08-03T15:26:10.067 10000 1993-08-03T15:26:10.067 201 456678 false 1993-08-03T15:26:09 010-18741034 {"wTS":"1596439569","rfId":"010-18741034","remark":"数据","weight":161.756} 710078936338096128 1596439569-010-18741034 -139866715170627584 10000 1993-08-03T15:26:10.955 10000 1993-08-03T15:26:10.955 201 456678 false 1993-08-03T15:26:09 010-18741035 {"wTS":"1596439569","rfId":"010-18741035","remark":"数据","weight":61.382} 710078936338096128 1596439569-010-18741035 -139866715187404800 10000 1993-08-03T15:26:10.959 10000 1993-08-03T15:26:10.959 201 456678 false 1993-08-03T15:26:09 010-18741036 {"wTS":"1596439569","rfId":"010-18741036","remark":"数据","weight":134.936} 710078936338096128 1596439569-010-18741036 -139866715195793408 10000 1993-08-03T15:26:10.961 10000 1993-08-03T15:26:10.961 201 456678 false 1993-08-03T15:26:09 010-18741037 {"wTS":"1596439569","rfId":"010-18741037","remark":"数据","weight":156.557} 710078936338096128 1596439569-010-18741037 -139866715225153536 10000 1993-08-03T15:26:10.968 10000 1993-08-03T15:26:10.968 201 456678 false 1993-08-03T15:26:09 010-18741038 {"wTS":"1596439569","rfId":"010-18741038","remark":"数据","weight":70.404} 710078936338096128 1596439569-010-18741038 -139866715313233920 10000 1993-08-03T15:26:10.989 10000 1993-08-03T15:26:10.989 201 456678 false 1993-08-03T15:26:09 010-18741040 {"wTS":"1596439569","rfId":"010-18741040","remark":"数据","weight":76.74} 710078936338096128 1596439569-010-18741040 -139866715413897216 10000 1993-08-03T15:26:11.013 10000 1993-08-03T15:26:11.013 201 456678 false 1993-08-03T15:26:09 010-18741032 {"wTS":"1596439569","rfId":"010-18741032","remark":"数据","weight":101.273} 710078936338096128 1596439569-010-18741032 -139866715711692800 10000 1993-08-03T15:26:11.084 10000 1993-08-03T15:26:11.084 201 456678 false 1993-08-03T15:26:09 010-18741033 {"wTS":"1596439569","rfId":"010-18741033","remark":"数据","weight":69.742} 710078936338096128 1596439569-010-18741033 -139866716114345984 10000 1993-08-03T15:26:11.180 10000 1993-08-03T15:26:11.180 201 456678 false 1993-08-03T15:26:09 010-18741031 {"wTS":"1596439569","rfId":"010-18741031","remark":"数据","weight":79.699} 710078936338096128 1596439569-010-18741031 -139866838638354432 10000 1993-08-03T15:26:40.392 10000 1993-08-03T15:26:40.392 201 456678 false 1993-08-03T15:26:40 010-18741039 {"wTS":"1596439600","rfId":"010-18741039","remark":"数据","weight":166.655} 710078936338096128 1596439600-010-18741039 -139866838952927232 10000 1993-08-03T15:26:40.467 10000 1993-08-03T15:26:40.467 201 456678 false 1993-08-03T15:26:40 010-18741031 {"wTS":"1596439600","rfId":"010-18741031","remark":"数据","weight":79.699} 710078936338096128 1596439600-010-18741031 -139866839275888640 10000 1993-08-03T15:26:40.544 10000 1993-08-03T15:26:40.544 201 456678 false 1993-08-03T15:26:40 010-18741037 {"wTS":"1596439600","rfId":"010-18741037","remark":"数据","weight":156.557} 710078936338096128 1596439600-010-18741037 -139866840232189952 10000 1993-08-03T15:26:40.772 10000 1993-08-03T15:26:40.772 201 456678 false 1993-08-03T15:26:40 010-18741032 {"wTS":"1596439600","rfId":"010-18741032","remark":"数据","weight":101.273} 710078936338096128 1596439600-010-18741032 -139866841603721100 10000 1993-08-03T15:26:41.099 10000 1993-08-03T15:26:41.099 201 456678 false 1993-08-03T15:26:40 010-18741035 {"wTS":"1596439600","rfId":"010-18741035","remark":"数据","weight":61.382} 710078936338096128 1596439600-010-18741035 -139866841725362176 10000 1993-08-03T15:26:41.128 10000 1993-08-03T15:26:41.128 201 456678 false 1993-08-03T15:26:40 010-18741034 {"wTS":"1596439600","rfId":"010-18741034","remark":"数据","weight":161.756} 710078936338096128 1596439600-010-18741034 -139866841767305216 10000 1993-08-03T15:26:41.138 10000 1993-08-03T15:26:41.138 201 456678 false 1993-08-03T15:26:40 010-18741033 {"wTS":"1596439600","rfId":"010-18741033","remark":"数据","weight":69.742} 710078936338096128 1596439600-010-18741033 -139866841779888128 10000 1993-08-03T15:26:41.141 10000 1993-08-03T15:26:41.141 201 456678 false 1993-08-03T15:26:40 010-18741036 {"wTS":"1596439600","rfId":"010-18741036","remark":"数据","weight":134.936} 710078936338096128 1596439600-010-18741036 -139866842065100800 10000 1993-08-03T15:26:41.209 10000 1993-08-03T15:26:41.209 201 456678 false 1993-08-03T15:26:40 010-18741038 {"wTS":"1596439600","rfId":"010-18741038","remark":"数据","weight":70.404} 710078936338096128 1596439600-010-18741038 -139866843038179328 10000 1993-08-03T15:26:41.441 10000 1993-08-03T15:26:41.441 201 456678 false 1993-08-03T15:26:40 010-18741040 {"wTS":"1596439600","rfId":"010-18741040","remark":"数据","weight":76.74} 710078936338096128 1596439600-010-18741040 -139866964631052288 10000 1993-08-03T15:27:10.431 10000 1993-08-03T15:27:10.431 201 456678 false 1993-08-03T15:27:10 010-18741031 {"wTS":"1596439630","rfId":"010-18741031","remark":"数据","weight":79.699} 710078936338096128 1596439630-010-18741031 -139866965037899776 10000 1993-08-03T15:27:10.528 10000 1993-08-03T15:27:10.528 201 456678 false 1993-08-03T15:27:10 010-18741039 {"wTS":"1596439630","rfId":"010-18741039","remark":"数据","weight":166.655} 710078936338096128 1596439630-010-18741039 -139866965356666880 10000 1993-08-03T15:27:10.604 10000 1993-08-03T15:27:10.604 201 456678 false 1993-08-03T15:27:10 010-18741036 {"wTS":"1596439630","rfId":"010-18741036","remark":"数据","weight":134.936} 710078936338096128 1596439630-010-18741036 -139866966094864384 10000 1993-08-03T15:27:10.780 10000 1993-08-03T15:27:10.780 201 456678 false 1993-08-03T15:27:10 010-18741032 {"wTS":"1596439630","rfId":"010-18741032","remark":"数据","weight":101.273} 710078936338096128 1596439630-010-18741032 -139866966635929600 10000 1993-08-03T15:27:10.909 10000 1993-08-03T15:27:10.909 201 456678 false 1993-08-03T15:27:10 010-18741037 {"wTS":"1596439630","rfId":"010-18741037","remark":"数据","weight":156.557} 710078936338096128 1596439630-010-18741037 -139866967017611264 10000 1993-08-03T15:27:11 10000 1993-08-03T15:27:11 201 456678 false 1993-08-03T15:27:10 010-18741040 {"wTS":"1596439630","rfId":"010-18741040","remark":"数据","weight":76.74} 710078936338096128 1596439630-010-18741040 -139866967785168896 10000 1993-08-03T15:27:11.183 10000 1993-08-03T15:27:11.183 201 456678 false 1993-08-03T15:27:10 010-18741033 {"wTS":"1596439630","rfId":"010-18741033","remark":"数据","weight":69.742} 710078936338096128 1596439630-010-18741033 -139866968745664512 10000 1993-08-03T15:27:11.412 10000 1993-08-03T15:27:11.412 201 456678 false 1993-08-03T15:27:10 010-18741038 {"wTS":"1596439630","rfId":"010-18741038","remark":"数据","weight":70.404} 710078936338096128 1596439630-010-18741038 -139866971123834880 10000 1993-08-03T15:27:11.979 10000 1993-08-03T15:27:11.979 201 456678 false 1993-08-03T15:27:10 010-18741034 {"wTS":"1596439630","rfId":"010-18741034","remark":"数据","weight":161.756} 710078936338096128 1596439630-010-18741034 -139866971149000704 10000 1993-08-03T15:27:11.985 10000 1993-08-03T15:27:11.985 201 456678 false 1993-08-03T15:27:10 010-18741035 {"wTS":"1596439630","rfId":"010-18741035","remark":"数据","weight":61.382} 710078936338096128 1596439630-010-18741035 -139867090732802048 10000 1993-08-03T15:27:40.496 10000 1993-08-03T15:27:40.496 201 456678 false 1993-08-03T15:27:40 010-18741037 {"wTS":"1596439660","rfId":"010-18741037","remark":"数据","weight":156.557} 710078936338096128 1596439660-010-18741037 -139867091068346368 10000 1993-08-03T15:27:40.576 10000 1993-08-03T15:27:40.576 201 456678 false 1993-08-03T15:27:40 010-18741039 {"wTS":"1596439660","rfId":"010-18741039","remark":"数据","weight":166.655} 710078936338096128 1596439660-010-18741039 -139867091512942592 10000 1993-08-03T15:27:40.682 10000 1993-08-03T15:27:40.682 201 456678 false 1993-08-03T15:27:40 010-18741036 {"wTS":"1596439660","rfId":"010-18741036","remark":"数据","weight":134.936} 710078936338096128 1596439660-010-18741036 -139867092112728064 10000 1993-08-03T15:27:40.825 10000 1993-08-03T15:27:40.825 201 456678 false 1993-08-03T15:27:40 010-18741031 {"wTS":"1596439660","rfId":"010-18741031","remark":"数据","weight":79.699} 710078936338096128 1596439660-010-18741031 -139867092284694528 10000 1993-08-03T15:27:40.866 10000 1993-08-03T15:27:40.866 201 456678 false 1993-08-03T15:27:40 010-18741032 {"wTS":"1596439660","rfId":"010-18741032","remark":"数据","weight":101.273} 710078936338096128 1596439660-010-18741032 -139867093572345856 10000 1993-08-03T15:27:41.173 10000 1993-08-03T15:27:41.173 201 456678 false 1993-08-03T15:27:40 010-18741038 {"wTS":"1596439660","rfId":"010-18741038","remark":"数据","weight":70.404} 710078936338096128 1596439660-010-18741038 -139867094369263616 10000 1993-08-03T15:27:41.363 10000 1993-08-03T15:27:41.363 201 456678 false 1993-08-03T15:27:40 010-18741033 {"wTS":"1596439660","rfId":"010-18741033","remark":"数据","weight":69.742} 710078936338096128 1596439660-010-18741033 -139867217174290432 10000 1993-08-03T15:28:10.642 10000 1993-08-03T15:28:10.642 201 456678 false 1993-08-03T15:28:10 010-18741031 {"wTS":"1596439690","rfId":"010-18741031","remark":"数据","weight":79.699} 710078936338096128 1596439690-010-18741031 -139867217245593600 10000 1993-08-03T15:28:10.659 10000 1993-08-03T15:28:10.659 201 456678 false 1993-08-03T15:28:10 010-18741039 {"wTS":"1596439690","rfId":"010-18741039","remark":"数据","weight":166.655} 710078936338096128 1596439690-010-18741039 -139867217677606912 10000 1993-08-03T15:28:10.762 10000 1993-08-03T15:28:10.762 201 456678 false 1993-08-03T15:28:10 010-18741037 {"wTS":"1596439690","rfId":"010-18741037","remark":"数据","weight":156.557} 710078936338096128 1596439690-010-18741037 -139867218214477824 10000 1993-08-03T15:28:10.890 10000 1993-08-03T15:28:10.890 201 456678 false 1993-08-03T15:28:10 010-18741035 {"wTS":"1596439690","rfId":"010-18741035","remark":"数据","weight":61.382} 710078936338096128 1596439690-010-18741035 -139867218508079104 10000 1993-08-03T15:28:10.960 10000 1993-08-03T15:28:10.960 201 456678 false 1993-08-03T15:28:10 010-18741032 {"wTS":"1596439690","rfId":"010-18741032","remark":"数据","weight":101.273} 710078936338096128 1596439690-010-18741032 -139867219925753856 10000 1993-08-03T15:28:11.298 10000 1993-08-03T15:28:11.298 201 456678 false 1993-08-03T15:28:10 010-18741033 {"wTS":"1596439690","rfId":"010-18741033","remark":"数据","weight":69.742} 710078936338096128 1596439690-010-18741033 -139867220013834240 10000 1993-08-03T15:28:11.320 10000 1993-08-03T15:28:11.320 201 456678 false 1993-08-03T15:28:10 010-18741036 {"wTS":"1596439690","rfId":"010-18741036","remark":"数据","weight":134.936} 710078936338096128 1596439690-010-18741036 -139867220076748800 10000 1993-08-03T15:28:11.334 10000 1993-08-03T15:28:11.334 201 456678 false 1993-08-03T15:28:10 010-18741034 {"wTS":"1596439690","rfId":"010-18741034","remark":"数据","weight":161.756} 710078936338096128 1596439690-010-18741034 -139867220299046912 10000 1993-08-03T15:28:11.387 10000 1993-08-03T15:28:11.387 201 456678 false 1993-08-03T15:28:10 010-18741040 {"wTS":"1596439690","rfId":"010-18741040","remark":"数据","weight":76.74} 710078936338096128 1596439690-010-18741040 -139867342911135744 10000 1993-08-03T15:28:40.620 10000 1993-08-03T15:28:40.620 201 456678 false 1993-08-03T15:28:40 010-18741037 {"wTS":"1596439720","rfId":"010-18741037","remark":"数据","weight":156.557} 710078936338096128 1596439720-010-18741037 -139867343108268032 10000 1993-08-03T15:28:40.667 10000 1993-08-03T15:28:40.667 201 456678 false 1993-08-03T15:28:40 010-18741031 {"wTS":"1596439720","rfId":"010-18741031","remark":"数据","weight":79.699} 710078936338096128 1596439720-010-18741031 -139867343733219328 10000 1993-08-03T15:28:40.816 10000 1993-08-03T15:28:40.816 201 456678 false 1993-08-03T15:28:40 010-18741039 {"wTS":"1596439720","rfId":"010-18741039","remark":"数据","weight":166.655} 710078936338096128 1596439720-010-18741039 -139867344068763648 10000 1993-08-03T15:28:40.896 10000 1993-08-03T15:28:40.896 201 456678 false 1993-08-03T15:28:40 010-18741034 {"wTS":"1596439720","rfId":"010-18741034","remark":"数据","weight":161.756} 710078936338096128 1596439720-010-18741034 -139867344895041536 10000 1993-08-03T15:28:41.093 10000 1993-08-03T15:28:41.093 201 456678 false 1993-08-03T15:28:40 010-18741032 {"wTS":"1596439720","rfId":"010-18741032","remark":"数据","weight":101.273} 710078936338096128 1596439720-010-18741032 -139867345054425088 10000 1993-08-03T15:28:41.131 10000 1993-08-03T15:28:41.131 201 456678 false 1993-08-03T15:28:40 010-18741035 {"wTS":"1596439720","rfId":"010-18741035","remark":"数据","weight":61.382} 710078936338096128 1596439720-010-18741035 -139867346480488448 10000 1993-08-03T15:28:41.471 10000 1993-08-03T15:28:41.471 201 456678 false 1993-08-03T15:28:40 010-18741038 {"wTS":"1596439720","rfId":"010-18741038","remark":"数据","weight":70.404} 710078936338096128 1596439720-010-18741038 -139867347524870144 10000 1993-08-03T15:28:41.721 10000 1993-08-03T15:28:41.721 201 456678 false 1993-08-03T15:28:40 010-18741040 {"wTS":"1596439720","rfId":"010-18741040","remark":"数据","weight":76.74} 710078936338096128 1596439720-010-18741040 -139867469239377920 10000 1993-08-03T15:29:10.739 10000 1993-08-03T15:29:10.739 201 456678 false 1993-08-03T15:29:10 010-18741037 {"wTS":"1596439750","rfId":"010-18741037","remark":"数据","weight":156.557} 710078936338096128 1596439750-010-18741037 -139867469352624128 10000 1993-08-03T15:29:10.766 10000 1993-08-03T15:29:10.766 201 456678 false 1993-08-03T15:29:10 010-18741039 {"wTS":"1596439750","rfId":"010-18741039","remark":"数据","weight":166.655} 710078936338096128 1596439750-010-18741039 -139867469516201984 10000 1993-08-03T15:29:10.805 10000 1993-08-03T15:29:10.805 201 456678 false 1993-08-03T15:29:10 010-18741031 {"wTS":"1596439750","rfId":"010-18741031","remark":"数据","weight":79.699} 710078936338096128 1596439750-010-18741031 -139867469654614016 10000 1993-08-03T15:29:10.838 10000 1993-08-03T15:29:10.838 201 456678 false 1993-08-03T15:29:10 010-18741036 {"wTS":"1596439750","rfId":"010-18741036","remark":"数据","weight":134.936} 710078936338096128 1596439750-010-18741036 -139867471651102720 10000 1993-08-03T15:29:11.314 10000 1993-08-03T15:29:11.314 201 456678 false 1993-08-03T15:29:10 010-18741032 {"wTS":"1596439750","rfId":"010-18741032","remark":"数据","weight":101.273} 710078936338096128 1596439750-010-18741032 -139867471726600192 10000 1993-08-03T15:29:11.332 10000 1993-08-03T15:29:11.332 201 456678 false 1993-08-03T15:29:10 010-18741035 {"wTS":"1596439750","rfId":"010-18741035","remark":"数据","weight":61.382} 710078936338096128 1596439750-010-18741035 -139867471730794496 10000 1993-08-03T15:29:11.333 10000 1993-08-03T15:29:11.333 201 456678 false 1993-08-03T15:29:10 010-18741034 {"wTS":"1596439750","rfId":"010-18741034","remark":"数据","weight":161.756} 710078936338096128 1596439750-010-18741034 -139867471776931840 10000 1993-08-03T15:29:11.344 10000 1993-08-03T15:29:11.344 201 456678 false 1993-08-03T15:29:10 010-18741038 {"wTS":"1596439750","rfId":"010-18741038","remark":"数据","weight":70.404} 710078936338096128 1596439750-010-18741038 -139867473987330048 10000 1993-08-03T15:29:11.871 10000 1993-08-03T15:29:11.871 201 456678 false 1993-08-03T15:29:10 010-18741033 {"wTS":"1596439750","rfId":"010-18741033","remark":"数据","weight":69.742} 710078936338096128 1596439750-010-18741033 -139867474566144000 10000 1993-08-03T15:29:12.009 10000 1993-08-03T15:29:12.009 201 456678 false 1993-08-03T15:29:10 010-18741040 {"wTS":"1596439750","rfId":"010-18741040","remark":"数据","weight":76.74} 710078936338096128 1596439750-010-18741040 -139867596326789120 10000 1993-08-03T15:29:41.039 10000 1993-08-03T15:29:41.039 201 456678 false 1993-08-03T15:29:40 010-18741034 {"wTS":"1596439780","rfId":"010-18741034","remark":"数据","weight":161.756} 710078936338096128 1596439780-010-18741034 -139867596335177728 10000 1993-08-03T15:29:41.041 10000 1993-08-03T15:29:41.041 201 456678 false 1993-08-03T15:29:40 010-18741035 {"wTS":"1596439780","rfId":"010-18741035","remark":"数据","weight":61.382} 710078936338096128 1596439780-010-18741035 -139867596356149248 10000 1993-08-03T15:29:41.046 10000 1993-08-03T15:29:41.046 201 456678 false 1993-08-03T15:29:40 010-18741037 {"wTS":"1596439780","rfId":"010-18741037","remark":"数据","weight":156.557} 710078936338096128 1596439780-010-18741037 -139867596586835968 10000 1993-08-03T15:29:41.101 10000 1993-08-03T15:29:41.101 201 456678 false 1993-08-03T15:29:40 010-18741039 {"wTS":"1596439780","rfId":"010-18741039","remark":"数据","weight":166.655} 710078936338096128 1596439780-010-18741039 -139867596616196096 10000 1993-08-03T15:29:41.108 10000 1993-08-03T15:29:41.108 201 456678 false 1993-08-03T15:29:40 010-18741031 {"wTS":"1596439780","rfId":"010-18741031","remark":"数据","weight":79.699} 710078936338096128 1596439780-010-18741031 -139867596620390400 10000 1993-08-03T15:29:41.109 10000 1993-08-03T15:29:41.109 201 456678 false 1993-08-03T15:29:40 010-18741036 {"wTS":"1596439780","rfId":"010-18741036","remark":"数据","weight":134.936} 710078936338096128 1596439780-010-18741036 -139867597828349952 10000 1993-08-03T15:29:41.397 10000 1993-08-03T15:29:41.397 201 456678 false 1993-08-03T15:29:40 010-18741038 {"wTS":"1596439780","rfId":"010-18741038","remark":"数据","weight":70.404} 710078936338096128 1596439780-010-18741038 -139867598340055040 10000 1993-08-03T15:29:41.519 10000 1993-08-03T15:29:41.520 201 456678 false 1993-08-03T15:29:40 010-18741040 {"wTS":"1596439780","rfId":"010-18741040","remark":"数据","weight":76.74} 710078936338096128 1596439780-010-18741040 -139867600818888704 10000 1993-08-03T15:29:42.110 10000 1993-08-03T15:29:42.110 201 456678 false 1993-08-03T15:29:40 010-18741033 {"wTS":"1596439780","rfId":"010-18741033","remark":"数据","weight":69.742} 710078936338096128 1596439780-010-18741033 -139867601850687488 10000 1993-08-03T15:29:42.356 10000 1993-08-03T15:29:42.357 201 456678 false 1993-08-03T15:29:40 010-18741032 {"wTS":"1596439780","rfId":"010-18741032","remark":"数据","weight":101.273} 710078936338096128 1596439780-010-18741032 -139867721308659712 10000 1993-08-03T15:30:10.838 10000 1993-08-03T15:30:10.838 201 456678 false 1993-08-03T15:30:10 010-18741037 {"wTS":"1596439810","rfId":"010-18741037","remark":"数据","weight":156.557} 710078936338096128 1596439810-010-18741037 -139867721384157184 10000 1993-08-03T15:30:10.855 10000 1993-08-03T15:30:10.855 201 456678 false 1993-08-03T15:30:10 010-18741039 {"wTS":"1596439810","rfId":"010-18741039","remark":"数据","weight":166.655} 710078936338096128 1596439810-010-18741039 -139867721694535680 10000 1993-08-03T15:30:10.929 10000 1993-08-03T15:30:10.929 201 456678 false 1993-08-03T15:30:10 010-18741034 {"wTS":"1596439810","rfId":"010-18741034","remark":"数据","weight":161.756} 710078936338096128 1596439810-010-18741034 -139867723183513600 10000 1993-08-03T15:30:11.284 10000 1993-08-03T15:30:11.284 201 456678 false 1993-08-03T15:30:10 010-18741032 {"wTS":"1596439810","rfId":"010-18741032","remark":"数据","weight":101.273} 710078936338096128 1596439810-010-18741032 -139867723208679424 10000 1993-08-03T15:30:11.290 10000 1993-08-03T15:30:11.290 201 456678 false 1993-08-03T15:30:10 010-18741033 {"wTS":"1596439810","rfId":"010-18741033","remark":"数据","weight":69.742} 710078936338096128 1596439810-010-18741033 -139867723229650944 10000 1993-08-03T15:30:11.295 10000 1993-08-03T15:30:11.295 201 456678 false 1993-08-03T15:30:10 010-18741035 {"wTS":"1596439810","rfId":"010-18741035","remark":"数据","weight":61.382} 710078936338096128 1596439810-010-18741035 -139867723238039552 10000 1993-08-03T15:30:11.297 10000 1993-08-03T15:30:11.297 201 456678 false 1993-08-03T15:30:10 010-18741036 {"wTS":"1596439810","rfId":"010-18741036","remark":"数据","weight":134.936} 710078936338096128 1596439810-010-18741036 -139867724131426304 10000 1993-08-03T15:30:11.510 10000 1993-08-03T15:30:11.510 201 456678 false 1993-08-03T15:30:10 010-18741040 {"wTS":"1596439810","rfId":"010-18741040","remark":"数据","weight":76.74} 710078936338096128 1596439810-010-18741040 -139867724534079488 10000 1993-08-03T15:30:11.606 10000 1993-08-03T15:30:11.606 201 456678 false 1993-08-03T15:30:10 010-18741038 {"wTS":"1596439810","rfId":"010-18741038","remark":"数据","weight":70.404} 710078936338096128 1596439810-010-18741038 -139867847209082880 10000 1993-08-03T15:30:40.854 10000 1993-08-03T15:30:40.854 201 456678 false 1993-08-03T15:30:40 010-18741037 {"wTS":"1596439840","rfId":"010-18741037","remark":"数据","weight":156.557} 710078936338096128 1596439840-010-18741037 -139867848484151296 10000 1993-08-03T15:30:41.158 10000 1993-08-03T15:30:41.158 201 456678 false 1993-08-03T15:30:40 010-18741031 {"wTS":"1596439840","rfId":"010-18741031","remark":"数据","weight":79.699} 710078936338096128 1596439840-010-18741031 -139867848492539904 10000 1993-08-03T15:30:41.160 10000 1993-08-03T15:30:41.160 201 456678 false 1993-08-03T15:30:40 010-18741036 {"wTS":"1596439840","rfId":"010-18741036","remark":"数据","weight":134.936} 710078936338096128 1596439840-010-18741036 -139867848492539905 10000 1993-08-03T15:30:41.160 10000 1993-08-03T15:30:41.160 201 456678 false 1993-08-03T15:30:40 010-18741034 {"wTS":"1596439840","rfId":"010-18741034","remark":"数据","weight":161.756} 710078936338096128 1596439840-010-18741034 -139867848521900032 10000 1993-08-03T15:30:41.167 10000 1993-08-03T15:30:41.167 201 456678 false 1993-08-03T15:30:40 010-18741039 {"wTS":"1596439840","rfId":"010-18741039","remark":"数据","weight":166.655} 710078936338096128 1596439840-010-18741039 -139867848660312064 10000 1993-08-03T15:30:41.200 10000 1993-08-03T15:30:41.200 201 456678 false 1993-08-03T15:30:40 010-18741035 {"wTS":"1596439840","rfId":"010-18741035","remark":"数据","weight":61.382} 710078936338096128 1596439840-010-18741035 -139867850090569728 10000 1993-08-03T15:30:41.541 10000 1993-08-03T15:30:41.541 201 456678 false 1993-08-03T15:30:40 010-18741040 {"wTS":"1596439840","rfId":"010-18741040","remark":"数据","weight":76.74} 710078936338096128 1596439840-010-18741040 -139867851755708416 10000 1993-08-03T15:30:41.938 10000 1993-08-03T15:30:41.938 201 456678 false 1993-08-03T15:30:40 010-18741033 {"wTS":"1596439840","rfId":"010-18741033","remark":"数据","weight":69.742} 710078936338096128 1596439840-010-18741033 -139867852619735040 10000 1993-08-03T15:30:42.144 10000 1993-08-03T15:30:42.144 201 456678 false 1993-08-03T15:30:40 010-18741038 {"wTS":"1596439840","rfId":"010-18741038","remark":"数据","weight":70.404} 710078936338096128 1596439840-010-18741038 -139867854192599040 10000 1993-08-03T15:30:42.519 10000 1993-08-03T15:30:42.519 201 456678 false 1993-08-03T15:30:40 010-18741032 {"wTS":"1596439840","rfId":"010-18741032","remark":"数据","weight":101.273} 710078936338096128 1596439840-010-18741032 -139867911025405440 10000 1993-08-03T15:31:10.994 10000 1993-08-03T15:31:10.994 201 456678 false 1993-08-03T15:31:10 010-18741037 {"wTS":"1596439870","rfId":"010-18741037","remark":"数据","weight":156.557} 710078936338096128 1596439870-010-18741037 -139867911079931392 10000 1993-08-03T15:31:11.007 10000 1993-08-03T15:31:11.007 201 456678 false 1993-08-03T15:31:10 010-18741039 {"wTS":"1596439870","rfId":"010-18741039","remark":"数据","weight":166.655} 710078936338096128 1596439870-010-18741039 -139867973185003520 10000 1993-08-03T15:31:10.889 10000 1993-08-03T15:31:10.889 201 456678 false 1993-08-03T15:31:10 010-18741031 {"wTS":"1596439870","rfId":"010-18741031","remark":"数据","weight":79.699} 710078936338096128 1596439870-010-18741031 -139867974439100416 10000 1993-08-03T15:31:11.188 10000 1993-08-03T15:31:11.188 201 456678 false 1993-08-03T15:31:10 010-18741036 {"wTS":"1596439870","rfId":"010-18741036","remark":"数据","weight":134.936} 710078936338096128 1596439870-010-18741036 -139867974640427008 10000 1993-08-03T15:31:11.236 10000 1993-08-03T15:31:11.237 201 456678 false 1993-08-03T15:31:10 010-18741034 {"wTS":"1596439870","rfId":"010-18741034","remark":"数据","weight":161.756} 710078936338096128 1596439870-010-18741034 -139867975483482112 10000 1993-08-03T15:31:11.437 10000 1993-08-03T15:31:11.437 201 456678 false 1993-08-03T15:31:10 010-18741035 {"wTS":"1596439870","rfId":"010-18741035","remark":"数据","weight":61.382} 710078936338096128 1596439870-010-18741035 -139867976561418240 10000 1993-08-03T15:31:11.694 10000 1993-08-03T15:31:11.694 201 456678 false 1993-08-03T15:31:10 010-18741038 {"wTS":"1596439870","rfId":"010-18741038","remark":"数据","weight":70.404} 710078936338096128 1596439870-010-18741038 -139868173785980928 10000 1993-08-03T15:31:58.716 10000 1993-08-03T15:31:58.716 201 456678 false 1993-08-03T15:31:58 010-18741039 {"wTS":"1596439918","rfId":"010-18741039","remark":"数据","weight":111.068} 710078936338096128 1596439918-010-18741039 -139868173853089792 10000 1993-08-03T15:31:58.732 10000 1993-08-03T15:31:58.732 201 456678 false 1993-08-03T15:31:58 010-18741037 {"wTS":"1596439918","rfId":"010-18741037","remark":"数据","weight":139.592} 710078936338096128 1596439918-010-18741037 -139868174079582208 10000 1993-08-03T15:31:58.786 10000 1993-08-03T15:31:58.786 201 456678 false 1993-08-03T15:31:58 010-18741031 {"wTS":"1596439918","rfId":"010-18741031","remark":"数据","weight":85.044} 710078936338096128 1596439918-010-18741031 -139868174394155008 10000 1993-08-03T15:31:58.861 10000 1993-08-03T15:31:58.861 201 456678 false 1993-08-03T15:31:58 010-18741034 {"wTS":"1596439918","rfId":"010-18741034","remark":"数据","weight":157.6} 710078936338096128 1596439918-010-18741034 -139868174528372110 10000 1993-08-03T15:31:58.893 10000 1993-08-03T15:31:58.893 201 456678 false 1993-08-03T15:31:58 010-18741033 {"wTS":"1596439918","rfId":"010-18741033","remark":"数据","weight":66.858} 710078936338096128 1596439918-010-18741033 -139868175157518336 10000 1993-08-03T15:31:59.043 10000 1993-08-03T15:31:59.043 201 456678 false 1993-08-03T15:31:58 010-18741038 {"wTS":"1596439918","rfId":"010-18741038","remark":"数据","weight":122.591} 710078936338096128 1596439918-010-18741038 -139868175782469632 10000 1993-08-03T15:31:59.192 10000 1993-08-03T15:31:59.192 201 456678 false 1993-08-03T15:31:58 010-18741036 {"wTS":"1596439918","rfId":"010-18741036","remark":"数据","weight":75.728} 710078936338096128 1596439918-010-18741036 -139868176168345600 10000 1993-08-03T15:31:59.284 10000 1993-08-03T15:31:59.284 201 456678 false 1993-08-03T15:31:58 010-18741040 {"wTS":"1596439918","rfId":"010-18741040","remark":"数据","weight":133.627} 710078936338096128 1596439918-010-18741040 -139868176168345601 10000 1993-08-03T15:31:59.284 10000 1993-08-03T15:31:59.284 201 456678 false 1993-08-03T15:31:58 010-18741032 {"wTS":"1596439918","rfId":"010-18741032","remark":"数据","weight":108.355} 710078936338096128 1596439918-010-18741032 -139868176940097536 10000 1993-08-03T15:31:59.468 10000 1993-08-03T15:31:59.468 201 456678 false 1993-08-03T15:31:58 010-18741035 {"wTS":"1596439918","rfId":"010-18741035","remark":"数据","weight":133.986} 710078936338096128 1596439918-010-18741035 -139868302886658048 10000 1993-08-03T15:32:29.496 10000 1993-08-03T15:32:29.496 201 456678 false 1993-08-03T15:32:29 010-18741032 {"wTS":"1596439949","rfId":"010-18741032","remark":"数据","weight":108.355} 710078936338096128 1596439949-010-18741032 -139868303213813760 10000 1993-08-03T15:32:29.574 10000 1993-08-03T15:32:29.574 201 456678 false 1993-08-03T15:32:29 010-18741031 {"wTS":"1596439949","rfId":"010-18741031","remark":"数据","weight":85.044} 710078936338096128 1596439949-010-18741031 -139868303238979584 10000 1993-08-03T15:32:29.580 10000 1993-08-03T15:32:29.580 201 456678 false 1993-08-03T15:32:29 010-18741034 {"wTS":"1596439949","rfId":"010-18741034","remark":"数据","weight":157.6} 710078936338096128 1596439949-010-18741034 -139868303616466944 10000 1993-08-03T15:32:29.670 10000 1993-08-03T15:32:29.670 201 456678 false 1993-08-03T15:32:29 010-18741036 {"wTS":"1596439949","rfId":"010-18741036","remark":"数据","weight":75.728} 710078936338096128 1596439949-010-18741036 -139868303721324544 10000 1993-08-03T15:32:29.695 10000 1993-08-03T15:32:29.695 201 456678 false 1993-08-03T15:32:29 010-18741035 {"wTS":"1596439949","rfId":"010-18741035","remark":"数据","weight":133.986} 710078936338096128 1596439949-010-18741035 -139868304795066368 10000 1993-08-03T15:32:29.951 10000 1993-08-03T15:32:29.951 201 456678 false 1993-08-03T15:32:29 010-18741040 {"wTS":"1596439949","rfId":"010-18741040","remark":"数据","weight":133.627} 710078936338096128 1596439949-010-18741040 -139868306833498112 10000 1993-08-03T15:32:30.437 10000 1993-08-03T15:32:30.437 201 456678 false 1993-08-03T15:32:29 010-18741037 {"wTS":"1596439949","rfId":"010-18741037","remark":"数据","weight":139.592} 710078936338096128 1596439949-010-18741037 -139868306904801280 10000 1993-08-03T15:32:30.454 10000 1993-08-03T15:32:30.454 201 456678 false 1993-08-03T15:32:29 010-18741038 {"wTS":"1596439949","rfId":"010-18741038","remark":"数据","weight":122.591} 710078936338096128 1596439949-010-18741038 -139868307030630400 10000 1993-08-03T15:32:30.484 10000 1993-08-03T15:32:30.484 201 456678 false 1993-08-03T15:32:29 010-18741033 {"wTS":"1596439949","rfId":"010-18741033","remark":"数据","weight":66.858} 710078936338096128 1596439949-010-18741033 -139868307257122816 10000 1993-08-03T15:32:30.538 10000 1993-08-03T15:32:30.538 201 456678 false 1993-08-03T15:32:29 010-18741039 {"wTS":"1596439949","rfId":"010-18741039","remark":"数据","weight":111.068} 710078936338096128 1596439949-010-18741039 -139868429219094528 10000 1993-08-03T15:32:59.616 10000 1993-08-03T15:32:59.616 201 456678 false 1993-08-03T15:32:59 010-18741036 {"wTS":"1596439979","rfId":"010-18741036","remark":"数据","weight":75.728} 710078936338096128 1596439979-010-18741036 -139868429403643904 10000 1993-08-03T15:32:59.660 10000 1993-08-03T15:32:59.660 201 456678 false 1993-08-03T15:32:59 010-18741032 {"wTS":"1596439979","rfId":"010-18741032","remark":"数据","weight":108.355} 710078936338096128 1596439979-010-18741032 -139868429835657216 10000 1993-08-03T15:32:59.763 10000 1993-08-03T15:32:59.763 201 456678 false 1993-08-03T15:32:59 010-18741031 {"wTS":"1596439979","rfId":"010-18741031","remark":"数据","weight":85.044} 710078936338096128 1596439979-010-18741031 -139868429919543296 10000 1993-08-03T15:32:59.783 10000 1993-08-03T15:32:59.783 201 456678 false 1993-08-03T15:32:59 010-18741034 {"wTS":"1596439979","rfId":"010-18741034","remark":"数据","weight":157.6} 710078936338096128 1596439979-010-18741034 -139868430703878144 10000 1993-08-03T15:32:59.970 10000 1993-08-03T15:32:59.970 201 456678 false 1993-08-03T15:32:59 010-18741037 {"wTS":"1596439979","rfId":"010-18741037","remark":"数据","weight":139.592} 710078936338096128 1596439979-010-18741037 -139868431274303488 10000 1993-08-03T15:33:00.106 10000 1993-08-03T15:33:00.106 201 456678 false 1993-08-03T15:32:59 010-18741035 {"wTS":"1596439979","rfId":"010-18741035","remark":"数据","weight":133.986} 710078936338096128 1596439979-010-18741035 -139868431551127552 10000 1993-08-03T15:33:00.172 10000 1993-08-03T15:33:00.172 201 456678 false 1993-08-03T15:32:59 010-18741038 {"wTS":"1596439979","rfId":"010-18741038","remark":"数据","weight":122.591} 710078936338096128 1596439979-010-18741038 -139868432771670016 10000 1993-08-03T15:33:00.463 10000 1993-08-03T15:33:00.463 201 456678 false 1993-08-03T15:32:59 010-18741040 {"wTS":"1596439979","rfId":"010-18741040","remark":"数据","weight":133.627} 710078936338096128 1596439979-010-18741040 -139868434906570752 10000 1993-08-03T15:33:00.972 10000 1993-08-03T15:33:00.972 201 456678 false 1993-08-03T15:32:59 010-18741033 {"wTS":"1596439979","rfId":"010-18741033","remark":"数据","weight":66.858} 710078936338096128 1596439979-010-18741033 -139868434969485312 10000 1993-08-03T15:33:00.987 10000 1993-08-03T15:33:00.987 201 456678 false 1993-08-03T15:32:59 010-18741039 {"wTS":"1596439979","rfId":"010-18741039","remark":"数据","weight":111.068} 710078936338096128 1596439979-010-18741039 -139868554591035392 10000 1993-08-03T15:33:29.507 10000 1993-08-03T15:33:29.507 201 456678 false 1993-08-03T15:33:29 010-18741036 {"wTS":"1596440009","rfId":"010-18741036","remark":"数据","weight":75.728} 710078936338096128 1596440009-010-18741036 -139868554830110720 10000 1993-08-03T15:33:29.564 10000 1993-08-03T15:33:29.564 201 456678 false 1993-08-03T15:33:29 010-18741032 {"wTS":"1596440009","rfId":"010-18741032","remark":"数据","weight":108.355} 710078936338096128 1596440009-010-18741032 -139868555350204416 10000 1993-08-03T15:33:29.688 10000 1993-08-03T15:33:29.688 201 456678 false 1993-08-03T15:33:29 010-18741031 {"wTS":"1596440009","rfId":"010-18741031","remark":"数据","weight":85.044} 710078936338096128 1596440009-010-18741031 -139868555471839232 10000 1993-08-03T15:33:29.717 10000 1993-08-03T15:33:29.717 201 456678 false 1993-08-03T15:33:29 010-18741034 {"wTS":"1596440009","rfId":"010-18741034","remark":"数据","weight":157.6} 710078936338096128 1596440009-010-18741034 -139868556222619648 10000 1993-08-03T15:33:29.896 10000 1993-08-03T15:33:29.896 201 456678 false 1993-08-03T15:33:29 010-18741039 {"wTS":"1596440009","rfId":"010-18741039","remark":"数据","weight":111.068} 710078936338096128 1596440009-010-18741039 -139868557208281088 10000 1993-08-03T15:33:30.131 10000 1993-08-03T15:33:30.131 201 456678 false 1993-08-03T15:33:29 010-18741033 {"wTS":"1596440009","rfId":"010-18741033","remark":"数据","weight":66.858} 710078936338096128 1596440009-010-18741033 -139868559221547008 10000 1993-08-03T15:33:30.611 10000 1993-08-03T15:33:30.611 201 456678 false 1993-08-03T15:33:29 010-18741037 {"wTS":"1596440009","rfId":"010-18741037","remark":"数据","weight":139.592} 710078936338096128 1596440009-010-18741037 -139868559234129920 10000 1993-08-03T15:33:30.614 10000 1993-08-03T15:33:30.614 201 456678 false 1993-08-03T15:33:29 010-18741038 {"wTS":"1596440009","rfId":"010-18741038","remark":"数据","weight":122.591} 710078936338096128 1596440009-010-18741038 -139868559435456512 10000 1993-08-03T15:33:30.662 10000 1993-08-03T15:33:30.662 201 456678 false 1993-08-03T15:33:29 010-18741040 {"wTS":"1596440009","rfId":"010-18741040","remark":"数据","weight":133.627} 710078936338096128 1596440009-010-18741040 -139868559452233728 10000 1993-08-03T15:33:30.666 10000 1993-08-03T15:33:30.666 201 456678 false 1993-08-03T15:33:29 010-18741035 {"wTS":"1596440009","rfId":"010-18741035","remark":"数据","weight":133.986} 710078936338096128 1596440009-010-18741035 -139868680860557312 10000 1993-08-03T15:33:59.612 10000 1993-08-03T15:33:59.613 201 456678 false 1993-08-03T15:33:59 010-18741032 {"wTS":"1596440039","rfId":"010-18741032","remark":"数据","weight":108.355} 710078936338096128 1596440039-010-18741032 -139868680994775040 10000 1993-08-03T15:33:59.644 10000 1993-08-03T15:33:59.644 201 456678 false 1993-08-03T15:33:59 010-18741036 {"wTS":"1596440039","rfId":"010-18741036","remark":"数据","weight":75.728} 710078936338096128 1596440039-010-18741036 -139868681170935808 10000 1993-08-03T15:33:59.686 10000 1993-08-03T15:33:59.686 201 456678 false 1993-08-03T15:33:59 010-18741031 {"wTS":"1596440039","rfId":"010-18741031","remark":"数据","weight":85.044} 710078936338096128 1596440039-010-18741031 -139868681850413056 10000 1993-08-03T15:33:59.848 10000 1993-08-03T15:33:59.848 201 456678 false 1993-08-03T15:33:59 010-18741034 {"wTS":"1596440039","rfId":"010-18741034","remark":"数据","weight":157.6} 710078936338096128 1596440039-010-18741034 -139868682123042816 10000 1993-08-03T15:33:59.913 10000 1993-08-03T15:33:59.913 201 456678 false 1993-08-03T15:33:59 010-18741035 {"wTS":"1596440039","rfId":"010-18741035","remark":"数据","weight":133.986} 710078936338096128 1596440039-010-18741035 -139868682487947264 10000 1993-08-03T15:34 10000 1993-08-03T15:34 201 456678 false 1993-08-03T15:33:59 010-18741039 {"wTS":"1596440039","rfId":"010-18741039","remark":"数据","weight":111.068} 710078936338096128 1596440039-010-18741039 -139868683733655552 10000 1993-08-03T15:34:00.297 10000 1993-08-03T15:34:00.297 201 456678 false 1993-08-03T15:33:59 010-18741040 {"wTS":"1596440039","rfId":"010-18741040","remark":"数据","weight":133.627} 710078936338096128 1596440039-010-18741040 -139868685809836032 10000 1993-08-03T15:34:00.792 10000 1993-08-03T15:34:00.792 201 456678 false 1993-08-03T15:33:59 010-18741037 {"wTS":"1596440039","rfId":"010-18741037","remark":"数据","weight":139.592} 710078936338096128 1596440039-010-18741037 -139868687265259520 10000 1993-08-03T15:34:01.139 10000 1993-08-03T15:34:01.139 201 456678 false 1993-08-03T15:33:59 010-18741033 {"wTS":"1596440039","rfId":"010-18741033","remark":"数据","weight":66.858} 710078936338096128 1596440039-010-18741033 -139868687596609536 10000 1993-08-03T15:34:01.218 10000 1993-08-03T15:34:01.218 201 456678 false 1993-08-03T15:33:59 010-18741038 {"wTS":"1596440039","rfId":"010-18741038","remark":"数据","weight":122.591} 710078936338096128 1596440039-010-18741038 -139868807180410880 10000 1993-08-03T15:34:29.729 10000 1993-08-03T15:34:29.729 201 456678 false 1993-08-03T15:34:29 010-18741036 {"wTS":"1596440069","rfId":"010-18741036","remark":"数据","weight":75.728} 710078936338096128 1596440069-010-18741036 -139868807306240000 10000 1993-08-03T15:34:29.759 10000 1993-08-03T15:34:29.759 201 456678 false 1993-08-03T15:34:29 010-18741032 {"wTS":"1596440069","rfId":"010-18741032","remark":"数据","weight":108.355} 710078936338096128 1596440069-010-18741032 -139868807461429248 10000 1993-08-03T15:34:29.796 10000 1993-08-03T15:34:29.796 201 456678 false 1993-08-03T15:34:29 010-18741035 {"wTS":"1596440069","rfId":"010-18741035","remark":"数据","weight":133.986} 710078936338096128 1596440069-010-18741035 -139868807478206464 10000 1993-08-03T15:34:29.800 10000 1993-08-03T15:34:29.800 201 456678 false 1993-08-03T15:34:29 010-18741031 {"wTS":"1596440069","rfId":"010-18741031","remark":"数据","weight":85.044} 710078936338096128 1596440069-010-18741031 -139868808539365376 10000 1993-08-03T15:34:30.053 10000 1993-08-03T15:34:30.053 201 456678 false 1993-08-03T15:34:29 010-18741038 {"wTS":"1596440069","rfId":"010-18741038","remark":"数据","weight":122.591} 710078936338096128 1596440069-010-18741038 -139868809302728704 10000 1993-08-03T15:34:30.235 10000 1993-08-03T15:34:30.235 201 456678 false 1993-08-03T15:34:29 010-18741034 {"wTS":"1596440069","rfId":"010-18741034","remark":"数据","weight":157.6} 710078936338096128 1596440069-010-18741034 -139868809797656576 10000 1993-08-03T15:34:30.353 10000 1993-08-03T15:34:30.353 201 456678 false 1993-08-03T15:34:29 010-18741037 {"wTS":"1596440069","rfId":"010-18741037","remark":"数据","weight":139.592} 710078936338096128 1596440069-010-18741037 -139868810493911040 10000 1993-08-03T15:34:30.519 10000 1993-08-03T15:34:30.519 201 456678 false 1993-08-03T15:34:29 010-18741039 {"wTS":"1596440069","rfId":"010-18741039","remark":"数据","weight":111.068} 710078936338096128 1596440069-010-18741039 -139868811185971200 10000 1993-08-03T15:34:30.684 10000 1993-08-03T15:34:30.684 201 456678 false 1993-08-03T15:34:29 010-18741040 {"wTS":"1596440069","rfId":"010-18741040","remark":"数据","weight":133.627} 710078936338096128 1596440069-010-18741040 -139868812108718080 10000 1993-08-03T15:34:30.904 10000 1993-08-03T15:34:30.904 201 456678 false 1993-08-03T15:34:29 010-18741033 {"wTS":"1596440069","rfId":"010-18741033","remark":"数据","weight":66.858} 710078936338096128 1596440069-010-18741033 -139868933114388480 10000 1993-08-03T15:34:59.754 10000 1993-08-03T15:34:59.754 201 456678 false 1993-08-03T15:34:59 010-18741032 {"wTS":"1596440099","rfId":"010-18741032","remark":"数据","weight":108.355} 710078936338096128 1596440099-010-18741032 -139868933353463808 10000 1993-08-03T15:34:59.811 10000 1993-08-03T15:34:59.811 201 456678 false 1993-08-03T15:34:59 010-18741031 {"wTS":"1596440099","rfId":"010-18741031","remark":"数据","weight":85.044} 710078936338096128 1596440099-010-18741031 -139868933798060032 10000 1993-08-03T15:34:59.917 10000 1993-08-03T15:34:59.917 201 456678 false 1993-08-03T15:34:59 010-18741036 {"wTS":"1596440099","rfId":"010-18741036","remark":"数据","weight":75.728} 710078936338096128 1596440099-010-18741036 -139868933974220800 10000 1993-08-03T15:34:59.959 10000 1993-08-03T15:34:59.959 201 456678 false 1993-08-03T15:34:59 010-18741035 {"wTS":"1596440099","rfId":"010-18741035","remark":"数据","weight":133.986} 710078936338096128 1596440099-010-18741035 -139868935110877184 10000 1993-08-03T15:35:00.230 10000 1993-08-03T15:35:00.230 201 456678 false 1993-08-03T15:34:59 010-18741033 {"wTS":"1596440099","rfId":"010-18741033","remark":"数据","weight":66.858} 710078936338096128 1596440099-010-18741033 -139868935331109600 10000 1993-08-03T15:35:00.284 10000 1993-08-03T15:35:00.284 201 456678 false 1993-08-03T15:34:59 010-18741039 {"wTS":"1596440099","rfId":"010-18741039","remark":"数据","weight":111.068} 710078936338096128 1596440099-010-18741039 -139868935459004416 10000 1993-08-03T15:35:00.313 10000 1993-08-03T15:35:00.313 201 456678 false 1993-08-03T15:34:59 010-18741040 {"wTS":"1596440099","rfId":"010-18741040","remark":"数据","weight":133.627} 710078936338096128 1596440099-010-18741040 -139868935823908864 10000 1993-08-03T15:35:00.400 10000 1993-08-03T15:35:00.400 201 456678 false 1993-08-03T15:34:59 010-18741038 {"wTS":"1596440099","rfId":"010-18741038","remark":"数据","weight":122.591} 710078936338096128 1596440099-010-18741038 -139869059098697728 10000 1993-08-03T15:35:29.791 10000 1993-08-03T15:35:29.791 201 456678 false 1993-08-03T15:35:29 010-18741032 {"wTS":"1596440129","rfId":"010-18741032","remark":"数据","weight":108.355} 710078936338096128 1596440129-010-18741032 -139869059476185088 10000 1993-08-03T15:35:29.881 10000 1993-08-03T15:35:29.881 201 456678 false 1993-08-03T15:35:29 010-18741035 {"wTS":"1596440129","rfId":"010-18741035","remark":"数据","weight":133.986} 710078936338096128 1596440129-010-18741035 -139869059669123072 10000 1993-08-03T15:35:29.927 10000 1993-08-03T15:35:29.927 201 456678 false 1993-08-03T15:35:29 010-18741031 {"wTS":"1596440129","rfId":"010-18741031","remark":"数据","weight":85.044} 710078936338096128 1596440129-010-18741031 -139869060927414272 10000 1993-08-03T15:35:30.227 10000 1993-08-03T15:35:30.227 201 456678 false 1993-08-03T15:35:29 010-18741039 {"wTS":"1596440129","rfId":"010-18741039","remark":"数据","weight":111.068} 710078936338096128 1596440129-010-18741039 -139869061061632000 10000 1993-08-03T15:35:30.259 10000 1993-08-03T15:35:30.259 201 456678 false 1993-08-03T15:35:29 010-18741034 {"wTS":"1596440129","rfId":"010-18741034","remark":"数据","weight":157.6} 710078936338096128 1596440129-010-18741034 -139869061623668110 10000 1993-08-03T15:35:30.393 10000 1993-08-03T15:35:30.393 201 456678 false 1993-08-03T15:35:29 010-18741037 {"wTS":"1596440129","rfId":"010-18741037","remark":"数据","weight":139.592} 710078936338096128 1596440129-010-18741037 -139869062315728896 10000 1993-08-03T15:35:30.558 10000 1993-08-03T15:35:30.558 201 456678 false 1993-08-03T15:35:29 010-18741040 {"wTS":"1596440129","rfId":"010-18741040","remark":"数据","weight":133.627} 710078936338096128 1596440129-010-18741040 -139869063414636544 10000 1993-08-03T15:35:30.821 10000 1993-08-03T15:35:30.821 201 456678 false 1993-08-03T15:35:29 010-18741038 {"wTS":"1596440129","rfId":"010-18741038","remark":"数据","weight":122.591} 710078936338096128 1596440129-010-18741038 -139869065004277760 10000 1993-08-03T15:35:31.199 10000 1993-08-03T15:35:31.199 201 456678 false 1993-08-03T15:35:29 010-18741033 {"wTS":"1596440129","rfId":"010-18741033","remark":"数据","weight":66.858} 710078936338096128 1596440129-010-18741033 -139869186022531072 10000 1993-08-03T15:36:00.052 10000 1993-08-03T15:36:00.052 201 456678 false 1993-08-03T15:35:59 010-18741035 {"wTS":"1596440159","rfId":"010-18741035","remark":"数据","weight":133.986} 710078936338096128 1596440159-010-18741035 -139869186030919680 10000 1993-08-03T15:36:00.054 10000 1993-08-03T15:36:00.054 201 456678 false 1993-08-03T15:35:59 010-18741032 {"wTS":"1596440159","rfId":"010-18741032","remark":"数据","weight":108.355} 710078936338096128 1596440159-010-18741032 -139869186064474112 10000 1993-08-03T15:36:00.062 10000 1993-08-03T15:36:00.062 201 456678 false 1993-08-03T15:35:59 010-18741036 {"wTS":"1596440159","rfId":"010-18741036","remark":"数据","weight":75.728} 710078936338096128 1596440159-010-18741036 -139869186500681728 10000 1993-08-03T15:36:00.166 10000 1993-08-03T15:36:00.166 201 456678 false 1993-08-03T15:35:59 010-18741034 {"wTS":"1596440159","rfId":"010-18741034","remark":"数据","weight":157.6} 710078936338096128 1596440159-010-18741034 -139869187461177344 10000 1993-08-03T15:36:00.395 10000 1993-08-03T15:36:00.395 201 456678 false 1993-08-03T15:35:59 010-18741039 {"wTS":"1596440159","rfId":"010-18741039","remark":"数据","weight":111.068} 710078936338096128 1596440159-010-18741039 -139869188274872320 10000 1993-08-03T15:36:00.589 10000 1993-08-03T15:36:00.589 201 456678 false 1993-08-03T15:35:59 010-18741033 {"wTS":"1596440159","rfId":"010-18741033","remark":"数据","weight":66.858} 710078936338096128 1596440159-010-18741033 -139869188656553984 10000 1993-08-03T15:36:00.680 10000 1993-08-03T15:36:00.680 201 456678 false 1993-08-03T15:35:59 010-18741037 {"wTS":"1596440159","rfId":"010-18741037","remark":"数据","weight":139.592} 710078936338096128 1596440159-010-18741037 -139869189369585664 10000 1993-08-03T15:36:00.850 10000 1993-08-03T15:36:00.850 201 456678 false 1993-08-03T15:35:59 010-18741038 {"wTS":"1596440159","rfId":"010-18741038","remark":"数据","weight":122.591} 710078936338096128 1596440159-010-18741038 -139869190275555328 10000 1993-08-03T15:36:01.066 10000 1993-08-03T15:36:01.066 201 456678 false 1993-08-03T15:35:59 010-18741040 {"wTS":"1596440159","rfId":"010-18741040","remark":"数据","weight":133.627} 710078936338096128 1596440159-010-18741040 -139869238396805120 10000 1993-08-03T15:36:12.539 10000 1993-08-03T15:36:12.539 201 456678 false 1993-08-03T15:36:11 010-18741029 {"wTS":"1596440171","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440171-010-18741029 -139869238489079808 10000 1993-08-03T15:36:12.561 10000 1993-08-03T15:36:12.561 201 456678 false 1993-08-03T15:36:11 010-18741031 {"wTS":"1596440171","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440171-010-18741031 -139869239512489984 10000 1993-08-03T15:36:12.805 10000 1993-08-03T15:36:12.806 201 456678 false 1993-08-03T15:36:11 010-18741021 {"wTS":"1596440171","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440171-010-18741021 -139869241085353984 10000 1993-08-03T15:36:13.180 10000 1993-08-03T15:36:13.180 201 456678 false 1993-08-03T15:36:11 010-18741027 {"wTS":"1596440171","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440171-010-18741027 -139869241093742592 10000 1993-08-03T15:36:13.182 10000 1993-08-03T15:36:13.182 201 456678 false 1993-08-03T15:36:11 010-18741026 {"wTS":"1596440171","rfId":"010-18741026","remark":"数据","weight":147.465} 710078936338096128 1596440171-010-18741026 -139869241106325504 10000 1993-08-03T15:36:13.185 10000 1993-08-03T15:36:13.185 201 456678 false 1993-08-03T15:36:11 010-18741022 {"wTS":"1596440171","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440171-010-18741022 -139869241160851456 10000 1993-08-03T15:36:13.198 10000 1993-08-03T15:36:13.198 201 456678 false 1993-08-03T15:36:11 010-18741028 {"wTS":"1596440171","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440171-010-18741028 -139869241186017280 10000 1993-08-03T15:36:13.204 10000 1993-08-03T15:36:13.204 201 456678 false 1993-08-03T15:36:11 010-18741023 {"wTS":"1596440171","rfId":"010-18741023","remark":"数据","weight":118.114} 710078936338096128 1596440171-010-18741023 -139869241307652096 10000 1993-08-03T15:36:13.233 10000 1993-08-03T15:36:13.233 201 456678 false 1993-08-03T15:36:11 010-18741024 {"wTS":"1596440171","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440171-010-18741024 -139869242075209728 10000 1993-08-03T15:36:13.416 10000 1993-08-03T15:36:13.416 201 456678 false 1993-08-03T15:36:11 010-18741030 {"wTS":"1596440171","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440171-010-18741030 -139869243685822464 10000 1993-08-03T15:36:13.800 10000 1993-08-03T15:36:13.800 201 456678 false 1993-08-03T15:36:11 010-18741025 {"wTS":"1596440171","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440171-010-18741025 -139869311054720512 10000 1993-08-03T15:36:44.787 10000 1993-08-03T15:36:44.787 201 456678 false 1993-08-03T15:36:42 010-18741028 {"wTS":"1596440202","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440202-010-18741028 -139869366893502464 10000 1993-08-03T15:36:43.175 10000 1993-08-03T15:36:43.175 201 456678 false 1993-08-03T15:36:42 010-18741035 {"wTS":"1596440202","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440202-010-18741035 -139869367610728448 10000 1993-08-03T15:36:43.346 10000 1993-08-03T15:36:43.346 201 456678 false 1993-08-03T15:36:42 010-18741039 {"wTS":"1596440202","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440202-010-18741039 -139869368155987968 10000 1993-08-03T15:36:43.476 10000 1993-08-03T15:36:43.476 201 456678 false 1993-08-03T15:36:42 010-18741038 {"wTS":"1596440202","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440202-010-18741038 -139869368168570880 10000 1993-08-03T15:36:43.480 10000 1993-08-03T15:36:43.480 201 456678 false 1993-08-03T15:36:42 010-18741024 {"wTS":"1596440202","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440202-010-18741024 -139869368235679744 10000 1993-08-03T15:36:43.495 10000 1993-08-03T15:36:43.495 201 456678 false 1993-08-03T15:36:42 010-18741040 {"wTS":"1596440202","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440202-010-18741040 -139869368374091776 10000 1993-08-03T15:36:43.528 10000 1993-08-03T15:36:43.528 201 456678 false 1993-08-03T15:36:42 010-18741034 {"wTS":"1596440202","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440202-010-18741034 -139869368776744960 10000 1993-08-03T15:36:43.624 10000 1993-08-03T15:36:43.624 201 456678 false 1993-08-03T15:36:42 010-18741022 {"wTS":"1596440202","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440202-010-18741022 -139869370446077952 10000 1993-08-03T15:36:44.022 10000 1993-08-03T15:36:44.022 201 456678 false 1993-08-03T15:36:42 010-18741036 {"wTS":"1596440202","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440202-010-18741036 -139869372174131200 10000 1993-08-03T15:36:44.434 10000 1993-08-03T15:36:44.434 201 456678 false 1993-08-03T15:36:42 010-18741025 {"wTS":"1596440202","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440202-010-18741025 -139869373784743936 10000 1993-08-03T15:36:44.818 10000 1993-08-03T15:36:44.818 201 456678 false 1993-08-03T15:36:42 010-18741027 {"wTS":"1596440202","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440202-010-18741027 -139869374111899648 10000 1993-08-03T15:36:44.896 10000 1993-08-03T15:36:44.896 201 456678 false 1993-08-03T15:36:42 010-18741030 {"wTS":"1596440202","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440202-010-18741030 -139869374170619904 10000 1993-08-03T15:36:44.910 10000 1993-08-03T15:36:44.910 201 456678 false 1993-08-03T15:36:42 010-18741026 {"wTS":"1596440202","rfId":"010-18741026","remark":"数据","weight":147.465} 710078936338096128 1596440202-010-18741026 -139869374652964864 10000 1993-08-03T15:36:45.025 10000 1993-08-03T15:36:45.025 201 456678 false 1993-08-03T15:36:42 010-18741033 {"wTS":"1596440202","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440202-010-18741033 -139869376062251008 10000 1993-08-03T15:36:45.361 10000 1993-08-03T15:36:45.361 201 456678 false 1993-08-03T15:36:42 010-18741031 {"wTS":"1596440202","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440202-010-18741031 -139869376079028224 10000 1993-08-03T15:36:45.365 10000 1993-08-03T15:36:45.365 201 456678 false 1993-08-03T15:36:42 010-18741023 {"wTS":"1596440202","rfId":"010-18741023","remark":"数据","weight":118.114} 710078936338096128 1596440202-010-18741023 -139869376330686464 10000 1993-08-03T15:36:45.425 10000 1993-08-03T15:36:45.425 201 456678 false 1993-08-03T15:36:42 010-18741037 {"wTS":"1596440202","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440202-010-18741037 -139869376339075072 10000 1993-08-03T15:36:45.427 10000 1993-08-03T15:36:45.427 201 456678 false 1993-08-03T15:36:42 010-18741029 {"wTS":"1596440202","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440202-010-18741029 -139869377677058048 10000 1993-08-03T15:36:45.746 10000 1993-08-03T15:36:45.746 201 456678 false 1993-08-03T15:36:42 010-18741021 {"wTS":"1596440202","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440202-010-18741021 -139869379086344192 10000 1993-08-03T15:36:46.082 10000 1993-08-03T15:36:46.082 201 456678 false 1993-08-03T15:36:42 010-18741032 {"wTS":"1596440202","rfId":"010-18741032","remark":"数据","weight":118.863} 710078936338096128 1596440202-010-18741032 -139869494853328896 10000 1993-08-03T15:37:13.683 10000 1993-08-03T15:37:13.683 201 456678 false 1993-08-03T15:37:12 010-18741022 {"wTS":"1596440232","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440232-010-18741022 -139869494886883328 10000 1993-08-03T15:37:13.691 10000 1993-08-03T15:37:13.691 201 456678 false 1993-08-03T15:37:13 010-18741028 {"wTS":"1596440233","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440233-010-18741028 -139869494907854848 10000 1993-08-03T15:37:13.696 10000 1993-08-03T15:37:13.696 201 456678 false 1993-08-03T15:37:12 010-18741040 {"wTS":"1596440232","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440232-010-18741040 -139869494937214976 10000 1993-08-03T15:37:13.703 10000 1993-08-03T15:37:13.703 201 456678 false 1993-08-03T15:37:12 010-18741039 {"wTS":"1596440232","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440232-010-18741039 -139869495016906752 10000 1993-08-03T15:37:13.722 10000 1993-08-03T15:37:13.722 201 456678 false 1993-08-03T15:37:13 010-18741025 {"wTS":"1596440233","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440233-010-18741025 -139869495058849792 10000 1993-08-03T15:37:13.732 10000 1993-08-03T15:37:13.732 201 456678 false 1993-08-03T15:37:12 010-18741038 {"wTS":"1596440232","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440232-010-18741038 -139869495209844110 10000 1993-08-03T15:37:13.768 10000 1993-08-03T15:37:13.769 201 456678 false 1993-08-03T15:37:12 010-18741035 {"wTS":"1596440232","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440232-010-18741035 -139869495616692224 10000 1993-08-03T15:37:13.865 10000 1993-08-03T15:37:13.865 201 456678 false 1993-08-03T15:37:13 010-18741036 {"wTS":"1596440233","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440233-010-18741036 -139869496199700480 10000 1993-08-03T15:37:14.004 10000 1993-08-03T15:37:14.004 201 456678 false 1993-08-03T15:37:12 010-18741024 {"wTS":"1596440232","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440232-010-18741024 -139869496459747328 10000 1993-08-03T15:37:14.066 10000 1993-08-03T15:37:14.066 201 456678 false 1993-08-03T15:37:12 010-18741034 {"wTS":"1596440232","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440232-010-18741034 -139869497097281536 10000 1993-08-03T15:37:14.218 10000 1993-08-03T15:37:14.218 201 456678 false 1993-08-03T15:37:12 010-18741033 {"wTS":"1596440232","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440232-010-18741033 -139869498498179072 10000 1993-08-03T15:37:14.552 10000 1993-08-03T15:37:14.552 201 456678 false 1993-08-03T15:37:13 010-18741027 {"wTS":"1596440233","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440233-010-18741027 -139869500343672832 10000 1993-08-03T15:37:14.992 10000 1993-08-03T15:37:14.992 201 456678 false 1993-08-03T15:37:12 010-18741031 {"wTS":"1596440232","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440232-010-18741031 -139869507037782016 10000 1993-08-03T15:37:16.588 10000 1993-08-03T15:37:16.589 201 456678 false 1993-08-03T15:37:13 010-18741026 {"wTS":"1596440233","rfId":"010-18741026","remark":"数据","weight":147.465} 710078936338096128 1596440233-010-18741026 -139869507054559232 10000 1993-08-03T15:37:16.592 10000 1993-08-03T15:37:16.592 201 456678 false 1993-08-03T15:37:13 010-18741023 {"wTS":"1596440233","rfId":"010-18741023","remark":"数据","weight":118.114} 710078936338096128 1596440233-010-18741023 -139869507281051648 10000 1993-08-03T15:37:16.646 10000 1993-08-03T15:37:16.646 201 456678 false 1993-08-03T15:37:13 010-18741029 {"wTS":"1596440233","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440233-010-18741029 -139869507830505472 10000 1993-08-03T15:37:16.777 10000 1993-08-03T15:37:16.777 201 456678 false 1993-08-03T15:37:13 010-18741037 {"wTS":"1596440233","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440233-010-18741037 -139869508501594112 10000 1993-08-03T15:37:16.937 10000 1993-08-03T15:37:16.937 201 456678 false 1993-08-03T15:37:13 010-18741021 {"wTS":"1596440233","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440233-010-18741021 -139869509084602368 10000 1993-08-03T15:37:17.076 10000 1993-08-03T15:37:17.076 201 456678 false 1993-08-03T15:37:13 010-18741030 {"wTS":"1596440233","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440233-010-18741030 -139869509629861888 10000 1993-08-03T15:37:17.206 10000 1993-08-03T15:37:17.206 201 456678 false 1993-08-03T15:37:13 010-18741032 {"wTS":"1596440233","rfId":"010-18741032","remark":"数据","weight":118.863} 710078936338096128 1596440233-010-18741032 -139869619306717184 10000 1993-08-03T15:37:43.355 10000 1993-08-03T15:37:43.355 201 456678 false 1993-08-03T15:37:42 010-18741038 {"wTS":"1596440262","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440262-010-18741038 -139869620531453952 10000 1993-08-03T15:37:43.647 10000 1993-08-03T15:37:43.647 201 456678 false 1993-08-03T15:37:43 010-18741022 {"wTS":"1596440263","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440263-010-18741022 -139869620544036864 10000 1993-08-03T15:37:43.650 10000 1993-08-03T15:37:43.650 201 456678 false 1993-08-03T15:37:42 010-18741035 {"wTS":"1596440262","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440262-010-18741035 -139869620552425472 10000 1993-08-03T15:37:43.652 10000 1993-08-03T15:37:43.652 201 456678 false 1993-08-03T15:37:43 010-18741040 {"wTS":"1596440263","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440263-010-18741040 -139869620573396992 10000 1993-08-03T15:37:43.657 10000 1993-08-03T15:37:43.657 201 456678 false 1993-08-03T15:37:42 010-18741039 {"wTS":"1596440262","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440262-010-18741039 -139869620602757120 10000 1993-08-03T15:37:43.664 10000 1993-08-03T15:37:43.664 201 456678 false 1993-08-03T15:37:43 010-18741024 {"wTS":"1596440263","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440263-010-18741024 -139869620732780544 10000 1993-08-03T15:37:43.695 10000 1993-08-03T15:37:43.695 201 456678 false 1993-08-03T15:37:43 010-18741031 {"wTS":"1596440263","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440263-010-18741031 -139869621395480576 10000 1993-08-03T15:37:43.853 10000 1993-08-03T15:37:43.853 201 456678 false 1993-08-03T15:37:43 010-18741028 {"wTS":"1596440263","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440263-010-18741028 -139869623563935744 10000 1993-08-03T15:37:44.370 10000 1993-08-03T15:37:44.370 201 456678 false 1993-08-03T15:37:43 010-18741025 {"wTS":"1596440263","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440263-010-18741025 -139869626030186496 10000 1993-08-03T15:37:44.958 10000 1993-08-03T15:37:44.958 201 456678 false 1993-08-03T15:37:43 010-18741036 {"wTS":"1596440263","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440263-010-18741036 -139869626176987136 10000 1993-08-03T15:37:44.993 10000 1993-08-03T15:37:44.993 201 456678 false 1993-08-03T15:37:43 010-18741033 {"wTS":"1596440263","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440263-010-18741033 -139869626688692224 10000 1993-08-03T15:37:45.115 10000 1993-08-03T15:37:45.115 201 456678 false 1993-08-03T15:37:43 010-18741021 {"wTS":"1596440263","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440263-010-18741021 -139869627057790976 10000 1993-08-03T15:37:45.203 10000 1993-08-03T15:37:45.203 201 456678 false 1993-08-03T15:37:43 010-18741037 {"wTS":"1596440263","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440263-010-18741037 -139869627733073920 10000 1993-08-03T15:37:45.364 10000 1993-08-03T15:37:45.364 201 456678 false 1993-08-03T15:37:43 010-18741029 {"wTS":"1596440263","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440263-010-18741029 -139869628341248000 10000 1993-08-03T15:37:45.509 10000 1993-08-03T15:37:45.509 201 456678 false 1993-08-03T15:37:43 010-18741032 {"wTS":"1596440263","rfId":"010-18741032","remark":"数据","weight":118.863} 710078936338096128 1596440263-010-18741032 -139869628395773952 10000 1993-08-03T15:37:45.522 10000 1993-08-03T15:37:45.522 201 456678 false 1993-08-03T15:37:43 010-18741030 {"wTS":"1596440263","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440263-010-18741030 -139869629545013248 10000 1993-08-03T15:37:45.796 10000 1993-08-03T15:37:45.796 201 456678 false 1993-08-03T15:37:43 010-18741023 {"wTS":"1596440263","rfId":"010-18741023","remark":"数据","weight":118.114} 710078936338096128 1596440263-010-18741023 -139869631533113344 10000 1993-08-03T15:37:46.270 10000 1993-08-03T15:37:46.270 201 456678 false 1993-08-03T15:37:43 010-18741026 {"wTS":"1596440263","rfId":"010-18741026","remark":"数据","weight":147.465} 710078936338096128 1596440263-010-18741026 -139869631830908928 10000 1993-08-03T15:37:46.341 10000 1993-08-03T15:37:46.341 201 456678 false 1993-08-03T15:37:43 010-18741027 {"wTS":"1596440263","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440263-010-18741027 -139869744959676416 10000 1993-08-03T15:38:13.313 10000 1993-08-03T15:38:13.313 201 456678 false 1993-08-03T15:38:12 010-18741038 {"wTS":"1596440292","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440292-010-18741038 -139869745026785280 10000 1993-08-03T15:38:13.329 10000 1993-08-03T15:38:13.329 201 456678 false 1993-08-03T15:38:12 010-18741035 {"wTS":"1596440292","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440292-010-18741035 -139869745072922624 10000 1993-08-03T15:38:13.340 10000 1993-08-03T15:38:13.340 201 456678 false 1993-08-03T15:38:12 010-18741039 {"wTS":"1596440292","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440292-010-18741039 -139869745978892288 10000 1993-08-03T15:38:13.556 10000 1993-08-03T15:38:13.556 201 456678 false 1993-08-03T15:38:13 010-18741040 {"wTS":"1596440293","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440293-010-18741040 -139869746314436608 10000 1993-08-03T15:38:13.636 10000 1993-08-03T15:38:13.636 201 456678 false 1993-08-03T15:38:13 010-18741022 {"wTS":"1596440293","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440293-010-18741022 -139869746951970816 10000 1993-08-03T15:38:13.788 10000 1993-08-03T15:38:13.788 201 456678 false 1993-08-03T15:38:13 010-18741034 {"wTS":"1596440293","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440293-010-18741034 -139869747128131584 10000 1993-08-03T15:38:13.830 10000 1993-08-03T15:38:13.830 201 456678 false 1993-08-03T15:38:13 010-18741033 {"wTS":"1596440293","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440293-010-18741033 -139869747832774656 10000 1993-08-03T15:38:13.998 10000 1993-08-03T15:38:13.998 201 456678 false 1993-08-03T15:38:13 010-18741024 {"wTS":"1596440293","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440293-010-18741024 -139869749611000002 10000 1993-08-03T15:38:14.422 10000 1993-08-03T15:38:14.422 201 456678 false 1993-08-03T15:38:13 010-18741025 {"wTS":"1596440293","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440293-010-18741025 -139869751498596352 10000 1993-08-03T15:38:14.872 10000 1993-08-03T15:38:14.872 201 456678 false 1993-08-03T15:38:13 010-18741029 {"wTS":"1596440293","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440293-010-18741029 -139869751574093824 10000 1993-08-03T15:38:14.890 10000 1993-08-03T15:38:14.890 201 456678 false 1993-08-03T15:38:13 010-18741023 {"wTS":"1596440293","rfId":"010-18741023","remark":"数据","weight":118.114} 710078936338096128 1596440293-010-18741023 -139869751859306496 10000 1993-08-03T15:38:14.958 10000 1993-08-03T15:38:14.958 201 456678 false 1993-08-03T15:38:13 010-18741031 {"wTS":"1596440293","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440293-010-18741031 -139869752371011584 10000 1993-08-03T15:38:15.080 10000 1993-08-03T15:38:15.080 201 456678 false 1993-08-03T15:38:13 010-18741036 {"wTS":"1596440293","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440293-010-18741036 -139869754161979392 10000 1993-08-03T15:38:15.507 10000 1993-08-03T15:38:15.507 201 456678 false 1993-08-03T15:38:13 010-18741028 {"wTS":"1596440293","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440293-010-18741028 -139869754262642688 10000 1993-08-03T15:38:15.531 10000 1993-08-03T15:38:15.531 201 456678 false 1993-08-03T15:38:13 010-18741032 {"wTS":"1596440293","rfId":"010-18741032","remark":"数据","weight":118.863} 710078936338096128 1596440293-010-18741032 -139869755197972480 10000 1993-08-03T15:38:15.754 10000 1993-08-03T15:38:15.754 201 456678 false 1993-08-03T15:38:13 010-18741026 {"wTS":"1596440293","rfId":"010-18741026","remark":"数据","weight":147.465} 710078936338096128 1596440293-010-18741026 -139869756246548480 10000 1993-08-03T15:38:16.004 10000 1993-08-03T15:38:16.004 201 456678 false 1993-08-03T15:38:13 010-18741037 {"wTS":"1596440293","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440293-010-18741037 -139869756783419392 10000 1993-08-03T15:38:16.132 10000 1993-08-03T15:38:16.132 201 456678 false 1993-08-03T15:38:13 010-18741021 {"wTS":"1596440293","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440293-010-18741021 -139869871002705920 10000 1993-08-03T15:38:43.364 10000 1993-08-03T15:38:43.364 201 456678 false 1993-08-03T15:38:43 010-18741038 {"wTS":"1596440323","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440323-010-18741038 -139869871027871744 10000 1993-08-03T15:38:43.370 10000 1993-08-03T15:38:43.370 201 456678 false 1993-08-03T15:38:43 010-18741035 {"wTS":"1596440323","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440323-010-18741035 -139869871292112896 10000 1993-08-03T15:38:43.433 10000 1993-08-03T15:38:43.433 201 456678 false 1993-08-03T15:38:43 010-18741039 {"wTS":"1596440323","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440323-010-18741039 -139869872256802816 10000 1993-08-03T15:38:43.663 10000 1993-08-03T15:38:43.663 201 456678 false 1993-08-03T15:38:43 010-18741022 {"wTS":"1596440323","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440323-010-18741022 -139869872802062336 10000 1993-08-03T15:38:43.793 10000 1993-08-03T15:38:43.793 201 456678 false 1993-08-03T15:38:43 010-18741024 {"wTS":"1596440323","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440323-010-18741024 -139869872965640192 10000 1993-08-03T15:38:43.832 10000 1993-08-03T15:38:43.832 201 456678 false 1993-08-03T15:38:43 010-18741034 {"wTS":"1596440323","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440323-010-18741034 -139869873049526272 10000 1993-08-03T15:38:43.852 10000 1993-08-03T15:38:43.852 201 456678 false 1993-08-03T15:38:43 010-18741040 {"wTS":"1596440323","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440323-010-18741040 -139869874500755456 10000 1993-08-03T15:38:44.198 10000 1993-08-03T15:38:44.198 201 456678 false 1993-08-03T15:38:43 010-18741033 {"wTS":"1596440323","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440323-010-18741033 -139869875629023232 10000 1993-08-03T15:38:44.467 10000 1993-08-03T15:38:44.468 201 456678 false 1993-08-03T15:38:43 010-18741030 {"wTS":"1596440323","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440323-010-18741030 -139869877268996096 10000 1993-08-03T15:38:44.858 10000 1993-08-03T15:38:44.858 201 456678 false 1993-08-03T15:38:43 010-18741027 {"wTS":"1596440323","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440323-010-18741027 -139869877277384704 10000 1993-08-03T15:38:44.860 10000 1993-08-03T15:38:44.860 201 456678 false 1993-08-03T15:38:43 010-18741025 {"wTS":"1596440323","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440323-010-18741025 -139869877495488512 10000 1993-08-03T15:38:44.912 10000 1993-08-03T15:38:44.912 201 456678 false 1993-08-03T15:38:43 010-18741023 {"wTS":"1596440323","rfId":"010-18741023","remark":"数据","weight":118.114} 710078936338096128 1596440323-010-18741023 -139869878300794880 10000 1993-08-03T15:38:45.104 10000 1993-08-03T15:38:45.104 201 456678 false 1993-08-03T15:38:43 010-18741032 {"wTS":"1596440323","rfId":"010-18741032","remark":"数据","weight":118.863} 710078936338096128 1596440323-010-18741032 -139869880230174720 10000 1993-08-03T15:38:45.564 10000 1993-08-03T15:38:45.564 201 456678 false 1993-08-03T15:38:43 010-18741021 {"wTS":"1596440323","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440323-010-18741021 -139869997196730368 10000 1993-08-03T15:39:13.451 10000 1993-08-03T15:39:13.451 201 456678 false 1993-08-03T15:39:13 010-18741038 {"wTS":"1596440353","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440353-010-18741038 -139869997305782272 10000 1993-08-03T15:39:13.477 10000 1993-08-03T15:39:13.477 201 456678 false 1993-08-03T15:39:13 010-18741039 {"wTS":"1596440353","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440353-010-18741039 -139869997645520896 10000 1993-08-03T15:39:13.558 10000 1993-08-03T15:39:13.558 201 456678 false 1993-08-03T15:39:13 010-18741035 {"wTS":"1596440353","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440353-010-18741035 -139869997792321536 10000 1993-08-03T15:39:13.593 10000 1993-08-03T15:39:13.593 201 456678 false 1993-08-03T15:39:13 010-18741022 {"wTS":"1596440353","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440353-010-18741022 -139869999402934272 10000 1993-08-03T15:39:13.977 10000 1993-08-03T15:39:13.977 201 456678 false 1993-08-03T15:39:13 010-18741034 {"wTS":"1596440353","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440353-010-18741034 -139869999667175424 10000 1993-08-03T15:39:14.040 10000 1993-08-03T15:39:14.040 201 456678 false 1993-08-03T15:39:13 010-18741040 {"wTS":"1596440353","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440353-010-18741040 -139870001030324224 10000 1993-08-03T15:39:14.365 10000 1993-08-03T15:39:14.365 201 456678 false 1993-08-03T15:39:13 010-18741033 {"wTS":"1596440353","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440353-010-18741033 -139870001357479936 10000 1993-08-03T15:39:14.443 10000 1993-08-03T15:39:14.443 201 456678 false 1993-08-03T15:39:13 010-18741031 {"wTS":"1596440353","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440353-010-18741031 -139870003332997120 10000 1993-08-03T15:39:14.914 10000 1993-08-03T15:39:14.914 201 456678 false 1993-08-03T15:39:13 010-18741024 {"wTS":"1596440353","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440353-010-18741024 -139870003379134464 10000 1993-08-03T15:39:14.925 10000 1993-08-03T15:39:14.925 201 456678 false 1993-08-03T15:39:13 010-18741028 {"wTS":"1596440353","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440353-010-18741028 -139870003387523072 10000 1993-08-03T15:39:14.927 10000 1993-08-03T15:39:14.927 201 456678 false 1993-08-03T15:39:13 010-18741036 {"wTS":"1596440353","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440353-010-18741036 -139870003450437632 10000 1993-08-03T15:39:14.942 10000 1993-08-03T15:39:14.942 201 456678 false 1993-08-03T15:39:13 010-18741037 {"wTS":"1596440353","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440353-010-18741037 -139870004415127552 10000 1993-08-03T15:39:15.172 10000 1993-08-03T15:39:15.172 201 456678 false 1993-08-03T15:39:13 010-18741023 {"wTS":"1596440353","rfId":"010-18741023","remark":"数据","weight":118.114} 710078936338096128 1596440353-010-18741023 -139870006185123840 10000 1993-08-03T15:39:15.594 10000 1993-08-03T15:39:15.594 201 456678 false 1993-08-03T15:39:13 010-18741029 {"wTS":"1596440353","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440353-010-18741029 -139870006772326400 10000 1993-08-03T15:39:15.734 10000 1993-08-03T15:39:15.734 201 456678 false 1993-08-03T15:39:13 010-18741025 {"wTS":"1596440353","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440353-010-18741025 -139870007275642880 10000 1993-08-03T15:39:15.854 10000 1993-08-03T15:39:15.854 201 456678 false 1993-08-03T15:39:13 010-18741026 {"wTS":"1596440353","rfId":"010-18741026","remark":"数据","weight":147.465} 710078936338096128 1596440353-010-18741026 -139870007862845440 10000 1993-08-03T15:39:15.994 10000 1993-08-03T15:39:15.994 201 456678 false 1993-08-03T15:39:13 010-18741030 {"wTS":"1596440353","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440353-010-18741030 -139870008290664448 10000 1993-08-03T15:39:16.096 10000 1993-08-03T15:39:16.096 201 456678 false 1993-08-03T15:39:13 010-18741027 {"wTS":"1596440353","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440353-010-18741027 -139870008965947392 10000 1993-08-03T15:39:16.257 10000 1993-08-03T15:39:16.257 201 456678 false 1993-08-03T15:39:13 010-18741032 {"wTS":"1596440353","rfId":"010-18741032","remark":"数据","weight":118.863} 710078936338096128 1596440353-010-18741032 -139870123344617472 10000 1993-08-03T15:39:43.527 10000 1993-08-03T15:39:43.527 201 456678 false 1993-08-03T15:39:43 010-18741038 {"wTS":"1596440383","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440383-010-18741038 -139870123432697856 10000 1993-08-03T15:39:43.548 10000 1993-08-03T15:39:43.549 201 456678 false 1993-08-03T15:39:43 010-18741039 {"wTS":"1596440383","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440383-010-18741039 -139870123650801664 10000 1993-08-03T15:39:43.600 10000 1993-08-03T15:39:43.600 201 456678 false 1993-08-03T15:39:43 010-18741035 {"wTS":"1596440383","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440383-010-18741035 -139870124363833344 10000 1993-08-03T15:39:43.770 10000 1993-08-03T15:39:43.770 201 456678 false 1993-08-03T15:39:43 010-18741022 {"wTS":"1596440383","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440383-010-18741022 -139870125013950464 10000 1993-08-03T15:39:43.925 10000 1993-08-03T15:39:43.925 201 456678 false 1993-08-03T15:39:43 010-18741034 {"wTS":"1596440383","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440383-010-18741034 -139870125345300480 10000 1993-08-03T15:39:44.004 10000 1993-08-03T15:39:44.004 201 456678 false 1993-08-03T15:39:43 010-18741040 {"wTS":"1596440383","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440383-010-18741040 -139870126607785984 10000 1993-08-03T15:39:44.305 10000 1993-08-03T15:39:44.305 201 456678 false 1993-08-03T15:39:43 010-18741028 {"wTS":"1596440383","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440383-010-18741028 -139870127023022080 10000 1993-08-03T15:39:44.404 10000 1993-08-03T15:39:44.404 201 456678 false 1993-08-03T15:39:43 010-18741024 {"wTS":"1596440383","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440383-010-18741024 -139870127559892992 10000 1993-08-03T15:39:44.532 10000 1993-08-03T15:39:44.532 201 456678 false 1993-08-03T15:39:43 010-18741033 {"wTS":"1596440383","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440383-010-18741033 -139870128121929728 10000 1993-08-03T15:39:44.666 10000 1993-08-03T15:39:44.666 201 456678 false 1993-08-03T15:39:43 010-18741023 {"wTS":"1596440383","rfId":"010-18741023","remark":"数据","weight":118.114} 710078936338096128 1596440383-010-18741023 -139870129015316480 10000 1993-08-03T15:39:44.879 10000 1993-08-03T15:39:44.879 201 456678 false 1993-08-03T15:39:43 010-18741037 {"wTS":"1596440383","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440383-010-18741037 -139870129606713344 10000 1993-08-03T15:39:45.020 10000 1993-08-03T15:39:45.020 201 456678 false 1993-08-03T15:39:43 010-18741021 {"wTS":"1596440383","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440383-010-18741021 -139870130911141888 10000 1993-08-03T15:39:45.331 10000 1993-08-03T15:39:45.331 201 456678 false 1993-08-03T15:39:43 010-18741031 {"wTS":"1596440383","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440383-010-18741031 -139870130936307712 10000 1993-08-03T15:39:45.337 10000 1993-08-03T15:39:45.337 201 456678 false 1993-08-03T15:39:43 010-18741036 {"wTS":"1596440383","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440383-010-18741036 -139870130969862144 10000 1993-08-03T15:39:45.345 10000 1993-08-03T15:39:45.345 201 456678 false 1993-08-03T15:39:43 010-18741025 {"wTS":"1596440383","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440383-010-18741025 -139870131003416576 10000 1993-08-03T15:39:45.353 10000 1993-08-03T15:39:45.353 201 456678 false 1993-08-03T15:39:43 010-18741029 {"wTS":"1596440383","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440383-010-18741029 -139870249345703936 10000 1993-08-03T15:40:13.568 10000 1993-08-03T15:40:13.568 201 456678 false 1993-08-03T15:40:13 010-18741038 {"wTS":"1596440413","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440413-010-18741038 -139870250390085632 10000 1993-08-03T15:40:13.817 10000 1993-08-03T15:40:13.817 201 456678 false 1993-08-03T15:40:13 010-18741022 {"wTS":"1596440413","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440413-010-18741022 -139870250432028672 10000 1993-08-03T15:40:13.827 10000 1993-08-03T15:40:13.827 201 456678 false 1993-08-03T15:40:13 010-18741034 {"wTS":"1596440413","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440413-010-18741034 -139870250453000192 10000 1993-08-03T15:40:13.832 10000 1993-08-03T15:40:13.832 201 456678 false 1993-08-03T15:40:13 010-18741039 {"wTS":"1596440413","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440413-010-18741039 -139870250490748928 10000 1993-08-03T15:40:13.841 10000 1993-08-03T15:40:13.841 201 456678 false 1993-08-03T15:40:13 010-18741035 {"wTS":"1596440413","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440413-010-18741035 -139870250880819200 10000 1993-08-03T15:40:13.934 10000 1993-08-03T15:40:13.934 201 456678 false 1993-08-03T15:40:13 010-18741024 {"wTS":"1596440413","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440413-010-18741024 -139870251040202752 10000 1993-08-03T15:40:13.972 10000 1993-08-03T15:40:13.972 201 456678 false 1993-08-03T15:40:13 010-18741033 {"wTS":"1596440413","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440413-010-18741033 -139870251698708480 10000 1993-08-03T15:40:14.129 10000 1993-08-03T15:40:14.129 201 456678 false 1993-08-03T15:40:13 010-18741040 {"wTS":"1596440413","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440413-010-18741040 -139870253548396544 10000 1993-08-03T15:40:14.570 10000 1993-08-03T15:40:14.570 201 456678 false 1993-08-03T15:40:13 010-18741027 {"wTS":"1596440413","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440413-010-18741027 -139870254777327616 10000 1993-08-03T15:40:14.863 10000 1993-08-03T15:40:14.863 201 456678 false 1993-08-03T15:40:13 010-18741030 {"wTS":"1596440413","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440413-010-18741030 -139870256186613760 10000 1993-08-03T15:40:15.199 10000 1993-08-03T15:40:15.199 201 456678 false 1993-08-03T15:40:13 010-18741026 {"wTS":"1596440413","rfId":"010-18741026","remark":"数据","weight":147.465} 710078936338096128 1596440413-010-18741026 -139870256429883392 10000 1993-08-03T15:40:15.257 10000 1993-08-03T15:40:15.257 201 456678 false 1993-08-03T15:40:13 010-18741021 {"wTS":"1596440413","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440413-010-18741021 -139870257587511296 10000 1993-08-03T15:40:15.533 10000 1993-08-03T15:40:15.533 201 456678 false 1993-08-03T15:40:13 010-18741037 {"wTS":"1596440413","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440413-010-18741037 -139870257650425856 10000 1993-08-03T15:40:15.548 10000 1993-08-03T15:40:15.548 201 456678 false 1993-08-03T15:40:13 010-18741032 {"wTS":"1596440413","rfId":"010-18741032","remark":"数据","weight":118.863} 710078936338096128 1596440413-010-18741032 -139870375501979648 10000 1993-08-03T15:40:43.646 10000 1993-08-03T15:40:43.646 201 456678 false 1993-08-03T15:40:43 010-18741035 {"wTS":"1596440443","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440443-010-18741035 -139870375611031552 10000 1993-08-03T15:40:43.672 10000 1993-08-03T15:40:43.672 201 456678 false 1993-08-03T15:40:43 010-18741038 {"wTS":"1596440443","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440443-010-18741038 -139870375803969536 10000 1993-08-03T15:40:43.718 10000 1993-08-03T15:40:43.718 201 456678 false 1993-08-03T15:40:43 010-18741039 {"wTS":"1596440443","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440443-010-18741039 -139870376328257536 10000 1993-08-03T15:40:43.843 10000 1993-08-03T15:40:43.843 201 456678 false 1993-08-03T15:40:43 010-18741022 {"wTS":"1596440443","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440443-010-18741022 -139870376856739840 10000 1993-08-03T15:40:43.969 10000 1993-08-03T15:40:43.969 201 456678 false 1993-08-03T15:40:43 010-18741034 {"wTS":"1596440443","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440443-010-18741034 -139870377087426560 10000 1993-08-03T15:40:44.024 10000 1993-08-03T15:40:44.024 201 456678 false 1993-08-03T15:40:43 010-18741024 {"wTS":"1596440443","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440443-010-18741024 -139870378702233600 10000 1993-08-03T15:40:44.409 10000 1993-08-03T15:40:44.409 201 456678 false 1993-08-03T15:40:43 010-18741033 {"wTS":"1596440443","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440443-010-18741033 -139870378828062720 10000 1993-08-03T15:40:44.440 10000 1993-08-03T15:40:44.440 201 456678 false 1993-08-03T15:40:43 010-18741040 {"wTS":"1596440443","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440443-010-18741040 -139870380052799488 10000 1993-08-03T15:40:44.731 10000 1993-08-03T15:40:44.731 201 456678 false 1993-08-03T15:40:43 010-18741030 {"wTS":"1596440443","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440443-010-18741030 -139870380765831168 10000 1993-08-03T15:40:44.901 10000 1993-08-03T15:40:44.901 201 456678 false 1993-08-03T15:40:44 010-18741037 {"wTS":"1596440444","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440444-010-18741037 -139870381059432448 10000 1993-08-03T15:40:44.971 10000 1993-08-03T15:40:44.971 201 456678 false 1993-08-03T15:40:43 010-18741036 {"wTS":"1596440443","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440443-010-18741036 -139870381566943232 10000 1993-08-03T15:40:45.092 10000 1993-08-03T15:40:45.092 201 456678 false 1993-08-03T15:40:43 010-18741031 {"wTS":"1596440443","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440443-010-18741031 -139870383727009792 10000 1993-08-03T15:40:45.607 10000 1993-08-03T15:40:45.607 201 456678 false 1993-08-03T15:40:43 010-18741028 {"wTS":"1596440443","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440443-010-18741028 -139870384465207296 10000 1993-08-03T15:40:45.783 10000 1993-08-03T15:40:45.783 201 456678 false 1993-08-03T15:40:43 010-18741027 {"wTS":"1596440443","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440443-010-18741027 -139870385815773184 10000 1993-08-03T15:40:46.105 10000 1993-08-03T15:40:46.105 201 456678 false 1993-08-03T15:40:43 010-18741029 {"wTS":"1596440443","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440443-010-18741029 -139870385983545344 10000 1993-08-03T15:40:46.145 10000 1993-08-03T15:40:46.145 201 456678 false 1993-08-03T15:40:43 010-18741025 {"wTS":"1596440443","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440443-010-18741025 -139870385987739648 10000 1993-08-03T15:40:46.146 10000 1993-08-03T15:40:46.146 201 456678 false 1993-08-03T15:40:44 010-18741032 {"wTS":"1596440444","rfId":"010-18741032","remark":"数据","weight":118.863} 710078936338096128 1596440444-010-18741032 -139870386075820032 10000 1993-08-03T15:40:46.167 10000 1993-08-03T15:40:46.167 201 456678 false 1993-08-03T15:40:43 010-18741023 {"wTS":"1596440443","rfId":"010-18741023","remark":"数据","weight":118.114} 710078936338096128 1596440443-010-18741023 -139870386138734592 10000 1993-08-03T15:40:46.182 10000 1993-08-03T15:40:46.182 201 456678 false 1993-08-03T15:40:43 010-18741021 {"wTS":"1596440443","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440443-010-18741021 -139870501465317376 10000 1993-08-03T15:41:13.678 10000 1993-08-03T15:41:13.678 201 456678 false 1993-08-03T15:41:13 010-18741038 {"wTS":"1596440473","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440473-010-18741038 -139870502438395904 10000 1993-08-03T15:41:13.910 10000 1993-08-03T15:41:13.910 201 456678 false 1993-08-03T15:41:13 010-18741039 {"wTS":"1596440473","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440473-010-18741039 -139870502526476288 10000 1993-08-03T15:41:13.931 10000 1993-08-03T15:41:13.931 201 456678 false 1993-08-03T15:41:13 010-18741022 {"wTS":"1596440473","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440473-010-18741022 -139870502539059200 10000 1993-08-03T15:41:13.934 10000 1993-08-03T15:41:13.934 201 456678 false 1993-08-03T15:41:13 010-18741035 {"wTS":"1596440473","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440473-010-18741035 -139870502673276928 10000 1993-08-03T15:41:13.966 10000 1993-08-03T15:41:13.966 201 456678 false 1993-08-03T15:41:13 010-18741040 {"wTS":"1596440473","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440473-010-18741040 -139870502711025664 10000 1993-08-03T15:41:13.975 10000 1993-08-03T15:41:13.975 201 456678 false 1993-08-03T15:41:13 010-18741034 {"wTS":"1596440473","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440473-010-18741034 -139870503939956110 10000 1993-08-03T15:41:14.268 10000 1993-08-03T15:41:14.268 201 456678 false 1993-08-03T15:41:13 010-18741024 {"wTS":"1596440473","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440473-010-18741024 -139870506985021440 10000 1993-08-03T15:41:14.994 10000 1993-08-03T15:41:14.994 201 456678 false 1993-08-03T15:41:13 010-18741033 {"wTS":"1596440473","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440473-010-18741033 -139870507026964480 10000 1993-08-03T15:41:15.004 10000 1993-08-03T15:41:15.004 201 456678 false 1993-08-03T15:41:13 010-18741030 {"wTS":"1596440473","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440473-010-18741030 -139870507123433472 10000 1993-08-03T15:41:15.027 10000 1993-08-03T15:41:15.027 201 456678 false 1993-08-03T15:41:13 010-18741029 {"wTS":"1596440473","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440473-010-18741029 -139870507173765120 10000 1993-08-03T15:41:15.039 10000 1993-08-03T15:41:15.039 201 456678 false 1993-08-03T15:41:13 010-18741026 {"wTS":"1596440473","rfId":"010-18741026","remark":"数据","weight":147.465} 710078936338096128 1596440473-010-18741026 -139870508058763264 10000 1993-08-03T15:41:15.250 10000 1993-08-03T15:41:15.250 201 456678 false 1993-08-03T15:41:13 010-18741025 {"wTS":"1596440473","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440473-010-18741025 -139870508696297472 10000 1993-08-03T15:41:15.402 10000 1993-08-03T15:41:15.402 201 456678 false 1993-08-03T15:41:14 010-18741032 {"wTS":"1596440474","rfId":"010-18741032","remark":"数据","weight":118.863} 710078936338096128 1596440474-010-18741032 -139870509912645632 10000 1993-08-03T15:41:15.692 10000 1993-08-03T15:41:15.692 201 456678 false 1993-08-03T15:41:14 010-18741037 {"wTS":"1596440474","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440474-010-18741037 -139870627244105728 10000 1993-08-03T15:41:43.666 10000 1993-08-03T15:41:43.666 201 456678 false 1993-08-03T15:41:43 010-18741038 {"wTS":"1596440503","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440503-010-18741038 -139870627609010176 10000 1993-08-03T15:41:43.753 10000 1993-08-03T15:41:43.753 201 456678 false 1993-08-03T15:41:43 010-18741039 {"wTS":"1596440503","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440503-010-18741039 -139870627869057024 10000 1993-08-03T15:41:43.815 10000 1993-08-03T15:41:43.815 201 456678 false 1993-08-03T15:41:43 010-18741035 {"wTS":"1596440503","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440503-010-18741035 -139870628359790592 10000 1993-08-03T15:41:43.932 10000 1993-08-03T15:41:43.932 201 456678 false 1993-08-03T15:41:43 010-18741022 {"wTS":"1596440503","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440503-010-18741022 -139870628686946304 10000 1993-08-03T15:41:44.010 10000 1993-08-03T15:41:44.010 201 456678 false 1993-08-03T15:41:43 010-18741034 {"wTS":"1596440503","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440503-010-18741034 -139870629555167232 10000 1993-08-03T15:41:44.217 10000 1993-08-03T15:41:44.217 201 456678 false 1993-08-03T15:41:43 010-18741040 {"wTS":"1596440503","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440503-010-18741040 -139870629928460288 10000 1993-08-03T15:41:44.306 10000 1993-08-03T15:41:44.306 201 456678 false 1993-08-03T15:41:43 010-18741024 {"wTS":"1596440503","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440503-010-18741024 -139870630901538816 10000 1993-08-03T15:41:44.538 10000 1993-08-03T15:41:44.538 201 456678 false 1993-08-03T15:41:43 010-18741033 {"wTS":"1596440503","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440503-010-18741033 -139870632268881920 10000 1993-08-03T15:41:44.864 10000 1993-08-03T15:41:44.864 201 456678 false 1993-08-03T15:41:43 010-18741031 {"wTS":"1596440503","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440503-010-18741031 -139870632424071168 10000 1993-08-03T15:41:44.901 10000 1993-08-03T15:41:44.901 201 456678 false 1993-08-03T15:41:43 010-18741036 {"wTS":"1596440503","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440503-010-18741036 -139870632436654080 10000 1993-08-03T15:41:44.904 10000 1993-08-03T15:41:44.904 201 456678 false 1993-08-03T15:41:43 010-18741027 {"wTS":"1596440503","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440503-010-18741027 -139870632596037632 10000 1993-08-03T15:41:44.942 10000 1993-08-03T15:41:44.942 201 456678 false 1993-08-03T15:41:43 010-18741028 {"wTS":"1596440503","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440503-010-18741028 -139870632826724352 10000 1993-08-03T15:41:44.997 10000 1993-08-03T15:41:44.997 201 456678 false 1993-08-03T15:41:44 010-18741029 {"wTS":"1596440504","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440504-010-18741029 -139870634311507968 10000 1993-08-03T15:41:45.351 10000 1993-08-03T15:41:45.352 201 456678 false 1993-08-03T15:41:44 010-18741026 {"wTS":"1596440504","rfId":"010-18741026","remark":"数据","weight":147.465} 710078936338096128 1596440504-010-18741026 -139870636047949824 10000 1993-08-03T15:41:45.765 10000 1993-08-03T15:41:45.765 201 456678 false 1993-08-03T15:41:44 010-18741037 {"wTS":"1596440504","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440504-010-18741037 -139870636157001728 10000 1993-08-03T15:41:45.791 10000 1993-08-03T15:41:45.791 201 456678 false 1993-08-03T15:41:43 010-18741030 {"wTS":"1596440503","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440503-010-18741030 -139870636282830848 10000 1993-08-03T15:41:45.821 10000 1993-08-03T15:41:45.821 201 456678 false 1993-08-03T15:41:44 010-18741025 {"wTS":"1596440504","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440504-010-18741025 -139870636903587840 10000 1993-08-03T15:41:45.969 10000 1993-08-03T15:41:45.969 201 456678 false 1993-08-03T15:41:44 010-18741023 {"wTS":"1596440504","rfId":"010-18741023","remark":"数据","weight":118.114} 710078936338096128 1596440504-010-18741023 -139870637834723328 10000 1993-08-03T15:41:46.191 10000 1993-08-03T15:41:46.191 201 456678 false 1993-08-03T15:41:44 010-18741021 {"wTS":"1596440504","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440504-010-18741021 -139870638811996160 10000 1993-08-03T15:41:46.424 10000 1993-08-03T15:41:46.424 201 456678 false 1993-08-03T15:41:44 010-18741032 {"wTS":"1596440504","rfId":"010-18741032","remark":"数据","weight":118.863} 710078936338096128 1596440504-010-18741032 -139870753387798528 10000 1993-08-03T15:42:13.741 10000 1993-08-03T15:42:13.741 201 456678 false 1993-08-03T15:42:13 010-18741038 {"wTS":"1596440533","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440533-010-18741038 -139870753656233984 10000 1993-08-03T15:42:13.805 10000 1993-08-03T15:42:13.805 201 456678 false 1993-08-03T15:42:13 010-18741039 {"wTS":"1596440533","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440533-010-18741039 -139870753794646016 10000 1993-08-03T15:42:13.838 10000 1993-08-03T15:42:13.838 201 456678 false 1993-08-03T15:42:13 010-18741035 {"wTS":"1596440533","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440533-010-18741035 -139870755124240384 10000 1993-08-03T15:42:14.155 10000 1993-08-03T15:42:14.155 201 456678 false 1993-08-03T15:42:13 010-18741022 {"wTS":"1596440533","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440533-010-18741022 -139870755224903680 10000 1993-08-03T15:42:14.179 10000 1993-08-03T15:42:14.179 201 456678 false 1993-08-03T15:42:13 010-18741034 {"wTS":"1596440533","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440533-010-18741034 -139870756336394240 10000 1993-08-03T15:42:14.444 10000 1993-08-03T15:42:14.444 201 456678 false 1993-08-03T15:42:13 010-18741024 {"wTS":"1596440533","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440533-010-18741024 -139870758077030400 10000 1993-08-03T15:42:14.859 10000 1993-08-03T15:42:14.859 201 456678 false 1993-08-03T15:42:13 010-18741040 {"wTS":"1596440533","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440533-010-18741040 -139870758123167744 10000 1993-08-03T15:42:14.870 10000 1993-08-03T15:42:14.870 201 456678 false 1993-08-03T15:42:13 010-18741028 {"wTS":"1596440533","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440533-010-18741028 -139870758207053824 10000 1993-08-03T15:42:14.890 10000 1993-08-03T15:42:14.890 201 456678 false 1993-08-03T15:42:13 010-18741033 {"wTS":"1596440533","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440533-010-18741033 -139870759146577920 10000 1993-08-03T15:42:15.114 10000 1993-08-03T15:42:15.114 201 456678 false 1993-08-03T15:42:13 010-18741036 {"wTS":"1596440533","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440533-010-18741036 -139870760664915968 10000 1993-08-03T15:42:15.476 10000 1993-08-03T15:42:15.476 201 456678 false 1993-08-03T15:42:13 010-18741031 {"wTS":"1596440533","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440533-010-18741031 -139870760815910912 10000 1993-08-03T15:42:15.512 10000 1993-08-03T15:42:15.512 201 456678 false 1993-08-03T15:42:13 010-18741027 {"wTS":"1596440533","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440533-010-18741027 -139870762791428096 10000 1993-08-03T15:42:15.983 10000 1993-08-03T15:42:15.983 201 456678 false 1993-08-03T15:42:14 010-18741023 {"wTS":"1596440534","rfId":"010-18741023","remark":"数据","weight":118.114} 710078936338096128 1596440534-010-18741023 -139870764880191488 10000 1993-08-03T15:42:16.481 10000 1993-08-03T15:42:16.481 201 456678 false 1993-08-03T15:42:14 010-18741021 {"wTS":"1596440534","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440534-010-18741021 -139870764964077568 10000 1993-08-03T15:42:16.501 10000 1993-08-03T15:42:16.501 201 456678 false 1993-08-03T15:42:14 010-18741032 {"wTS":"1596440534","rfId":"010-18741032","remark":"数据","weight":118.863} 710078936338096128 1596440534-010-18741032 -139870765077323776 10000 1993-08-03T15:42:16.528 10000 1993-08-03T15:42:16.528 201 456678 false 1993-08-03T15:42:14 010-18741037 {"wTS":"1596440534","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440534-010-18741037 -139870879627960320 10000 1993-08-03T15:42:43.839 10000 1993-08-03T15:42:43.839 201 456678 false 1993-08-03T15:42:43 010-18741038 {"wTS":"1596440563","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440563-010-18741038 -139870879699263488 10000 1993-08-03T15:42:43.856 10000 1993-08-03T15:42:43.856 201 456678 false 1993-08-03T15:42:43 010-18741039 {"wTS":"1596440563","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440563-010-18741039 -139870879997059072 10000 1993-08-03T15:42:43.927 10000 1993-08-03T15:42:43.927 201 456678 false 1993-08-03T15:42:43 010-18741035 {"wTS":"1596440563","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440563-010-18741035 -139870882274566144 10000 1993-08-03T15:42:44.470 10000 1993-08-03T15:42:44.470 201 456678 false 1993-08-03T15:42:43 010-18741040 {"wTS":"1596440563","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440563-010-18741040 -139870882735939584 10000 1993-08-03T15:42:44.580 10000 1993-08-03T15:42:44.580 201 456678 false 1993-08-03T15:42:43 010-18741024 {"wTS":"1596440563","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440563-010-18741024 -139870883767738368 10000 1993-08-03T15:42:44.826 10000 1993-08-03T15:42:44.826 201 456678 false 1993-08-03T15:42:43 010-18741036 {"wTS":"1596440563","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440563-010-18741036 -139870883843235840 10000 1993-08-03T15:42:44.844 10000 1993-08-03T15:42:44.844 201 456678 false 1993-08-03T15:42:43 010-18741033 {"wTS":"1596440563","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440563-010-18741033 -139870883943899136 10000 1993-08-03T15:42:44.868 10000 1993-08-03T15:42:44.868 201 456678 false 1993-08-03T15:42:43 010-18741031 {"wTS":"1596440563","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440563-010-18741031 -139870884774371328 10000 1993-08-03T15:42:45.066 10000 1993-08-03T15:42:45.066 201 456678 false 1993-08-03T15:42:44 010-18741030 {"wTS":"1596440564","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440564-010-18741030 -139870885562900480 10000 1993-08-03T15:42:45.254 10000 1993-08-03T15:42:45.254 201 456678 false 1993-08-03T15:42:43 010-18741028 {"wTS":"1596440563","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440563-010-18741028 -139870887806853120 10000 1993-08-03T15:42:45.789 10000 1993-08-03T15:42:45.789 201 456678 false 1993-08-03T15:42:44 010-18741025 {"wTS":"1596440564","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440564-010-18741025 -139870889190973440 10000 1993-08-03T15:42:46.119 10000 1993-08-03T15:42:46.119 201 456678 false 1993-08-03T15:42:44 010-18741026 {"wTS":"1596440564","rfId":"010-18741026","remark":"数据","weight":147.465} 710078936338096128 1596440564-010-18741026 -139870889476186112 10000 1993-08-03T15:42:46.187 10000 1993-08-03T15:42:46.187 201 456678 false 1993-08-03T15:42:44 010-18741029 {"wTS":"1596440564","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440564-010-18741029 -139871005406748672 10000 1993-08-03T15:43:13.827 10000 1993-08-03T15:43:13.827 201 456678 false 1993-08-03T15:43:13 010-18741038 {"wTS":"1596440593","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440593-010-18741038 -139871005830373376 10000 1993-08-03T15:43:13.928 10000 1993-08-03T15:43:13.928 201 456678 false 1993-08-03T15:43:13 010-18741039 {"wTS":"1596440593","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440593-010-18741039 -139871006937669632 10000 1993-08-03T15:43:14.192 10000 1993-08-03T15:43:14.192 201 456678 false 1993-08-03T15:43:13 010-18741022 {"wTS":"1596440593","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440593-010-18741022 -139871007000584192 10000 1993-08-03T15:43:14.207 10000 1993-08-03T15:43:14.207 201 456678 false 1993-08-03T15:43:13 010-18741035 {"wTS":"1596440593","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440593-010-18741035 -139871007210299392 10000 1993-08-03T15:43:14.257 10000 1993-08-03T15:43:14.257 201 456678 false 1993-08-03T15:43:13 010-18741034 {"wTS":"1596440593","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440593-010-18741034 -139871008309207040 10000 1993-08-03T15:43:14.519 10000 1993-08-03T15:43:14.519 201 456678 false 1993-08-03T15:43:13 010-18741033 {"wTS":"1596440593","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440593-010-18741033 -139871009856905216 10000 1993-08-03T15:43:14.888 10000 1993-08-03T15:43:14.888 201 456678 false 1993-08-03T15:43:14 010-18741027 {"wTS":"1596440594","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440594-010-18741027 -139871009932402688 10000 1993-08-03T15:43:14.906 10000 1993-08-03T15:43:14.906 201 456678 false 1993-08-03T15:43:14 010-18741023 {"wTS":"1596440594","rfId":"010-18741023","remark":"数据","weight":118.114} 710078936338096128 1596440594-010-18741023 -139871011622707200 10000 1993-08-03T15:43:15.309 10000 1993-08-03T15:43:15.309 201 456678 false 1993-08-03T15:43:14 010-18741021 {"wTS":"1596440594","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440594-010-18741021 -139871011752730624 10000 1993-08-03T15:43:15.340 10000 1993-08-03T15:43:15.340 201 456678 false 1993-08-03T15:43:14 010-18741032 {"wTS":"1596440594","rfId":"010-18741032","remark":"数据","weight":118.863} 710078936338096128 1596440594-010-18741032 -139871011983417344 10000 1993-08-03T15:43:15.395 10000 1993-08-03T15:43:15.395 201 456678 false 1993-08-03T15:43:14 010-18741037 {"wTS":"1596440594","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440594-010-18741037 -139871131340726272 10000 1993-08-03T15:43:43.852 10000 1993-08-03T15:43:43.852 201 456678 false 1993-08-03T15:43:43 010-18741038 {"wTS":"1596440623","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440623-010-18741038 -139871131785322496 10000 1993-08-03T15:43:43.958 10000 1993-08-03T15:43:43.958 201 456678 false 1993-08-03T15:43:43 010-18741039 {"wTS":"1596440623","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440623-010-18741039 -139871132246695936 10000 1993-08-03T15:43:44.068 10000 1993-08-03T15:43:44.068 201 456678 false 1993-08-03T15:43:43 010-18741035 {"wTS":"1596440623","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440623-010-18741035 -139871132657737728 10000 1993-08-03T15:43:44.167 10000 1993-08-03T15:43:44.167 201 456678 false 1993-08-03T15:43:43 010-18741034 {"wTS":"1596440623","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440623-010-18741034 -139871133219774464 10000 1993-08-03T15:43:44.300 10000 1993-08-03T15:43:44.300 201 456678 false 1993-08-03T15:43:43 010-18741040 {"wTS":"1596440623","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440623-010-18741040 -139871133760839680 10000 1993-08-03T15:43:44.429 10000 1993-08-03T15:43:44.429 201 456678 false 1993-08-03T15:43:43 010-18741022 {"wTS":"1596440623","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440623-010-18741022 -139871133895057408 10000 1993-08-03T15:43:44.461 10000 1993-08-03T15:43:44.461 201 456678 false 1993-08-03T15:43:43 010-18741024 {"wTS":"1596440623","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440623-010-18741024 -139871134603894784 10000 1993-08-03T15:43:44.630 10000 1993-08-03T15:43:44.630 201 456678 false 1993-08-03T15:43:44 010-18741030 {"wTS":"1596440624","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440624-010-18741030 -139871135446949888 10000 1993-08-03T15:43:44.831 10000 1993-08-03T15:43:44.831 201 456678 false 1993-08-03T15:43:44 010-18741028 {"wTS":"1596440624","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440624-010-18741028 -139871135690219520 10000 1993-08-03T15:43:44.889 10000 1993-08-03T15:43:44.889 201 456678 false 1993-08-03T15:43:44 010-18741036 {"wTS":"1596440624","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440624-010-18741036 -139871136432611328 10000 1993-08-03T15:43:45.067 10000 1993-08-03T15:43:45.067 201 456678 false 1993-08-03T15:43:44 010-18741023 {"wTS":"1596440624","rfId":"010-18741023","remark":"数据","weight":118.114} 710078936338096128 1596440624-010-18741023 -139871138248744960 10000 1993-08-03T15:43:45.499 10000 1993-08-03T15:43:45.499 201 456678 false 1993-08-03T15:43:44 010-18741031 {"wTS":"1596440624","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440624-010-18741031 -139871138403934208 10000 1993-08-03T15:43:45.536 10000 1993-08-03T15:43:45.536 201 456678 false 1993-08-03T15:43:44 010-18741025 {"wTS":"1596440624","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440624-010-18741025 -139871138877890560 10000 1993-08-03T15:43:45.649 10000 1993-08-03T15:43:45.649 201 456678 false 1993-08-03T15:43:44 010-18741026 {"wTS":"1596440624","rfId":"010-18741026","remark":"数据","weight":147.465} 710078936338096128 1596440624-010-18741026 -139871138970165248 10000 1993-08-03T15:43:45.671 10000 1993-08-03T15:43:45.671 201 456678 false 1993-08-03T15:43:44 010-18741029 {"wTS":"1596440624","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440624-010-18741029 -139871139590922240 10000 1993-08-03T15:43:45.819 10000 1993-08-03T15:43:45.819 201 456678 false 1993-08-03T15:43:44 010-18741037 {"wTS":"1596440624","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440624-010-18741037 -139871257476030464 10000 1993-08-03T15:44:13.925 10000 1993-08-03T15:44:13.925 201 456678 false 1993-08-03T15:44:13 010-18741038 {"wTS":"1596440653","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440653-010-18741038 -139871257740271616 10000 1993-08-03T15:44:13.988 10000 1993-08-03T15:44:13.988 201 456678 false 1993-08-03T15:44:13 010-18741039 {"wTS":"1596440653","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440653-010-18741039 -139871258302308352 10000 1993-08-03T15:44:14.122 10000 1993-08-03T15:44:14.122 201 456678 false 1993-08-03T15:44:13 010-18741035 {"wTS":"1596440653","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440653-010-18741035 -139871259027922944 10000 1993-08-03T15:44:14.295 10000 1993-08-03T15:44:14.295 201 456678 false 1993-08-03T15:44:13 010-18741034 {"wTS":"1596440653","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440653-010-18741034 -139871259682234368 10000 1993-08-03T15:44:14.451 10000 1993-08-03T15:44:14.451 201 456678 false 1993-08-03T15:44:13 010-18741040 {"wTS":"1596440653","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440653-010-18741040 -139871260231688192 10000 1993-08-03T15:44:14.582 10000 1993-08-03T15:44:14.582 201 456678 false 1993-08-03T15:44:13 010-18741024 {"wTS":"1596440653","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440653-010-18741024 -139871260512706560 10000 1993-08-03T15:44:14.649 10000 1993-08-03T15:44:14.649 201 456678 false 1993-08-03T15:44:14 010-18741033 {"wTS":"1596440654","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440654-010-18741033 -139871261041188864 10000 1993-08-03T15:44:14.775 10000 1993-08-03T15:44:14.775 201 456678 false 1993-08-03T15:44:14 010-18741028 {"wTS":"1596440654","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440654-010-18741028 -139871261758414848 10000 1993-08-03T15:44:14.946 10000 1993-08-03T15:44:14.946 201 456678 false 1993-08-03T15:44:14 010-18741030 {"wTS":"1596440654","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440654-010-18741030 -139871262639218688 10000 1993-08-03T15:44:15.156 10000 1993-08-03T15:44:15.156 201 456678 false 1993-08-03T15:44:14 010-18741029 {"wTS":"1596440654","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440654-010-18741029 -139871263805235200 10000 1993-08-03T15:44:15.434 10000 1993-08-03T15:44:15.434 201 456678 false 1993-08-03T15:44:14 010-18741036 {"wTS":"1596440654","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440654-010-18741036 -139871264044310528 10000 1993-08-03T15:44:15.491 10000 1993-08-03T15:44:15.491 201 456678 false 1993-08-03T15:44:14 010-18741027 {"wTS":"1596440654","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440654-010-18741027 -139871264610541568 10000 1993-08-03T15:44:15.626 10000 1993-08-03T15:44:15.627 201 456678 false 1993-08-03T15:44:14 010-18741031 {"wTS":"1596440654","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440654-010-18741031 -139871265361321984 10000 1993-08-03T15:44:15.805 10000 1993-08-03T15:44:15.805 201 456678 false 1993-08-03T15:44:14 010-18741037 {"wTS":"1596440654","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440654-010-18741037 -139871265893998592 10000 1993-08-03T15:44:15.932 10000 1993-08-03T15:44:15.932 201 456678 false 1993-08-03T15:44:14 010-18741026 {"wTS":"1596440654","rfId":"010-18741026","remark":"数据","weight":147.465} 710078936338096128 1596440654-010-18741026 -139871266917408768 10000 1993-08-03T15:44:16.176 10000 1993-08-03T15:44:16.176 201 456678 false 1993-08-03T15:44:14 010-18741025 {"wTS":"1596440654","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440654-010-18741025 -139871267198427136 10000 1993-08-03T15:44:16.243 10000 1993-08-03T15:44:16.243 201 456678 false 1993-08-03T15:44:14 010-18741021 {"wTS":"1596440654","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440654-010-18741021 -139871267999539200 10000 1993-08-03T15:44:16.434 10000 1993-08-03T15:44:16.434 201 456678 false 1993-08-03T15:44:14 010-18741032 {"wTS":"1596440654","rfId":"010-18741032","remark":"数据","weight":118.863} 710078936338096128 1596440654-010-18741032 -139871383804272640 10000 1993-08-03T15:44:44.044 10000 1993-08-03T15:44:44.044 201 456678 false 1993-08-03T15:44:43 010-18741039 {"wTS":"1596440683","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440683-010-18741039 -139871383946878976 10000 1993-08-03T15:44:44.078 10000 1993-08-03T15:44:44.078 201 456678 false 1993-08-03T15:44:43 010-18741038 {"wTS":"1596440683","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440683-010-18741038 -139871384173371392 10000 1993-08-03T15:44:44.132 10000 1993-08-03T15:44:44.132 201 456678 false 1993-08-03T15:44:43 010-18741035 {"wTS":"1596440683","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440683-010-18741035 -139871384362115072 10000 1993-08-03T15:44:44.177 10000 1993-08-03T15:44:44.177 201 456678 false 1993-08-03T15:44:43 010-18741022 {"wTS":"1596440683","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440683-010-18741022 -139871385318416384 10000 1993-08-03T15:44:44.405 10000 1993-08-03T15:44:44.405 201 456678 false 1993-08-03T15:44:43 010-18741034 {"wTS":"1596440683","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440683-010-18741034 -139871385553297408 10000 1993-08-03T15:44:44.461 10000 1993-08-03T15:44:44.461 201 456678 false 1993-08-03T15:44:43 010-18741040 {"wTS":"1596440683","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440683-010-18741040 -139871385851092992 10000 1993-08-03T15:44:44.532 10000 1993-08-03T15:44:44.532 201 456678 false 1993-08-03T15:44:44 010-18741024 {"wTS":"1596440684","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440684-010-18741024 -139871386111139840 10000 1993-08-03T15:44:44.594 10000 1993-08-03T15:44:44.594 201 456678 false 1993-08-03T15:44:44 010-18741033 {"wTS":"1596440684","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440684-010-18741033 -139871387046469632 10000 1993-08-03T15:44:44.817 10000 1993-08-03T15:44:44.817 201 456678 false 1993-08-03T15:44:44 010-18741030 {"wTS":"1596440684","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440684-010-18741030 -139871388103434240 10000 1993-08-03T15:44:45.069 10000 1993-08-03T15:44:45.069 201 456678 false 1993-08-03T15:44:44 010-18741028 {"wTS":"1596440684","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440684-010-18741028 -139871388552224768 10000 1993-08-03T15:44:45.176 10000 1993-08-03T15:44:45.176 201 456678 false 1993-08-03T15:44:44 010-18741036 {"wTS":"1596440684","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440684-010-18741036 -139871388929712128 10000 1993-08-03T15:44:45.266 10000 1993-08-03T15:44:45.266 201 456678 false 1993-08-03T15:44:44 010-18741027 {"wTS":"1596440684","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440684-010-18741027 -139871390653571072 10000 1993-08-03T15:44:45.677 10000 1993-08-03T15:44:45.677 201 456678 false 1993-08-03T15:44:44 010-18741021 {"wTS":"1596440684","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440684-010-18741021 -139871390666153984 10000 1993-08-03T15:44:45.680 10000 1993-08-03T15:44:45.680 201 456678 false 1993-08-03T15:44:44 010-18741023 {"wTS":"1596440684","rfId":"010-18741023","remark":"数据","weight":118.114} 710078936338096128 1596440684-010-18741023 -139871391165276160 10000 1993-08-03T15:44:45.799 10000 1993-08-03T15:44:45.799 201 456678 false 1993-08-03T15:44:44 010-18741029 {"wTS":"1596440684","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440684-010-18741029 -139871392822026240 10000 1993-08-03T15:44:46.194 10000 1993-08-03T15:44:46.194 201 456678 false 1993-08-03T15:44:44 010-18741037 {"wTS":"1596440684","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440684-010-18741037 -139871509830524928 10000 1993-08-03T15:45:14.091 10000 1993-08-03T15:45:14.091 201 456678 false 1993-08-03T15:45:13 010-18741039 {"wTS":"1596440713","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440713-010-18741039 -139871510983958528 10000 1993-08-03T15:45:14.366 10000 1993-08-03T15:45:14.366 201 456678 false 1993-08-03T15:45:13 010-18741040 {"wTS":"1596440713","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440713-010-18741040 -139871511017512960 10000 1993-08-03T15:45:14.374 10000 1993-08-03T15:45:14.374 201 456678 false 1993-08-03T15:45:13 010-18741035 {"wTS":"1596440713","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440713-010-18741035 -139871511042678784 10000 1993-08-03T15:45:14.380 10000 1993-08-03T15:45:14.380 201 456678 false 1993-08-03T15:45:13 010-18741022 {"wTS":"1596440713","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440713-010-18741022 -139871511072038912 10000 1993-08-03T15:45:14.387 10000 1993-08-03T15:45:14.387 201 456678 false 1993-08-03T15:45:14 010-18741024 {"wTS":"1596440714","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440714-010-18741024 -139871511088816128 10000 1993-08-03T15:45:14.391 10000 1993-08-03T15:45:14.391 201 456678 false 1993-08-03T15:45:13 010-18741034 {"wTS":"1596440713","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440713-010-18741034 -139871513341157376 10000 1993-08-03T15:45:14.928 10000 1993-08-03T15:45:14.928 201 456678 false 1993-08-03T15:45:14 010-18741033 {"wTS":"1596440714","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440714-010-18741033 -139871513706061824 10000 1993-08-03T15:45:15.015 10000 1993-08-03T15:45:15.015 201 456678 false 1993-08-03T15:45:14 010-18741031 {"wTS":"1596440714","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440714-010-18741031 -139871515257954304 10000 1993-08-03T15:45:15.385 10000 1993-08-03T15:45:15.385 201 456678 false 1993-08-03T15:45:14 010-18741028 {"wTS":"1596440714","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440714-010-18741028 -139871516579160064 10000 1993-08-03T15:45:15.700 10000 1993-08-03T15:45:15.700 201 456678 false 1993-08-03T15:45:14 010-18741030 {"wTS":"1596440714","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440714-010-18741030 -139871517271220224 10000 1993-08-03T15:45:15.865 10000 1993-08-03T15:45:15.865 201 456678 false 1993-08-03T15:45:14 010-18741027 {"wTS":"1596440714","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440714-010-18741027 -139871517942308864 10000 1993-08-03T15:45:16.025 10000 1993-08-03T15:45:16.025 201 456678 false 1993-08-03T15:45:14 010-18741032 {"wTS":"1596440714","rfId":"010-18741032","remark":"数据","weight":118.863} 710078936338096128 1596440714-010-18741032 -139871518294630400 10000 1993-08-03T15:45:16.109 10000 1993-08-03T15:45:16.109 201 456678 false 1993-08-03T15:45:14 010-18741025 {"wTS":"1596440714","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440714-010-18741025 -139871519984934912 10000 1993-08-03T15:45:16.512 10000 1993-08-03T15:45:16.512 201 456678 false 1993-08-03T15:45:14 010-18741026 {"wTS":"1596440714","rfId":"010-18741026","remark":"数据","weight":147.465} 710078936338096128 1596440714-010-18741026 -139871520580526080 10000 1993-08-03T15:45:16.654 10000 1993-08-03T15:45:16.654 201 456678 false 1993-08-03T15:45:14 010-18741023 {"wTS":"1596440714","rfId":"010-18741023","remark":"数据","weight":118.114} 710078936338096128 1596440714-010-18741023 -139871520614080512 10000 1993-08-03T15:45:16.662 10000 1993-08-03T15:45:16.662 201 456678 false 1993-08-03T15:45:14 010-18741021 {"wTS":"1596440714","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440714-010-18741021 -139871635961634816 10000 1993-08-03T15:45:44.163 10000 1993-08-03T15:45:44.163 201 456678 false 1993-08-03T15:45:43 010-18741039 {"wTS":"1596440743","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440743-010-18741039 -139871636230070272 10000 1993-08-03T15:45:44.227 10000 1993-08-03T15:45:44.227 201 456678 false 1993-08-03T15:45:43 010-18741038 {"wTS":"1596440743","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440743-010-18741038 -139871637765185536 10000 1993-08-03T15:45:44.593 10000 1993-08-03T15:45:44.593 201 456678 false 1993-08-03T15:45:43 010-18741022 {"wTS":"1596440743","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440743-010-18741022 -139871637777768448 10000 1993-08-03T15:45:44.596 10000 1993-08-03T15:45:44.597 201 456678 false 1993-08-03T15:45:43 010-18741034 {"wTS":"1596440743","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440743-010-18741034 -139871637794545664 10000 1993-08-03T15:45:44.600 10000 1993-08-03T15:45:44.600 201 456678 false 1993-08-03T15:45:44 010-18741024 {"wTS":"1596440744","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440744-010-18741024 -139871637811322880 10000 1993-08-03T15:45:44.604 10000 1993-08-03T15:45:44.604 201 456678 false 1993-08-03T15:45:44 010-18741040 {"wTS":"1596440744","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440744-010-18741040 -139871637844877312 10000 1993-08-03T15:45:44.612 10000 1993-08-03T15:45:44.612 201 456678 false 1993-08-03T15:45:43 010-18741035 {"wTS":"1596440743","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440743-010-18741035 -139871639900086272 10000 1993-08-03T15:45:45.102 10000 1993-08-03T15:45:45.102 201 456678 false 1993-08-03T15:45:44 010-18741031 {"wTS":"1596440744","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440744-010-18741031 -139871641212903424 10000 1993-08-03T15:45:45.415 10000 1993-08-03T15:45:45.415 201 456678 false 1993-08-03T15:45:44 010-18741036 {"wTS":"1596440744","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440744-010-18741036 -139871641628139520 10000 1993-08-03T15:45:45.514 10000 1993-08-03T15:45:45.514 201 456678 false 1993-08-03T15:45:44 010-18741025 {"wTS":"1596440744","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440744-010-18741025 -139871641854631936 10000 1993-08-03T15:45:45.568 10000 1993-08-03T15:45:45.568 201 456678 false 1993-08-03T15:45:44 010-18741033 {"wTS":"1596440744","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440744-010-18741033 -139871643645599744 10000 1993-08-03T15:45:45.995 10000 1993-08-03T15:45:45.995 201 456678 false 1993-08-03T15:45:44 010-18741029 {"wTS":"1596440744","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440744-010-18741029 -139871643964366848 10000 1993-08-03T15:45:46.071 10000 1993-08-03T15:45:46.071 201 456678 false 1993-08-03T15:45:44 010-18741027 {"wTS":"1596440744","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440744-010-18741027 -139871644929056768 10000 1993-08-03T15:45:46.301 10000 1993-08-03T15:45:46.301 201 456678 false 1993-08-03T15:45:44 010-18741037 {"wTS":"1596440744","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440744-010-18741037 -139871645679837184 10000 1993-08-03T15:45:46.480 10000 1993-08-03T15:45:46.480 201 456678 false 1993-08-03T15:45:44 010-18741021 {"wTS":"1596440744","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440744-010-18741021 -139871645889552384 10000 1993-08-03T15:45:46.530 10000 1993-08-03T15:45:46.530 201 456678 false 1993-08-03T15:45:44 010-18741032 {"wTS":"1596440744","rfId":"010-18741032","remark":"数据","weight":118.863} 710078936338096128 1596440744-010-18741032 -139871762545729536 10000 1993-08-03T15:46:14.343 10000 1993-08-03T15:46:14.343 201 456678 false 1993-08-03T15:46:13 010-18741039 {"wTS":"1596440773","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440773-010-18741039 -139871762562506752 10000 1993-08-03T15:46:14.347 10000 1993-08-03T15:46:14.347 201 456678 false 1993-08-03T15:46:13 010-18741038 {"wTS":"1596440773","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440773-010-18741038 -139871762843525120 10000 1993-08-03T15:46:14.414 10000 1993-08-03T15:46:14.414 201 456678 false 1993-08-03T15:46:14 010-18741022 {"wTS":"1596440774","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440774-010-18741022 -139871763732717568 10000 1993-08-03T15:46:14.626 10000 1993-08-03T15:46:14.626 201 456678 false 1993-08-03T15:46:14 010-18741034 {"wTS":"1596440774","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440774-010-18741034 -139871764470915072 10000 1993-08-03T15:46:14.802 10000 1993-08-03T15:46:14.802 201 456678 false 1993-08-03T15:46:14 010-18741024 {"wTS":"1596440774","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440774-010-18741024 -139871765951504384 10000 1993-08-03T15:46:15.155 10000 1993-08-03T15:46:15.155 201 456678 false 1993-08-03T15:46:14 010-18741036 {"wTS":"1596440774","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440774-010-18741036 -139871766014418944 10000 1993-08-03T15:46:15.170 10000 1993-08-03T15:46:15.170 201 456678 false 1993-08-03T15:46:14 010-18741028 {"wTS":"1596440774","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440774-010-18741028 -139871766459015168 10000 1993-08-03T15:46:15.276 10000 1993-08-03T15:46:15.276 201 456678 false 1993-08-03T15:46:14 010-18741032 {"wTS":"1596440774","rfId":"010-18741032","remark":"数据","weight":118.863} 710078936338096128 1596440774-010-18741032 -139871767213989888 10000 1993-08-03T15:46:15.456 10000 1993-08-03T15:46:15.457 201 456678 false 1993-08-03T15:46:14 010-18741031 {"wTS":"1596440774","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440774-010-18741031 -139871768082210816 10000 1993-08-03T15:46:15.663 10000 1993-08-03T15:46:15.663 201 456678 false 1993-08-03T15:46:14 010-18741037 {"wTS":"1596440774","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440774-010-18741037 -139871768556167168 10000 1993-08-03T15:46:15.776 10000 1993-08-03T15:46:15.776 201 456678 false 1993-08-03T15:46:14 010-18741033 {"wTS":"1596440774","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440774-010-18741033 -139871769566994432 10000 1993-08-03T15:46:16.017 10000 1993-08-03T15:46:16.017 201 456678 false 1993-08-03T15:46:14 010-18741030 {"wTS":"1596440774","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440774-010-18741030 -139871771605426176 10000 1993-08-03T15:46:16.503 10000 1993-08-03T15:46:16.503 201 456678 false 1993-08-03T15:46:14 010-18741026 {"wTS":"1596440774","rfId":"010-18741026","remark":"数据","weight":147.465} 710078936338096128 1596440774-010-18741026 -139871771618009088 10000 1993-08-03T15:46:16.506 10000 1993-08-03T15:46:16.506 201 456678 false 1993-08-03T15:46:14 010-18741025 {"wTS":"1596440774","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440774-010-18741025 -139871772301680640 10000 1993-08-03T15:46:16.669 10000 1993-08-03T15:46:16.669 201 456678 false 1993-08-03T15:46:14 010-18741029 {"wTS":"1596440774","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440774-010-18741029 -139871772809191424 10000 1993-08-03T15:46:16.790 10000 1993-08-03T15:46:16.790 201 456678 false 1993-08-03T15:46:14 010-18741023 {"wTS":"1596440774","rfId":"010-18741023","remark":"数据","weight":118.114} 710078936338096128 1596440774-010-18741023 -139871888001556480 10000 1993-08-03T15:46:44.254 10000 1993-08-03T15:46:44.254 201 456678 false 1993-08-03T15:46:43 010-18741039 {"wTS":"1596440803","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440803-010-18741039 -139871888240631808 10000 1993-08-03T15:46:44.311 10000 1993-08-03T15:46:44.311 201 456678 false 1993-08-03T15:46:43 010-18741038 {"wTS":"1596440803","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440803-010-18741038 -139871888370655232 10000 1993-08-03T15:46:44.342 10000 1993-08-03T15:46:44.342 201 456678 false 1993-08-03T15:46:43 010-18741035 {"wTS":"1596440803","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440803-010-18741035 -139871889373093888 10000 1993-08-03T15:46:44.581 10000 1993-08-03T15:46:44.581 201 456678 false 1993-08-03T15:46:44 010-18741040 {"wTS":"1596440804","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440804-010-18741040 -139871889549254656 10000 1993-08-03T15:46:44.623 10000 1993-08-03T15:46:44.623 201 456678 false 1993-08-03T15:46:44 010-18741022 {"wTS":"1596440804","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440804-010-18741022 -139871889691860992 10000 1993-08-03T15:46:44.657 10000 1993-08-03T15:46:44.657 201 456678 false 1993-08-03T15:46:44 010-18741034 {"wTS":"1596440804","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440804-010-18741034 -139871890451030016 10000 1993-08-03T15:46:44.838 10000 1993-08-03T15:46:44.838 201 456678 false 1993-08-03T15:46:44 010-18741024 {"wTS":"1596440804","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440804-010-18741024 -139871892028088320 10000 1993-08-03T15:46:45.214 10000 1993-08-03T15:46:45.214 201 456678 false 1993-08-03T15:46:44 010-18741028 {"wTS":"1596440804","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440804-010-18741028 -139871893294768128 10000 1993-08-03T15:46:45.516 10000 1993-08-03T15:46:45.516 201 456678 false 1993-08-03T15:46:44 010-18741027 {"wTS":"1596440804","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440804-010-18741027 -139871893517066240 10000 1993-08-03T15:46:45.569 10000 1993-08-03T15:46:45.569 201 456678 false 1993-08-03T15:46:44 010-18741021 {"wTS":"1596440804","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440804-010-18741021 -139871894364315648 10000 1993-08-03T15:46:45.771 10000 1993-08-03T15:46:45.771 201 456678 false 1993-08-03T15:46:44 010-18741032 {"wTS":"1596440804","rfId":"010-18741032","remark":"数据","weight":118.863} 710078936338096128 1596440804-010-18741032 -139872014090723328 10000 1993-08-03T15:47:14.316 10000 1993-08-03T15:47:14.316 201 456678 false 1993-08-03T15:47:13 010-18741039 {"wTS":"1596440833","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440833-010-18741039 -139872014187192320 10000 1993-08-03T15:47:14.339 10000 1993-08-03T15:47:14.339 201 456678 false 1993-08-03T15:47:13 010-18741038 {"wTS":"1596440833","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440833-010-18741038 -139872014229135360 10000 1993-08-03T15:47:14.349 10000 1993-08-03T15:47:14.349 201 456678 false 1993-08-03T15:47:14 010-18741035 {"wTS":"1596440834","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440834-010-18741035 -139872015739084800 10000 1993-08-03T15:47:14.709 10000 1993-08-03T15:47:14.709 201 456678 false 1993-08-03T15:47:14 010-18741040 {"wTS":"1596440834","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440834-010-18741040 -139872015860719616 10000 1993-08-03T15:47:14.738 10000 1993-08-03T15:47:14.738 201 456678 false 1993-08-03T15:47:14 010-18741034 {"wTS":"1596440834","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440834-010-18741034 -139872016007520256 10000 1993-08-03T15:47:14.773 10000 1993-08-03T15:47:14.773 201 456678 false 1993-08-03T15:47:14 010-18741024 {"wTS":"1596440834","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440834-010-18741024 -139872016183681024 10000 1993-08-03T15:47:14.815 10000 1993-08-03T15:47:14.816 201 456678 false 1993-08-03T15:47:14 010-18741022 {"wTS":"1596440834","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440834-010-18741022 -139872018347941888 10000 1993-08-03T15:47:15.331 10000 1993-08-03T15:47:15.331 201 456678 false 1993-08-03T15:47:14 010-18741031 {"wTS":"1596440834","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440834-010-18741031 -139872018700263424 10000 1993-08-03T15:47:15.415 10000 1993-08-03T15:47:15.415 201 456678 false 1993-08-03T15:47:14 010-18741036 {"wTS":"1596440834","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440834-010-18741036 -139872019648176128 10000 1993-08-03T15:47:15.641 10000 1993-08-03T15:47:15.641 201 456678 false 1993-08-03T15:47:14 010-18741033 {"wTS":"1596440834","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440834-010-18741033 -139872020088578048 10000 1993-08-03T15:47:15.746 10000 1993-08-03T15:47:15.746 201 456678 false 1993-08-03T15:47:14 010-18741030 {"wTS":"1596440834","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440834-010-18741030 -139872020520591360 10000 1993-08-03T15:47:15.849 10000 1993-08-03T15:47:15.849 201 456678 false 1993-08-03T15:47:14 010-18741028 {"wTS":"1596440834","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440834-010-18741028 -139872021506252800 10000 1993-08-03T15:47:16.084 10000 1993-08-03T15:47:16.084 201 456678 false 1993-08-03T15:47:14 010-18741029 {"wTS":"1596440834","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440834-010-18741029 -139872022290587648 10000 1993-08-03T15:47:16.271 10000 1993-08-03T15:47:16.271 201 456678 false 1993-08-03T15:47:14 010-18741021 {"wTS":"1596440834","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440834-010-18741021 -139872022869401600 10000 1993-08-03T15:47:16.409 10000 1993-08-03T15:47:16.409 201 456678 false 1993-08-03T15:47:14 010-18741025 {"wTS":"1596440834","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440834-010-18741025 -139872023464992768 10000 1993-08-03T15:47:16.551 10000 1993-08-03T15:47:16.551 201 456678 false 1993-08-03T15:47:14 010-18741023 {"wTS":"1596440834","rfId":"010-18741023","remark":"数据","weight":118.114} 710078936338096128 1596440834-010-18741023 -139872023523713024 10000 1993-08-03T15:47:16.566 10000 1993-08-03T15:47:16.566 201 456678 false 1993-08-03T15:47:14 010-18741026 {"wTS":"1596440834","rfId":"010-18741026","remark":"数据","weight":147.465} 710078936338096128 1596440834-010-18741026 -139872024970747904 10000 1993-08-03T15:47:16.910 10000 1993-08-03T15:47:16.910 201 456678 false 1993-08-03T15:47:14 010-18741037 {"wTS":"1596440834","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440834-010-18741037 -139872140372828160 10000 1993-08-03T15:47:44.424 10000 1993-08-03T15:47:44.425 201 456678 false 1993-08-03T15:47:43 010-18741038 {"wTS":"1596440863","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440863-010-18741038 -139872140435742720 10000 1993-08-03T15:47:44.439 10000 1993-08-03T15:47:44.439 201 456678 false 1993-08-03T15:47:44 010-18741035 {"wTS":"1596440864","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440864-010-18741035 -139872140939059200 10000 1993-08-03T15:47:44.559 10000 1993-08-03T15:47:44.559 201 456678 false 1993-08-03T15:47:43 010-18741039 {"wTS":"1596440863","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440863-010-18741039 -139872142331568128 10000 1993-08-03T15:47:44.891 10000 1993-08-03T15:47:44.891 201 456678 false 1993-08-03T15:47:44 010-18741040 {"wTS":"1596440864","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440864-010-18741040 -139872142365122560 10000 1993-08-03T15:47:44.899 10000 1993-08-03T15:47:44.899 201 456678 false 1993-08-03T15:47:44 010-18741034 {"wTS":"1596440864","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440864-010-18741034 -139872142381899776 10000 1993-08-03T15:47:44.903 10000 1993-08-03T15:47:44.903 201 456678 false 1993-08-03T15:47:44 010-18741024 {"wTS":"1596440864","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440864-010-18741024 -139872142407065600 10000 1993-08-03T15:47:44.909 10000 1993-08-03T15:47:44.909 201 456678 false 1993-08-03T15:47:44 010-18741022 {"wTS":"1596440864","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440864-010-18741022 -139872144193839104 10000 1993-08-03T15:47:45.335 10000 1993-08-03T15:47:45.335 201 456678 false 1993-08-03T15:47:44 010-18741031 {"wTS":"1596440864","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440864-010-18741031 -139872145619902464 10000 1993-08-03T15:47:45.675 10000 1993-08-03T15:47:45.675 201 456678 false 1993-08-03T15:47:44 010-18741036 {"wTS":"1596440864","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440864-010-18741036 -139872145930280960 10000 1993-08-03T15:47:45.749 10000 1993-08-03T15:47:45.749 201 456678 false 1993-08-03T15:47:44 010-18741029 {"wTS":"1596440864","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440864-010-18741029 -139872147243098112 10000 1993-08-03T15:47:46.062 10000 1993-08-03T15:47:46.062 201 456678 false 1993-08-03T15:47:44 010-18741030 {"wTS":"1596440864","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440864-010-18741030 -139872147276652544 10000 1993-08-03T15:47:46.070 10000 1993-08-03T15:47:46.070 201 456678 false 1993-08-03T15:47:44 010-18741028 {"wTS":"1596440864","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440864-010-18741028 -139872147285041152 10000 1993-08-03T15:47:46.072 10000 1993-08-03T15:47:46.072 201 456678 false 1993-08-03T15:47:44 010-18741027 {"wTS":"1596440864","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440864-010-18741027 -139872147398281100 10000 1993-08-03T15:47:46.099 10000 1993-08-03T15:47:46.099 201 456678 false 1993-08-03T15:47:44 010-18741023 {"wTS":"1596440864","rfId":"010-18741023","remark":"数据","weight":118.114} 710078936338096128 1596440864-010-18741023 -139872147620585472 10000 1993-08-03T15:47:46.152 10000 1993-08-03T15:47:46.152 201 456678 false 1993-08-03T15:47:44 010-18741026 {"wTS":"1596440864","rfId":"010-18741026","remark":"数据","weight":147.465} 710078936338096128 1596440864-010-18741026 -139872149973590016 10000 1993-08-03T15:47:46.713 10000 1993-08-03T15:47:46.713 201 456678 false 1993-08-03T15:47:44 010-18741025 {"wTS":"1596440864","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440864-010-18741025 -139872151349321728 10000 1993-08-03T15:47:47.041 10000 1993-08-03T15:47:47.041 201 456678 false 1993-08-03T15:47:44 010-18741032 {"wTS":"1596440864","rfId":"010-18741032","remark":"数据","weight":118.863} 710078936338096128 1596440864-010-18741032 -139872151533871104 10000 1993-08-03T15:47:47.085 10000 1993-08-03T15:47:47.085 201 456678 false 1993-08-03T15:47:44 010-18741021 {"wTS":"1596440864","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440864-010-18741021 -139872151705837568 10000 1993-08-03T15:47:47.126 10000 1993-08-03T15:47:47.126 201 456678 false 1993-08-03T15:47:45 010-18741037 {"wTS":"1596440865","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440865-010-18741037 -139872266231308288 10000 1993-08-03T15:48:14.431 10000 1993-08-03T15:48:14.431 201 456678 false 1993-08-03T15:48:14 010-18741038 {"wTS":"1596440894","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440894-010-18741038 -139872266407469056 10000 1993-08-03T15:48:14.473 10000 1993-08-03T15:48:14.473 201 456678 false 1993-08-03T15:48:14 010-18741035 {"wTS":"1596440894","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440894-010-18741035 -139872266491355136 10000 1993-08-03T15:48:14.493 10000 1993-08-03T15:48:14.493 201 456678 false 1993-08-03T15:48:14 010-18741039 {"wTS":"1596440894","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440894-010-18741039 -139872267909029888 10000 1993-08-03T15:48:14.831 10000 1993-08-03T15:48:14.831 201 456678 false 1993-08-03T15:48:14 010-18741040 {"wTS":"1596440894","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440894-010-18741040 -139872268039053312 10000 1993-08-03T15:48:14.862 10000 1993-08-03T15:48:14.862 201 456678 false 1993-08-03T15:48:14 010-18741024 {"wTS":"1596440894","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440894-010-18741024 -139872269435756544 10000 1993-08-03T15:48:15.195 10000 1993-08-03T15:48:15.195 201 456678 false 1993-08-03T15:48:14 010-18741031 {"wTS":"1596440894","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440894-010-18741031 -139872269532225536 10000 1993-08-03T15:48:15.218 10000 1993-08-03T15:48:15.218 201 456678 false 1993-08-03T15:48:14 010-18741036 {"wTS":"1596440894","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440894-010-18741036 -139872269859381248 10000 1993-08-03T15:48:15.296 10000 1993-08-03T15:48:15.296 201 456678 false 1993-08-03T15:48:14 010-18741030 {"wTS":"1596440894","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440894-010-18741030 -139872270924734464 10000 1993-08-03T15:48:15.550 10000 1993-08-03T15:48:15.550 201 456678 false 1993-08-03T15:48:14 010-18741033 {"wTS":"1596440894","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440894-010-18741033 -139872272392740864 10000 1993-08-03T15:48:15.900 10000 1993-08-03T15:48:15.900 201 456678 false 1993-08-03T15:48:14 010-18741028 {"wTS":"1596440894","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440894-010-18741028 -139872272791199744 10000 1993-08-03T15:48:15.995 10000 1993-08-03T15:48:15.995 201 456678 false 1993-08-03T15:48:14 010-18741027 {"wTS":"1596440894","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440894-010-18741027 -139872274309537792 10000 1993-08-03T15:48:16.357 10000 1993-08-03T15:48:16.357 201 456678 false 1993-08-03T15:48:14 010-18741021 {"wTS":"1596440894","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440894-010-18741021 -139872392240783360 10000 1993-08-03T15:48:44.474 10000 1993-08-03T15:48:44.474 201 456678 false 1993-08-03T15:48:44 010-18741039 {"wTS":"1596440924","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440924-010-18741039 -139872392354029568 10000 1993-08-03T15:48:44.501 10000 1993-08-03T15:48:44.501 201 456678 false 1993-08-03T15:48:44 010-18741035 {"wTS":"1596440924","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440924-010-18741035 -139872392484052992 10000 1993-08-03T15:48:44.532 10000 1993-08-03T15:48:44.532 201 456678 false 1993-08-03T15:48:44 010-18741038 {"wTS":"1596440924","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440924-010-18741038 -139872393570377728 10000 1993-08-03T15:48:44.791 10000 1993-08-03T15:48:44.791 201 456678 false 1993-08-03T15:48:44 010-18741034 {"wTS":"1596440924","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440924-010-18741034 -139872393742344192 10000 1993-08-03T15:48:44.832 10000 1993-08-03T15:48:44.832 201 456678 false 1993-08-03T15:48:44 010-18741022 {"wTS":"1596440924","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440924-010-18741022 -139872394040139776 10000 1993-08-03T15:48:44.903 10000 1993-08-03T15:48:44.904 201 456678 false 1993-08-03T15:48:44 010-18741040 {"wTS":"1596440924","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440924-010-18741040 -139872394543456256 10000 1993-08-03T15:48:45.023 10000 1993-08-03T15:48:45.023 201 456678 false 1993-08-03T15:48:44 010-18741024 {"wTS":"1596440924","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440924-010-18741024 -139872396456058880 10000 1993-08-03T15:48:45.479 10000 1993-08-03T15:48:45.479 201 456678 false 1993-08-03T15:48:44 010-18741031 {"wTS":"1596440924","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440924-010-18741031 -139872398083448832 10000 1993-08-03T15:48:45.867 10000 1993-08-03T15:48:45.867 201 456678 false 1993-08-03T15:48:44 010-18741036 {"wTS":"1596440924","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440924-010-18741036 -139872398150557696 10000 1993-08-03T15:48:45.883 10000 1993-08-03T15:48:45.883 201 456678 false 1993-08-03T15:48:44 010-18741027 {"wTS":"1596440924","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440924-010-18741027 -139872398276386816 10000 1993-08-03T15:48:45.913 10000 1993-08-03T15:48:45.913 201 456678 false 1993-08-03T15:48:44 010-18741029 {"wTS":"1596440924","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440924-010-18741029 -139872398356078592 10000 1993-08-03T15:48:45.932 10000 1993-08-03T15:48:45.932 201 456678 false 1993-08-03T15:48:44 010-18741023 {"wTS":"1596440924","rfId":"010-18741023","remark":"数据","weight":118.114} 710078936338096128 1596440924-010-18741023 -139872400310624256 10000 1993-08-03T15:48:46.398 10000 1993-08-03T15:48:46.398 201 456678 false 1993-08-03T15:48:44 010-18741026 {"wTS":"1596440924","rfId":"010-18741026","remark":"数据","weight":147.465} 710078936338096128 1596440924-010-18741026 -139872402147729408 10000 1993-08-03T15:48:46.836 10000 1993-08-03T15:48:46.836 201 456678 false 1993-08-03T15:48:45 010-18741025 {"wTS":"1596440925","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440925-010-18741025 -13987240259651987 10000 1993-08-03T15:48:46.943 10000 1993-08-03T15:48:46.943 201 456678 false 1993-08-03T15:48:45 010-18741032 {"wTS":"1596440925","rfId":"010-18741032","remark":"数据","weight":118.863} 710078936338096128 1596440925-010-18741032 -139872403179528192 10000 1993-08-03T15:48:47.082 10000 1993-08-03T15:48:47.082 201 456678 false 1993-08-03T15:48:45 010-18741021 {"wTS":"1596440925","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440925-010-18741021 -139872403720593408 10000 1993-08-03T15:48:47.211 10000 1993-08-03T15:48:47.211 201 456678 false 1993-08-03T15:48:45 010-18741037 {"wTS":"1596440925","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440925-010-18741037 -139872404051943424 10000 1993-08-03T15:48:47.290 10000 1993-08-03T15:48:47.290 201 456678 false 1993-08-03T15:48:44 010-18741028 {"wTS":"1596440924","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440924-010-18741028 -139872404689477632 10000 1993-08-03T15:48:47.442 10000 1993-08-03T15:48:47.442 201 456678 false 1993-08-03T15:48:44 010-18741030 {"wTS":"1596440924","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440924-010-18741030 -139872518397059072 10000 1993-08-03T15:49:14.552 10000 1993-08-03T15:49:14.552 201 456678 false 1993-08-03T15:49:14 010-18741038 {"wTS":"1596440954","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440954-010-18741038 -139872518405447680 10000 1993-08-03T15:49:14.554 10000 1993-08-03T15:49:14.554 201 456678 false 1993-08-03T15:49:14 010-18741035 {"wTS":"1596440954","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440954-010-18741035 -139872518539665408 10000 1993-08-03T15:49:14.586 10000 1993-08-03T15:49:14.586 201 456678 false 1993-08-03T15:49:14 010-18741039 {"wTS":"1596440954","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440954-010-18741039 -139872519579852800 10000 1993-08-03T15:49:14.834 10000 1993-08-03T15:49:14.834 201 456678 false 1993-08-03T15:49:14 010-18741040 {"wTS":"1596440954","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440954-010-18741040 -139872520058003456 10000 1993-08-03T15:49:14.948 10000 1993-08-03T15:49:14.948 201 456678 false 1993-08-03T15:49:14 010-18741034 {"wTS":"1596440954","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440954-010-18741034 -139872520221581312 10000 1993-08-03T15:49:14.987 10000 1993-08-03T15:49:14.987 201 456678 false 1993-08-03T15:49:14 010-18741024 {"wTS":"1596440954","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440954-010-18741024 -139872520645206016 10000 1993-08-03T15:49:15.088 10000 1993-08-03T15:49:15.088 201 456678 false 1993-08-03T15:49:14 010-18741022 {"wTS":"1596440954","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440954-010-18741022 -139872521467289600 10000 1993-08-03T15:49:15.284 10000 1993-08-03T15:49:15.284 201 456678 false 1993-08-03T15:49:14 010-18741033 {"wTS":"1596440954","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440954-010-18741033 -139872523287617536 10000 1993-08-03T15:49:15.718 10000 1993-08-03T15:49:15.718 201 456678 false 1993-08-03T15:49:14 010-18741027 {"wTS":"1596440954","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440954-010-18741027 -139872524629794816 10000 1993-08-03T15:49:16.038 10000 1993-08-03T15:49:16.038 201 456678 false 1993-08-03T15:49:14 010-18741029 {"wTS":"1596440954","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440954-010-18741029 -139872524915007488 10000 1993-08-03T15:49:16.106 10000 1993-08-03T15:49:16.106 201 456678 false 1993-08-03T15:49:14 010-18741036 {"wTS":"1596440954","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440954-010-18741036 -139872525456072704 10000 1993-08-03T15:49:16.235 10000 1993-08-03T15:49:16.235 201 456678 false 1993-08-03T15:49:14 010-18741023 {"wTS":"1596440954","rfId":"010-18741023","remark":"数据","weight":118.114} 710078936338096128 1596440954-010-18741023 -139872527263817728 10000 1993-08-03T15:49:16.666 10000 1993-08-03T15:49:16.666 201 456678 false 1993-08-03T15:49:15 010-18741026 {"wTS":"1596440955","rfId":"010-18741026","remark":"数据","weight":147.465} 710078936338096128 1596440955-010-18741026 -139872527469338624 10000 1993-08-03T15:49:16.715 10000 1993-08-03T15:49:16.715 201 456678 false 1993-08-03T15:49:15 010-18741021 {"wTS":"1596440955","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440955-010-18741021 -139872528224313344 10000 1993-08-03T15:49:16.895 10000 1993-08-03T15:49:16.895 201 456678 false 1993-08-03T15:49:15 010-18741037 {"wTS":"1596440955","rfId":"010-18741037","remark":"数据","weight":114.952} 710078936338096128 1596440955-010-18741037 -139872644595277824 10000 1993-08-03T15:49:44.640 10000 1993-08-03T15:49:44.640 201 456678 false 1993-08-03T15:49:44 010-18741039 {"wTS":"1596440984","rfId":"010-18741039","remark":"数据","weight":152.171} 710078936338096128 1596440984-010-18741039 -139872644674969600 10000 1993-08-03T15:49:44.659 10000 1993-08-03T15:49:44.659 201 456678 false 1993-08-03T15:49:44 010-18741035 {"wTS":"1596440984","rfId":"010-18741035","remark":"数据","weight":158.451} 710078936338096128 1596440984-010-18741035 -139872644922433536 10000 1993-08-03T15:49:44.718 10000 1993-08-03T15:49:44.718 201 456678 false 1993-08-03T15:49:44 010-18741038 {"wTS":"1596440984","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596440984-010-18741038 -139872645878734848 10000 1993-08-03T15:49:44.946 10000 1993-08-03T15:49:44.946 201 456678 false 1993-08-03T15:49:44 010-18741040 {"wTS":"1596440984","rfId":"010-18741040","remark":"数据","weight":145.513} 710078936338096128 1596440984-010-18741040 -139872646386245632 10000 1993-08-03T15:49:45.067 10000 1993-08-03T15:49:45.067 201 456678 false 1993-08-03T15:49:44 010-18741034 {"wTS":"1596440984","rfId":"010-18741034","remark":"数据","weight":110.069} 710078936338096128 1596440984-010-18741034 -139872646591766528 10000 1993-08-03T15:49:45.116 10000 1993-08-03T15:49:45.116 201 456678 false 1993-08-03T15:49:44 010-18741022 {"wTS":"1596440984","rfId":"010-18741022","remark":"数据","weight":104.797} 710078936338096128 1596440984-010-18741022 -139872647241883648 10000 1993-08-03T15:49:45.271 10000 1993-08-03T15:49:45.271 201 456678 false 1993-08-03T15:49:44 010-18741024 {"wTS":"1596440984","rfId":"010-18741024","remark":"数据","weight":62.852} 710078936338096128 1596440984-010-18741024 -139872648152047616 10000 1993-08-03T15:49:45.488 10000 1993-08-03T15:49:45.488 201 456678 false 1993-08-03T15:49:44 010-18741033 {"wTS":"1596440984","rfId":"010-18741033","remark":"数据","weight":119.771} 710078936338096128 1596440984-010-18741033 -139872649997541376 10000 1993-08-03T15:49:45.928 10000 1993-08-03T15:49:45.928 201 456678 false 1993-08-03T15:49:44 010-18741031 {"wTS":"1596440984","rfId":"010-18741031","remark":"数据","weight":102.374} 710078936338096128 1596440984-010-18741031 -139872650000171200 10000 1993-08-03T15:49:46.309 10000 1993-08-03T15:49:46.309 201 456678 false 1993-08-03T15:49:45 010-18741027 {"wTS":"1596440985","rfId":"010-18741027","remark":"数据","weight":89.6} 710078936338096128 1596440985-010-18741027 -139872651008368640 10000 1993-08-03T15:49:46.169 10000 1993-08-03T15:49:46.169 201 456678 false 1993-08-03T15:49:44 010-18741030 {"wTS":"1596440984","rfId":"010-18741030","remark":"数据","weight":137.051} 710078936338096128 1596440984-010-18741030 -139872653399121920 10000 1993-08-03T15:49:46.739 10000 1993-08-03T15:49:46.739 201 456678 false 1993-08-03T15:49:44 010-18741028 {"wTS":"1596440984","rfId":"010-18741028","remark":"数据","weight":62.048} 710078936338096128 1596440984-010-18741028 -139872653420093440 10000 1993-08-03T15:49:46.744 10000 1993-08-03T15:49:46.744 201 456678 false 1993-08-03T15:49:44 010-18741036 {"wTS":"1596440984","rfId":"010-18741036","remark":"数据","weight":141.555} 710078936338096128 1596440984-010-18741036 -139872654166679552 10000 1993-08-03T15:49:46.923 10000 1993-08-03T15:49:46.923 201 456678 false 1993-08-03T15:49:45 010-18741029 {"wTS":"1596440985","rfId":"010-18741029","remark":"数据","weight":123.022} 710078936338096128 1596440985-010-18741029 -139872655022317568 10000 1993-08-03T15:49:47.126 10000 1993-08-03T15:49:47.126 201 456678 false 1993-08-03T15:49:45 010-18741025 {"wTS":"1596440985","rfId":"010-18741025","remark":"数据","weight":110.27} 710078936338096128 1596440985-010-18741025 -139872655416582144 10000 1993-08-03T15:49:47.220 10000 1993-08-03T15:49:47.220 201 456678 false 1993-08-03T15:49:45 010-18741032 {"wTS":"1596440985","rfId":"010-18741032","remark":"数据","weight":118.863} 710078936338096128 1596440985-010-18741032 -139872656439992320 10000 1993-08-03T15:49:47.464 10000 1993-08-03T15:49:47.464 201 456678 false 1993-08-03T15:49:45 010-18741021 {"wTS":"1596440985","rfId":"010-18741021","remark":"数据","weight":169.033} 710078936338096128 1596440985-010-18741021 -139872770659278848 10000 1993-08-03T15:50:14.696 10000 1993-08-03T15:50:14.696 201 456678 false 1993-08-03T15:50:14 010-18741038 {"wTS":"1596441014","rfId":"010-18741038","remark":"数据","weight":154.728} 710078936338096128 1596441014-010-18741038 -149906320110972801 10000 2021-12-24T08:19:59.449 10000 2021-12-24T08:19:59.449 201 123321 false 2021-12-24T08:19:59 65322-002867696000 {"wTS":"1598833199","rfId":"65322-002867696000","remark":"数据","weight":57.5} 1233211234567891011 1598833199-65322-002867696000 -185561918179860480 10000 1993-12-07T17:42:36.328 10000 1993-12-07T17:42:36.328 201 123321 false 1993-12-07T17:42:35 0410-137110012 {"wTS":"1607334155","rfId":"0410-137110012","remark":"数据","weight":560} 1233211234567891011 1607334155-0410-137110012 -185562285613473792 10000 1993-12-07T17:44:03.931 10000 1993-12-07T17:44:03.931 201 123321 false 1993-12-07T17:44:03 0410-137110012 {"wTS":"1607334243","rfId":"0410-137110012","remark":"数据","weight":580} 1233211234567891011 1607334243-0410-137110012 -185564316508053504 10000 1993-12-07T17:52:08.134 10000 1993-12-07T17:52:08.134 201 123321 false 1993-12-07T17:52:07 0410-137110012 {"wTS":"1607334727","rfId":"0410-137110012","remark":"数据","weight":480} 1233211234567891011 1607334727-0410-137110012 -864565750682841088 10000 2021-07-13T17:55:37.817 10000 2021-07-13T17:55:37.817 201 123321 false 2021-07-13T17:55:22 000-000000000000 {"wTS":"1626170122","rfId":"000-000000000000","remark":"数据","weight":78} 720303223645310976 1626170122-000-000000000000 -864566846897745920 10000 2021-07-13T17:59:59.175 10000 2021-07-13T17:59:59.175 201 123321 false 2021-07-13T17:59:30 000-000000000000 {"wTS":"1626170370","rfId":"000-000000000000","remark":"数据","weight":77.1} 720303223645310976 1626170370-000-000000000000 -864568598468456448 10000 2021-07-13T18:06:56.782 10000 2021-07-13T18:06:56.782 201 123321 false 2021-07-13T18:04:44 000-000000000000 {"wTS":"1626170684","rfId":"000-000000000000","remark":"数据","weight":77.7} 720303223645310976 1626170684-000-000000000000 -864568901754384384 10000 2021-07-13T18:08:09.091 10000 2021-07-13T18:08:09.091 201 123321 false 2021-07-13T18:07:29 000-000000000000 {"wTS":"1626170849","rfId":"000-000000000000","remark":"数据","weight":2000} 720303223645310976 1626170849-000-000000000000 -864568950307647488 10000 2021-07-13T18:08:20.667 10000 2021-07-13T18:08:20.667 201 123321 false 2021-07-13T18:08:06 000-000000000000 {"wTS":"1626170886","rfId":"000-000000000000","remark":"数据","weight":2000} 720303223645310976 1626170886-000-000000000000 -864577514845728768 10000 2021-07-13T18:42:22.612 10000 2021-07-13T18:42:22.612 201 123321 false 2021-07-13T18:41:05 000-000000000000 {"wTS":"1626172865","rfId":"000-000000000000","remark":"数据","weight":30.7} 720303223645310976 1626172865-000-000000000000 -864585483348045824 10000 2021-07-13T19:14:02.451 10000 2021-07-13T19:14:02.451 201 123321 false 2021-07-13T19:13:48 999-020052022634 {"wTS":"1626174828","rfId":"999-020052022634","remark":"数据","weight":2000} 720303223645310976 1626174828-999-020052022634 -864595543231000576 10000 2021-07-13T19:54:00.914 10000 2021-07-13T19:54:00.914 201 123321 false 2021-07-13T19:53:56 000-000000000000 {"wTS":"1626177236","rfId":"000-000000000000","remark":"数据","weight":2000} 720303223645310976 1626177236-000-000000000000 -864595601850593280 10000 2021-07-13T19:54:14.890 10000 2021-07-13T19:54:14.890 201 123321 false 2021-07-13T19:54:05 999-020052022634 {"wTS":"1626177245","rfId":"999-020052022634","remark":"数据","weight":43.7} 720303223645310976 1626177245-999-020052022634 -864595723665764352 10000 2021-07-13T19:54:43.933 10000 2021-07-13T19:54:43.933 201 123321 false 2021-07-13T19:54:39 999-020052022634 {"wTS":"1626177279","rfId":"999-020052022634","remark":"数据","weight":2000} 720303223645310976 1626177279-999-020052022634 -864595868411195392 10000 2021-07-13T19:55:18.443 10000 2021-07-13T19:55:18.443 201 123321 false 2021-07-13T19:55:14 999-020052022634 {"wTS":"1626177314","rfId":"999-020052022634","remark":"数据","weight":128.1} 720303223645310976 1626177314-999-020052022634 -864596000837955584 10000 2021-07-13T19:55:50.016 10000 2021-07-13T19:55:50.016 201 123321 false 2021-07-13T19:55:46 999-020052022634 {"wTS":"1626177346","rfId":"999-020052022634","remark":"数据","weight":2000} 720303223645310976 1626177346-999-020052022634 -864596477000511488 10000 2021-07-13T19:57:43.542 10000 2021-07-13T19:57:43.542 201 123321 false 2021-07-13T19:57:39 999-020052022634 {"wTS":"1626177459","rfId":"999-020052022634","remark":"数据","weight":2000} 720303223645310976 1626177459-999-020052022634 -864598919201124352 10000 2021-07-13T20:07:25.808 10000 2021-07-13T20:07:25.808 201 123321 false 2021-07-13T20:06:56 999-020052022634 {"wTS":"1626178016","rfId":"999-020052022634","remark":"数据","weight":2000} 720303223645310976 1626178016-999-020052022634 -864598949525942272 10000 2021-07-13T20:07:33.038 10000 2021-07-13T20:07:33.038 201 123321 false 2021-07-13T20:07:28 999-020052022634 {"wTS":"1626178048","rfId":"999-020052022634","remark":"数据","weight":2000} 720303223645310976 1626178048-999-020052022634 -864599123581169664 10000 2021-07-13T20:08:14.536 10000 2021-07-13T20:08:14.536 201 123321 false 2021-07-13T20:08:11 999-020052022634 {"wTS":"1626178091","rfId":"999-020052022634","remark":"数据","weight":2000} 720303223645310976 1626178091-999-020052022634 -864599252606349312 10000 2021-07-13T20:08:45.299 10000 2021-07-13T20:08:45.299 201 123321 false 2021-07-13T20:08:41 999-020052022634 {"wTS":"1626178121","rfId":"999-020052022634","remark":"数据","weight":2000} 720303223645310976 1626178121-999-020052022634 -864599388354998272 10000 2021-07-13T20:09:17.663 10000 2021-07-13T20:09:17.663 201 123321 false 2021-07-13T20:09:13 999-020052022634 {"wTS":"1626178153","rfId":"999-020052022634","remark":"数据","weight":31.2} 720303223645310976 1626178153-999-020052022634 -864599520311996416 10000 2021-07-13T20:09:49.124 10000 2021-07-13T20:09:49.124 201 123321 false 2021-07-13T20:09:45 999-020052022634 {"wTS":"1626178185","rfId":"999-020052022634","remark":"数据","weight":31.4} 720303223645310976 1626178185-999-020052022634 -864599651631460352 10000 2021-07-13T20:10:20.433 10000 2021-07-13T20:10:20.433 201 123321 false 2021-07-13T20:10:16 999-020052022634 {"wTS":"1626178216","rfId":"999-020052022634","remark":"数据","weight":95.7} 720303223645310976 1626178216-999-020052022634 -864599775870939136 10000 2021-07-13T20:10:50.054 10000 2021-07-13T20:10:50.054 201 123321 false 2021-07-13T20:10:45 999-020052022634 {"wTS":"1626178245","rfId":"999-020052022634","remark":"数据","weight":31.3} 720303223645310976 1626178245-999-020052022634 -925010014523199488 10000 2021-12-27T12:59:33.317 10000 2021-12-27T12:59:33.317 201 123321 false 2021-12-27T13:20:21 984-000100748724 {"wTS":"1640582421","rfId":"984-000100748724","remark":"数据","weight":0.123} 1233211234567891011 1640582421-984-000100748724 -972179152830517248 10000 2022-05-06T16:52:52.250 10000 2022-05-06T16:52:52.250 202 123321 false 2022-05-06T16:52:49 0410-137110005 {"wTS":"1651827169","rfId":"0410-137110005","remark":"","weight":200} 1233211234567891011 1651827169-0410-137110005 -972179288327507968 10000 2022-05-06T16:53:24.555 10000 2022-05-06T16:53:24.555 202 123321 false 2022-05-06T16:53:22 0410-137110005 {"wTS":"1651827202","rfId":"0410-137110005","remark":"","weight":100} 1233211234567891011 1651827202-0410-137110005 -972182670182240256 10000 2022-05-06T17:06:50.852 10000 2022-05-06T17:06:50.852 201 123321 false 2022-05-06T17:06:47 984-000100744406 {"wTS":"1651828007","rfId":"984-000100744406","remark":"数据","weight":1.119} 1233211234567891011 1651828007-984-000100744406 -972185021504868352 10000 2022-05-06T17:16:11.451 10000 2022-05-06T17:16:11.451 201 123321 false 2022-05-06T17:16:08 984-000100744406 {"wTS":"1651828568","rfId":"984-000100744406","remark":"数据","weight":11.19} 1233211234567891011 1651828568-984-000100744406 -972186411757260800 10000 2022-05-06T17:21:42.913 10000 2022-05-06T17:21:42.913 201 123321 false 2022-05-06T17:21:39 984-000100744406 {"wTS":"1651828899","rfId":"984-000100744406","remark":"数据","weight":111.9} 1233211234567891011 1651828899-984-000100744406 -972187393287311360 10000 2022-05-06T17:25:36.928 10000 2022-05-06T17:25:36.928 201 123321 false 2022-05-06T17:25:33 984-000100744406 {"wTS":"1651829133","rfId":"984-000100744406","remark":"数据","weight":121.9} 1233211234567891011 1651829133-984-000100744406 +110169402202689536 10000 1993-07-24T10:34:22.851 10000 1993-07-24T10:34:22.851 201 123321 false 1993-07-24T10:34:22 xyz {"wTS": "abc58062", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc58062-xyz +110175095697739776 10000 1993-07-24T10:57:00.286 10000 1993-07-24T10:57:00.286 201 123321 false 1993-07-24T10:56:59 xyz {"wTS": "abc59419", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc59419-xyz +110176899974074368 10000 1993-07-24T11:04:10.459 10000 1993-07-24T11:04:10.459 201 123321 false 1993-07-24T11:04:09 xyz {"wTS": "abc59849", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc59849-xyz +110178694460903424 10000 1993-07-24T11:11:18.298 10000 1993-07-24T11:11:18.298 201 123321 false 1993-07-24T11:11:17 xyz {"wTS": "abc60277", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 12332112345 +110178823263784960 10000 1993-07-24T11:11:49.007 10000 1993-07-24T11:11:49.007 201 123321 false 1993-07-24T11:11:48 xyz {"wTS": "abc60308", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60308-xyz +110178991782531072 10000 1993-07-24T11:12:29.185 10000 1993-07-24T11:12:29.186 201 123321 false 1993-07-24T11:12:28 xyz {"wTS": "abc60348", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60348-xyz +110179118664421376 10000 1993-07-24T11:12:59.436 10000 1993-07-24T11:12:59.436 201 123321 false 1993-07-24T11:12:58 xyz {"wTS": "abc60378", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60378-xyz +110179245076549632 10000 1993-07-24T11:13:29.575 10000 1993-07-24T11:13:29.575 201 123321 false 1993-07-24T11:13:28 xyz {"wTS": "abc60408", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60408-xyz +110179371597729792 10000 1993-07-24T11:13:59.740 10000 1993-07-24T11:13:59.740 201 123321 false 1993-07-24T11:13:59 xyz {"wTS": "abc60439", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60439-xyz +110179498144075776 10000 1993-07-24T11:14:29.911 10000 1993-07-24T11:14:29.911 201 123321 false 1993-07-24T11:14:29 xyz {"wTS": "abc60469", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60469-xyz +110179625034354688 10000 1993-07-24T11:15:00.164 10000 1993-07-24T11:15:00.164 201 123321 false 1993-07-24T11:14:59 xyz {"wTS": "abc60499", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60499-xyz +110179751375179776 10000 1993-07-24T11:15:30.286 10000 1993-07-24T11:15:30.286 201 123321 false 1993-07-24T11:15:29 xyz {"wTS": "abc60529", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60529-xyz +110179877997023232 10000 1993-07-24T11:16:00.475 10000 1993-07-24T11:16:00.475 201 123321 false 1993-07-24T11:15:59 xyz {"wTS": "abc60559", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60559-xyz +110180004425928704 10000 1993-07-24T11:16:30.618 10000 1993-07-24T11:16:30.618 201 123321 false 1993-07-24T11:16:30 xyz {"wTS": "abc60590", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60590-xyz +110180131114881024 10000 1993-07-24T11:17:00.823 10000 1993-07-24T11:17:00.823 201 123321 false 1993-07-24T11:17 xyz {"wTS": "abc60620", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60620-xyz +110180257979994112 10000 1993-07-24T11:17:31.070 10000 1993-07-24T11:17:31.070 201 123321 false 1993-07-24T11:17:30 xyz {"wTS": "abc60650", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60650-xyz +110180395045654528 10000 1993-07-24T11:18:03.749 10000 1993-07-24T11:18:03.749 201 123321 false 1993-07-24T11:18 xyz {"wTS": "abc60680", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60680-xyz +110180510913302528 10000 1993-07-24T11:18:31.374 10000 1993-07-24T11:18:31.374 201 123321 false 1993-07-24T11:18:30 xyz {"wTS": "abc60710", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60710-xyz +110180637086355456 10000 1993-07-24T11:19:01.456 10000 1993-07-24T11:19:01.456 201 123321 false 1993-07-24T11:19 xyz {"wTS": "abc60740", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60740-xyz +110180763712393216 10000 1993-07-24T11:19:31.646 10000 1993-07-24T11:19:31.646 201 123321 false 1993-07-24T11:19:31 xyz {"wTS": "abc60771", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60771-xyz +110180890539757568 10000 1993-07-24T11:20:01.884 10000 1993-07-24T11:20:01.884 201 123321 false 1993-07-24T11:20:01 xyz {"wTS": "abc60801", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60801-xyz +110181018361171968 10000 1993-07-24T11:20:32.359 10000 1993-07-24T11:20:32.359 201 123321 false 1993-07-24T11:20:31 xyz {"wTS": "abc60831", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60831-xyz +110181143443705856 10000 1993-07-24T11:21:02.181 10000 1993-07-24T11:21:02.181 201 123321 false 1993-07-24T11:21:01 xyz {"wTS": "abc60861", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60861-xyz +110181270325596160 10000 1993-07-24T11:21:32.432 10000 1993-07-24T11:21:32.432 201 123321 false 1993-07-24T11:21:31 xyz {"wTS": "abc60891", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60891-xyz +110181396674809856 10000 1993-07-24T11:22:02.556 10000 1993-07-24T11:22:02.556 201 123321 false 1993-07-24T11:22:01 xyz {"wTS": "abc60921", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60921-xyz +110181522885611520 10000 1993-07-24T11:22:32.647 10000 1993-07-24T11:22:32.647 201 123321 false 1993-07-24T11:22:32 xyz {"wTS": "abc60952", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60952-xyz +110181649587146752 10000 1993-07-24T11:23:02.855 10000 1993-07-24T11:23:02.855 201 123321 false 1993-07-24T11:23:02 xyz {"wTS": "abc60982", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc60982-xyz +110181776418705408 10000 1993-07-24T11:23:33.094 10000 1993-07-24T11:23:33.094 201 123321 false 1993-07-24T11:23:32 xyz {"wTS": "abc61012", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc61012-xyz +110181902755336192 10000 1993-07-24T11:24:03.215 10000 1993-07-24T11:24:03.215 201 123321 false 1993-07-24T11:24:02 xyz {"wTS": "abc61042", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc61042-xyz +110182029335236608 10000 1993-07-24T11:24:33.394 10000 1993-07-24T11:24:33.394 201 123321 false 1993-07-24T11:24:32 xyz {"wTS": "abc61072", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc61072-xyz +110182155852222464 10000 1993-07-24T11:25:03.558 10000 1993-07-24T11:25:03.558 201 123321 false 1993-07-24T11:25:02 xyz {"wTS": "abc61102", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc61102-xyz +110182282490843136 10000 1993-07-24T11:25:33.751 10000 1993-07-24T11:25:33.751 201 123321 false 1993-07-24T11:25:33 xyz {"wTS": "abc61133", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc61133-xyz +110182409053966336 10000 1993-07-24T11:26:03.926 10000 1993-07-24T11:26:03.926 201 123321 false 1993-07-24T11:26:03 xyz {"wTS": "abc61163", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc61163-xyz +110182535369625600 10000 1993-07-24T11:26:34.042 10000 1993-07-24T11:26:34.042 201 123321 false 1993-07-24T11:26:33 xyz {"wTS": "abc61193", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc61193-xyz +110182662033412096 10000 1993-07-24T11:27:04.241 10000 1993-07-24T11:27:04.241 201 123321 false 1993-07-24T11:27:03 xyz {"wTS": "abc61223", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc61223-xyz +110182790806933504 10000 1993-07-24T11:27:34.943 10000 1993-07-24T11:27:34.943 201 123321 false 1993-07-24T11:27:33 xyz {"wTS": "abc61253", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc61253-xyz +110182915377762304 10000 1993-07-24T11:28:04.643 10000 1993-07-24T11:28:04.643 201 123321 false 1993-07-24T11:28:03 xyz {"wTS": "abc61283", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc61283-xyz +110183041982828544 10000 1993-07-24T11:28:34.828 10000 1993-07-24T11:28:34.828 201 123321 false 1993-07-24T11:28:34 xyz {"wTS": "abc61314", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc61314-xyz +110188354488274944 10000 1993-07-24T11:49:41.428 10000 1993-07-24T11:49:41.428 201 123321 false 1993-07-24T11:49:40 xyz {"wTS": "abc62580", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc62580-xyz +110188483035303936 10000 1993-07-24T11:50:12.076 10000 1993-07-24T11:50:12.076 201 123321 false 1993-07-24T11:50:11 xyz {"wTS": "abc62611", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc62611-xyz +110188609283854336 10000 1993-07-24T11:50:42.176 10000 1993-07-24T11:50:42.176 201 123321 false 1993-07-24T11:50:41 xyz {"wTS": "abc62641", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc62641-xyz +110188860698824704 10000 1993-07-24T11:51:42.118 10000 1993-07-24T11:51:42.118 201 123321 false 1993-07-24T11:51:41 xyz {"wTS": "abc62701", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc62701-xyz +110188994555842560 10000 1993-07-24T11:52:14.032 10000 1993-07-24T11:52:14.032 201 123321 false 1993-07-24T11:52:11 xyz {"wTS": "abc62731", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc62731-xyz +110189113330143232 10000 1993-07-24T11:52:42.350 10000 1993-07-24T11:52:42.350 201 123321 false 1993-07-24T11:52:41 xyz {"wTS": "abc62761", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc62761-xyz +110189239381561344 10000 1993-07-24T11:53:12.403 10000 1993-07-24T11:53:12.403 201 123321 false 1993-07-24T11:53:11 xyz {"wTS": "abc62791", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc62791-xyz +110189365340704768 10000 1993-07-24T11:53:42.434 10000 1993-07-24T11:53:42.434 201 123321 false 1993-07-24T11:53:41 xyz {"wTS": "abc62821", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc62821-xyz +110189491983519744 10000 1993-07-24T11:54:12.629 10000 1993-07-24T11:54:12.629 201 123321 false 1993-07-24T11:54:11 xyz {"wTS": "abc62851", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc62851-xyz +110189618773135360 10000 1993-07-24T11:54:42.857 10000 1993-07-24T11:54:42.857 201 123321 false 1993-07-24T11:54:41 xyz {"wTS": "abc62881", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc62881-xyz +110189794522861568 10000 1993-07-24T11:55:24.759 10000 1993-07-24T11:55:24.759 201 123321 false 1993-07-24T11:55:11 xyz {"wTS": "abc62911", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc62911-xyz +110189870402015232 10000 1993-07-24T11:55:42.850 10000 1993-07-24T11:55:42.850 201 123321 false 1993-07-24T11:55:41 xyz {"wTS": "abc62941", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc62941-xyz +110189996034002944 10000 1993-07-24T11:56:12.803 10000 1993-07-24T11:56:12.803 201 123321 false 1993-07-24T11:56:12 xyz {"wTS": "abc62972", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc62972-xyz +110190121573715968 10000 1993-07-24T11:56:42.734 10000 1993-07-24T11:56:42.734 201 123321 false 1993-07-24T11:56:42 xyz {"wTS": "abc63002", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63002-xyz +110190247864209408 10000 1993-07-24T11:57:12.844 10000 1993-07-24T11:57:12.844 201 123321 false 1993-07-24T11:57:12 xyz {"wTS": "abc63032", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63032-xyz +110190373731078144 10000 1993-07-24T11:57:42.853 10000 1993-07-24T11:57:42.853 201 123321 false 1993-07-24T11:57:42 xyz {"wTS": "abc63062", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63062-xyz +110190500046737408 10000 1993-07-24T11:58:12.969 10000 1993-07-24T11:58:12.969 201 123321 false 1993-07-24T11:58:12 xyz {"wTS": "abc63092", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63092-xyz +110190626324647936 10000 1993-07-24T11:58:43.076 10000 1993-07-24T11:58:43.076 201 123321 false 1993-07-24T11:58:42 xyz {"wTS": "abc63122", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63122-xyz +110190752267014144 10000 1993-07-24T11:59:13.103 10000 1993-07-24T11:59:13.103 201 123321 false 1993-07-24T11:59:12 xyz {"wTS": "abc63152", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63152-xyz +110190878452649984 10000 1993-07-24T11:59:43.188 10000 1993-07-24T11:59:43.188 201 123321 false 1993-07-24T11:59:42 xyz {"wTS": "abc63182", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63182-xyz +110191004369850368 10000 1993-07-24T12:00:13.209 10000 1993-07-24T12:00:13.209 201 123321 false 1993-07-24T12:00:12 xyz {"wTS": "abc63212", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63212-xyz +110191130433851392 10000 1993-07-24T12:00:43.265 10000 1993-07-24T12:00:43.265 201 123321 false 1993-07-24T12:00:42 xyz {"wTS": "abc63242", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63242-xyz +110191256502046720 10000 1993-07-24T12:01:13.323 10000 1993-07-24T12:01:13.323 201 123321 false 1993-07-24T12:01:12 xyz {"wTS": "abc63272", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63272-xyz +110191382591213568 10000 1993-07-24T12:01:43.384 10000 1993-07-24T12:01:43.384 201 123321 false 1993-07-24T12:01:42 xyz {"wTS": "abc63302", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63302-xyz +110191508500025344 10000 1993-07-24T12:02:13.403 10000 1993-07-24T12:02:13.403 201 123321 false 1993-07-24T12:02:12 xyz {"wTS": "abc63332", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63332-xyz +110191634446585856 10000 1993-07-24T12:02:43.431 10000 1993-07-24T12:02:43.431 201 123321 false 1993-07-24T12:02:42 xyz {"wTS": "abc63362", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63362-xyz +110191760862908416 10000 1993-07-24T12:03:13.571 10000 1993-07-24T12:03:13.571 201 123321 false 1993-07-24T12:03:12 xyz {"wTS": "abc63392", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63392-xyz +110191886729777152 10000 1993-07-24T12:03:43.580 10000 1993-07-24T12:03:43.580 201 123321 false 1993-07-24T12:03:42 xyz {"wTS": "abc63422", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63422-xyz +110192013620056064 10000 1993-07-24T12:04:13.833 10000 1993-07-24T12:04:13.833 201 123321 false 1993-07-24T12:04:12 xyz {"wTS": "abc63452", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63452-xyz +110192140048961536 10000 1993-07-24T12:04:43.976 10000 1993-07-24T12:04:43.976 201 123321 false 1993-07-24T12:04:42 xyz {"wTS": "abc63482", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63482-xyz +110192390960615424 10000 1993-07-24T12:05:43.798 10000 1993-07-24T12:05:43.798 201 123321 false 1993-07-24T12:05:43 xyz {"wTS": "abc63543", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63543-xyz +110192642979565568 10000 1993-07-24T12:06:43.884 10000 1993-07-24T12:06:43.884 201 123321 false 1993-07-24T12:06:43 xyz {"wTS": "abc63603", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63603-xyz +110192769030983680 10000 1993-07-24T12:07:13.937 10000 1993-07-24T12:07:13.937 201 123321 false 1993-07-24T12:07:13 xyz {"wTS": "abc63633", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63633-xyz +110192895040458752 10000 1993-07-24T12:07:43.980 10000 1993-07-24T12:07:43.980 201 123321 false 1993-07-24T12:07:43 xyz {"wTS": "abc63663", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63663-xyz +110193021058322432 10000 1993-07-24T12:08:14.025 10000 1993-07-24T12:08:14.025 201 123321 false 1993-07-24T12:08:13 xyz {"wTS": "abc63693", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63693-xyz +110193147101351936 10000 1993-07-24T12:08:44.076 10000 1993-07-24T12:08:44.076 201 123321 false 1993-07-24T12:08:43 xyz {"wTS": "abc63723", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63723-xyz +110193273085661184 10000 1993-07-24T12:09:14.113 10000 1993-07-24T12:09:14.113 201 123321 false 1993-07-24T12:09:13 xyz {"wTS": "abc63753", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63753-xyz +110193399204188160 10000 1993-07-24T12:09:44.182 10000 1993-07-24T12:09:44.182 201 123321 false 1993-07-24T12:09:43 xyz {"wTS": "abc63783", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63783-xyz +110193525255606272 10000 1993-07-24T12:10:14.235 10000 1993-07-24T12:10:14.235 201 123321 false 1993-07-24T12:10:13 xyz {"wTS": "abc63813", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63813-xyz +110193651541905408 10000 1993-07-24T12:10:44.344 10000 1993-07-24T12:10:44.344 201 123321 false 1993-07-24T12:10:43 xyz {"wTS": "abc63843", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63843-xyz +110193777673015296 10000 1993-07-24T12:11:14.416 10000 1993-07-24T12:11:14.416 201 123321 false 1993-07-24T12:11:13 xyz {"wTS": "abc63873", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63873-xyz +110193903774765056 10000 1993-07-24T12:11:44.481 10000 1993-07-24T12:11:44.481 201 123321 false 1993-07-24T12:11:43 xyz {"wTS": "abc63903", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63903-xyz +110194030237224960 10000 1993-07-24T12:12:14.633 10000 1993-07-24T12:12:14.633 201 123321 false 1993-07-24T12:12:13 xyz {"wTS": "abc63933", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63933-xyz +110194156341103328 10000 1993-07-24T12:12:44.699 10000 1993-07-24T12:12:44.699 201 123321 false 1993-07-24T12:12:43 xyz {"wTS": "abc63963", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63963-xyz +110194282004516864 10000 1993-07-24T12:13:14.658 10000 1993-07-24T12:13:14.658 201 123321 false 1993-07-24T12:13:14 xyz {"wTS": "abc63994", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc63994-xyz +110194408274038784 10000 1993-07-24T12:13:44.763 10000 1993-07-24T12:13:44.763 201 123321 false 1993-07-24T12:13:44 xyz {"wTS": "abc64024", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64024-xyz +110194534354817024 10000 1993-07-24T12:14:14.823 10000 1993-07-24T12:14:14.823 201 123321 false 1993-07-24T12:14:14 xyz {"wTS": "abc64054", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64054-xyz +110194660330737664 10000 1993-07-24T12:14:44.858 10000 1993-07-24T12:14:44.858 201 123321 false 1993-07-24T12:14:44 xyz {"wTS": "abc64084", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64084-xyz +110194786499596288 10000 1993-07-24T12:15:14.939 10000 1993-07-24T12:15:14.939 201 123321 false 1993-07-24T12:15:14 xyz {"wTS": "abc64114", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64114-xyz +110194912936890368 10000 1993-07-24T12:15:45.084 10000 1993-07-24T12:15:45.084 201 123321 false 1993-07-24T12:15:44 xyz {"wTS": "abc64144", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64144-xyz +110195039147692032 10000 1993-07-24T12:16:15.175 10000 1993-07-24T12:16:15.175 201 123321 false 1993-07-24T12:16:14 xyz {"wTS": "abc64174", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64174-xyz +110195290638159872 10000 1993-07-24T12:17:15.135 10000 1993-07-24T12:17:15.135 201 123321 false 1993-07-24T12:17:14 xyz {"wTS": "abc64234", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64234-xyz +110195416685383680 10000 1993-07-24T12:17:45.187 10000 1993-07-24T12:17:45.187 201 123321 false 1993-07-24T12:17:44 xyz {"wTS": "abc64264", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64264-xyz +110195542942322688 10000 1993-07-24T12:18:15.289 10000 1993-07-24T12:18:15.289 201 123321 false 1993-07-24T12:18:14 xyz {"wTS": "abc64294", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64294-xyz +110195668876300288 10000 1993-07-24T12:18:45.314 10000 1993-07-24T12:18:45.314 201 123321 false 1993-07-24T12:18:44 xyz {"wTS": "abc64324", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64324-xyz +110195795045158912 10000 1993-07-24T12:19:15.395 10000 1993-07-24T12:19:15.395 201 123321 false 1993-07-24T12:19:14 xyz {"wTS": "abc64354", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64354-xyz +110195920932999168 10000 1993-07-24T12:19:45.409 10000 1993-07-24T12:19:45.409 201 123321 false 1993-07-24T12:19:44 xyz {"wTS": "abc64384", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64384-xyz +110196047487733760 10000 1993-07-24T12:20:15.582 10000 1993-07-24T12:20:15.582 201 123321 false 1993-07-24T12:20:14 xyz {"wTS": "abc64414", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64414-xyz +110196173232967680 10000 1993-07-24T12:20:45.562 10000 1993-07-24T12:20:45.562 201 123321 false 1993-07-24T12:20:44 xyz {"wTS": "abc64444", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64444-xyz +110196299162750976 10000 1993-07-24T12:21:15.586 10000 1993-07-24T12:21:15.586 201 123321 false 1993-07-24T12:21:14 xyz {"wTS": "abc64474", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64474-xyz +110196425423884288 10000 1993-07-24T12:21:45.689 10000 1993-07-24T12:21:45.689 201 123321 false 1993-07-24T12:21:45 xyz {"wTS": "abc64505", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64505-xyz +110196551647268864 10000 1993-07-24T12:22:15.783 10000 1993-07-24T12:22:15.783 201 123321 false 1993-07-24T12:22:15 xyz {"wTS": "abc64535", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64535-xyz +110196677765795840 10000 1993-07-24T12:22:45.852 10000 1993-07-24T12:22:45.852 201 123321 false 1993-07-24T12:22:45 xyz {"wTS": "abc64565", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64565-xyz +110196803972403200 10000 1993-07-24T12:23:15.943 10000 1993-07-24T12:23:15.943 201 123321 false 1993-07-24T12:23:15 xyz {"wTS": "abc64595", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64595-xyz +110196930053181440 10000 1993-07-24T12:23:46.002 10000 1993-07-24T12:23:46.002 201 123321 false 1993-07-24T12:23:45 xyz {"wTS": "abc64625", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64625-xyz +110197056020713472 10000 1993-07-24T12:24:16.035 10000 1993-07-24T12:24:16.035 201 123321 false 1993-07-24T12:24:15 xyz {"wTS": "abc64655", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc64655-xyz +110217731217330176 10000 1993-07-24T13:46:25.386 10000 1993-07-24T13:46:25.386 201 123321 false 1993-07-24T13:46:05 xyz {"wTS": "abc69565", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc69565-xyz +110217731531902976 10000 1993-07-24T13:46:25.461 10000 1993-07-24T13:46:25.461 201 123321 false 1993-07-24T13:45:35 xyz {"wTS": "abc69535", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc69535-xyz +110217731724840960 10000 1993-07-24T13:46:25.507 10000 1993-07-24T13:46:25.507 201 123321 false 1993-07-24T13:44:34 xyz {"wTS": "abc69474", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc69474-xyz +110217731951333376 10000 1993-07-24T13:46:25.561 10000 1993-07-24T13:46:25.561 201 123321 false 1993-07-24T13:45:04 xyz {"wTS": "abc69504", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc69504-xyz +110217898096103424 10000 1993-07-24T13:47:05.173 10000 1993-07-24T13:47:05.173 201 123321 false 1993-07-24T13:47:05 xyz {"wTS": "abc69625", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc69625-xyz +110218024139132928 10000 1993-07-24T13:47:35.224 10000 1993-07-24T13:47:35.224 201 123321 false 1993-07-24T13:47:35 xyz {"wTS": "abc69655", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc69655-xyz +110218150177968128 10000 1993-07-24T13:48:05.274 10000 1993-07-24T13:48:05.274 201 123321 false 1993-07-24T13:48:05 xyz {"wTS": "abc69685", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc69685-xyz +110218276145500160 10000 1993-07-24T13:48:35.307 10000 1993-07-24T13:48:35.307 201 123321 false 1993-07-24T13:48:35 xyz {"wTS": "abc69715", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc69715-xyz +110218402381467648 10000 1993-07-24T13:49:05.404 10000 1993-07-24T13:49:05.404 201 123321 false 1993-07-24T13:49:05 xyz {"wTS": "abc69745", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc69745-xyz +110218528369971200 10000 1993-07-24T13:49:35.442 10000 1993-07-24T13:49:35.442 201 123321 false 1993-07-24T13:49:35 xyz {"wTS": "abc69775", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc69775-xyz +110218765297815552 10000 1993-07-24T13:50:31.930 10000 1993-07-24T13:50:31.930 201 123321 false 1993-07-24T13:50:31 xyz {"wTS": "abc69831", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc69831-xyz +110218891194044416 10000 1993-07-24T13:51:01.946 10000 1993-07-24T13:51:01.946 201 123321 false 1993-07-24T13:51:02 xyz {"wTS": "abc69862", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc69862-xyz +110219017400651776 10000 1993-07-24T13:51:32.036 10000 1993-07-24T13:51:32.036 201 123321 false 1993-07-24T13:51:32 xyz {"wTS": "abc69892", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc69892-xyz +110219143410126848 10000 1993-07-24T13:52:02.079 10000 1993-07-24T13:52:02.079 201 123321 false 1993-07-24T13:52:02 xyz {"wTS": "abc69922", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc69922-xyz +110219269524459520 10000 1993-07-24T13:52:32.147 10000 1993-07-24T13:52:32.147 201 123321 false 1993-07-24T13:52:32 xyz {"wTS": "abc69952", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc69952-xyz +110219395626209280 10000 1993-07-24T13:53:02.212 10000 1993-07-24T13:53:02.212 201 123321 false 1993-07-24T13:53:02 xyz {"wTS": "abc69982", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc69982-xyz +110219522440990720 10000 1993-07-24T13:53:32.447 10000 1993-07-24T13:53:32.447 201 123321 false 1993-07-24T13:53:32 xyz {"wTS": "abc70012", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70012-xyz +110219648433688576 10000 1993-07-24T13:54:02.486 10000 1993-07-24T13:54:02.486 201 123321 false 1993-07-24T13:54:02 xyz {"wTS": "abc70042", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70042-xyz +110219773797240832 10000 1993-07-24T13:54:32.375 10000 1993-07-24T13:54:32.375 201 123321 false 1993-07-24T13:54:32 xyz {"wTS": "abc70072", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70072-xyz +110219899630555136 10000 1993-07-24T13:55:02.376 10000 1993-07-24T13:55:02.376 201 123321 false 1993-07-24T13:55:02 xyz {"wTS": "abc70102", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70102-xyz +110220025518395392 10000 1993-07-24T13:55:32.390 10000 1993-07-24T13:55:32.390 201 123321 false 1993-07-24T13:55:32 xyz {"wTS": "abc70132", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70132-xyz +110220151716614144 10000 1993-07-24T13:56:02.478 10000 1993-07-24T13:56:02.478 201 123321 false 1993-07-24T13:56:02 xyz {"wTS": "abc70162", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70162-xyz +110220277596065792 10000 1993-07-24T13:56:32.490 10000 1993-07-24T13:56:32.490 201 123321 false 1993-07-24T13:56:32 xyz {"wTS": "abc70192", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70192-xyz +110220403542626304 10000 1993-07-24T13:57:02.518 10000 1993-07-24T13:57:02.518 201 123321 false 1993-07-24T13:57:02 xyz {"wTS": "abc70222", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70222-xyz +110220529875062784 10000 1993-07-24T13:57:32.638 10000 1993-07-24T13:57:32.638 201 123321 false 1993-07-24T13:57:32 xyz {"wTS": "abc70252", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70252-xyz +110220781780766720 10000 1993-07-24T13:58:32.697 10000 1993-07-24T13:58:32.697 201 123321 false 1993-07-24T13:58:32 xyz {"wTS": "abc70312", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70312-xyz +110220907869933568 10000 1993-07-24T13:59:02.759 10000 1993-07-24T13:59:02.759 201 123321 false 1993-07-24T13:59:02 xyz {"wTS": "abc70342", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70342-xyz +110221033896185856 10000 1993-07-24T13:59:32.806 10000 1993-07-24T13:59:32.806 201 123321 false 1993-07-24T13:59:32 xyz {"wTS": "abc70372", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70372-xyz +110221159939215360 10000 1993-07-24T14:00:02.857 10000 1993-07-24T14:00:02.857 201 123321 false 1993-07-24T14:00:02 xyz {"wTS": "abc70402", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70402-xyz +110221287076958208 10000 1993-07-24T14:00:33.170 10000 1993-07-24T14:00:33.170 201 123321 false 1993-07-24T14:00:32 xyz {"wTS": "abc70432", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70432-xyz +110221412973187072 10000 1993-07-24T14:01:03.185 10000 1993-07-24T14:01:03.185 201 123321 false 1993-07-24T14:01:02 xyz {"wTS": "abc70462", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70462-xyz +110221538944913408 10000 1993-07-24T14:01:33.219 10000 1993-07-24T14:01:33.219 201 123321 false 1993-07-24T14:01:33 xyz {"wTS": "abc70493", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70493-xyz +110221789957230592 10000 1993-07-24T14:02:33.065 10000 1993-07-24T14:02:33.065 201 123321 false 1993-07-24T14:02:33 xyz {"wTS": "abc70553", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70553-xyz +110221916008648704 10000 1993-07-24T14:03:03.118 10000 1993-07-24T14:03:03.118 201 123321 false 1993-07-24T14:03:03 xyz {"wTS": "abc70583", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70583-xyz +110222042202673152 10000 1993-07-24T14:03:33.205 10000 1993-07-24T14:03:33.205 201 123321 false 1993-07-24T14:03:33 xyz {"wTS": "abc70613", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70613-xyz +110222168291840000 10000 1993-07-24T14:04:03.267 10000 1993-07-24T14:04:03.267 201 123321 false 1993-07-24T14:04:03 xyz {"wTS": "abc70643", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70643-xyz +110222361863163904 10000 1993-07-24T14:04:49.418 10000 1993-07-24T14:04:49.418 201 123321 false 1993-07-24T14:04:49 xyz {"wTS": "abc70689", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70689-xyz +110222488090742784 10000 1993-07-24T14:05:19.513 10000 1993-07-24T14:05:19.513 201 123321 false 1993-07-24T14:05:19 xyz {"wTS": "abc70719", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70719-xyz +110222613961805824 10000 1993-07-24T14:05:49.523 10000 1993-07-24T14:05:49.523 201 123321 false 1993-07-24T14:05:49 xyz {"wTS": "abc70749", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70749-xyz +110222740306825216 10000 1993-07-24T14:06:19.647 10000 1993-07-24T14:06:19.647 201 123321 false 1993-07-24T14:06:19 xyz {"wTS": "abc70779", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70779-xyz +110222866127556608 10000 1993-07-24T14:06:49.644 10000 1993-07-24T14:06:49.644 201 123321 false 1993-07-24T14:06:49 xyz {"wTS": "abc70809", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70809-xyz +110222992099282944 10000 1993-07-24T14:07:19.678 10000 1993-07-24T14:07:19.678 201 123321 false 1993-07-24T14:07:19 xyz {"wTS": "abc70839", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70839-xyz +110223118272335872 10000 1993-07-24T14:07:49.760 10000 1993-07-24T14:07:49.760 201 123321 false 1993-07-24T14:07:49 xyz {"wTS": "abc70869", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70869-xyz +110223245355552768 10000 1993-07-24T14:08:20.059 10000 1993-07-24T14:08:20.059 201 123321 false 1993-07-24T14:08:19 xyz {"wTS": "abc70899", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70899-xyz +110223371398582272 10000 1993-07-24T14:08:50.110 10000 1993-07-24T14:08:50.110 201 123321 false 1993-07-24T14:08:49 xyz {"wTS": "abc70929", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70929-xyz +110223496602750976 10000 1993-07-24T14:09:19.961 10000 1993-07-24T14:09:19.961 201 123321 false 1993-07-24T14:09:19 xyz {"wTS": "abc70959", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70959-xyz +110223623874711552 10000 1993-07-24T14:09:50.305 10000 1993-07-24T14:09:50.305 201 123321 false 1993-07-24T14:09:49 xyz {"wTS": "abc70989", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc70989-xyz +110223748852387840 10000 1993-07-24T14:10:20.102 10000 1993-07-24T14:10:20.102 201 123321 false 1993-07-24T14:10:19 xyz {"wTS": "abc71019", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71019-xyz +110223874110033792 10000 1993-07-24T14:10:50.115 10000 1993-07-24T14:10:50.115 201 123321 false 1993-07-24T14:10:49 xyz {"wTS": "abc71049", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71049-xyz +110224000581931008 10000 1993-07-24T14:11:20.119 10000 1993-07-24T14:11:20.119 201 123321 false 1993-07-24T14:11:19 xyz {"wTS": "abc71079", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71079-xyz +110224127031808000 10000 1993-07-24T14:11:50.267 10000 1993-07-24T14:11:50.267 201 123321 false 1993-07-24T14:11:49 xyz {"wTS": "abc71109", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71109-xyz +110224253032894464 10000 1993-07-24T14:12:20.308 10000 1993-07-24T14:12:20.309 201 123321 false 1993-07-24T14:12:19 xyz {"wTS": "abc71139", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71139-xyz +110224379105284096 10000 1993-07-24T14:12:50.366 10000 1993-07-24T14:12:50.366 201 123321 false 1993-07-24T14:12:49 xyz {"wTS": "abc71169", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71169-xyz +110224505295114240 10000 1993-07-24T14:13:20.453 10000 1993-07-24T14:13:20.453 201 123321 false 1993-07-24T14:13:19 xyz {"wTS": "abc71199", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71199-xyz +110224631157788672 10000 1993-07-24T14:13:50.460 10000 1993-07-24T14:13:50.460 201 123321 false 1993-07-24T14:13:49 xyz {"wTS": "abc71229", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71229-xyz +110224757565722624 10000 1993-07-24T14:14:20.598 10000 1993-07-24T14:14:20.598 201 123321 false 1993-07-24T14:14:19 xyz {"wTS": "abc71259", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71259-xyz +110224883621335040 10000 1993-07-24T14:14:50.652 10000 1993-07-24T14:14:50.652 201 123321 false 1993-07-24T14:14:50 xyz {"wTS": "abc71290", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71290-xyz +110225009978937344 10000 1993-07-24T14:15:20.778 10000 1993-07-24T14:15:20.778 201 123321 false 1993-07-24T14:15:20 xyz {"wTS": "abc71320", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71320-xyz +110225135732559872 10000 1993-07-24T14:15:50.760 10000 1993-07-24T14:15:50.760 201 123321 false 1993-07-24T14:15:50 xyz {"wTS": "abc71350", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71350-xyz +110225261687508992 10000 1993-07-24T14:16:20.790 10000 1993-07-24T14:16:20.790 201 123321 false 1993-07-24T14:16:20 xyz {"wTS": "abc71380", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71380-xyz +110225388510679040 10000 1993-07-24T14:16:51.027 10000 1993-07-24T14:16:51.027 201 123321 false 1993-07-24T14:16:50 xyz {"wTS": "abc71410", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71410-xyz +110225514373353472 10000 1993-07-24T14:17:21.035 10000 1993-07-24T14:17:21.035 201 123321 false 1993-07-24T14:17:20 xyz {"wTS": "abc71440", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71440-xyz +110225640051478528 10000 1993-07-24T14:17:50.999 10000 1993-07-24T14:17:50.999 201 123321 false 1993-07-24T14:17:50 xyz {"wTS": "abc71470", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71470-xyz +110225892120760320 10000 1993-07-24T14:18:51.097 10000 1993-07-24T14:18:51.097 201 123321 false 1993-07-24T14:18:50 xyz {"wTS": "abc71530", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71530-xyz +110226018105069568 10000 1993-07-24T14:19:21.134 10000 1993-07-24T14:19:21.134 201 123321 false 1993-07-24T14:19:20 xyz {"wTS": "abc71560", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71560-xyz +110226144315871232 10000 1993-07-24T14:19:51.225 10000 1993-07-24T14:19:51.225 201 123321 false 1993-07-24T14:19:50 xyz {"wTS": "abc71590", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71590-xyz +110226270233071616 10000 1993-07-24T14:20:21.246 10000 1993-07-24T14:20:21.246 201 123321 false 1993-07-24T14:20:20 xyz {"wTS": "abc71620", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71620-xyz +110226396309655552 10000 1993-07-24T14:20:51.305 10000 1993-07-24T14:20:51.305 201 123321 false 1993-07-24T14:20:50 xyz {"wTS": "abc71650", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71650-xyz +110226522470125568 10000 1993-07-24T14:21:21.384 10000 1993-07-24T14:21:21.384 201 123321 false 1993-07-24T14:21:20 xyz {"wTS": "abc71680", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71680-xyz +110226648936779776 10000 1993-07-24T14:21:51.536 10000 1993-07-24T14:21:51.536 201 123321 false 1993-07-24T14:21:50 xyz {"wTS": "abc71710", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71710-xyz +110226774807842816 10000 1993-07-24T14:22:21.546 10000 1993-07-24T14:22:21.546 201 123321 false 1993-07-24T14:22:20 xyz {"wTS": "abc71740", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71740-xyz +110226900670517248 10000 1993-07-24T14:22:51.554 10000 1993-07-24T14:22:51.555 201 123321 false 1993-07-24T14:22:50 xyz {"wTS": "abc71770", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71770-xyz +110227026835181568 10000 1993-07-24T14:23:21.634 10000 1993-07-24T14:23:21.634 201 123321 false 1993-07-24T14:23:20 xyz {"wTS": "abc71800", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71800-xyz +110227153578659840 10000 1993-07-24T14:23:51.852 10000 1993-07-24T14:23:51.852 201 123321 false 1993-07-24T14:23:50 xyz {"wTS": "abc71830", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71830-xyz +110227279382614016 10000 1993-07-24T14:24:21.846 10000 1993-07-24T14:24:21.846 201 123321 false 1993-07-24T14:24:21 xyz {"wTS": "abc71861", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71861-xyz +110227404746166272 10000 1993-07-24T14:24:51.735 10000 1993-07-24T14:24:51.735 201 123321 false 1993-07-24T14:24:51 xyz {"wTS": "abc71891", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71891-xyz +110227530835333120 10000 1993-07-24T14:25:21.797 10000 1993-07-24T14:25:21.797 201 123321 false 1993-07-24T14:25:21 xyz {"wTS": "abc71921", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71921-xyz +110227783114330112 10000 1993-07-24T14:26:21.945 10000 1993-07-24T14:26:21.945 201 123321 false 1993-07-24T14:26:21 xyz {"wTS": "abc71981", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc71981-xyz +110228035099725824 10000 1993-07-24T14:27:22.023 10000 1993-07-24T14:27:22.023 201 123321 false 1993-07-24T14:27:21 xyz {"wTS": "abc72041", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72041-xyz +110228161386024960 10000 1993-07-24T14:27:52.132 10000 1993-07-24T14:27:52.132 201 123321 false 1993-07-24T14:27:51 xyz {"wTS": "abc72071", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72071-xyz +110228287328391168 10000 1993-07-24T14:28:22.159 10000 1993-07-24T14:28:22.159 201 123321 false 1993-07-24T14:28:21 xyz {"wTS": "abc72101", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72101-xyz +110228539603193856 10000 1993-07-24T14:29:22.306 10000 1993-07-24T14:29:22.306 201 123321 false 1993-07-24T14:29:21 xyz {"wTS": "abc72161", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72161-xyz +110228792138043392 10000 1993-07-24T14:30:22.515 10000 1993-07-24T14:30:22.515 201 123321 false 1993-07-24T14:30:21 xyz {"wTS": "abc72221", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72221-xyz +110228917686145024 10000 1993-07-24T14:30:52.448 10000 1993-07-24T14:30:52.448 201 123321 false 1993-07-24T14:30:51 xyz {"wTS": "abc72251", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72251-xyz +110229045104906240 10000 1993-07-24T14:31:22.827 10000 1993-07-24T14:31:22.827 201 123321 false 1993-07-24T14:31:21 xyz {"wTS": "abc72281", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72281-xyz +110229169679929344 10000 1993-07-24T14:31:52.528 10000 1993-07-24T14:31:52.528 201 123321 false 1993-07-24T14:31:51 xyz {"wTS": "abc72311", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72311-xyz +110229295722958848 10000 1993-07-24T14:32:22.579 10000 1993-07-24T14:32:22.579 201 123321 false 1993-07-24T14:32:21 xyz {"wTS": "abc72341", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72341-xyz +110229421824708608 10000 1993-07-24T14:32:52.644 10000 1993-07-24T14:32:52.644 201 123321 false 1993-07-24T14:32:52 xyz {"wTS": "abc72372", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72372-xyz +110229547901292544 10000 1993-07-24T14:33:22.703 10000 1993-07-24T14:33:22.703 201 123321 false 1993-07-24T14:33:22 xyz {"wTS": "abc72402", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72402-xyz +110229674154037248 10000 1993-07-24T14:33:52.804 10000 1993-07-24T14:33:52.804 201 123321 false 1993-07-24T14:33:52 xyz {"wTS": "abc72432", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72432-xyz +110229800067043328 10000 1993-07-24T14:34:22.824 10000 1993-07-24T14:34:22.824 201 123321 false 1993-07-24T14:34:22 xyz {"wTS": "abc72462", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72462-xyz +110229925980049408 10000 1993-07-24T14:34:52.844 10000 1993-07-24T14:34:52.844 201 123321 false 1993-07-24T14:34:52 xyz {"wTS": "abc72492", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72492-xyz +110230052274737152 10000 1993-07-24T14:35:22.955 10000 1993-07-24T14:35:22.955 201 123321 false 1993-07-24T14:35:22 xyz {"wTS": "abc72522", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72522-xyz +110230178028359680 10000 1993-07-24T14:35:52.937 10000 1993-07-24T14:35:52.937 201 123321 false 1993-07-24T14:35:52 xyz {"wTS": "abc72552", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72552-xyz +110230304310464512 10000 1993-07-24T14:36:23.045 10000 1993-07-24T14:36:23.045 201 123321 false 1993-07-24T14:36:22 xyz {"wTS": "abc72582", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72582-xyz +110230430491906048 10000 1993-07-24T14:36:53.129 10000 1993-07-24T14:36:53.129 201 123321 false 1993-07-24T14:36:52 xyz {"wTS": "abc72612", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72612-xyz +110230556434272256 10000 1993-07-24T14:37:23.156 10000 1993-07-24T14:37:23.156 201 123321 false 1993-07-24T14:37:22 xyz {"wTS": "abc72642", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72642-xyz +110230682510856192 10000 1993-07-24T14:37:53.215 10000 1993-07-24T14:37:53.215 201 123321 false 1993-07-24T14:37:52 xyz {"wTS": "abc72672", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72672-xyz +110230808419667968 10000 1993-07-24T14:38:23.234 10000 1993-07-24T14:38:23.234 201 123321 false 1993-07-24T14:38:22 xyz {"wTS": "abc72702", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72702-xyz +110230935968452608 10000 1993-07-24T14:38:53.644 10000 1993-07-24T14:38:53.644 201 123321 false 1993-07-24T14:38:52 xyz {"wTS": "abc72732", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72732-xyz +110231061294256128 10000 1993-07-24T14:39:23.524 10000 1993-07-24T14:39:23.524 201 123321 false 1993-07-24T14:39:22 xyz {"wTS": "abc72762", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72762-xyz +110231186666196992 10000 1993-07-24T14:39:53.415 10000 1993-07-24T14:39:53.415 201 123321 false 1993-07-24T14:39:52 xyz {"wTS": "abc72792", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72792-xyz +110231312654700544 10000 1993-07-24T14:40:23.453 10000 1993-07-24T14:40:23.453 201 123321 false 1993-07-24T14:40:22 xyz {"wTS": "abc72822", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72822-xyz +110231438513180672 10000 1993-07-24T14:40:53.460 10000 1993-07-24T14:40:53.460 201 123321 false 1993-07-24T14:40:52 xyz {"wTS": "abc72852", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72852-xyz +110233741108480768 10000 1993-07-24T14:50:03.934 10000 1993-07-24T14:50:03.934 201 123321 false 1993-07-24T14:45:53 xyz {"wTS": "abc73153", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73153-xyz +110233743476166656 10000 1993-07-24T14:50:03.006 10000 1993-07-24T14:50:03.006 201 123321 false 1993-07-24T14:41:22 xyz {"wTS": "abc72882", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72882-xyz +110233744231141376 10000 1993-07-24T14:50:03.186 10000 1993-07-24T14:50:03.186 201 123321 false 1993-07-24T14:42:23 xyz {"wTS": "abc72943", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72943-xyz +110233744503771136 10000 1993-07-24T14:50:03.251 10000 1993-07-24T14:50:03.251 201 123321 false 1993-07-24T14:41:52 xyz {"wTS": "abc72912", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72912-xyz +110233745111945216 10000 1993-07-24T14:50:03.396 10000 1993-07-24T14:50:03.396 201 123321 false 1993-07-24T14:42:53 xyz {"wTS": "abc72973", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc72973-xyz +110233745283911680 10000 1993-07-24T14:50:03.437 10000 1993-07-24T14:50:03.437 201 123321 false 1993-07-24T14:43:53 xyz {"wTS": "abc73033", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73033-xyz +110233745485238272 10000 1993-07-24T14:50:03.485 10000 1993-07-24T14:50:03.485 201 123321 false 1993-07-24T14:43:23 xyz {"wTS": "abc73003", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73003-xyz +110233745703342080 10000 1993-07-24T14:50:03.537 10000 1993-07-24T14:50:03.537 201 123321 false 1993-07-24T14:44:53 xyz {"wTS": "abc73093", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73093-xyz +110233746064052224 10000 1993-07-24T14:50:03.623 10000 1993-07-24T14:50:03.623 201 123321 false 1993-07-24T14:45:23 xyz {"wTS": "abc73123", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73123-xyz +110233746315710464 10000 1993-07-24T14:50:03.683 10000 1993-07-24T14:50:03.683 201 123321 false 1993-07-24T14:44:23 xyz {"wTS": "abc73063", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73063-xyz +110233746919690240 10000 1993-07-24T14:50:03.827 10000 1993-07-24T14:50:03.827 201 123321 false 1993-07-24T14:46:53 xyz {"wTS": "abc73213", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73213-xyz +110233747469144064 10000 1993-07-24T14:50:03.958 10000 1993-07-24T14:50:03.958 201 123321 false 1993-07-24T14:46:23 xyz {"wTS": "abc73183", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73183-xyz +110233747863408640 10000 1993-07-24T14:50:04.052 10000 1993-07-24T14:50:04.052 201 123321 false 1993-07-24T14:47:53 xyz {"wTS": "abc73273", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73273-xyz +110233748387696640 10000 1993-07-24T14:50:04.177 10000 1993-07-24T14:50:04.177 201 123321 false 1993-07-24T14:48:23 xyz {"wTS": "abc73303", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73303-xyz +110233748760989696 10000 1993-07-24T14:50:04.266 10000 1993-07-24T14:50:04.266 201 123321 false 1993-07-24T14:47:23 xyz {"wTS": "abc73243", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73243-xyz +110233748987482112 10000 1993-07-24T14:50:04.320 10000 1993-07-24T14:50:04.320 201 123321 false 1993-07-24T14:48:53 xyz {"wTS": "abc73333", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73333-xyz +110233749117505536 10000 1993-07-24T14:50:04.351 10000 1993-07-24T14:50:04.351 201 123321 false 1993-07-24T14:49:23 xyz {"wTS": "abc73363", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73363-xyz +110233749348192256 10000 1993-07-24T14:50:04.406 10000 1993-07-24T14:50:04.406 201 123321 false 1993-07-24T14:49:53 xyz {"wTS": "abc73393", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73393-xyz +110233834433843200 10000 1993-07-24T14:50:24.692 10000 1993-07-24T14:50:24.692 201 123321 false 1993-07-24T14:50:23 xyz {"wTS": "abc73423", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73423-xyz +110233960820805632 10000 1993-07-24T14:50:54.825 10000 1993-07-24T14:50:54.825 201 123321 false 1993-07-24T14:50:54 xyz {"wTS": "abc73454", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73454-xyz +110234212168667136 10000 1993-07-24T14:51:54.751 10000 1993-07-24T14:51:54.751 201 123321 false 1993-07-24T14:51:54 xyz {"wTS": "abc73514", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73514-xyz +110234338161364992 10000 1993-07-24T14:52:24.790 10000 1993-07-24T14:52:24.790 201 123321 false 1993-07-24T14:52:24 xyz {"wTS": "abc73544", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73544-xyz +110234590239035392 10000 1993-07-24T14:53:24.890 10000 1993-07-24T14:53:24.890 201 123321 false 1993-07-24T14:53:24 xyz {"wTS": "abc11004", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc11004-xyz +110234716424671232 10000 1993-07-24T14:53:54.975 10000 1993-07-24T14:53:54.975 201 123321 false 1993-07-24T14:53:54 xyz {"wTS": "abc11034", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc11034-xyz +110235560025034752 10000 1993-07-24T14:57:16.105 10000 1993-07-24T14:57:16.106 201 123321 false 1993-07-24T14:57:15 xyz {"wTS": "abc73835", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73835-xyz +110235688232325120 10000 1993-07-24T14:57:46.672 10000 1993-07-24T14:57:46.672 201 123321 false 1993-07-24T14:57:46 xyz {"wTS": "abc73866", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73866-xyz +110235814371823616 10000 1993-07-24T14:58:16.746 10000 1993-07-24T14:58:16.746 201 123321 false 1993-07-24T14:58:16 xyz {"wTS": "abc73896", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73896-xyz +110235940360327168 10000 1993-07-24T14:58:46.784 10000 1993-07-24T14:58:46.784 201 123321 false 1993-07-24T14:58:46 xyz {"wTS": "abc73926", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73926-xyz +110236067707785216 10000 1993-07-24T14:59:17.146 10000 1993-07-24T14:59:17.146 201 123321 false 1993-07-24T14:59:16 xyz {"wTS": "abc73956", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73956-xyz +110236192224088064 10000 1993-07-24T14:59:46.833 10000 1993-07-24T14:59:46.833 201 123321 false 1993-07-24T14:59:46 xyz {"wTS": "abc73986", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc73986-xyz +110236318577496064 10000 1993-07-24T15:00:16.958 10000 1993-07-24T15:00:16.958 201 123321 false 1993-07-24T15:00:16 xyz {"wTS": "abc74016", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74016-xyz +110236570869075968 10000 1993-07-24T15:01:17.109 10000 1993-07-24T15:01:17.109 201 123321 false 1993-07-24T15:01:16 xyz {"wTS": "abc74076", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74076-xyz +110236697297981440 10000 1993-07-24T15:01:47.252 10000 1993-07-24T15:01:47.252 201 123321 false 1993-07-24T15:01:46 xyz {"wTS": "abc74106", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74106-xyz +110236823001272320 10000 1993-07-24T15:02:17.222 10000 1993-07-24T15:02:17.222 201 123321 false 1993-07-24T15:02:16 xyz {"wTS": "abc74136", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74136-xyz +110236948876529664 10000 1993-07-24T15:02:47.233 10000 1993-07-24T15:02:47.233 201 123321 false 1993-07-24T15:02:46 xyz {"wTS": "abc74166", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74166-xyz +110237074726621184 10000 1993-07-24T15:03:17.238 10000 1993-07-24T15:03:17.238 201 123321 false 1993-07-24T15:03:16 xyz {"wTS": "abc74196", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74196-xyz +110238336427790336 10000 1993-07-24T15:08:18.051 10000 1993-07-24T15:08:18.051 201 123321 false 1993-07-24T15:08:17 xyz {"wTS": "abc74497", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74497-xyz +110238461564850176 10000 1993-07-24T15:08:47.886 10000 1993-07-24T15:08:47.886 201 123321 false 1993-07-24T15:08:47 xyz {"wTS": "abc74527", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74527-xyz +110238587549159424 10000 1993-07-24T15:09:17.923 10000 1993-07-24T15:09:17.923 201 123321 false 1993-07-24T15:09:17 xyz {"wTS": "abc74557", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74557-xyz +110238713772544000 10000 1993-07-24T15:09:48.017 10000 1993-07-24T15:09:48.017 201 123321 false 1993-07-24T15:09:47 xyz {"wTS": "abc74587", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74587-xyz +110238839517777920 10000 1993-07-24T15:10:17.997 10000 1993-07-24T15:10:17.997 201 123321 false 1993-07-24T15:10:17 xyz {"wTS": "abc74617", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74617-xyz +110238965250428928 10000 1993-07-24T15:10:47.974 10000 1993-07-24T15:10:47.974 201 123321 false 1993-07-24T15:10:47 xyz {"wTS": "abc74647", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74647-xyz +110239216921251840 10000 1993-07-24T15:11:47.977 10000 1993-07-24T15:11:47.977 201 123321 false 1993-07-24T15:11:46 xyz {"wTS": "abc74706", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74706-xyz +110239342381273088 10000 1993-07-24T15:12:17.889 10000 1993-07-24T15:12:17.889 201 123321 false 1993-07-24T15:12:16 xyz {"wTS": "abc74110", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74110-xyz +110239467153428480 10000 1993-07-24T15:12:47.637 10000 1993-07-24T15:12:47.637 201 123321 false 1993-07-24T15:12:46 xyz {"wTS": "abc74766", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74766-xyz +110239593200652288 10000 1993-07-24T15:13:17.689 10000 1993-07-24T15:13:17.689 201 123321 false 1993-07-24T15:13:17 xyz {"wTS": "abc74797", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74797-xyz +110239719096881152 10000 1993-07-24T15:13:47.705 10000 1993-07-24T15:13:47.705 201 123321 false 1993-07-24T15:13:47 xyz {"wTS": "abc74827", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74827-xyz +110239845207019520 10000 1993-07-24T15:14:17.772 10000 1993-07-24T15:14:17.772 201 123321 false 1993-07-24T15:14:17 xyz {"wTS": "abc74857", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74857-xyz +110239971363295232 10000 1993-07-24T15:14:47.850 10000 1993-07-24T15:14:47.850 201 123321 false 1993-07-24T15:14:47 xyz {"wTS": "abc74887", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74887-xyz +110240098626867200 10000 1993-07-24T15:15:18.192 10000 1993-07-24T15:15:18.192 201 123321 false 1993-07-24T15:15:17 xyz {"wTS": "abc74917", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74917-xyz +110240224049139712 10000 1993-07-24T15:15:48.095 10000 1993-07-24T15:15:48.095 201 123321 false 1993-07-24T15:15:47 xyz {"wTS": "abc74947", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74947-xyz +110240349840510976 10000 1993-07-24T15:16:18.086 10000 1993-07-24T15:16:18.086 201 123321 false 1993-07-24T15:16:17 xyz {"wTS": "abc74977", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc74977-xyz +110240476395245568 10000 1993-07-24T15:16:48.259 10000 1993-07-24T15:16:48.259 201 123321 false 1993-07-24T15:16:47 xyz {"wTS": "abc75007", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75007-xyz +110240601515528192 10000 1993-07-24T15:17:18.090 10000 1993-07-24T15:17:18.090 201 123321 false 1993-07-24T15:17:17 xyz {"wTS": "abc75037", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75037-xyz +110240727730524160 10000 1993-07-24T15:17:48.182 10000 1993-07-24T15:17:48.182 201 123321 false 1993-07-24T15:17:47 xyz {"wTS": "abc75067", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75067-xyz +110240853538672640 10000 1993-07-24T15:18:18.178 10000 1993-07-24T15:18:18.178 201 123321 false 1993-07-24T15:18:17 xyz {"wTS": "abc75097", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75097-xyz +110240979623645184 10000 1993-07-24T15:18:48.238 10000 1993-07-24T15:18:48.238 201 123321 false 1993-07-24T15:18:47 xyz {"wTS": "abc75127", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75127-xyz +110241106841079808 10000 1993-07-24T15:19:18.569 10000 1993-07-24T15:19:18.569 201 123321 false 1993-07-24T15:19:17 xyz {"wTS": "abc75157", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75157-xyz +110241110250920960 10000 1993-07-24T15:21:48.632 10000 1993-07-24T15:21:48.632 201 123321 false 1993-07-24T15:21:47 xyz {"wTS": "abc75307", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75307-xyz +110241231827144704 10000 1993-07-24T15:19:48.368 10000 1993-07-24T15:19:48.368 201 123321 false 1993-07-24T15:19:47 xyz {"wTS": "abc75187", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75187-xyz +110241357886951424 10000 1993-07-24T15:20:18.423 10000 1993-07-24T15:20:18.423 201 123321 false 1993-07-24T15:20:17 xyz {"wTS": "abc75217", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75217-xyz +110241484072587264 10000 1993-07-24T15:20:48.508 10000 1993-07-24T15:20:48.508 201 123321 false 1993-07-24T15:20:47 xyz {"wTS": "abc75247", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75247-xyz +110241610224668672 10000 1993-07-24T15:21:18.585 10000 1993-07-24T15:21:18.585 201 123321 false 1993-07-24T15:21:17 xyz {"wTS": "abc75277", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75277-xyz +110241862428168192 10000 1993-07-24T15:22:18.715 10000 1993-07-24T15:22:18.715 201 123321 false 1993-07-24T15:22:17 xyz {"wTS": "abc75337", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75337-xyz +110241988550889472 10000 1993-07-24T15:22:48.785 10000 1993-07-24T15:22:48.785 201 123321 false 1993-07-24T15:22:48 xyz {"wTS": "abc75368", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75368-xyz +110242114287734784 10000 1993-07-24T15:23:18.763 10000 1993-07-24T15:23:18.763 201 123321 false 1993-07-24T15:23:18 xyz {"wTS": "abc75398", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75398-xyz +110242240129437696 10000 1993-07-24T15:23:48.766 10000 1993-07-24T15:23:48.766 201 123321 false 1993-07-24T15:23:48 xyz {"wTS": "abc75428", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75428-xyz +110242366264741888 10000 1993-07-24T15:24:18.839 10000 1993-07-24T15:24:18.839 201 123321 false 1993-07-24T15:24:18 xyz {"wTS": "abc75458", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75458-xyz +110242492534263808 10000 1993-07-24T15:24:48.945 10000 1993-07-24T15:24:48.945 201 123321 false 1993-07-24T15:24:48 xyz {"wTS": "abc75488", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75488-xyz +110242618149474304 10000 1993-07-24T15:25:18.893 10000 1993-07-24T15:25:18.893 201 123321 false 1993-07-24T15:25:18 xyz {"wTS": "abc75518", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75518-xyz +110242744481910784 10000 1993-07-24T15:25:49.013 10000 1993-07-24T15:25:49.013 201 123321 false 1993-07-24T15:25:48 xyz {"wTS": "abc75548", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75548-xyz +110242870382333952 10000 1993-07-24T15:26:19.030 10000 1993-07-24T15:26:19.030 201 123321 false 1993-07-24T15:26:18 xyz {"wTS": "abc75578", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75578-xyz +110242996479889408 10000 1993-07-24T15:26:49.094 10000 1993-07-24T15:26:49.094 201 123321 false 1993-07-24T15:26:48 xyz {"wTS": "abc75608", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75608-xyz +110243122464198656 10000 1993-07-24T15:27:19.131 10000 1993-07-24T15:27:19.131 201 123321 false 1993-07-24T15:27:18 xyz {"wTS": "abc75638", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75638-xyz +110243248985378816 10000 1993-07-24T15:27:49.296 10000 1993-07-24T15:27:49.296 201 123321 false 1993-07-24T15:27:48 xyz {"wTS": "abc75668", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75668-xyz +110243374441205760 10000 1993-07-24T15:28:19.207 10000 1993-07-24T15:28:19.207 201 123321 false 1993-07-24T15:28:18 xyz {"wTS": "abc75698", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75698-xyz +110243500798808064 10000 1993-07-24T15:28:49.333 10000 1993-07-24T15:28:49.333 201 123321 false 1993-07-24T15:28:48 xyz {"wTS": "abc75728", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75728-xyz +110243627026386944 10000 1993-07-24T15:29:19.428 10000 1993-07-24T15:29:19.428 201 123321 false 1993-07-24T15:29:18 xyz {"wTS": "abc75758", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75758-xyz +110243754067660800 10000 1993-07-24T15:29:49.717 10000 1993-07-24T15:29:49.717 201 123321 false 1993-07-24T15:29:48 xyz {"wTS": "abc75788", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75788-xyz +110243878957256704 10000 1993-07-24T15:30:19.493 10000 1993-07-24T15:30:19.493 201 123321 false 1993-07-24T15:30:18 xyz {"wTS": "abc75818", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75818-xyz +110244004870262784 10000 1993-07-24T15:30:49.513 10000 1993-07-24T15:30:49.513 201 123321 false 1993-07-24T15:30:48 xyz {"wTS": "abc75848", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75848-xyz +110244130875543552 10000 1993-07-24T15:31:19.555 10000 1993-07-24T15:31:19.555 201 123321 false 1993-07-24T15:31:18 xyz {"wTS": "abc75878", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75878-xyz +110244256989876224 10000 1993-07-24T15:31:49.623 10000 1993-07-24T15:31:49.624 201 123321 false 1993-07-24T15:31:49 xyz {"wTS": "abc75909", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75909-xyz +110244383129374720 10000 1993-07-24T15:32:19.697 10000 1993-07-24T15:32:19.697 201 123321 false 1993-07-24T15:32:19 xyz {"wTS": "abc75939", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75939-xyz +110244509516337152 10000 1993-07-24T15:32:49.830 10000 1993-07-24T15:32:49.830 201 123321 false 1993-07-24T15:32:49 xyz {"wTS": "abc75969", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75969-xyz +110244635370622976 10000 1993-07-24T15:33:19.836 10000 1993-07-24T15:33:19.836 201 123321 false 1993-07-24T15:33:19 xyz {"wTS": "abc75999", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc75999-xyz +110244761581424640 10000 1993-07-24T15:33:49.927 10000 1993-07-24T15:33:49.927 201 123321 false 1993-07-24T15:33:49 xyz {"wTS": "abc76029", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76029-xyz +110244887590899712 10000 1993-07-24T15:34:19.970 10000 1993-07-24T15:34:19.970 201 123321 false 1993-07-24T15:34:19 xyz {"wTS": "abc76059", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76059-xyz +110245013550043136 10000 1993-07-24T15:34:50.001 10000 1993-07-24T15:34:50.002 201 123321 false 1993-07-24T15:34:49 xyz {"wTS": "abc76089", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76089-xyz +110245139765039104 10000 1993-07-24T15:35:20.093 10000 1993-07-24T15:35:20.093 201 123321 false 1993-07-24T15:35:19 xyz {"wTS": "abc76119", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76119-xyz +110245265950674944 10000 1993-07-24T15:35:50.178 10000 1993-07-24T15:35:50.178 201 123321 false 1993-07-24T15:35:49 xyz {"wTS": "abc76149", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76149-xyz +110245392366997504 10000 1993-07-24T15:36:20.318 10000 1993-07-24T15:36:20.318 201 123321 false 1993-07-24T15:36:19 xyz {"wTS": "abc76179", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76179-xyz +110245517818630144 10000 1993-07-24T15:36:50.228 10000 1993-07-24T15:36:50.228 201 123321 false 1993-07-24T15:36:49 xyz {"wTS": "abc76209", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76209-xyz +110245644071374848 10000 1993-07-24T15:37:20.329 10000 1993-07-24T15:37:20.329 201 123321 false 1993-07-24T15:37:19 xyz {"wTS": "abc76239", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76239-xyz +110245770043101184 10000 1993-07-24T15:37:50.363 10000 1993-07-24T15:37:50.363 201 123321 false 1993-07-24T15:37:49 xyz {"wTS": "abc76269", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76269-xyz +110245896203571200 10000 1993-07-24T15:38:20.442 10000 1993-07-24T15:38:20.442 201 123321 false 1993-07-24T15:38:19 xyz {"wTS": "abc76299", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76299-xyz +110246022175297536 10000 1993-07-24T15:38:50.476 10000 1993-07-24T15:38:50.476 201 123321 false 1993-07-24T15:38:49 xyz {"wTS": "abc76329", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76329-xyz +110246147996028928 10000 1993-07-24T15:39:20.474 10000 1993-07-24T15:39:20.474 201 123321 false 1993-07-24T15:39:19 xyz {"wTS": "abc76359", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76359-xyz +110246274026475520 10000 1993-07-24T15:39:50.522 10000 1993-07-24T15:39:50.522 201 123321 false 1993-07-24T15:39:49 xyz {"wTS": "abc76389", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76389-xyz +110246400031756288 10000 1993-07-24T15:40:20.564 10000 1993-07-24T15:40:20.565 201 123321 false 1993-07-24T15:40:19 xyz {"wTS": "abc76419", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76419-xyz +110246526204809216 10000 1993-07-24T15:40:50.646 10000 1993-07-24T15:40:50.646 201 123321 false 1993-07-24T15:40:49 xyz {"wTS": "abc76449", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76449-xyz +110246651966820352 10000 1993-07-24T15:41:20.630 10000 1993-07-24T15:41:20.630 201 123321 false 1993-07-24T15:41:20 xyz {"wTS": "abc76480", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76480-xyz +110246778110513152 10000 1993-07-24T15:41:50.705 10000 1993-07-24T15:41:50.705 201 123321 false 1993-07-24T15:41:50 xyz {"wTS": "abc76510", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76510-xyz +110246904396812288 10000 1993-07-24T15:42:20.814 10000 1993-07-24T15:42:20.814 201 123321 false 1993-07-24T15:42:20 xyz {"wTS": "abc76540", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76540-xyz +110247030221737984 10000 1993-07-24T15:42:50.813 10000 1993-07-24T15:42:50.813 201 123321 false 1993-07-24T15:42:50 xyz {"wTS": "abc76570", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76570-xyz +110247157414006784 10000 1993-07-24T15:43:21.138 10000 1993-07-24T15:43:21.138 201 123321 false 1993-07-24T15:43:20 xyz {"wTS": "abc76600", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76600-xyz +110247282832084992 10000 1993-07-24T15:43:51.040 10000 1993-07-24T15:43:51.040 201 123321 false 1993-07-24T15:43:50 xyz {"wTS": "abc76630", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76630-xyz +110247408644427776 10000 1993-07-24T15:44:21.036 10000 1993-07-24T15:44:21.036 201 123321 false 1993-07-24T15:44:20 xyz {"wTS": "abc76660", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76660-xyz +110247534695845888 10000 1993-07-24T15:44:51.089 10000 1993-07-24T15:44:51.089 201 123321 false 1993-07-24T15:44:50 xyz {"wTS": "abc76690", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76690-xyz +110247660474634240 10000 1993-07-24T15:45:21.077 10000 1993-07-24T15:45:21.077 201 123321 false 1993-07-24T15:45:20 xyz {"wTS": "abc76720", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76720-xyz +110247786630909952 10000 1993-07-24T15:45:51.155 10000 1993-07-24T15:45:51.155 201 123321 false 1993-07-24T15:45:50 xyz {"wTS": "abc76750", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76750-xyz +110247912862683136 10000 1993-07-24T15:46:21.251 10000 1993-07-24T15:46:21.251 201 123321 false 1993-07-24T15:46:20 xyz {"wTS": "abc76780", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76780-xyz +110248038553391104 10000 1993-07-24T15:46:51.218 10000 1993-07-24T15:46:51.218 201 123321 false 1993-07-24T15:46:50 xyz {"wTS": "abc76810", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76810-xyz +110248164768387072 10000 1993-07-24T15:47:21.310 10000 1993-07-24T15:47:21.310 201 123321 false 1993-07-24T15:47:20 xyz {"wTS": "abc76840", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76840-xyz +110248290807222272 10000 1993-07-24T15:47:51.360 10000 1993-07-24T15:47:51.360 201 123321 false 1993-07-24T15:47:50 xyz {"wTS": "abc76870", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76870-xyz +110248417680723968 10000 1993-07-24T15:48:21.609 10000 1993-07-24T15:48:21.609 201 123321 false 1993-07-24T15:48:20 xyz {"wTS": "abc76900", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76900-xyz +110248543044276224 10000 1993-07-24T15:48:51.498 10000 1993-07-24T15:48:51.498 201 123321 false 1993-07-24T15:48:50 xyz {"wTS": "abc76930", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76930-xyz +110248668856619008 10000 1993-07-24T15:49:21.494 10000 1993-07-24T15:49:21.494 201 123321 false 1993-07-24T15:49:20 xyz {"wTS": "abc76960", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76960-xyz +110248794903842816 10000 1993-07-24T15:49:51.546 10000 1993-07-24T15:49:51.546 201 123321 false 1993-07-24T15:49:50 xyz {"wTS": "abc76990", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc76990-xyz +110248922033197056 10000 1993-07-24T15:50:21.856 10000 1993-07-24T15:50:21.856 201 123321 false 1993-07-24T15:50:20 xyz {"wTS": "abc77020", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc77020-xyz +110249047547744256 10000 1993-07-24T15:50:51.781 10000 1993-07-24T15:50:51.781 201 123321 false 1993-07-24T15:50:51 xyz {"wTS": "abc77051", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc77051-xyz +110249173741768704 10000 1993-07-24T15:51:21.868 10000 1993-07-24T15:51:21.868 201 123321 false 1993-07-24T15:51:21 xyz {"wTS": "abc77081", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc77081-xyz +110249299239538688 10000 1993-07-24T15:51:51.789 10000 1993-07-24T15:51:51.789 201 123321 false 1993-07-24T15:51:51 xyz {"wTS": "abc77111", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc77111-xyz +110249425433563136 10000 1993-07-24T15:52:21.876 10000 1993-07-24T15:52:21.876 201 123321 false 1993-07-24T15:52:21 xyz {"wTS": "abc77141", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc77141-xyz +110249551396900864 10000 1993-07-24T15:52:51.908 10000 1993-07-24T15:52:51.908 201 123321 false 1993-07-24T15:52:51 xyz {"wTS": "abc77171", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc77171-xyz +110249677473484800 10000 1993-07-24T15:53:21.967 10000 1993-07-24T15:53:21.967 201 123321 false 1993-07-24T15:53:21 xyz {"wTS": "abc77201", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc77201-xyz +110249803487154176 10000 1993-07-24T15:53:52.011 10000 1993-07-24T15:53:52.011 201 123321 false 1993-07-24T15:53:51 xyz {"wTS": "abc77231", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc77231-xyz +110249929664401408 10000 1993-07-24T15:54:22.094 10000 1993-07-24T15:54:22.094 201 123321 false 1993-07-24T15:54:21 xyz {"wTS": "abc77261", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc77261-xyz +110250055740985344 10000 1993-07-24T15:54:52.153 10000 1993-07-24T15:54:52.153 201 123321 false 1993-07-24T15:54:51 xyz {"wTS": "abc77291", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc77291-xyz +110250181804986368 10000 1993-07-24T15:55:22.209 10000 1993-07-24T15:55:22.209 201 123321 false 1993-07-24T15:55:21 xyz {"wTS": "abc77321", "rfId": "xyz", "remark": "数据", "weight": 100} 1233211234567891011 abc77321-xyz +110708314073595904 10000 1993-07-25T22:15:49.453 10000 1993-07-25T22:15:49.453 201 123321 false 1993-07-25T22:15:48 0410-137ijk {"wTS": "1595686548", "rfId": "0410-137ijk", "remark": "数据", "weight": 0.9} 1233211234567891011 1595686548-0410-137ijk +110708469292204032 10000 1993-07-25T22:16:26.460 10000 1993-07-25T22:16:26.461 201 123321 false 1993-07-25T22:16:25 0410-137ijk {"wTS": "1595686585", "rfId": "0410-137ijk", "remark": "数据", "weight": 0.9} 1233211234567891011 1595686585-0410-137ijk +135164473750999040 10000 1993-07-21T16:01:09.237 10000 1993-07-21T16:01:09.237 201 123321 false 0410-137ijk {"wTS": "1595318468052", "rfId": "0410-137ijk", "remark": "数据", "weight": 0.2} 1233211234567891011 1595318468052-0410-137ijk +135165117484388352 10000 1993-07-21T16:03:42.715 10000 1993-07-21T16:03:42.715 201 123321 false 1993-07-21T16:03:41 0410-137ijk {"wTS": "1595318621", "rfId": "0410-137ijk", "remark": "数据", "weight": 0.2} 1233211234567891011 1595318621-0410-137ijk +135167443414700032 10000 1993-07-21T16:12:57.260 10000 1993-07-21T16:12:57.260 201 123321 false 1993-07-21T16:12:56 0410-137ijk {"wTS": "1595319176", "rfId": "0410-137ijk", "remark": "数据", "weight": 0.2} 1233211234567891011 1595319176-0410-137ijk +135167805882257408 10000 1993-07-21T16:14:23.679 10000 1993-07-21T16:14:23.679 201 123321 false 1993-07-21T16:14:22 0410-137ijk {"wTS": "1595319262", "rfId": "0410-137ijk", "remark": "数据", "weight": 2.0} 1233211234567891011 1595319262-0410-137ijk +135167875595784192 10000 1993-07-21T16:14:40.300 10000 1993-07-21T16:14:40.300 201 123321 false 1993-07-21T16:14:39 0410-137ijk {"wTS": "1595319279", "rfId": "0410-137ijk", "remark": "数据", "weight": 2.0} 1233211234567891011 1595319279-0410-137ijk +138903405079396352 10000 1993-07-31T23:38:19.932 10000 1993-07-31T23:38:19.932 201 123321 false 1993-07-31T23:38:18 0410-137110005 {"wTS": "1596209898", "rfId": "0410-137110005", "remark": "数据", "weight": 30.0} 1233211234567891011 1596209898-0410-137110005 +138905785832038400 10000 1993-07-31T23:47:47.547 10000 1993-07-31T23:47:47.547 201 123321 false 1993-07-31T23:47:45 0410-137110005 {"wTS": "1596210465", "rfId": "0410-137110005", "remark": "数据", "weight": 500.0} 1233211234567891011 1596210465-0410-137110005 +138905859697926144 10000 1993-07-31T23:48:05.158 10000 1993-07-31T23:48:05.159 201 123321 false 1993-07-31T23:48:03 0410-137110005 {"wTS": "1596210483", "rfId": "0410-137110005", "remark": "数据", "weight": 500.001} 1233211234567891011 1596210483-0410-137110005 +138906169749266432 10000 1993-07-31T23:49:19.080 10000 1993-07-31T23:49:19.080 201 123321 false 1993-07-31T23:49:17 0410-137110005 {"wTS": "1596210557", "rfId": "0410-137110005", "remark": "数据", "weight": 0.5} 1233211234567891011 1596210557-0410-137110005 +13978385225 10000 1993-08-03T15:28:11.301 10000 1993-08-03T15:28:11.301 201 456678 false 1993-08-03T15:28:10 010-18741038 {"wTS": "1596439690", "rfId": "010-18741038", "remark": "数据", "weight": 70.404} 710078936338096128 1596439690-010-18741038 +139866186189201408 10000 1993-08-03T15:24:04.836 10000 1993-08-03T15:24:04.836 201 456678 false 1993-08-03T15:24:04 010-18741039 {"wTS": "1596439444", "rfId": "010-18741039", "remark": "数据", "weight": 100} 710078936338096128 1596439444-010-18741039 +139866186201784320 10000 1993-08-03T15:24:04.839 10000 1993-08-03T15:24:04.839 201 456678 false 1993-08-03T15:24:04 010-18741035 {"wTS": "1596439444", "rfId": "010-18741035", "remark": "数据", "weight": 100} 710078936338096128 1596439444-010-18741035 +139866186575077376 10000 1993-08-03T15:24:04.928 10000 1993-08-03T15:24:04.928 201 456678 false 1993-08-03T15:24:04 010-18741036 {"wTS": "1596439444", "rfId": "010-18741036", "remark": "数据", "weight": 100} 710078936338096128 1596439444-010-18741036 +139866187183251456 10000 1993-08-03T15:24:05.073 10000 1993-08-03T15:24:05.073 201 456678 false 1993-08-03T15:24:04 010-18741040 {"wTS": "1596439444", "rfId": "010-18741040", "remark": "数据", "weight": 100} 710078936338096128 1596439444-010-18741040 +139866187455881216 10000 1993-08-03T15:24:05.138 10000 1993-08-03T15:24:05.138 201 456678 false 1993-08-03T15:24:04 010-18741033 {"wTS": "1596439444", "rfId": "010-18741033", "remark": "数据", "weight": 100} 710078936338096128 1596439444-010-18741033 +139866187862728704 10000 1993-08-03T15:24:05.235 10000 1993-08-03T15:24:05.235 201 456678 false 1993-08-03T15:24:04 010-18741034 {"wTS": "1596439444", "rfId": "010-18741034", "remark": "数据", "weight": 100} 710078936338096128 1596439444-010-18741034 +139866188911304704 10000 1993-08-03T15:24:05.486 10000 1993-08-03T15:24:05.486 201 456678 false 1993-08-03T15:24:04 010-18741038 {"wTS": "1596439444", "rfId": "010-18741038", "remark": "数据", "weight": 100} 710078936338096128 1596439444-010-18741038 +139866189112631296 10000 1993-08-03T15:24:05.533 10000 1993-08-03T15:24:05.533 201 456678 false 1993-08-03T15:24:04 010-18741032 {"wTS": "1596439444", "rfId": "010-18741032", "remark": "数据", "weight": 100} 710078936338096128 1596439444-010-18741032 +139866189481730048 10000 1993-08-03T15:24:05.621 10000 1993-08-03T15:24:05.621 201 456678 false 1993-08-03T15:24:04 010-18741031 {"wTS": "1596439444", "rfId": "010-18741031", "remark": "数据", "weight": 100} 710078936338096128 1596439444-010-18741031 +139866190014406656 10000 1993-08-03T15:24:05.748 10000 1993-08-03T15:24:05.748 201 456678 false 1993-08-03T15:24:04 010-18741037 {"wTS": "1596439444", "rfId": "010-18741037", "remark": "数据", "weight": 100} 710078936338096128 1596439444-010-18741037 +139866315625422848 10000 1993-08-03T15:24:35.696 10000 1993-08-03T15:24:35.696 201 456678 false 1993-08-03T15:24:35 010-18741039 {"wTS": "1596439475", "rfId": "010-18741039", "remark": "数据", "weight": 100} 710078936338096128 1596439475-010-18741039 +139866315784806400 10000 1993-08-03T15:24:35.734 10000 1993-08-03T15:24:35.734 201 456678 false 1993-08-03T15:24:35 010-18741040 {"wTS": "1596439475", "rfId": "010-18741040", "remark": "数据", "weight": 100} 710078936338096128 1596439475-010-18741040 +139866316082601984 10000 1993-08-03T15:24:35.805 10000 1993-08-03T15:24:35.805 201 456678 false 1993-08-03T15:24:35 010-18741037 {"wTS": "1596439475", "rfId": "010-18741037", "remark": "数据", "weight": 100} 710078936338096128 1596439475-010-18741037 +139866316590112768 10000 1993-08-03T15:24:35.926 10000 1993-08-03T15:24:35.926 201 456678 false 1993-08-03T15:24:35 010-18741034 {"wTS": "1596439475", "rfId": "010-18741034", "remark": "数据", "weight": 100} 710078936338096128 1596439475-010-18741034 +139866317424779264 10000 1993-08-03T15:24:36.125 10000 1993-08-03T15:24:36.125 201 456678 false 1993-08-03T15:24:35 010-18741036 {"wTS": "1596439475", "rfId": "010-18741036", "remark": "数据", "weight": 100} 710078936338096128 1596439475-010-18741036 +139866317953261568 10000 1993-08-03T15:24:36.251 10000 1993-08-03T15:24:36.251 201 456678 false 1993-08-03T15:24:35 010-18741033 {"wTS": "1596439475", "rfId": "010-18741033", "remark": "数据", "weight": 100} 710078936338096128 1596439475-010-18741033 +139866318209114112 10000 1993-08-03T15:24:36.312 10000 1993-08-03T15:24:36.312 201 456678 false 1993-08-03T15:24:35 010-18741032 {"wTS": "1596439475", "rfId": "010-18741032", "remark": "数据", "weight": 100} 710078936338096128 1596439475-010-18741032 +139866318750179328 10000 1993-08-03T15:24:36.441 10000 1993-08-03T15:24:36.441 201 456678 false 1993-08-03T15:24:35 010-18741035 {"wTS": "1596439475", "rfId": "010-18741035", "remark": "数据", "weight": 100} 710078936338096128 1596439475-010-18741035 +139866319706480640 10000 1993-08-03T15:24:36.669 10000 1993-08-03T15:24:36.669 201 456678 false 1993-08-03T15:24:35 010-18741031 {"wTS": "1596439475", "rfId": "010-18741031", "remark": "数据", "weight": 100} 710078936338096128 1596439475-010-18741031 +139866320658587648 10000 1993-08-03T15:24:36.896 10000 1993-08-03T15:24:36.896 201 456678 false 1993-08-03T15:24:35 010-18741038 {"wTS": "1596439475", "rfId": "010-18741038", "remark": "数据", "weight": 100} 710078936338096128 1596439475-010-18741038 +139866709332156416 10000 1993-08-03T15:26:09.563 10000 1993-08-03T15:26:09.563 201 456678 false 1993-08-03T15:26:09 010-18741039 {"wTS": "1596439569", "rfId": "010-18741039", "remark": "数据", "weight": 166.655} 710078936338096128 1596439569-010-18741039 +139866711446085632 10000 1993-08-03T15:26:10.067 10000 1993-08-03T15:26:10.067 201 456678 false 1993-08-03T15:26:09 010-18741034 {"wTS": "1596439569", "rfId": "010-18741034", "remark": "数据", "weight": 161.756} 710078936338096128 1596439569-010-18741034 +139866715170627584 10000 1993-08-03T15:26:10.955 10000 1993-08-03T15:26:10.955 201 456678 false 1993-08-03T15:26:09 010-18741035 {"wTS": "1596439569", "rfId": "010-18741035", "remark": "数据", "weight": 61.382} 710078936338096128 1596439569-010-18741035 +139866715187404800 10000 1993-08-03T15:26:10.959 10000 1993-08-03T15:26:10.959 201 456678 false 1993-08-03T15:26:09 010-18741036 {"wTS": "1596439569", "rfId": "010-18741036", "remark": "数据", "weight": 134.936} 710078936338096128 1596439569-010-18741036 +139866715195793408 10000 1993-08-03T15:26:10.961 10000 1993-08-03T15:26:10.961 201 456678 false 1993-08-03T15:26:09 010-18741037 {"wTS": "1596439569", "rfId": "010-18741037", "remark": "数据", "weight": 156.557} 710078936338096128 1596439569-010-18741037 +139866715225153536 10000 1993-08-03T15:26:10.968 10000 1993-08-03T15:26:10.968 201 456678 false 1993-08-03T15:26:09 010-18741038 {"wTS": "1596439569", "rfId": "010-18741038", "remark": "数据", "weight": 70.404} 710078936338096128 1596439569-010-18741038 +139866715313233920 10000 1993-08-03T15:26:10.989 10000 1993-08-03T15:26:10.989 201 456678 false 1993-08-03T15:26:09 010-18741040 {"wTS": "1596439569", "rfId": "010-18741040", "remark": "数据", "weight": 76.74} 710078936338096128 1596439569-010-18741040 +139866715413897216 10000 1993-08-03T15:26:11.013 10000 1993-08-03T15:26:11.013 201 456678 false 1993-08-03T15:26:09 010-18741032 {"wTS": "1596439569", "rfId": "010-18741032", "remark": "数据", "weight": 101.273} 710078936338096128 1596439569-010-18741032 +139866715711692800 10000 1993-08-03T15:26:11.084 10000 1993-08-03T15:26:11.084 201 456678 false 1993-08-03T15:26:09 010-18741033 {"wTS": "1596439569", "rfId": "010-18741033", "remark": "数据", "weight": 69.742} 710078936338096128 1596439569-010-18741033 +139866716114345984 10000 1993-08-03T15:26:11.180 10000 1993-08-03T15:26:11.180 201 456678 false 1993-08-03T15:26:09 010-18741031 {"wTS": "1596439569", "rfId": "010-18741031", "remark": "数据", "weight": 79.699} 710078936338096128 1596439569-010-18741031 +139866838638354432 10000 1993-08-03T15:26:40.392 10000 1993-08-03T15:26:40.392 201 456678 false 1993-08-03T15:26:40 010-18741039 {"wTS": "1596439600", "rfId": "010-18741039", "remark": "数据", "weight": 166.655} 710078936338096128 1596439600-010-18741039 +139866838952927232 10000 1993-08-03T15:26:40.467 10000 1993-08-03T15:26:40.467 201 456678 false 1993-08-03T15:26:40 010-18741031 {"wTS": "1596439600", "rfId": "010-18741031", "remark": "数据", "weight": 79.699} 710078936338096128 1596439600-010-18741031 +139866839275888640 10000 1993-08-03T15:26:40.544 10000 1993-08-03T15:26:40.544 201 456678 false 1993-08-03T15:26:40 010-18741037 {"wTS": "1596439600", "rfId": "010-18741037", "remark": "数据", "weight": 156.557} 710078936338096128 1596439600-010-18741037 +139866840232189952 10000 1993-08-03T15:26:40.772 10000 1993-08-03T15:26:40.772 201 456678 false 1993-08-03T15:26:40 010-18741032 {"wTS": "1596439600", "rfId": "010-18741032", "remark": "数据", "weight": 101.273} 710078936338096128 1596439600-010-18741032 +139866841603721100 10000 1993-08-03T15:26:41.099 10000 1993-08-03T15:26:41.099 201 456678 false 1993-08-03T15:26:40 010-18741035 {"wTS": "1596439600", "rfId": "010-18741035", "remark": "数据", "weight": 61.382} 710078936338096128 1596439600-010-18741035 +139866841725362176 10000 1993-08-03T15:26:41.128 10000 1993-08-03T15:26:41.128 201 456678 false 1993-08-03T15:26:40 010-18741034 {"wTS": "1596439600", "rfId": "010-18741034", "remark": "数据", "weight": 161.756} 710078936338096128 1596439600-010-18741034 +139866841767305216 10000 1993-08-03T15:26:41.138 10000 1993-08-03T15:26:41.138 201 456678 false 1993-08-03T15:26:40 010-18741033 {"wTS": "1596439600", "rfId": "010-18741033", "remark": "数据", "weight": 69.742} 710078936338096128 1596439600-010-18741033 +139866841779888128 10000 1993-08-03T15:26:41.141 10000 1993-08-03T15:26:41.141 201 456678 false 1993-08-03T15:26:40 010-18741036 {"wTS": "1596439600", "rfId": "010-18741036", "remark": "数据", "weight": 134.936} 710078936338096128 1596439600-010-18741036 +139866842065100800 10000 1993-08-03T15:26:41.209 10000 1993-08-03T15:26:41.209 201 456678 false 1993-08-03T15:26:40 010-18741038 {"wTS": "1596439600", "rfId": "010-18741038", "remark": "数据", "weight": 70.404} 710078936338096128 1596439600-010-18741038 +139866843038179328 10000 1993-08-03T15:26:41.441 10000 1993-08-03T15:26:41.441 201 456678 false 1993-08-03T15:26:40 010-18741040 {"wTS": "1596439600", "rfId": "010-18741040", "remark": "数据", "weight": 76.74} 710078936338096128 1596439600-010-18741040 +139866964631052288 10000 1993-08-03T15:27:10.431 10000 1993-08-03T15:27:10.431 201 456678 false 1993-08-03T15:27:10 010-18741031 {"wTS": "1596439630", "rfId": "010-18741031", "remark": "数据", "weight": 79.699} 710078936338096128 1596439630-010-18741031 +139866965037899776 10000 1993-08-03T15:27:10.528 10000 1993-08-03T15:27:10.528 201 456678 false 1993-08-03T15:27:10 010-18741039 {"wTS": "1596439630", "rfId": "010-18741039", "remark": "数据", "weight": 166.655} 710078936338096128 1596439630-010-18741039 +139866965356666880 10000 1993-08-03T15:27:10.604 10000 1993-08-03T15:27:10.604 201 456678 false 1993-08-03T15:27:10 010-18741036 {"wTS": "1596439630", "rfId": "010-18741036", "remark": "数据", "weight": 134.936} 710078936338096128 1596439630-010-18741036 +139866966094864384 10000 1993-08-03T15:27:10.780 10000 1993-08-03T15:27:10.780 201 456678 false 1993-08-03T15:27:10 010-18741032 {"wTS": "1596439630", "rfId": "010-18741032", "remark": "数据", "weight": 101.273} 710078936338096128 1596439630-010-18741032 +139866966635929600 10000 1993-08-03T15:27:10.909 10000 1993-08-03T15:27:10.909 201 456678 false 1993-08-03T15:27:10 010-18741037 {"wTS": "1596439630", "rfId": "010-18741037", "remark": "数据", "weight": 156.557} 710078936338096128 1596439630-010-18741037 +139866967017611264 10000 1993-08-03T15:27:11 10000 1993-08-03T15:27:11 201 456678 false 1993-08-03T15:27:10 010-18741040 {"wTS": "1596439630", "rfId": "010-18741040", "remark": "数据", "weight": 76.74} 710078936338096128 1596439630-010-18741040 +139866967785168896 10000 1993-08-03T15:27:11.183 10000 1993-08-03T15:27:11.183 201 456678 false 1993-08-03T15:27:10 010-18741033 {"wTS": "1596439630", "rfId": "010-18741033", "remark": "数据", "weight": 69.742} 710078936338096128 1596439630-010-18741033 +139866968745664512 10000 1993-08-03T15:27:11.412 10000 1993-08-03T15:27:11.412 201 456678 false 1993-08-03T15:27:10 010-18741038 {"wTS": "1596439630", "rfId": "010-18741038", "remark": "数据", "weight": 70.404} 710078936338096128 1596439630-010-18741038 +139866971123834880 10000 1993-08-03T15:27:11.979 10000 1993-08-03T15:27:11.979 201 456678 false 1993-08-03T15:27:10 010-18741034 {"wTS": "1596439630", "rfId": "010-18741034", "remark": "数据", "weight": 161.756} 710078936338096128 1596439630-010-18741034 +139866971149000704 10000 1993-08-03T15:27:11.985 10000 1993-08-03T15:27:11.985 201 456678 false 1993-08-03T15:27:10 010-18741035 {"wTS": "1596439630", "rfId": "010-18741035", "remark": "数据", "weight": 61.382} 710078936338096128 1596439630-010-18741035 +139867090732802048 10000 1993-08-03T15:27:40.496 10000 1993-08-03T15:27:40.496 201 456678 false 1993-08-03T15:27:40 010-18741037 {"wTS": "1596439660", "rfId": "010-18741037", "remark": "数据", "weight": 156.557} 710078936338096128 1596439660-010-18741037 +139867091068346368 10000 1993-08-03T15:27:40.576 10000 1993-08-03T15:27:40.576 201 456678 false 1993-08-03T15:27:40 010-18741039 {"wTS": "1596439660", "rfId": "010-18741039", "remark": "数据", "weight": 166.655} 710078936338096128 1596439660-010-18741039 +139867091512942592 10000 1993-08-03T15:27:40.682 10000 1993-08-03T15:27:40.682 201 456678 false 1993-08-03T15:27:40 010-18741036 {"wTS": "1596439660", "rfId": "010-18741036", "remark": "数据", "weight": 134.936} 710078936338096128 1596439660-010-18741036 +139867092112728064 10000 1993-08-03T15:27:40.825 10000 1993-08-03T15:27:40.825 201 456678 false 1993-08-03T15:27:40 010-18741031 {"wTS": "1596439660", "rfId": "010-18741031", "remark": "数据", "weight": 79.699} 710078936338096128 1596439660-010-18741031 +139867092284694528 10000 1993-08-03T15:27:40.866 10000 1993-08-03T15:27:40.866 201 456678 false 1993-08-03T15:27:40 010-18741032 {"wTS": "1596439660", "rfId": "010-18741032", "remark": "数据", "weight": 101.273} 710078936338096128 1596439660-010-18741032 +139867093572345856 10000 1993-08-03T15:27:41.173 10000 1993-08-03T15:27:41.173 201 456678 false 1993-08-03T15:27:40 010-18741038 {"wTS": "1596439660", "rfId": "010-18741038", "remark": "数据", "weight": 70.404} 710078936338096128 1596439660-010-18741038 +139867094369263616 10000 1993-08-03T15:27:41.363 10000 1993-08-03T15:27:41.363 201 456678 false 1993-08-03T15:27:40 010-18741033 {"wTS": "1596439660", "rfId": "010-18741033", "remark": "数据", "weight": 69.742} 710078936338096128 1596439660-010-18741033 +139867217174290432 10000 1993-08-03T15:28:10.642 10000 1993-08-03T15:28:10.642 201 456678 false 1993-08-03T15:28:10 010-18741031 {"wTS": "1596439690", "rfId": "010-18741031", "remark": "数据", "weight": 79.699} 710078936338096128 1596439690-010-18741031 +139867217245593600 10000 1993-08-03T15:28:10.659 10000 1993-08-03T15:28:10.659 201 456678 false 1993-08-03T15:28:10 010-18741039 {"wTS": "1596439690", "rfId": "010-18741039", "remark": "数据", "weight": 166.655} 710078936338096128 1596439690-010-18741039 +139867217677606912 10000 1993-08-03T15:28:10.762 10000 1993-08-03T15:28:10.762 201 456678 false 1993-08-03T15:28:10 010-18741037 {"wTS": "1596439690", "rfId": "010-18741037", "remark": "数据", "weight": 156.557} 710078936338096128 1596439690-010-18741037 +139867218214477824 10000 1993-08-03T15:28:10.890 10000 1993-08-03T15:28:10.890 201 456678 false 1993-08-03T15:28:10 010-18741035 {"wTS": "1596439690", "rfId": "010-18741035", "remark": "数据", "weight": 61.382} 710078936338096128 1596439690-010-18741035 +139867218508079104 10000 1993-08-03T15:28:10.960 10000 1993-08-03T15:28:10.960 201 456678 false 1993-08-03T15:28:10 010-18741032 {"wTS": "1596439690", "rfId": "010-18741032", "remark": "数据", "weight": 101.273} 710078936338096128 1596439690-010-18741032 +139867219925753856 10000 1993-08-03T15:28:11.298 10000 1993-08-03T15:28:11.298 201 456678 false 1993-08-03T15:28:10 010-18741033 {"wTS": "1596439690", "rfId": "010-18741033", "remark": "数据", "weight": 69.742} 710078936338096128 1596439690-010-18741033 +139867220013834240 10000 1993-08-03T15:28:11.320 10000 1993-08-03T15:28:11.320 201 456678 false 1993-08-03T15:28:10 010-18741036 {"wTS": "1596439690", "rfId": "010-18741036", "remark": "数据", "weight": 134.936} 710078936338096128 1596439690-010-18741036 +139867220076748800 10000 1993-08-03T15:28:11.334 10000 1993-08-03T15:28:11.334 201 456678 false 1993-08-03T15:28:10 010-18741034 {"wTS": "1596439690", "rfId": "010-18741034", "remark": "数据", "weight": 161.756} 710078936338096128 1596439690-010-18741034 +139867220299046912 10000 1993-08-03T15:28:11.387 10000 1993-08-03T15:28:11.387 201 456678 false 1993-08-03T15:28:10 010-18741040 {"wTS": "1596439690", "rfId": "010-18741040", "remark": "数据", "weight": 76.74} 710078936338096128 1596439690-010-18741040 +139867342911135744 10000 1993-08-03T15:28:40.620 10000 1993-08-03T15:28:40.620 201 456678 false 1993-08-03T15:28:40 010-18741037 {"wTS": "1596439720", "rfId": "010-18741037", "remark": "数据", "weight": 156.557} 710078936338096128 1596439720-010-18741037 +139867343108268032 10000 1993-08-03T15:28:40.667 10000 1993-08-03T15:28:40.667 201 456678 false 1993-08-03T15:28:40 010-18741031 {"wTS": "1596439720", "rfId": "010-18741031", "remark": "数据", "weight": 79.699} 710078936338096128 1596439720-010-18741031 +139867343733219328 10000 1993-08-03T15:28:40.816 10000 1993-08-03T15:28:40.816 201 456678 false 1993-08-03T15:28:40 010-18741039 {"wTS": "1596439720", "rfId": "010-18741039", "remark": "数据", "weight": 166.655} 710078936338096128 1596439720-010-18741039 +139867344068763648 10000 1993-08-03T15:28:40.896 10000 1993-08-03T15:28:40.896 201 456678 false 1993-08-03T15:28:40 010-18741034 {"wTS": "1596439720", "rfId": "010-18741034", "remark": "数据", "weight": 161.756} 710078936338096128 1596439720-010-18741034 +139867344895041536 10000 1993-08-03T15:28:41.093 10000 1993-08-03T15:28:41.093 201 456678 false 1993-08-03T15:28:40 010-18741032 {"wTS": "1596439720", "rfId": "010-18741032", "remark": "数据", "weight": 101.273} 710078936338096128 1596439720-010-18741032 +139867345054425088 10000 1993-08-03T15:28:41.131 10000 1993-08-03T15:28:41.131 201 456678 false 1993-08-03T15:28:40 010-18741035 {"wTS": "1596439720", "rfId": "010-18741035", "remark": "数据", "weight": 61.382} 710078936338096128 1596439720-010-18741035 +139867346480488448 10000 1993-08-03T15:28:41.471 10000 1993-08-03T15:28:41.471 201 456678 false 1993-08-03T15:28:40 010-18741038 {"wTS": "1596439720", "rfId": "010-18741038", "remark": "数据", "weight": 70.404} 710078936338096128 1596439720-010-18741038 +139867347524870144 10000 1993-08-03T15:28:41.721 10000 1993-08-03T15:28:41.721 201 456678 false 1993-08-03T15:28:40 010-18741040 {"wTS": "1596439720", "rfId": "010-18741040", "remark": "数据", "weight": 76.74} 710078936338096128 1596439720-010-18741040 +139867469239377920 10000 1993-08-03T15:29:10.739 10000 1993-08-03T15:29:10.739 201 456678 false 1993-08-03T15:29:10 010-18741037 {"wTS": "1596439750", "rfId": "010-18741037", "remark": "数据", "weight": 156.557} 710078936338096128 1596439750-010-18741037 +139867469352624128 10000 1993-08-03T15:29:10.766 10000 1993-08-03T15:29:10.766 201 456678 false 1993-08-03T15:29:10 010-18741039 {"wTS": "1596439750", "rfId": "010-18741039", "remark": "数据", "weight": 166.655} 710078936338096128 1596439750-010-18741039 +139867469516201984 10000 1993-08-03T15:29:10.805 10000 1993-08-03T15:29:10.805 201 456678 false 1993-08-03T15:29:10 010-18741031 {"wTS": "1596439750", "rfId": "010-18741031", "remark": "数据", "weight": 79.699} 710078936338096128 1596439750-010-18741031 +139867469654614016 10000 1993-08-03T15:29:10.838 10000 1993-08-03T15:29:10.838 201 456678 false 1993-08-03T15:29:10 010-18741036 {"wTS": "1596439750", "rfId": "010-18741036", "remark": "数据", "weight": 134.936} 710078936338096128 1596439750-010-18741036 +139867471651102720 10000 1993-08-03T15:29:11.314 10000 1993-08-03T15:29:11.314 201 456678 false 1993-08-03T15:29:10 010-18741032 {"wTS": "1596439750", "rfId": "010-18741032", "remark": "数据", "weight": 101.273} 710078936338096128 1596439750-010-18741032 +139867471726600192 10000 1993-08-03T15:29:11.332 10000 1993-08-03T15:29:11.332 201 456678 false 1993-08-03T15:29:10 010-18741035 {"wTS": "1596439750", "rfId": "010-18741035", "remark": "数据", "weight": 61.382} 710078936338096128 1596439750-010-18741035 +139867471730794496 10000 1993-08-03T15:29:11.333 10000 1993-08-03T15:29:11.333 201 456678 false 1993-08-03T15:29:10 010-18741034 {"wTS": "1596439750", "rfId": "010-18741034", "remark": "数据", "weight": 161.756} 710078936338096128 1596439750-010-18741034 +139867471776931840 10000 1993-08-03T15:29:11.344 10000 1993-08-03T15:29:11.344 201 456678 false 1993-08-03T15:29:10 010-18741038 {"wTS": "1596439750", "rfId": "010-18741038", "remark": "数据", "weight": 70.404} 710078936338096128 1596439750-010-18741038 +139867473987330048 10000 1993-08-03T15:29:11.871 10000 1993-08-03T15:29:11.871 201 456678 false 1993-08-03T15:29:10 010-18741033 {"wTS": "1596439750", "rfId": "010-18741033", "remark": "数据", "weight": 69.742} 710078936338096128 1596439750-010-18741033 +139867474566144000 10000 1993-08-03T15:29:12.009 10000 1993-08-03T15:29:12.009 201 456678 false 1993-08-03T15:29:10 010-18741040 {"wTS": "1596439750", "rfId": "010-18741040", "remark": "数据", "weight": 76.74} 710078936338096128 1596439750-010-18741040 +139867596326789120 10000 1993-08-03T15:29:41.039 10000 1993-08-03T15:29:41.039 201 456678 false 1993-08-03T15:29:40 010-18741034 {"wTS": "1596439780", "rfId": "010-18741034", "remark": "数据", "weight": 161.756} 710078936338096128 1596439780-010-18741034 +139867596335177728 10000 1993-08-03T15:29:41.041 10000 1993-08-03T15:29:41.041 201 456678 false 1993-08-03T15:29:40 010-18741035 {"wTS": "1596439780", "rfId": "010-18741035", "remark": "数据", "weight": 61.382} 710078936338096128 1596439780-010-18741035 +139867596356149248 10000 1993-08-03T15:29:41.046 10000 1993-08-03T15:29:41.046 201 456678 false 1993-08-03T15:29:40 010-18741037 {"wTS": "1596439780", "rfId": "010-18741037", "remark": "数据", "weight": 156.557} 710078936338096128 1596439780-010-18741037 +139867596586835968 10000 1993-08-03T15:29:41.101 10000 1993-08-03T15:29:41.101 201 456678 false 1993-08-03T15:29:40 010-18741039 {"wTS": "1596439780", "rfId": "010-18741039", "remark": "数据", "weight": 166.655} 710078936338096128 1596439780-010-18741039 +139867596616196096 10000 1993-08-03T15:29:41.108 10000 1993-08-03T15:29:41.108 201 456678 false 1993-08-03T15:29:40 010-18741031 {"wTS": "1596439780", "rfId": "010-18741031", "remark": "数据", "weight": 79.699} 710078936338096128 1596439780-010-18741031 +139867596620390400 10000 1993-08-03T15:29:41.109 10000 1993-08-03T15:29:41.109 201 456678 false 1993-08-03T15:29:40 010-18741036 {"wTS": "1596439780", "rfId": "010-18741036", "remark": "数据", "weight": 134.936} 710078936338096128 1596439780-010-18741036 +139867597828349952 10000 1993-08-03T15:29:41.397 10000 1993-08-03T15:29:41.397 201 456678 false 1993-08-03T15:29:40 010-18741038 {"wTS": "1596439780", "rfId": "010-18741038", "remark": "数据", "weight": 70.404} 710078936338096128 1596439780-010-18741038 +139867598340055040 10000 1993-08-03T15:29:41.519 10000 1993-08-03T15:29:41.520 201 456678 false 1993-08-03T15:29:40 010-18741040 {"wTS": "1596439780", "rfId": "010-18741040", "remark": "数据", "weight": 76.74} 710078936338096128 1596439780-010-18741040 +139867600818888704 10000 1993-08-03T15:29:42.110 10000 1993-08-03T15:29:42.110 201 456678 false 1993-08-03T15:29:40 010-18741033 {"wTS": "1596439780", "rfId": "010-18741033", "remark": "数据", "weight": 69.742} 710078936338096128 1596439780-010-18741033 +139867601850687488 10000 1993-08-03T15:29:42.356 10000 1993-08-03T15:29:42.357 201 456678 false 1993-08-03T15:29:40 010-18741032 {"wTS": "1596439780", "rfId": "010-18741032", "remark": "数据", "weight": 101.273} 710078936338096128 1596439780-010-18741032 +139867721308659712 10000 1993-08-03T15:30:10.838 10000 1993-08-03T15:30:10.838 201 456678 false 1993-08-03T15:30:10 010-18741037 {"wTS": "1596439810", "rfId": "010-18741037", "remark": "数据", "weight": 156.557} 710078936338096128 1596439810-010-18741037 +139867721384157184 10000 1993-08-03T15:30:10.855 10000 1993-08-03T15:30:10.855 201 456678 false 1993-08-03T15:30:10 010-18741039 {"wTS": "1596439810", "rfId": "010-18741039", "remark": "数据", "weight": 166.655} 710078936338096128 1596439810-010-18741039 +139867721694535680 10000 1993-08-03T15:30:10.929 10000 1993-08-03T15:30:10.929 201 456678 false 1993-08-03T15:30:10 010-18741034 {"wTS": "1596439810", "rfId": "010-18741034", "remark": "数据", "weight": 161.756} 710078936338096128 1596439810-010-18741034 +139867723183513600 10000 1993-08-03T15:30:11.284 10000 1993-08-03T15:30:11.284 201 456678 false 1993-08-03T15:30:10 010-18741032 {"wTS": "1596439810", "rfId": "010-18741032", "remark": "数据", "weight": 101.273} 710078936338096128 1596439810-010-18741032 +139867723208679424 10000 1993-08-03T15:30:11.290 10000 1993-08-03T15:30:11.290 201 456678 false 1993-08-03T15:30:10 010-18741033 {"wTS": "1596439810", "rfId": "010-18741033", "remark": "数据", "weight": 69.742} 710078936338096128 1596439810-010-18741033 +139867723229650944 10000 1993-08-03T15:30:11.295 10000 1993-08-03T15:30:11.295 201 456678 false 1993-08-03T15:30:10 010-18741035 {"wTS": "1596439810", "rfId": "010-18741035", "remark": "数据", "weight": 61.382} 710078936338096128 1596439810-010-18741035 +139867723238039552 10000 1993-08-03T15:30:11.297 10000 1993-08-03T15:30:11.297 201 456678 false 1993-08-03T15:30:10 010-18741036 {"wTS": "1596439810", "rfId": "010-18741036", "remark": "数据", "weight": 134.936} 710078936338096128 1596439810-010-18741036 +139867724131426304 10000 1993-08-03T15:30:11.510 10000 1993-08-03T15:30:11.510 201 456678 false 1993-08-03T15:30:10 010-18741040 {"wTS": "1596439810", "rfId": "010-18741040", "remark": "数据", "weight": 76.74} 710078936338096128 1596439810-010-18741040 +139867724534079488 10000 1993-08-03T15:30:11.606 10000 1993-08-03T15:30:11.606 201 456678 false 1993-08-03T15:30:10 010-18741038 {"wTS": "1596439810", "rfId": "010-18741038", "remark": "数据", "weight": 70.404} 710078936338096128 1596439810-010-18741038 +139867847209082880 10000 1993-08-03T15:30:40.854 10000 1993-08-03T15:30:40.854 201 456678 false 1993-08-03T15:30:40 010-18741037 {"wTS": "1596439840", "rfId": "010-18741037", "remark": "数据", "weight": 156.557} 710078936338096128 1596439840-010-18741037 +139867848484151296 10000 1993-08-03T15:30:41.158 10000 1993-08-03T15:30:41.158 201 456678 false 1993-08-03T15:30:40 010-18741031 {"wTS": "1596439840", "rfId": "010-18741031", "remark": "数据", "weight": 79.699} 710078936338096128 1596439840-010-18741031 +139867848492539904 10000 1993-08-03T15:30:41.160 10000 1993-08-03T15:30:41.160 201 456678 false 1993-08-03T15:30:40 010-18741036 {"wTS": "1596439840", "rfId": "010-18741036", "remark": "数据", "weight": 134.936} 710078936338096128 1596439840-010-18741036 +139867848492539905 10000 1993-08-03T15:30:41.160 10000 1993-08-03T15:30:41.160 201 456678 false 1993-08-03T15:30:40 010-18741034 {"wTS": "1596439840", "rfId": "010-18741034", "remark": "数据", "weight": 161.756} 710078936338096128 1596439840-010-18741034 +139867848521900032 10000 1993-08-03T15:30:41.167 10000 1993-08-03T15:30:41.167 201 456678 false 1993-08-03T15:30:40 010-18741039 {"wTS": "1596439840", "rfId": "010-18741039", "remark": "数据", "weight": 166.655} 710078936338096128 1596439840-010-18741039 +139867848660312064 10000 1993-08-03T15:30:41.200 10000 1993-08-03T15:30:41.200 201 456678 false 1993-08-03T15:30:40 010-18741035 {"wTS": "1596439840", "rfId": "010-18741035", "remark": "数据", "weight": 61.382} 710078936338096128 1596439840-010-18741035 +139867850090569728 10000 1993-08-03T15:30:41.541 10000 1993-08-03T15:30:41.541 201 456678 false 1993-08-03T15:30:40 010-18741040 {"wTS": "1596439840", "rfId": "010-18741040", "remark": "数据", "weight": 76.74} 710078936338096128 1596439840-010-18741040 +139867851755708416 10000 1993-08-03T15:30:41.938 10000 1993-08-03T15:30:41.938 201 456678 false 1993-08-03T15:30:40 010-18741033 {"wTS": "1596439840", "rfId": "010-18741033", "remark": "数据", "weight": 69.742} 710078936338096128 1596439840-010-18741033 +139867852619735040 10000 1993-08-03T15:30:42.144 10000 1993-08-03T15:30:42.144 201 456678 false 1993-08-03T15:30:40 010-18741038 {"wTS": "1596439840", "rfId": "010-18741038", "remark": "数据", "weight": 70.404} 710078936338096128 1596439840-010-18741038 +139867854192599040 10000 1993-08-03T15:30:42.519 10000 1993-08-03T15:30:42.519 201 456678 false 1993-08-03T15:30:40 010-18741032 {"wTS": "1596439840", "rfId": "010-18741032", "remark": "数据", "weight": 101.273} 710078936338096128 1596439840-010-18741032 +139867911025405440 10000 1993-08-03T15:31:10.994 10000 1993-08-03T15:31:10.994 201 456678 false 1993-08-03T15:31:10 010-18741037 {"wTS": "1596439870", "rfId": "010-18741037", "remark": "数据", "weight": 156.557} 710078936338096128 1596439870-010-18741037 +139867911079931392 10000 1993-08-03T15:31:11.007 10000 1993-08-03T15:31:11.007 201 456678 false 1993-08-03T15:31:10 010-18741039 {"wTS": "1596439870", "rfId": "010-18741039", "remark": "数据", "weight": 166.655} 710078936338096128 1596439870-010-18741039 +139867973185003520 10000 1993-08-03T15:31:10.889 10000 1993-08-03T15:31:10.889 201 456678 false 1993-08-03T15:31:10 010-18741031 {"wTS": "1596439870", "rfId": "010-18741031", "remark": "数据", "weight": 79.699} 710078936338096128 1596439870-010-18741031 +139867974439100416 10000 1993-08-03T15:31:11.188 10000 1993-08-03T15:31:11.188 201 456678 false 1993-08-03T15:31:10 010-18741036 {"wTS": "1596439870", "rfId": "010-18741036", "remark": "数据", "weight": 134.936} 710078936338096128 1596439870-010-18741036 +139867974640427008 10000 1993-08-03T15:31:11.236 10000 1993-08-03T15:31:11.237 201 456678 false 1993-08-03T15:31:10 010-18741034 {"wTS": "1596439870", "rfId": "010-18741034", "remark": "数据", "weight": 161.756} 710078936338096128 1596439870-010-18741034 +139867975483482112 10000 1993-08-03T15:31:11.437 10000 1993-08-03T15:31:11.437 201 456678 false 1993-08-03T15:31:10 010-18741035 {"wTS": "1596439870", "rfId": "010-18741035", "remark": "数据", "weight": 61.382} 710078936338096128 1596439870-010-18741035 +139867976561418240 10000 1993-08-03T15:31:11.694 10000 1993-08-03T15:31:11.694 201 456678 false 1993-08-03T15:31:10 010-18741038 {"wTS": "1596439870", "rfId": "010-18741038", "remark": "数据", "weight": 70.404} 710078936338096128 1596439870-010-18741038 +139868173785980928 10000 1993-08-03T15:31:58.716 10000 1993-08-03T15:31:58.716 201 456678 false 1993-08-03T15:31:58 010-18741039 {"wTS": "1596439918", "rfId": "010-18741039", "remark": "数据", "weight": 111.068} 710078936338096128 1596439918-010-18741039 +139868173853089792 10000 1993-08-03T15:31:58.732 10000 1993-08-03T15:31:58.732 201 456678 false 1993-08-03T15:31:58 010-18741037 {"wTS": "1596439918", "rfId": "010-18741037", "remark": "数据", "weight": 139.592} 710078936338096128 1596439918-010-18741037 +139868174079582208 10000 1993-08-03T15:31:58.786 10000 1993-08-03T15:31:58.786 201 456678 false 1993-08-03T15:31:58 010-18741031 {"wTS": "1596439918", "rfId": "010-18741031", "remark": "数据", "weight": 85.044} 710078936338096128 1596439918-010-18741031 +139868174394155008 10000 1993-08-03T15:31:58.861 10000 1993-08-03T15:31:58.861 201 456678 false 1993-08-03T15:31:58 010-18741034 {"wTS": "1596439918", "rfId": "010-18741034", "remark": "数据", "weight": 157.6} 710078936338096128 1596439918-010-18741034 +139868174528372110 10000 1993-08-03T15:31:58.893 10000 1993-08-03T15:31:58.893 201 456678 false 1993-08-03T15:31:58 010-18741033 {"wTS": "1596439918", "rfId": "010-18741033", "remark": "数据", "weight": 66.858} 710078936338096128 1596439918-010-18741033 +139868175157518336 10000 1993-08-03T15:31:59.043 10000 1993-08-03T15:31:59.043 201 456678 false 1993-08-03T15:31:58 010-18741038 {"wTS": "1596439918", "rfId": "010-18741038", "remark": "数据", "weight": 122.591} 710078936338096128 1596439918-010-18741038 +139868175782469632 10000 1993-08-03T15:31:59.192 10000 1993-08-03T15:31:59.192 201 456678 false 1993-08-03T15:31:58 010-18741036 {"wTS": "1596439918", "rfId": "010-18741036", "remark": "数据", "weight": 75.728} 710078936338096128 1596439918-010-18741036 +139868176168345600 10000 1993-08-03T15:31:59.284 10000 1993-08-03T15:31:59.284 201 456678 false 1993-08-03T15:31:58 010-18741040 {"wTS": "1596439918", "rfId": "010-18741040", "remark": "数据", "weight": 133.627} 710078936338096128 1596439918-010-18741040 +139868176168345601 10000 1993-08-03T15:31:59.284 10000 1993-08-03T15:31:59.284 201 456678 false 1993-08-03T15:31:58 010-18741032 {"wTS": "1596439918", "rfId": "010-18741032", "remark": "数据", "weight": 108.355} 710078936338096128 1596439918-010-18741032 +139868176940097536 10000 1993-08-03T15:31:59.468 10000 1993-08-03T15:31:59.468 201 456678 false 1993-08-03T15:31:58 010-18741035 {"wTS": "1596439918", "rfId": "010-18741035", "remark": "数据", "weight": 133.986} 710078936338096128 1596439918-010-18741035 +139868302886658048 10000 1993-08-03T15:32:29.496 10000 1993-08-03T15:32:29.496 201 456678 false 1993-08-03T15:32:29 010-18741032 {"wTS": "1596439949", "rfId": "010-18741032", "remark": "数据", "weight": 108.355} 710078936338096128 1596439949-010-18741032 +139868303213813760 10000 1993-08-03T15:32:29.574 10000 1993-08-03T15:32:29.574 201 456678 false 1993-08-03T15:32:29 010-18741031 {"wTS": "1596439949", "rfId": "010-18741031", "remark": "数据", "weight": 85.044} 710078936338096128 1596439949-010-18741031 +139868303238979584 10000 1993-08-03T15:32:29.580 10000 1993-08-03T15:32:29.580 201 456678 false 1993-08-03T15:32:29 010-18741034 {"wTS": "1596439949", "rfId": "010-18741034", "remark": "数据", "weight": 157.6} 710078936338096128 1596439949-010-18741034 +139868303616466944 10000 1993-08-03T15:32:29.670 10000 1993-08-03T15:32:29.670 201 456678 false 1993-08-03T15:32:29 010-18741036 {"wTS": "1596439949", "rfId": "010-18741036", "remark": "数据", "weight": 75.728} 710078936338096128 1596439949-010-18741036 +139868303721324544 10000 1993-08-03T15:32:29.695 10000 1993-08-03T15:32:29.695 201 456678 false 1993-08-03T15:32:29 010-18741035 {"wTS": "1596439949", "rfId": "010-18741035", "remark": "数据", "weight": 133.986} 710078936338096128 1596439949-010-18741035 +139868304795066368 10000 1993-08-03T15:32:29.951 10000 1993-08-03T15:32:29.951 201 456678 false 1993-08-03T15:32:29 010-18741040 {"wTS": "1596439949", "rfId": "010-18741040", "remark": "数据", "weight": 133.627} 710078936338096128 1596439949-010-18741040 +139868306833498112 10000 1993-08-03T15:32:30.437 10000 1993-08-03T15:32:30.437 201 456678 false 1993-08-03T15:32:29 010-18741037 {"wTS": "1596439949", "rfId": "010-18741037", "remark": "数据", "weight": 139.592} 710078936338096128 1596439949-010-18741037 +139868306904801280 10000 1993-08-03T15:32:30.454 10000 1993-08-03T15:32:30.454 201 456678 false 1993-08-03T15:32:29 010-18741038 {"wTS": "1596439949", "rfId": "010-18741038", "remark": "数据", "weight": 122.591} 710078936338096128 1596439949-010-18741038 +139868307030630400 10000 1993-08-03T15:32:30.484 10000 1993-08-03T15:32:30.484 201 456678 false 1993-08-03T15:32:29 010-18741033 {"wTS": "1596439949", "rfId": "010-18741033", "remark": "数据", "weight": 66.858} 710078936338096128 1596439949-010-18741033 +139868307257122816 10000 1993-08-03T15:32:30.538 10000 1993-08-03T15:32:30.538 201 456678 false 1993-08-03T15:32:29 010-18741039 {"wTS": "1596439949", "rfId": "010-18741039", "remark": "数据", "weight": 111.068} 710078936338096128 1596439949-010-18741039 +139868429219094528 10000 1993-08-03T15:32:59.616 10000 1993-08-03T15:32:59.616 201 456678 false 1993-08-03T15:32:59 010-18741036 {"wTS": "1596439979", "rfId": "010-18741036", "remark": "数据", "weight": 75.728} 710078936338096128 1596439979-010-18741036 +139868429403643904 10000 1993-08-03T15:32:59.660 10000 1993-08-03T15:32:59.660 201 456678 false 1993-08-03T15:32:59 010-18741032 {"wTS": "1596439979", "rfId": "010-18741032", "remark": "数据", "weight": 108.355} 710078936338096128 1596439979-010-18741032 +139868429835657216 10000 1993-08-03T15:32:59.763 10000 1993-08-03T15:32:59.763 201 456678 false 1993-08-03T15:32:59 010-18741031 {"wTS": "1596439979", "rfId": "010-18741031", "remark": "数据", "weight": 85.044} 710078936338096128 1596439979-010-18741031 +139868429919543296 10000 1993-08-03T15:32:59.783 10000 1993-08-03T15:32:59.783 201 456678 false 1993-08-03T15:32:59 010-18741034 {"wTS": "1596439979", "rfId": "010-18741034", "remark": "数据", "weight": 157.6} 710078936338096128 1596439979-010-18741034 +139868430703878144 10000 1993-08-03T15:32:59.970 10000 1993-08-03T15:32:59.970 201 456678 false 1993-08-03T15:32:59 010-18741037 {"wTS": "1596439979", "rfId": "010-18741037", "remark": "数据", "weight": 139.592} 710078936338096128 1596439979-010-18741037 +139868431274303488 10000 1993-08-03T15:33:00.106 10000 1993-08-03T15:33:00.106 201 456678 false 1993-08-03T15:32:59 010-18741035 {"wTS": "1596439979", "rfId": "010-18741035", "remark": "数据", "weight": 133.986} 710078936338096128 1596439979-010-18741035 +139868431551127552 10000 1993-08-03T15:33:00.172 10000 1993-08-03T15:33:00.172 201 456678 false 1993-08-03T15:32:59 010-18741038 {"wTS": "1596439979", "rfId": "010-18741038", "remark": "数据", "weight": 122.591} 710078936338096128 1596439979-010-18741038 +139868432771670016 10000 1993-08-03T15:33:00.463 10000 1993-08-03T15:33:00.463 201 456678 false 1993-08-03T15:32:59 010-18741040 {"wTS": "1596439979", "rfId": "010-18741040", "remark": "数据", "weight": 133.627} 710078936338096128 1596439979-010-18741040 +139868434906570752 10000 1993-08-03T15:33:00.972 10000 1993-08-03T15:33:00.972 201 456678 false 1993-08-03T15:32:59 010-18741033 {"wTS": "1596439979", "rfId": "010-18741033", "remark": "数据", "weight": 66.858} 710078936338096128 1596439979-010-18741033 +139868434969485312 10000 1993-08-03T15:33:00.987 10000 1993-08-03T15:33:00.987 201 456678 false 1993-08-03T15:32:59 010-18741039 {"wTS": "1596439979", "rfId": "010-18741039", "remark": "数据", "weight": 111.068} 710078936338096128 1596439979-010-18741039 +139868554591035392 10000 1993-08-03T15:33:29.507 10000 1993-08-03T15:33:29.507 201 456678 false 1993-08-03T15:33:29 010-18741036 {"wTS": "1596440009", "rfId": "010-18741036", "remark": "数据", "weight": 75.728} 710078936338096128 1596440009-010-18741036 +139868554830110720 10000 1993-08-03T15:33:29.564 10000 1993-08-03T15:33:29.564 201 456678 false 1993-08-03T15:33:29 010-18741032 {"wTS": "1596440009", "rfId": "010-18741032", "remark": "数据", "weight": 108.355} 710078936338096128 1596440009-010-18741032 +139868555350204416 10000 1993-08-03T15:33:29.688 10000 1993-08-03T15:33:29.688 201 456678 false 1993-08-03T15:33:29 010-18741031 {"wTS": "1596440009", "rfId": "010-18741031", "remark": "数据", "weight": 85.044} 710078936338096128 1596440009-010-18741031 +139868555471839232 10000 1993-08-03T15:33:29.717 10000 1993-08-03T15:33:29.717 201 456678 false 1993-08-03T15:33:29 010-18741034 {"wTS": "1596440009", "rfId": "010-18741034", "remark": "数据", "weight": 157.6} 710078936338096128 1596440009-010-18741034 +139868556222619648 10000 1993-08-03T15:33:29.896 10000 1993-08-03T15:33:29.896 201 456678 false 1993-08-03T15:33:29 010-18741039 {"wTS": "1596440009", "rfId": "010-18741039", "remark": "数据", "weight": 111.068} 710078936338096128 1596440009-010-18741039 +139868557208281088 10000 1993-08-03T15:33:30.131 10000 1993-08-03T15:33:30.131 201 456678 false 1993-08-03T15:33:29 010-18741033 {"wTS": "1596440009", "rfId": "010-18741033", "remark": "数据", "weight": 66.858} 710078936338096128 1596440009-010-18741033 +139868559221547008 10000 1993-08-03T15:33:30.611 10000 1993-08-03T15:33:30.611 201 456678 false 1993-08-03T15:33:29 010-18741037 {"wTS": "1596440009", "rfId": "010-18741037", "remark": "数据", "weight": 139.592} 710078936338096128 1596440009-010-18741037 +139868559234129920 10000 1993-08-03T15:33:30.614 10000 1993-08-03T15:33:30.614 201 456678 false 1993-08-03T15:33:29 010-18741038 {"wTS": "1596440009", "rfId": "010-18741038", "remark": "数据", "weight": 122.591} 710078936338096128 1596440009-010-18741038 +139868559435456512 10000 1993-08-03T15:33:30.662 10000 1993-08-03T15:33:30.662 201 456678 false 1993-08-03T15:33:29 010-18741040 {"wTS": "1596440009", "rfId": "010-18741040", "remark": "数据", "weight": 133.627} 710078936338096128 1596440009-010-18741040 +139868559452233728 10000 1993-08-03T15:33:30.666 10000 1993-08-03T15:33:30.666 201 456678 false 1993-08-03T15:33:29 010-18741035 {"wTS": "1596440009", "rfId": "010-18741035", "remark": "数据", "weight": 133.986} 710078936338096128 1596440009-010-18741035 +139868680860557312 10000 1993-08-03T15:33:59.612 10000 1993-08-03T15:33:59.613 201 456678 false 1993-08-03T15:33:59 010-18741032 {"wTS": "1596440039", "rfId": "010-18741032", "remark": "数据", "weight": 108.355} 710078936338096128 1596440039-010-18741032 +139868680994775040 10000 1993-08-03T15:33:59.644 10000 1993-08-03T15:33:59.644 201 456678 false 1993-08-03T15:33:59 010-18741036 {"wTS": "1596440039", "rfId": "010-18741036", "remark": "数据", "weight": 75.728} 710078936338096128 1596440039-010-18741036 +139868681170935808 10000 1993-08-03T15:33:59.686 10000 1993-08-03T15:33:59.686 201 456678 false 1993-08-03T15:33:59 010-18741031 {"wTS": "1596440039", "rfId": "010-18741031", "remark": "数据", "weight": 85.044} 710078936338096128 1596440039-010-18741031 +139868681850413056 10000 1993-08-03T15:33:59.848 10000 1993-08-03T15:33:59.848 201 456678 false 1993-08-03T15:33:59 010-18741034 {"wTS": "1596440039", "rfId": "010-18741034", "remark": "数据", "weight": 157.6} 710078936338096128 1596440039-010-18741034 +139868682123042816 10000 1993-08-03T15:33:59.913 10000 1993-08-03T15:33:59.913 201 456678 false 1993-08-03T15:33:59 010-18741035 {"wTS": "1596440039", "rfId": "010-18741035", "remark": "数据", "weight": 133.986} 710078936338096128 1596440039-010-18741035 +139868682487947264 10000 1993-08-03T15:34 10000 1993-08-03T15:34 201 456678 false 1993-08-03T15:33:59 010-18741039 {"wTS": "1596440039", "rfId": "010-18741039", "remark": "数据", "weight": 111.068} 710078936338096128 1596440039-010-18741039 +139868683733655552 10000 1993-08-03T15:34:00.297 10000 1993-08-03T15:34:00.297 201 456678 false 1993-08-03T15:33:59 010-18741040 {"wTS": "1596440039", "rfId": "010-18741040", "remark": "数据", "weight": 133.627} 710078936338096128 1596440039-010-18741040 +139868685809836032 10000 1993-08-03T15:34:00.792 10000 1993-08-03T15:34:00.792 201 456678 false 1993-08-03T15:33:59 010-18741037 {"wTS": "1596440039", "rfId": "010-18741037", "remark": "数据", "weight": 139.592} 710078936338096128 1596440039-010-18741037 +139868687265259520 10000 1993-08-03T15:34:01.139 10000 1993-08-03T15:34:01.139 201 456678 false 1993-08-03T15:33:59 010-18741033 {"wTS": "1596440039", "rfId": "010-18741033", "remark": "数据", "weight": 66.858} 710078936338096128 1596440039-010-18741033 +139868687596609536 10000 1993-08-03T15:34:01.218 10000 1993-08-03T15:34:01.218 201 456678 false 1993-08-03T15:33:59 010-18741038 {"wTS": "1596440039", "rfId": "010-18741038", "remark": "数据", "weight": 122.591} 710078936338096128 1596440039-010-18741038 +139868807180410880 10000 1993-08-03T15:34:29.729 10000 1993-08-03T15:34:29.729 201 456678 false 1993-08-03T15:34:29 010-18741036 {"wTS": "1596440069", "rfId": "010-18741036", "remark": "数据", "weight": 75.728} 710078936338096128 1596440069-010-18741036 +139868807306240000 10000 1993-08-03T15:34:29.759 10000 1993-08-03T15:34:29.759 201 456678 false 1993-08-03T15:34:29 010-18741032 {"wTS": "1596440069", "rfId": "010-18741032", "remark": "数据", "weight": 108.355} 710078936338096128 1596440069-010-18741032 +139868807461429248 10000 1993-08-03T15:34:29.796 10000 1993-08-03T15:34:29.796 201 456678 false 1993-08-03T15:34:29 010-18741035 {"wTS": "1596440069", "rfId": "010-18741035", "remark": "数据", "weight": 133.986} 710078936338096128 1596440069-010-18741035 +139868807478206464 10000 1993-08-03T15:34:29.800 10000 1993-08-03T15:34:29.800 201 456678 false 1993-08-03T15:34:29 010-18741031 {"wTS": "1596440069", "rfId": "010-18741031", "remark": "数据", "weight": 85.044} 710078936338096128 1596440069-010-18741031 +139868808539365376 10000 1993-08-03T15:34:30.053 10000 1993-08-03T15:34:30.053 201 456678 false 1993-08-03T15:34:29 010-18741038 {"wTS": "1596440069", "rfId": "010-18741038", "remark": "数据", "weight": 122.591} 710078936338096128 1596440069-010-18741038 +139868809302728704 10000 1993-08-03T15:34:30.235 10000 1993-08-03T15:34:30.235 201 456678 false 1993-08-03T15:34:29 010-18741034 {"wTS": "1596440069", "rfId": "010-18741034", "remark": "数据", "weight": 157.6} 710078936338096128 1596440069-010-18741034 +139868809797656576 10000 1993-08-03T15:34:30.353 10000 1993-08-03T15:34:30.353 201 456678 false 1993-08-03T15:34:29 010-18741037 {"wTS": "1596440069", "rfId": "010-18741037", "remark": "数据", "weight": 139.592} 710078936338096128 1596440069-010-18741037 +139868810493911040 10000 1993-08-03T15:34:30.519 10000 1993-08-03T15:34:30.519 201 456678 false 1993-08-03T15:34:29 010-18741039 {"wTS": "1596440069", "rfId": "010-18741039", "remark": "数据", "weight": 111.068} 710078936338096128 1596440069-010-18741039 +139868811185971200 10000 1993-08-03T15:34:30.684 10000 1993-08-03T15:34:30.684 201 456678 false 1993-08-03T15:34:29 010-18741040 {"wTS": "1596440069", "rfId": "010-18741040", "remark": "数据", "weight": 133.627} 710078936338096128 1596440069-010-18741040 +139868812108718080 10000 1993-08-03T15:34:30.904 10000 1993-08-03T15:34:30.904 201 456678 false 1993-08-03T15:34:29 010-18741033 {"wTS": "1596440069", "rfId": "010-18741033", "remark": "数据", "weight": 66.858} 710078936338096128 1596440069-010-18741033 +139868933114388480 10000 1993-08-03T15:34:59.754 10000 1993-08-03T15:34:59.754 201 456678 false 1993-08-03T15:34:59 010-18741032 {"wTS": "1596440099", "rfId": "010-18741032", "remark": "数据", "weight": 108.355} 710078936338096128 1596440099-010-18741032 +139868933353463808 10000 1993-08-03T15:34:59.811 10000 1993-08-03T15:34:59.811 201 456678 false 1993-08-03T15:34:59 010-18741031 {"wTS": "1596440099", "rfId": "010-18741031", "remark": "数据", "weight": 85.044} 710078936338096128 1596440099-010-18741031 +139868933798060032 10000 1993-08-03T15:34:59.917 10000 1993-08-03T15:34:59.917 201 456678 false 1993-08-03T15:34:59 010-18741036 {"wTS": "1596440099", "rfId": "010-18741036", "remark": "数据", "weight": 75.728} 710078936338096128 1596440099-010-18741036 +139868933974220800 10000 1993-08-03T15:34:59.959 10000 1993-08-03T15:34:59.959 201 456678 false 1993-08-03T15:34:59 010-18741035 {"wTS": "1596440099", "rfId": "010-18741035", "remark": "数据", "weight": 133.986} 710078936338096128 1596440099-010-18741035 +139868935110877184 10000 1993-08-03T15:35:00.230 10000 1993-08-03T15:35:00.230 201 456678 false 1993-08-03T15:34:59 010-18741033 {"wTS": "1596440099", "rfId": "010-18741033", "remark": "数据", "weight": 66.858} 710078936338096128 1596440099-010-18741033 +139868935331109600 10000 1993-08-03T15:35:00.284 10000 1993-08-03T15:35:00.284 201 456678 false 1993-08-03T15:34:59 010-18741039 {"wTS": "1596440099", "rfId": "010-18741039", "remark": "数据", "weight": 111.068} 710078936338096128 1596440099-010-18741039 +139868935459004416 10000 1993-08-03T15:35:00.313 10000 1993-08-03T15:35:00.313 201 456678 false 1993-08-03T15:34:59 010-18741040 {"wTS": "1596440099", "rfId": "010-18741040", "remark": "数据", "weight": 133.627} 710078936338096128 1596440099-010-18741040 +139868935823908864 10000 1993-08-03T15:35:00.400 10000 1993-08-03T15:35:00.400 201 456678 false 1993-08-03T15:34:59 010-18741038 {"wTS": "1596440099", "rfId": "010-18741038", "remark": "数据", "weight": 122.591} 710078936338096128 1596440099-010-18741038 +139869059098697728 10000 1993-08-03T15:35:29.791 10000 1993-08-03T15:35:29.791 201 456678 false 1993-08-03T15:35:29 010-18741032 {"wTS": "1596440129", "rfId": "010-18741032", "remark": "数据", "weight": 108.355} 710078936338096128 1596440129-010-18741032 +139869059476185088 10000 1993-08-03T15:35:29.881 10000 1993-08-03T15:35:29.881 201 456678 false 1993-08-03T15:35:29 010-18741035 {"wTS": "1596440129", "rfId": "010-18741035", "remark": "数据", "weight": 133.986} 710078936338096128 1596440129-010-18741035 +139869059669123072 10000 1993-08-03T15:35:29.927 10000 1993-08-03T15:35:29.927 201 456678 false 1993-08-03T15:35:29 010-18741031 {"wTS": "1596440129", "rfId": "010-18741031", "remark": "数据", "weight": 85.044} 710078936338096128 1596440129-010-18741031 +139869060927414272 10000 1993-08-03T15:35:30.227 10000 1993-08-03T15:35:30.227 201 456678 false 1993-08-03T15:35:29 010-18741039 {"wTS": "1596440129", "rfId": "010-18741039", "remark": "数据", "weight": 111.068} 710078936338096128 1596440129-010-18741039 +139869061061632000 10000 1993-08-03T15:35:30.259 10000 1993-08-03T15:35:30.259 201 456678 false 1993-08-03T15:35:29 010-18741034 {"wTS": "1596440129", "rfId": "010-18741034", "remark": "数据", "weight": 157.6} 710078936338096128 1596440129-010-18741034 +139869061623668110 10000 1993-08-03T15:35:30.393 10000 1993-08-03T15:35:30.393 201 456678 false 1993-08-03T15:35:29 010-18741037 {"wTS": "1596440129", "rfId": "010-18741037", "remark": "数据", "weight": 139.592} 710078936338096128 1596440129-010-18741037 +139869062315728896 10000 1993-08-03T15:35:30.558 10000 1993-08-03T15:35:30.558 201 456678 false 1993-08-03T15:35:29 010-18741040 {"wTS": "1596440129", "rfId": "010-18741040", "remark": "数据", "weight": 133.627} 710078936338096128 1596440129-010-18741040 +139869063414636544 10000 1993-08-03T15:35:30.821 10000 1993-08-03T15:35:30.821 201 456678 false 1993-08-03T15:35:29 010-18741038 {"wTS": "1596440129", "rfId": "010-18741038", "remark": "数据", "weight": 122.591} 710078936338096128 1596440129-010-18741038 +139869065004277760 10000 1993-08-03T15:35:31.199 10000 1993-08-03T15:35:31.199 201 456678 false 1993-08-03T15:35:29 010-18741033 {"wTS": "1596440129", "rfId": "010-18741033", "remark": "数据", "weight": 66.858} 710078936338096128 1596440129-010-18741033 +139869186022531072 10000 1993-08-03T15:36:00.052 10000 1993-08-03T15:36:00.052 201 456678 false 1993-08-03T15:35:59 010-18741035 {"wTS": "1596440159", "rfId": "010-18741035", "remark": "数据", "weight": 133.986} 710078936338096128 1596440159-010-18741035 +139869186030919680 10000 1993-08-03T15:36:00.054 10000 1993-08-03T15:36:00.054 201 456678 false 1993-08-03T15:35:59 010-18741032 {"wTS": "1596440159", "rfId": "010-18741032", "remark": "数据", "weight": 108.355} 710078936338096128 1596440159-010-18741032 +139869186064474112 10000 1993-08-03T15:36:00.062 10000 1993-08-03T15:36:00.062 201 456678 false 1993-08-03T15:35:59 010-18741036 {"wTS": "1596440159", "rfId": "010-18741036", "remark": "数据", "weight": 75.728} 710078936338096128 1596440159-010-18741036 +139869186500681728 10000 1993-08-03T15:36:00.166 10000 1993-08-03T15:36:00.166 201 456678 false 1993-08-03T15:35:59 010-18741034 {"wTS": "1596440159", "rfId": "010-18741034", "remark": "数据", "weight": 157.6} 710078936338096128 1596440159-010-18741034 +139869187461177344 10000 1993-08-03T15:36:00.395 10000 1993-08-03T15:36:00.395 201 456678 false 1993-08-03T15:35:59 010-18741039 {"wTS": "1596440159", "rfId": "010-18741039", "remark": "数据", "weight": 111.068} 710078936338096128 1596440159-010-18741039 +139869188274872320 10000 1993-08-03T15:36:00.589 10000 1993-08-03T15:36:00.589 201 456678 false 1993-08-03T15:35:59 010-18741033 {"wTS": "1596440159", "rfId": "010-18741033", "remark": "数据", "weight": 66.858} 710078936338096128 1596440159-010-18741033 +139869188656553984 10000 1993-08-03T15:36:00.680 10000 1993-08-03T15:36:00.680 201 456678 false 1993-08-03T15:35:59 010-18741037 {"wTS": "1596440159", "rfId": "010-18741037", "remark": "数据", "weight": 139.592} 710078936338096128 1596440159-010-18741037 +139869189369585664 10000 1993-08-03T15:36:00.850 10000 1993-08-03T15:36:00.850 201 456678 false 1993-08-03T15:35:59 010-18741038 {"wTS": "1596440159", "rfId": "010-18741038", "remark": "数据", "weight": 122.591} 710078936338096128 1596440159-010-18741038 +139869190275555328 10000 1993-08-03T15:36:01.066 10000 1993-08-03T15:36:01.066 201 456678 false 1993-08-03T15:35:59 010-18741040 {"wTS": "1596440159", "rfId": "010-18741040", "remark": "数据", "weight": 133.627} 710078936338096128 1596440159-010-18741040 +139869238396805120 10000 1993-08-03T15:36:12.539 10000 1993-08-03T15:36:12.539 201 456678 false 1993-08-03T15:36:11 010-18741029 {"wTS": "1596440171", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440171-010-18741029 +139869238489079808 10000 1993-08-03T15:36:12.561 10000 1993-08-03T15:36:12.561 201 456678 false 1993-08-03T15:36:11 010-18741031 {"wTS": "1596440171", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440171-010-18741031 +139869239512489984 10000 1993-08-03T15:36:12.805 10000 1993-08-03T15:36:12.806 201 456678 false 1993-08-03T15:36:11 010-18741021 {"wTS": "1596440171", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440171-010-18741021 +139869241085353984 10000 1993-08-03T15:36:13.180 10000 1993-08-03T15:36:13.180 201 456678 false 1993-08-03T15:36:11 010-18741027 {"wTS": "1596440171", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440171-010-18741027 +139869241093742592 10000 1993-08-03T15:36:13.182 10000 1993-08-03T15:36:13.182 201 456678 false 1993-08-03T15:36:11 010-18741026 {"wTS": "1596440171", "rfId": "010-18741026", "remark": "数据", "weight": 147.465} 710078936338096128 1596440171-010-18741026 +139869241106325504 10000 1993-08-03T15:36:13.185 10000 1993-08-03T15:36:13.185 201 456678 false 1993-08-03T15:36:11 010-18741022 {"wTS": "1596440171", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440171-010-18741022 +139869241160851456 10000 1993-08-03T15:36:13.198 10000 1993-08-03T15:36:13.198 201 456678 false 1993-08-03T15:36:11 010-18741028 {"wTS": "1596440171", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440171-010-18741028 +139869241186017280 10000 1993-08-03T15:36:13.204 10000 1993-08-03T15:36:13.204 201 456678 false 1993-08-03T15:36:11 010-18741023 {"wTS": "1596440171", "rfId": "010-18741023", "remark": "数据", "weight": 118.114} 710078936338096128 1596440171-010-18741023 +139869241307652096 10000 1993-08-03T15:36:13.233 10000 1993-08-03T15:36:13.233 201 456678 false 1993-08-03T15:36:11 010-18741024 {"wTS": "1596440171", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440171-010-18741024 +139869242075209728 10000 1993-08-03T15:36:13.416 10000 1993-08-03T15:36:13.416 201 456678 false 1993-08-03T15:36:11 010-18741030 {"wTS": "1596440171", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440171-010-18741030 +139869243685822464 10000 1993-08-03T15:36:13.800 10000 1993-08-03T15:36:13.800 201 456678 false 1993-08-03T15:36:11 010-18741025 {"wTS": "1596440171", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440171-010-18741025 +139869311054720512 10000 1993-08-03T15:36:44.787 10000 1993-08-03T15:36:44.787 201 456678 false 1993-08-03T15:36:42 010-18741028 {"wTS": "1596440202", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440202-010-18741028 +139869366893502464 10000 1993-08-03T15:36:43.175 10000 1993-08-03T15:36:43.175 201 456678 false 1993-08-03T15:36:42 010-18741035 {"wTS": "1596440202", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440202-010-18741035 +139869367610728448 10000 1993-08-03T15:36:43.346 10000 1993-08-03T15:36:43.346 201 456678 false 1993-08-03T15:36:42 010-18741039 {"wTS": "1596440202", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440202-010-18741039 +139869368155987968 10000 1993-08-03T15:36:43.476 10000 1993-08-03T15:36:43.476 201 456678 false 1993-08-03T15:36:42 010-18741038 {"wTS": "1596440202", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440202-010-18741038 +139869368168570880 10000 1993-08-03T15:36:43.480 10000 1993-08-03T15:36:43.480 201 456678 false 1993-08-03T15:36:42 010-18741024 {"wTS": "1596440202", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440202-010-18741024 +139869368235679744 10000 1993-08-03T15:36:43.495 10000 1993-08-03T15:36:43.495 201 456678 false 1993-08-03T15:36:42 010-18741040 {"wTS": "1596440202", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440202-010-18741040 +139869368374091776 10000 1993-08-03T15:36:43.528 10000 1993-08-03T15:36:43.528 201 456678 false 1993-08-03T15:36:42 010-18741034 {"wTS": "1596440202", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440202-010-18741034 +139869368776744960 10000 1993-08-03T15:36:43.624 10000 1993-08-03T15:36:43.624 201 456678 false 1993-08-03T15:36:42 010-18741022 {"wTS": "1596440202", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440202-010-18741022 +139869370446077952 10000 1993-08-03T15:36:44.022 10000 1993-08-03T15:36:44.022 201 456678 false 1993-08-03T15:36:42 010-18741036 {"wTS": "1596440202", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440202-010-18741036 +139869372174131200 10000 1993-08-03T15:36:44.434 10000 1993-08-03T15:36:44.434 201 456678 false 1993-08-03T15:36:42 010-18741025 {"wTS": "1596440202", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440202-010-18741025 +139869373784743936 10000 1993-08-03T15:36:44.818 10000 1993-08-03T15:36:44.818 201 456678 false 1993-08-03T15:36:42 010-18741027 {"wTS": "1596440202", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440202-010-18741027 +139869374111899648 10000 1993-08-03T15:36:44.896 10000 1993-08-03T15:36:44.896 201 456678 false 1993-08-03T15:36:42 010-18741030 {"wTS": "1596440202", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440202-010-18741030 +139869374170619904 10000 1993-08-03T15:36:44.910 10000 1993-08-03T15:36:44.910 201 456678 false 1993-08-03T15:36:42 010-18741026 {"wTS": "1596440202", "rfId": "010-18741026", "remark": "数据", "weight": 147.465} 710078936338096128 1596440202-010-18741026 +139869374652964864 10000 1993-08-03T15:36:45.025 10000 1993-08-03T15:36:45.025 201 456678 false 1993-08-03T15:36:42 010-18741033 {"wTS": "1596440202", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440202-010-18741033 +139869376062251008 10000 1993-08-03T15:36:45.361 10000 1993-08-03T15:36:45.361 201 456678 false 1993-08-03T15:36:42 010-18741031 {"wTS": "1596440202", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440202-010-18741031 +139869376079028224 10000 1993-08-03T15:36:45.365 10000 1993-08-03T15:36:45.365 201 456678 false 1993-08-03T15:36:42 010-18741023 {"wTS": "1596440202", "rfId": "010-18741023", "remark": "数据", "weight": 118.114} 710078936338096128 1596440202-010-18741023 +139869376330686464 10000 1993-08-03T15:36:45.425 10000 1993-08-03T15:36:45.425 201 456678 false 1993-08-03T15:36:42 010-18741037 {"wTS": "1596440202", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440202-010-18741037 +139869376339075072 10000 1993-08-03T15:36:45.427 10000 1993-08-03T15:36:45.427 201 456678 false 1993-08-03T15:36:42 010-18741029 {"wTS": "1596440202", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440202-010-18741029 +139869377677058048 10000 1993-08-03T15:36:45.746 10000 1993-08-03T15:36:45.746 201 456678 false 1993-08-03T15:36:42 010-18741021 {"wTS": "1596440202", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440202-010-18741021 +139869379086344192 10000 1993-08-03T15:36:46.082 10000 1993-08-03T15:36:46.082 201 456678 false 1993-08-03T15:36:42 010-18741032 {"wTS": "1596440202", "rfId": "010-18741032", "remark": "数据", "weight": 118.863} 710078936338096128 1596440202-010-18741032 +139869494853328896 10000 1993-08-03T15:37:13.683 10000 1993-08-03T15:37:13.683 201 456678 false 1993-08-03T15:37:12 010-18741022 {"wTS": "1596440232", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440232-010-18741022 +139869494886883328 10000 1993-08-03T15:37:13.691 10000 1993-08-03T15:37:13.691 201 456678 false 1993-08-03T15:37:13 010-18741028 {"wTS": "1596440233", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440233-010-18741028 +139869494907854848 10000 1993-08-03T15:37:13.696 10000 1993-08-03T15:37:13.696 201 456678 false 1993-08-03T15:37:12 010-18741040 {"wTS": "1596440232", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440232-010-18741040 +139869494937214976 10000 1993-08-03T15:37:13.703 10000 1993-08-03T15:37:13.703 201 456678 false 1993-08-03T15:37:12 010-18741039 {"wTS": "1596440232", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440232-010-18741039 +139869495016906752 10000 1993-08-03T15:37:13.722 10000 1993-08-03T15:37:13.722 201 456678 false 1993-08-03T15:37:13 010-18741025 {"wTS": "1596440233", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440233-010-18741025 +139869495058849792 10000 1993-08-03T15:37:13.732 10000 1993-08-03T15:37:13.732 201 456678 false 1993-08-03T15:37:12 010-18741038 {"wTS": "1596440232", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440232-010-18741038 +139869495209844110 10000 1993-08-03T15:37:13.768 10000 1993-08-03T15:37:13.769 201 456678 false 1993-08-03T15:37:12 010-18741035 {"wTS": "1596440232", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440232-010-18741035 +139869495616692224 10000 1993-08-03T15:37:13.865 10000 1993-08-03T15:37:13.865 201 456678 false 1993-08-03T15:37:13 010-18741036 {"wTS": "1596440233", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440233-010-18741036 +139869496199700480 10000 1993-08-03T15:37:14.004 10000 1993-08-03T15:37:14.004 201 456678 false 1993-08-03T15:37:12 010-18741024 {"wTS": "1596440232", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440232-010-18741024 +139869496459747328 10000 1993-08-03T15:37:14.066 10000 1993-08-03T15:37:14.066 201 456678 false 1993-08-03T15:37:12 010-18741034 {"wTS": "1596440232", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440232-010-18741034 +139869497097281536 10000 1993-08-03T15:37:14.218 10000 1993-08-03T15:37:14.218 201 456678 false 1993-08-03T15:37:12 010-18741033 {"wTS": "1596440232", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440232-010-18741033 +139869498498179072 10000 1993-08-03T15:37:14.552 10000 1993-08-03T15:37:14.552 201 456678 false 1993-08-03T15:37:13 010-18741027 {"wTS": "1596440233", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440233-010-18741027 +139869500343672832 10000 1993-08-03T15:37:14.992 10000 1993-08-03T15:37:14.992 201 456678 false 1993-08-03T15:37:12 010-18741031 {"wTS": "1596440232", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440232-010-18741031 +139869507037782016 10000 1993-08-03T15:37:16.588 10000 1993-08-03T15:37:16.589 201 456678 false 1993-08-03T15:37:13 010-18741026 {"wTS": "1596440233", "rfId": "010-18741026", "remark": "数据", "weight": 147.465} 710078936338096128 1596440233-010-18741026 +139869507054559232 10000 1993-08-03T15:37:16.592 10000 1993-08-03T15:37:16.592 201 456678 false 1993-08-03T15:37:13 010-18741023 {"wTS": "1596440233", "rfId": "010-18741023", "remark": "数据", "weight": 118.114} 710078936338096128 1596440233-010-18741023 +139869507281051648 10000 1993-08-03T15:37:16.646 10000 1993-08-03T15:37:16.646 201 456678 false 1993-08-03T15:37:13 010-18741029 {"wTS": "1596440233", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440233-010-18741029 +139869507830505472 10000 1993-08-03T15:37:16.777 10000 1993-08-03T15:37:16.777 201 456678 false 1993-08-03T15:37:13 010-18741037 {"wTS": "1596440233", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440233-010-18741037 +139869508501594112 10000 1993-08-03T15:37:16.937 10000 1993-08-03T15:37:16.937 201 456678 false 1993-08-03T15:37:13 010-18741021 {"wTS": "1596440233", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440233-010-18741021 +139869509084602368 10000 1993-08-03T15:37:17.076 10000 1993-08-03T15:37:17.076 201 456678 false 1993-08-03T15:37:13 010-18741030 {"wTS": "1596440233", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440233-010-18741030 +139869509629861888 10000 1993-08-03T15:37:17.206 10000 1993-08-03T15:37:17.206 201 456678 false 1993-08-03T15:37:13 010-18741032 {"wTS": "1596440233", "rfId": "010-18741032", "remark": "数据", "weight": 118.863} 710078936338096128 1596440233-010-18741032 +139869619306717184 10000 1993-08-03T15:37:43.355 10000 1993-08-03T15:37:43.355 201 456678 false 1993-08-03T15:37:42 010-18741038 {"wTS": "1596440262", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440262-010-18741038 +139869620531453952 10000 1993-08-03T15:37:43.647 10000 1993-08-03T15:37:43.647 201 456678 false 1993-08-03T15:37:43 010-18741022 {"wTS": "1596440263", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440263-010-18741022 +139869620544036864 10000 1993-08-03T15:37:43.650 10000 1993-08-03T15:37:43.650 201 456678 false 1993-08-03T15:37:42 010-18741035 {"wTS": "1596440262", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440262-010-18741035 +139869620552425472 10000 1993-08-03T15:37:43.652 10000 1993-08-03T15:37:43.652 201 456678 false 1993-08-03T15:37:43 010-18741040 {"wTS": "1596440263", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440263-010-18741040 +139869620573396992 10000 1993-08-03T15:37:43.657 10000 1993-08-03T15:37:43.657 201 456678 false 1993-08-03T15:37:42 010-18741039 {"wTS": "1596440262", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440262-010-18741039 +139869620602757120 10000 1993-08-03T15:37:43.664 10000 1993-08-03T15:37:43.664 201 456678 false 1993-08-03T15:37:43 010-18741024 {"wTS": "1596440263", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440263-010-18741024 +139869620732780544 10000 1993-08-03T15:37:43.695 10000 1993-08-03T15:37:43.695 201 456678 false 1993-08-03T15:37:43 010-18741031 {"wTS": "1596440263", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440263-010-18741031 +139869621395480576 10000 1993-08-03T15:37:43.853 10000 1993-08-03T15:37:43.853 201 456678 false 1993-08-03T15:37:43 010-18741028 {"wTS": "1596440263", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440263-010-18741028 +139869623563935744 10000 1993-08-03T15:37:44.370 10000 1993-08-03T15:37:44.370 201 456678 false 1993-08-03T15:37:43 010-18741025 {"wTS": "1596440263", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440263-010-18741025 +139869626030186496 10000 1993-08-03T15:37:44.958 10000 1993-08-03T15:37:44.958 201 456678 false 1993-08-03T15:37:43 010-18741036 {"wTS": "1596440263", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440263-010-18741036 +139869626176987136 10000 1993-08-03T15:37:44.993 10000 1993-08-03T15:37:44.993 201 456678 false 1993-08-03T15:37:43 010-18741033 {"wTS": "1596440263", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440263-010-18741033 +139869626688692224 10000 1993-08-03T15:37:45.115 10000 1993-08-03T15:37:45.115 201 456678 false 1993-08-03T15:37:43 010-18741021 {"wTS": "1596440263", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440263-010-18741021 +139869627057790976 10000 1993-08-03T15:37:45.203 10000 1993-08-03T15:37:45.203 201 456678 false 1993-08-03T15:37:43 010-18741037 {"wTS": "1596440263", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440263-010-18741037 +139869627733073920 10000 1993-08-03T15:37:45.364 10000 1993-08-03T15:37:45.364 201 456678 false 1993-08-03T15:37:43 010-18741029 {"wTS": "1596440263", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440263-010-18741029 +139869628341248000 10000 1993-08-03T15:37:45.509 10000 1993-08-03T15:37:45.509 201 456678 false 1993-08-03T15:37:43 010-18741032 {"wTS": "1596440263", "rfId": "010-18741032", "remark": "数据", "weight": 118.863} 710078936338096128 1596440263-010-18741032 +139869628395773952 10000 1993-08-03T15:37:45.522 10000 1993-08-03T15:37:45.522 201 456678 false 1993-08-03T15:37:43 010-18741030 {"wTS": "1596440263", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440263-010-18741030 +139869629545013248 10000 1993-08-03T15:37:45.796 10000 1993-08-03T15:37:45.796 201 456678 false 1993-08-03T15:37:43 010-18741023 {"wTS": "1596440263", "rfId": "010-18741023", "remark": "数据", "weight": 118.114} 710078936338096128 1596440263-010-18741023 +139869631533113344 10000 1993-08-03T15:37:46.270 10000 1993-08-03T15:37:46.270 201 456678 false 1993-08-03T15:37:43 010-18741026 {"wTS": "1596440263", "rfId": "010-18741026", "remark": "数据", "weight": 147.465} 710078936338096128 1596440263-010-18741026 +139869631830908928 10000 1993-08-03T15:37:46.341 10000 1993-08-03T15:37:46.341 201 456678 false 1993-08-03T15:37:43 010-18741027 {"wTS": "1596440263", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440263-010-18741027 +139869744959676416 10000 1993-08-03T15:38:13.313 10000 1993-08-03T15:38:13.313 201 456678 false 1993-08-03T15:38:12 010-18741038 {"wTS": "1596440292", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440292-010-18741038 +139869745026785280 10000 1993-08-03T15:38:13.329 10000 1993-08-03T15:38:13.329 201 456678 false 1993-08-03T15:38:12 010-18741035 {"wTS": "1596440292", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440292-010-18741035 +139869745072922624 10000 1993-08-03T15:38:13.340 10000 1993-08-03T15:38:13.340 201 456678 false 1993-08-03T15:38:12 010-18741039 {"wTS": "1596440292", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440292-010-18741039 +139869745978892288 10000 1993-08-03T15:38:13.556 10000 1993-08-03T15:38:13.556 201 456678 false 1993-08-03T15:38:13 010-18741040 {"wTS": "1596440293", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440293-010-18741040 +139869746314436608 10000 1993-08-03T15:38:13.636 10000 1993-08-03T15:38:13.636 201 456678 false 1993-08-03T15:38:13 010-18741022 {"wTS": "1596440293", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440293-010-18741022 +139869746951970816 10000 1993-08-03T15:38:13.788 10000 1993-08-03T15:38:13.788 201 456678 false 1993-08-03T15:38:13 010-18741034 {"wTS": "1596440293", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440293-010-18741034 +139869747128131584 10000 1993-08-03T15:38:13.830 10000 1993-08-03T15:38:13.830 201 456678 false 1993-08-03T15:38:13 010-18741033 {"wTS": "1596440293", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440293-010-18741033 +139869747832774656 10000 1993-08-03T15:38:13.998 10000 1993-08-03T15:38:13.998 201 456678 false 1993-08-03T15:38:13 010-18741024 {"wTS": "1596440293", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440293-010-18741024 +139869749611000002 10000 1993-08-03T15:38:14.422 10000 1993-08-03T15:38:14.422 201 456678 false 1993-08-03T15:38:13 010-18741025 {"wTS": "1596440293", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440293-010-18741025 +139869751498596352 10000 1993-08-03T15:38:14.872 10000 1993-08-03T15:38:14.872 201 456678 false 1993-08-03T15:38:13 010-18741029 {"wTS": "1596440293", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440293-010-18741029 +139869751574093824 10000 1993-08-03T15:38:14.890 10000 1993-08-03T15:38:14.890 201 456678 false 1993-08-03T15:38:13 010-18741023 {"wTS": "1596440293", "rfId": "010-18741023", "remark": "数据", "weight": 118.114} 710078936338096128 1596440293-010-18741023 +139869751859306496 10000 1993-08-03T15:38:14.958 10000 1993-08-03T15:38:14.958 201 456678 false 1993-08-03T15:38:13 010-18741031 {"wTS": "1596440293", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440293-010-18741031 +139869752371011584 10000 1993-08-03T15:38:15.080 10000 1993-08-03T15:38:15.080 201 456678 false 1993-08-03T15:38:13 010-18741036 {"wTS": "1596440293", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440293-010-18741036 +139869754161979392 10000 1993-08-03T15:38:15.507 10000 1993-08-03T15:38:15.507 201 456678 false 1993-08-03T15:38:13 010-18741028 {"wTS": "1596440293", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440293-010-18741028 +139869754262642688 10000 1993-08-03T15:38:15.531 10000 1993-08-03T15:38:15.531 201 456678 false 1993-08-03T15:38:13 010-18741032 {"wTS": "1596440293", "rfId": "010-18741032", "remark": "数据", "weight": 118.863} 710078936338096128 1596440293-010-18741032 +139869755197972480 10000 1993-08-03T15:38:15.754 10000 1993-08-03T15:38:15.754 201 456678 false 1993-08-03T15:38:13 010-18741026 {"wTS": "1596440293", "rfId": "010-18741026", "remark": "数据", "weight": 147.465} 710078936338096128 1596440293-010-18741026 +139869756246548480 10000 1993-08-03T15:38:16.004 10000 1993-08-03T15:38:16.004 201 456678 false 1993-08-03T15:38:13 010-18741037 {"wTS": "1596440293", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440293-010-18741037 +139869756783419392 10000 1993-08-03T15:38:16.132 10000 1993-08-03T15:38:16.132 201 456678 false 1993-08-03T15:38:13 010-18741021 {"wTS": "1596440293", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440293-010-18741021 +139869871002705920 10000 1993-08-03T15:38:43.364 10000 1993-08-03T15:38:43.364 201 456678 false 1993-08-03T15:38:43 010-18741038 {"wTS": "1596440323", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440323-010-18741038 +139869871027871744 10000 1993-08-03T15:38:43.370 10000 1993-08-03T15:38:43.370 201 456678 false 1993-08-03T15:38:43 010-18741035 {"wTS": "1596440323", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440323-010-18741035 +139869871292112896 10000 1993-08-03T15:38:43.433 10000 1993-08-03T15:38:43.433 201 456678 false 1993-08-03T15:38:43 010-18741039 {"wTS": "1596440323", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440323-010-18741039 +139869872256802816 10000 1993-08-03T15:38:43.663 10000 1993-08-03T15:38:43.663 201 456678 false 1993-08-03T15:38:43 010-18741022 {"wTS": "1596440323", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440323-010-18741022 +139869872802062336 10000 1993-08-03T15:38:43.793 10000 1993-08-03T15:38:43.793 201 456678 false 1993-08-03T15:38:43 010-18741024 {"wTS": "1596440323", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440323-010-18741024 +139869872965640192 10000 1993-08-03T15:38:43.832 10000 1993-08-03T15:38:43.832 201 456678 false 1993-08-03T15:38:43 010-18741034 {"wTS": "1596440323", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440323-010-18741034 +139869873049526272 10000 1993-08-03T15:38:43.852 10000 1993-08-03T15:38:43.852 201 456678 false 1993-08-03T15:38:43 010-18741040 {"wTS": "1596440323", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440323-010-18741040 +139869874500755456 10000 1993-08-03T15:38:44.198 10000 1993-08-03T15:38:44.198 201 456678 false 1993-08-03T15:38:43 010-18741033 {"wTS": "1596440323", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440323-010-18741033 +139869875629023232 10000 1993-08-03T15:38:44.467 10000 1993-08-03T15:38:44.468 201 456678 false 1993-08-03T15:38:43 010-18741030 {"wTS": "1596440323", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440323-010-18741030 +139869877268996096 10000 1993-08-03T15:38:44.858 10000 1993-08-03T15:38:44.858 201 456678 false 1993-08-03T15:38:43 010-18741027 {"wTS": "1596440323", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440323-010-18741027 +139869877277384704 10000 1993-08-03T15:38:44.860 10000 1993-08-03T15:38:44.860 201 456678 false 1993-08-03T15:38:43 010-18741025 {"wTS": "1596440323", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440323-010-18741025 +139869877495488512 10000 1993-08-03T15:38:44.912 10000 1993-08-03T15:38:44.912 201 456678 false 1993-08-03T15:38:43 010-18741023 {"wTS": "1596440323", "rfId": "010-18741023", "remark": "数据", "weight": 118.114} 710078936338096128 1596440323-010-18741023 +139869878300794880 10000 1993-08-03T15:38:45.104 10000 1993-08-03T15:38:45.104 201 456678 false 1993-08-03T15:38:43 010-18741032 {"wTS": "1596440323", "rfId": "010-18741032", "remark": "数据", "weight": 118.863} 710078936338096128 1596440323-010-18741032 +139869880230174720 10000 1993-08-03T15:38:45.564 10000 1993-08-03T15:38:45.564 201 456678 false 1993-08-03T15:38:43 010-18741021 {"wTS": "1596440323", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440323-010-18741021 +139869997196730368 10000 1993-08-03T15:39:13.451 10000 1993-08-03T15:39:13.451 201 456678 false 1993-08-03T15:39:13 010-18741038 {"wTS": "1596440353", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440353-010-18741038 +139869997305782272 10000 1993-08-03T15:39:13.477 10000 1993-08-03T15:39:13.477 201 456678 false 1993-08-03T15:39:13 010-18741039 {"wTS": "1596440353", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440353-010-18741039 +139869997645520896 10000 1993-08-03T15:39:13.558 10000 1993-08-03T15:39:13.558 201 456678 false 1993-08-03T15:39:13 010-18741035 {"wTS": "1596440353", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440353-010-18741035 +139869997792321536 10000 1993-08-03T15:39:13.593 10000 1993-08-03T15:39:13.593 201 456678 false 1993-08-03T15:39:13 010-18741022 {"wTS": "1596440353", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440353-010-18741022 +139869999402934272 10000 1993-08-03T15:39:13.977 10000 1993-08-03T15:39:13.977 201 456678 false 1993-08-03T15:39:13 010-18741034 {"wTS": "1596440353", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440353-010-18741034 +139869999667175424 10000 1993-08-03T15:39:14.040 10000 1993-08-03T15:39:14.040 201 456678 false 1993-08-03T15:39:13 010-18741040 {"wTS": "1596440353", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440353-010-18741040 +139870001030324224 10000 1993-08-03T15:39:14.365 10000 1993-08-03T15:39:14.365 201 456678 false 1993-08-03T15:39:13 010-18741033 {"wTS": "1596440353", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440353-010-18741033 +139870001357479936 10000 1993-08-03T15:39:14.443 10000 1993-08-03T15:39:14.443 201 456678 false 1993-08-03T15:39:13 010-18741031 {"wTS": "1596440353", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440353-010-18741031 +139870003332997120 10000 1993-08-03T15:39:14.914 10000 1993-08-03T15:39:14.914 201 456678 false 1993-08-03T15:39:13 010-18741024 {"wTS": "1596440353", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440353-010-18741024 +139870003379134464 10000 1993-08-03T15:39:14.925 10000 1993-08-03T15:39:14.925 201 456678 false 1993-08-03T15:39:13 010-18741028 {"wTS": "1596440353", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440353-010-18741028 +139870003387523072 10000 1993-08-03T15:39:14.927 10000 1993-08-03T15:39:14.927 201 456678 false 1993-08-03T15:39:13 010-18741036 {"wTS": "1596440353", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440353-010-18741036 +139870003450437632 10000 1993-08-03T15:39:14.942 10000 1993-08-03T15:39:14.942 201 456678 false 1993-08-03T15:39:13 010-18741037 {"wTS": "1596440353", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440353-010-18741037 +139870004415127552 10000 1993-08-03T15:39:15.172 10000 1993-08-03T15:39:15.172 201 456678 false 1993-08-03T15:39:13 010-18741023 {"wTS": "1596440353", "rfId": "010-18741023", "remark": "数据", "weight": 118.114} 710078936338096128 1596440353-010-18741023 +139870006185123840 10000 1993-08-03T15:39:15.594 10000 1993-08-03T15:39:15.594 201 456678 false 1993-08-03T15:39:13 010-18741029 {"wTS": "1596440353", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440353-010-18741029 +139870006772326400 10000 1993-08-03T15:39:15.734 10000 1993-08-03T15:39:15.734 201 456678 false 1993-08-03T15:39:13 010-18741025 {"wTS": "1596440353", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440353-010-18741025 +139870007275642880 10000 1993-08-03T15:39:15.854 10000 1993-08-03T15:39:15.854 201 456678 false 1993-08-03T15:39:13 010-18741026 {"wTS": "1596440353", "rfId": "010-18741026", "remark": "数据", "weight": 147.465} 710078936338096128 1596440353-010-18741026 +139870007862845440 10000 1993-08-03T15:39:15.994 10000 1993-08-03T15:39:15.994 201 456678 false 1993-08-03T15:39:13 010-18741030 {"wTS": "1596440353", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440353-010-18741030 +139870008290664448 10000 1993-08-03T15:39:16.096 10000 1993-08-03T15:39:16.096 201 456678 false 1993-08-03T15:39:13 010-18741027 {"wTS": "1596440353", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440353-010-18741027 +139870008965947392 10000 1993-08-03T15:39:16.257 10000 1993-08-03T15:39:16.257 201 456678 false 1993-08-03T15:39:13 010-18741032 {"wTS": "1596440353", "rfId": "010-18741032", "remark": "数据", "weight": 118.863} 710078936338096128 1596440353-010-18741032 +139870123344617472 10000 1993-08-03T15:39:43.527 10000 1993-08-03T15:39:43.527 201 456678 false 1993-08-03T15:39:43 010-18741038 {"wTS": "1596440383", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440383-010-18741038 +139870123432697856 10000 1993-08-03T15:39:43.548 10000 1993-08-03T15:39:43.549 201 456678 false 1993-08-03T15:39:43 010-18741039 {"wTS": "1596440383", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440383-010-18741039 +139870123650801664 10000 1993-08-03T15:39:43.600 10000 1993-08-03T15:39:43.600 201 456678 false 1993-08-03T15:39:43 010-18741035 {"wTS": "1596440383", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440383-010-18741035 +139870124363833344 10000 1993-08-03T15:39:43.770 10000 1993-08-03T15:39:43.770 201 456678 false 1993-08-03T15:39:43 010-18741022 {"wTS": "1596440383", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440383-010-18741022 +139870125013950464 10000 1993-08-03T15:39:43.925 10000 1993-08-03T15:39:43.925 201 456678 false 1993-08-03T15:39:43 010-18741034 {"wTS": "1596440383", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440383-010-18741034 +139870125345300480 10000 1993-08-03T15:39:44.004 10000 1993-08-03T15:39:44.004 201 456678 false 1993-08-03T15:39:43 010-18741040 {"wTS": "1596440383", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440383-010-18741040 +139870126607785984 10000 1993-08-03T15:39:44.305 10000 1993-08-03T15:39:44.305 201 456678 false 1993-08-03T15:39:43 010-18741028 {"wTS": "1596440383", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440383-010-18741028 +139870127023022080 10000 1993-08-03T15:39:44.404 10000 1993-08-03T15:39:44.404 201 456678 false 1993-08-03T15:39:43 010-18741024 {"wTS": "1596440383", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440383-010-18741024 +139870127559892992 10000 1993-08-03T15:39:44.532 10000 1993-08-03T15:39:44.532 201 456678 false 1993-08-03T15:39:43 010-18741033 {"wTS": "1596440383", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440383-010-18741033 +139870128121929728 10000 1993-08-03T15:39:44.666 10000 1993-08-03T15:39:44.666 201 456678 false 1993-08-03T15:39:43 010-18741023 {"wTS": "1596440383", "rfId": "010-18741023", "remark": "数据", "weight": 118.114} 710078936338096128 1596440383-010-18741023 +139870129015316480 10000 1993-08-03T15:39:44.879 10000 1993-08-03T15:39:44.879 201 456678 false 1993-08-03T15:39:43 010-18741037 {"wTS": "1596440383", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440383-010-18741037 +139870129606713344 10000 1993-08-03T15:39:45.020 10000 1993-08-03T15:39:45.020 201 456678 false 1993-08-03T15:39:43 010-18741021 {"wTS": "1596440383", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440383-010-18741021 +139870130911141888 10000 1993-08-03T15:39:45.331 10000 1993-08-03T15:39:45.331 201 456678 false 1993-08-03T15:39:43 010-18741031 {"wTS": "1596440383", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440383-010-18741031 +139870130936307712 10000 1993-08-03T15:39:45.337 10000 1993-08-03T15:39:45.337 201 456678 false 1993-08-03T15:39:43 010-18741036 {"wTS": "1596440383", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440383-010-18741036 +139870130969862144 10000 1993-08-03T15:39:45.345 10000 1993-08-03T15:39:45.345 201 456678 false 1993-08-03T15:39:43 010-18741025 {"wTS": "1596440383", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440383-010-18741025 +139870131003416576 10000 1993-08-03T15:39:45.353 10000 1993-08-03T15:39:45.353 201 456678 false 1993-08-03T15:39:43 010-18741029 {"wTS": "1596440383", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440383-010-18741029 +139870249345703936 10000 1993-08-03T15:40:13.568 10000 1993-08-03T15:40:13.568 201 456678 false 1993-08-03T15:40:13 010-18741038 {"wTS": "1596440413", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440413-010-18741038 +139870250390085632 10000 1993-08-03T15:40:13.817 10000 1993-08-03T15:40:13.817 201 456678 false 1993-08-03T15:40:13 010-18741022 {"wTS": "1596440413", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440413-010-18741022 +139870250432028672 10000 1993-08-03T15:40:13.827 10000 1993-08-03T15:40:13.827 201 456678 false 1993-08-03T15:40:13 010-18741034 {"wTS": "1596440413", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440413-010-18741034 +139870250453000192 10000 1993-08-03T15:40:13.832 10000 1993-08-03T15:40:13.832 201 456678 false 1993-08-03T15:40:13 010-18741039 {"wTS": "1596440413", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440413-010-18741039 +139870250490748928 10000 1993-08-03T15:40:13.841 10000 1993-08-03T15:40:13.841 201 456678 false 1993-08-03T15:40:13 010-18741035 {"wTS": "1596440413", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440413-010-18741035 +139870250880819200 10000 1993-08-03T15:40:13.934 10000 1993-08-03T15:40:13.934 201 456678 false 1993-08-03T15:40:13 010-18741024 {"wTS": "1596440413", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440413-010-18741024 +139870251040202752 10000 1993-08-03T15:40:13.972 10000 1993-08-03T15:40:13.972 201 456678 false 1993-08-03T15:40:13 010-18741033 {"wTS": "1596440413", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440413-010-18741033 +139870251698708480 10000 1993-08-03T15:40:14.129 10000 1993-08-03T15:40:14.129 201 456678 false 1993-08-03T15:40:13 010-18741040 {"wTS": "1596440413", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440413-010-18741040 +139870253548396544 10000 1993-08-03T15:40:14.570 10000 1993-08-03T15:40:14.570 201 456678 false 1993-08-03T15:40:13 010-18741027 {"wTS": "1596440413", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440413-010-18741027 +139870254777327616 10000 1993-08-03T15:40:14.863 10000 1993-08-03T15:40:14.863 201 456678 false 1993-08-03T15:40:13 010-18741030 {"wTS": "1596440413", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440413-010-18741030 +139870256186613760 10000 1993-08-03T15:40:15.199 10000 1993-08-03T15:40:15.199 201 456678 false 1993-08-03T15:40:13 010-18741026 {"wTS": "1596440413", "rfId": "010-18741026", "remark": "数据", "weight": 147.465} 710078936338096128 1596440413-010-18741026 +139870256429883392 10000 1993-08-03T15:40:15.257 10000 1993-08-03T15:40:15.257 201 456678 false 1993-08-03T15:40:13 010-18741021 {"wTS": "1596440413", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440413-010-18741021 +139870257587511296 10000 1993-08-03T15:40:15.533 10000 1993-08-03T15:40:15.533 201 456678 false 1993-08-03T15:40:13 010-18741037 {"wTS": "1596440413", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440413-010-18741037 +139870257650425856 10000 1993-08-03T15:40:15.548 10000 1993-08-03T15:40:15.548 201 456678 false 1993-08-03T15:40:13 010-18741032 {"wTS": "1596440413", "rfId": "010-18741032", "remark": "数据", "weight": 118.863} 710078936338096128 1596440413-010-18741032 +139870375501979648 10000 1993-08-03T15:40:43.646 10000 1993-08-03T15:40:43.646 201 456678 false 1993-08-03T15:40:43 010-18741035 {"wTS": "1596440443", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440443-010-18741035 +139870375611031552 10000 1993-08-03T15:40:43.672 10000 1993-08-03T15:40:43.672 201 456678 false 1993-08-03T15:40:43 010-18741038 {"wTS": "1596440443", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440443-010-18741038 +139870375803969536 10000 1993-08-03T15:40:43.718 10000 1993-08-03T15:40:43.718 201 456678 false 1993-08-03T15:40:43 010-18741039 {"wTS": "1596440443", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440443-010-18741039 +139870376328257536 10000 1993-08-03T15:40:43.843 10000 1993-08-03T15:40:43.843 201 456678 false 1993-08-03T15:40:43 010-18741022 {"wTS": "1596440443", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440443-010-18741022 +139870376856739840 10000 1993-08-03T15:40:43.969 10000 1993-08-03T15:40:43.969 201 456678 false 1993-08-03T15:40:43 010-18741034 {"wTS": "1596440443", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440443-010-18741034 +139870377087426560 10000 1993-08-03T15:40:44.024 10000 1993-08-03T15:40:44.024 201 456678 false 1993-08-03T15:40:43 010-18741024 {"wTS": "1596440443", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440443-010-18741024 +139870378702233600 10000 1993-08-03T15:40:44.409 10000 1993-08-03T15:40:44.409 201 456678 false 1993-08-03T15:40:43 010-18741033 {"wTS": "1596440443", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440443-010-18741033 +139870378828062720 10000 1993-08-03T15:40:44.440 10000 1993-08-03T15:40:44.440 201 456678 false 1993-08-03T15:40:43 010-18741040 {"wTS": "1596440443", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440443-010-18741040 +139870380052799488 10000 1993-08-03T15:40:44.731 10000 1993-08-03T15:40:44.731 201 456678 false 1993-08-03T15:40:43 010-18741030 {"wTS": "1596440443", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440443-010-18741030 +139870380765831168 10000 1993-08-03T15:40:44.901 10000 1993-08-03T15:40:44.901 201 456678 false 1993-08-03T15:40:44 010-18741037 {"wTS": "1596440444", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440444-010-18741037 +139870381059432448 10000 1993-08-03T15:40:44.971 10000 1993-08-03T15:40:44.971 201 456678 false 1993-08-03T15:40:43 010-18741036 {"wTS": "1596440443", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440443-010-18741036 +139870381566943232 10000 1993-08-03T15:40:45.092 10000 1993-08-03T15:40:45.092 201 456678 false 1993-08-03T15:40:43 010-18741031 {"wTS": "1596440443", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440443-010-18741031 +139870383727009792 10000 1993-08-03T15:40:45.607 10000 1993-08-03T15:40:45.607 201 456678 false 1993-08-03T15:40:43 010-18741028 {"wTS": "1596440443", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440443-010-18741028 +139870384465207296 10000 1993-08-03T15:40:45.783 10000 1993-08-03T15:40:45.783 201 456678 false 1993-08-03T15:40:43 010-18741027 {"wTS": "1596440443", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440443-010-18741027 +139870385815773184 10000 1993-08-03T15:40:46.105 10000 1993-08-03T15:40:46.105 201 456678 false 1993-08-03T15:40:43 010-18741029 {"wTS": "1596440443", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440443-010-18741029 +139870385983545344 10000 1993-08-03T15:40:46.145 10000 1993-08-03T15:40:46.145 201 456678 false 1993-08-03T15:40:43 010-18741025 {"wTS": "1596440443", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440443-010-18741025 +139870385987739648 10000 1993-08-03T15:40:46.146 10000 1993-08-03T15:40:46.146 201 456678 false 1993-08-03T15:40:44 010-18741032 {"wTS": "1596440444", "rfId": "010-18741032", "remark": "数据", "weight": 118.863} 710078936338096128 1596440444-010-18741032 +139870386075820032 10000 1993-08-03T15:40:46.167 10000 1993-08-03T15:40:46.167 201 456678 false 1993-08-03T15:40:43 010-18741023 {"wTS": "1596440443", "rfId": "010-18741023", "remark": "数据", "weight": 118.114} 710078936338096128 1596440443-010-18741023 +139870386138734592 10000 1993-08-03T15:40:46.182 10000 1993-08-03T15:40:46.182 201 456678 false 1993-08-03T15:40:43 010-18741021 {"wTS": "1596440443", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440443-010-18741021 +139870501465317376 10000 1993-08-03T15:41:13.678 10000 1993-08-03T15:41:13.678 201 456678 false 1993-08-03T15:41:13 010-18741038 {"wTS": "1596440473", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440473-010-18741038 +139870502438395904 10000 1993-08-03T15:41:13.910 10000 1993-08-03T15:41:13.910 201 456678 false 1993-08-03T15:41:13 010-18741039 {"wTS": "1596440473", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440473-010-18741039 +139870502526476288 10000 1993-08-03T15:41:13.931 10000 1993-08-03T15:41:13.931 201 456678 false 1993-08-03T15:41:13 010-18741022 {"wTS": "1596440473", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440473-010-18741022 +139870502539059200 10000 1993-08-03T15:41:13.934 10000 1993-08-03T15:41:13.934 201 456678 false 1993-08-03T15:41:13 010-18741035 {"wTS": "1596440473", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440473-010-18741035 +139870502673276928 10000 1993-08-03T15:41:13.966 10000 1993-08-03T15:41:13.966 201 456678 false 1993-08-03T15:41:13 010-18741040 {"wTS": "1596440473", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440473-010-18741040 +139870502711025664 10000 1993-08-03T15:41:13.975 10000 1993-08-03T15:41:13.975 201 456678 false 1993-08-03T15:41:13 010-18741034 {"wTS": "1596440473", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440473-010-18741034 +139870503939956110 10000 1993-08-03T15:41:14.268 10000 1993-08-03T15:41:14.268 201 456678 false 1993-08-03T15:41:13 010-18741024 {"wTS": "1596440473", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440473-010-18741024 +139870506985021440 10000 1993-08-03T15:41:14.994 10000 1993-08-03T15:41:14.994 201 456678 false 1993-08-03T15:41:13 010-18741033 {"wTS": "1596440473", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440473-010-18741033 +139870507026964480 10000 1993-08-03T15:41:15.004 10000 1993-08-03T15:41:15.004 201 456678 false 1993-08-03T15:41:13 010-18741030 {"wTS": "1596440473", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440473-010-18741030 +139870507123433472 10000 1993-08-03T15:41:15.027 10000 1993-08-03T15:41:15.027 201 456678 false 1993-08-03T15:41:13 010-18741029 {"wTS": "1596440473", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440473-010-18741029 +139870507173765120 10000 1993-08-03T15:41:15.039 10000 1993-08-03T15:41:15.039 201 456678 false 1993-08-03T15:41:13 010-18741026 {"wTS": "1596440473", "rfId": "010-18741026", "remark": "数据", "weight": 147.465} 710078936338096128 1596440473-010-18741026 +139870508058763264 10000 1993-08-03T15:41:15.250 10000 1993-08-03T15:41:15.250 201 456678 false 1993-08-03T15:41:13 010-18741025 {"wTS": "1596440473", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440473-010-18741025 +139870508696297472 10000 1993-08-03T15:41:15.402 10000 1993-08-03T15:41:15.402 201 456678 false 1993-08-03T15:41:14 010-18741032 {"wTS": "1596440474", "rfId": "010-18741032", "remark": "数据", "weight": 118.863} 710078936338096128 1596440474-010-18741032 +139870509912645632 10000 1993-08-03T15:41:15.692 10000 1993-08-03T15:41:15.692 201 456678 false 1993-08-03T15:41:14 010-18741037 {"wTS": "1596440474", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440474-010-18741037 +139870627244105728 10000 1993-08-03T15:41:43.666 10000 1993-08-03T15:41:43.666 201 456678 false 1993-08-03T15:41:43 010-18741038 {"wTS": "1596440503", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440503-010-18741038 +139870627609010176 10000 1993-08-03T15:41:43.753 10000 1993-08-03T15:41:43.753 201 456678 false 1993-08-03T15:41:43 010-18741039 {"wTS": "1596440503", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440503-010-18741039 +139870627869057024 10000 1993-08-03T15:41:43.815 10000 1993-08-03T15:41:43.815 201 456678 false 1993-08-03T15:41:43 010-18741035 {"wTS": "1596440503", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440503-010-18741035 +139870628359790592 10000 1993-08-03T15:41:43.932 10000 1993-08-03T15:41:43.932 201 456678 false 1993-08-03T15:41:43 010-18741022 {"wTS": "1596440503", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440503-010-18741022 +139870628686946304 10000 1993-08-03T15:41:44.010 10000 1993-08-03T15:41:44.010 201 456678 false 1993-08-03T15:41:43 010-18741034 {"wTS": "1596440503", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440503-010-18741034 +139870629555167232 10000 1993-08-03T15:41:44.217 10000 1993-08-03T15:41:44.217 201 456678 false 1993-08-03T15:41:43 010-18741040 {"wTS": "1596440503", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440503-010-18741040 +139870629928460288 10000 1993-08-03T15:41:44.306 10000 1993-08-03T15:41:44.306 201 456678 false 1993-08-03T15:41:43 010-18741024 {"wTS": "1596440503", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440503-010-18741024 +139870630901538816 10000 1993-08-03T15:41:44.538 10000 1993-08-03T15:41:44.538 201 456678 false 1993-08-03T15:41:43 010-18741033 {"wTS": "1596440503", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440503-010-18741033 +139870632268881920 10000 1993-08-03T15:41:44.864 10000 1993-08-03T15:41:44.864 201 456678 false 1993-08-03T15:41:43 010-18741031 {"wTS": "1596440503", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440503-010-18741031 +139870632424071168 10000 1993-08-03T15:41:44.901 10000 1993-08-03T15:41:44.901 201 456678 false 1993-08-03T15:41:43 010-18741036 {"wTS": "1596440503", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440503-010-18741036 +139870632436654080 10000 1993-08-03T15:41:44.904 10000 1993-08-03T15:41:44.904 201 456678 false 1993-08-03T15:41:43 010-18741027 {"wTS": "1596440503", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440503-010-18741027 +139870632596037632 10000 1993-08-03T15:41:44.942 10000 1993-08-03T15:41:44.942 201 456678 false 1993-08-03T15:41:43 010-18741028 {"wTS": "1596440503", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440503-010-18741028 +139870632826724352 10000 1993-08-03T15:41:44.997 10000 1993-08-03T15:41:44.997 201 456678 false 1993-08-03T15:41:44 010-18741029 {"wTS": "1596440504", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440504-010-18741029 +139870634311507968 10000 1993-08-03T15:41:45.351 10000 1993-08-03T15:41:45.352 201 456678 false 1993-08-03T15:41:44 010-18741026 {"wTS": "1596440504", "rfId": "010-18741026", "remark": "数据", "weight": 147.465} 710078936338096128 1596440504-010-18741026 +139870636047949824 10000 1993-08-03T15:41:45.765 10000 1993-08-03T15:41:45.765 201 456678 false 1993-08-03T15:41:44 010-18741037 {"wTS": "1596440504", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440504-010-18741037 +139870636157001728 10000 1993-08-03T15:41:45.791 10000 1993-08-03T15:41:45.791 201 456678 false 1993-08-03T15:41:43 010-18741030 {"wTS": "1596440503", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440503-010-18741030 +139870636282830848 10000 1993-08-03T15:41:45.821 10000 1993-08-03T15:41:45.821 201 456678 false 1993-08-03T15:41:44 010-18741025 {"wTS": "1596440504", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440504-010-18741025 +139870636903587840 10000 1993-08-03T15:41:45.969 10000 1993-08-03T15:41:45.969 201 456678 false 1993-08-03T15:41:44 010-18741023 {"wTS": "1596440504", "rfId": "010-18741023", "remark": "数据", "weight": 118.114} 710078936338096128 1596440504-010-18741023 +139870637834723328 10000 1993-08-03T15:41:46.191 10000 1993-08-03T15:41:46.191 201 456678 false 1993-08-03T15:41:44 010-18741021 {"wTS": "1596440504", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440504-010-18741021 +139870638811996160 10000 1993-08-03T15:41:46.424 10000 1993-08-03T15:41:46.424 201 456678 false 1993-08-03T15:41:44 010-18741032 {"wTS": "1596440504", "rfId": "010-18741032", "remark": "数据", "weight": 118.863} 710078936338096128 1596440504-010-18741032 +139870753387798528 10000 1993-08-03T15:42:13.741 10000 1993-08-03T15:42:13.741 201 456678 false 1993-08-03T15:42:13 010-18741038 {"wTS": "1596440533", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440533-010-18741038 +139870753656233984 10000 1993-08-03T15:42:13.805 10000 1993-08-03T15:42:13.805 201 456678 false 1993-08-03T15:42:13 010-18741039 {"wTS": "1596440533", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440533-010-18741039 +139870753794646016 10000 1993-08-03T15:42:13.838 10000 1993-08-03T15:42:13.838 201 456678 false 1993-08-03T15:42:13 010-18741035 {"wTS": "1596440533", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440533-010-18741035 +139870755124240384 10000 1993-08-03T15:42:14.155 10000 1993-08-03T15:42:14.155 201 456678 false 1993-08-03T15:42:13 010-18741022 {"wTS": "1596440533", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440533-010-18741022 +139870755224903680 10000 1993-08-03T15:42:14.179 10000 1993-08-03T15:42:14.179 201 456678 false 1993-08-03T15:42:13 010-18741034 {"wTS": "1596440533", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440533-010-18741034 +139870756336394240 10000 1993-08-03T15:42:14.444 10000 1993-08-03T15:42:14.444 201 456678 false 1993-08-03T15:42:13 010-18741024 {"wTS": "1596440533", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440533-010-18741024 +139870758077030400 10000 1993-08-03T15:42:14.859 10000 1993-08-03T15:42:14.859 201 456678 false 1993-08-03T15:42:13 010-18741040 {"wTS": "1596440533", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440533-010-18741040 +139870758123167744 10000 1993-08-03T15:42:14.870 10000 1993-08-03T15:42:14.870 201 456678 false 1993-08-03T15:42:13 010-18741028 {"wTS": "1596440533", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440533-010-18741028 +139870758207053824 10000 1993-08-03T15:42:14.890 10000 1993-08-03T15:42:14.890 201 456678 false 1993-08-03T15:42:13 010-18741033 {"wTS": "1596440533", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440533-010-18741033 +139870759146577920 10000 1993-08-03T15:42:15.114 10000 1993-08-03T15:42:15.114 201 456678 false 1993-08-03T15:42:13 010-18741036 {"wTS": "1596440533", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440533-010-18741036 +139870760664915968 10000 1993-08-03T15:42:15.476 10000 1993-08-03T15:42:15.476 201 456678 false 1993-08-03T15:42:13 010-18741031 {"wTS": "1596440533", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440533-010-18741031 +139870760815910912 10000 1993-08-03T15:42:15.512 10000 1993-08-03T15:42:15.512 201 456678 false 1993-08-03T15:42:13 010-18741027 {"wTS": "1596440533", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440533-010-18741027 +139870762791428096 10000 1993-08-03T15:42:15.983 10000 1993-08-03T15:42:15.983 201 456678 false 1993-08-03T15:42:14 010-18741023 {"wTS": "1596440534", "rfId": "010-18741023", "remark": "数据", "weight": 118.114} 710078936338096128 1596440534-010-18741023 +139870764880191488 10000 1993-08-03T15:42:16.481 10000 1993-08-03T15:42:16.481 201 456678 false 1993-08-03T15:42:14 010-18741021 {"wTS": "1596440534", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440534-010-18741021 +139870764964077568 10000 1993-08-03T15:42:16.501 10000 1993-08-03T15:42:16.501 201 456678 false 1993-08-03T15:42:14 010-18741032 {"wTS": "1596440534", "rfId": "010-18741032", "remark": "数据", "weight": 118.863} 710078936338096128 1596440534-010-18741032 +139870765077323776 10000 1993-08-03T15:42:16.528 10000 1993-08-03T15:42:16.528 201 456678 false 1993-08-03T15:42:14 010-18741037 {"wTS": "1596440534", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440534-010-18741037 +139870879627960320 10000 1993-08-03T15:42:43.839 10000 1993-08-03T15:42:43.839 201 456678 false 1993-08-03T15:42:43 010-18741038 {"wTS": "1596440563", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440563-010-18741038 +139870879699263488 10000 1993-08-03T15:42:43.856 10000 1993-08-03T15:42:43.856 201 456678 false 1993-08-03T15:42:43 010-18741039 {"wTS": "1596440563", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440563-010-18741039 +139870879997059072 10000 1993-08-03T15:42:43.927 10000 1993-08-03T15:42:43.927 201 456678 false 1993-08-03T15:42:43 010-18741035 {"wTS": "1596440563", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440563-010-18741035 +139870882274566144 10000 1993-08-03T15:42:44.470 10000 1993-08-03T15:42:44.470 201 456678 false 1993-08-03T15:42:43 010-18741040 {"wTS": "1596440563", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440563-010-18741040 +139870882735939584 10000 1993-08-03T15:42:44.580 10000 1993-08-03T15:42:44.580 201 456678 false 1993-08-03T15:42:43 010-18741024 {"wTS": "1596440563", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440563-010-18741024 +139870883767738368 10000 1993-08-03T15:42:44.826 10000 1993-08-03T15:42:44.826 201 456678 false 1993-08-03T15:42:43 010-18741036 {"wTS": "1596440563", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440563-010-18741036 +139870883843235840 10000 1993-08-03T15:42:44.844 10000 1993-08-03T15:42:44.844 201 456678 false 1993-08-03T15:42:43 010-18741033 {"wTS": "1596440563", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440563-010-18741033 +139870883943899136 10000 1993-08-03T15:42:44.868 10000 1993-08-03T15:42:44.868 201 456678 false 1993-08-03T15:42:43 010-18741031 {"wTS": "1596440563", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440563-010-18741031 +139870884774371328 10000 1993-08-03T15:42:45.066 10000 1993-08-03T15:42:45.066 201 456678 false 1993-08-03T15:42:44 010-18741030 {"wTS": "1596440564", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440564-010-18741030 +139870885562900480 10000 1993-08-03T15:42:45.254 10000 1993-08-03T15:42:45.254 201 456678 false 1993-08-03T15:42:43 010-18741028 {"wTS": "1596440563", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440563-010-18741028 +139870887806853120 10000 1993-08-03T15:42:45.789 10000 1993-08-03T15:42:45.789 201 456678 false 1993-08-03T15:42:44 010-18741025 {"wTS": "1596440564", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440564-010-18741025 +139870889190973440 10000 1993-08-03T15:42:46.119 10000 1993-08-03T15:42:46.119 201 456678 false 1993-08-03T15:42:44 010-18741026 {"wTS": "1596440564", "rfId": "010-18741026", "remark": "数据", "weight": 147.465} 710078936338096128 1596440564-010-18741026 +139870889476186112 10000 1993-08-03T15:42:46.187 10000 1993-08-03T15:42:46.187 201 456678 false 1993-08-03T15:42:44 010-18741029 {"wTS": "1596440564", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440564-010-18741029 +139871005406748672 10000 1993-08-03T15:43:13.827 10000 1993-08-03T15:43:13.827 201 456678 false 1993-08-03T15:43:13 010-18741038 {"wTS": "1596440593", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440593-010-18741038 +139871005830373376 10000 1993-08-03T15:43:13.928 10000 1993-08-03T15:43:13.928 201 456678 false 1993-08-03T15:43:13 010-18741039 {"wTS": "1596440593", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440593-010-18741039 +139871006937669632 10000 1993-08-03T15:43:14.192 10000 1993-08-03T15:43:14.192 201 456678 false 1993-08-03T15:43:13 010-18741022 {"wTS": "1596440593", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440593-010-18741022 +139871007000584192 10000 1993-08-03T15:43:14.207 10000 1993-08-03T15:43:14.207 201 456678 false 1993-08-03T15:43:13 010-18741035 {"wTS": "1596440593", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440593-010-18741035 +139871007210299392 10000 1993-08-03T15:43:14.257 10000 1993-08-03T15:43:14.257 201 456678 false 1993-08-03T15:43:13 010-18741034 {"wTS": "1596440593", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440593-010-18741034 +139871008309207040 10000 1993-08-03T15:43:14.519 10000 1993-08-03T15:43:14.519 201 456678 false 1993-08-03T15:43:13 010-18741033 {"wTS": "1596440593", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440593-010-18741033 +139871009856905216 10000 1993-08-03T15:43:14.888 10000 1993-08-03T15:43:14.888 201 456678 false 1993-08-03T15:43:14 010-18741027 {"wTS": "1596440594", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440594-010-18741027 +139871009932402688 10000 1993-08-03T15:43:14.906 10000 1993-08-03T15:43:14.906 201 456678 false 1993-08-03T15:43:14 010-18741023 {"wTS": "1596440594", "rfId": "010-18741023", "remark": "数据", "weight": 118.114} 710078936338096128 1596440594-010-18741023 +139871011622707200 10000 1993-08-03T15:43:15.309 10000 1993-08-03T15:43:15.309 201 456678 false 1993-08-03T15:43:14 010-18741021 {"wTS": "1596440594", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440594-010-18741021 +139871011752730624 10000 1993-08-03T15:43:15.340 10000 1993-08-03T15:43:15.340 201 456678 false 1993-08-03T15:43:14 010-18741032 {"wTS": "1596440594", "rfId": "010-18741032", "remark": "数据", "weight": 118.863} 710078936338096128 1596440594-010-18741032 +139871011983417344 10000 1993-08-03T15:43:15.395 10000 1993-08-03T15:43:15.395 201 456678 false 1993-08-03T15:43:14 010-18741037 {"wTS": "1596440594", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440594-010-18741037 +139871131340726272 10000 1993-08-03T15:43:43.852 10000 1993-08-03T15:43:43.852 201 456678 false 1993-08-03T15:43:43 010-18741038 {"wTS": "1596440623", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440623-010-18741038 +139871131785322496 10000 1993-08-03T15:43:43.958 10000 1993-08-03T15:43:43.958 201 456678 false 1993-08-03T15:43:43 010-18741039 {"wTS": "1596440623", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440623-010-18741039 +139871132246695936 10000 1993-08-03T15:43:44.068 10000 1993-08-03T15:43:44.068 201 456678 false 1993-08-03T15:43:43 010-18741035 {"wTS": "1596440623", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440623-010-18741035 +139871132657737728 10000 1993-08-03T15:43:44.167 10000 1993-08-03T15:43:44.167 201 456678 false 1993-08-03T15:43:43 010-18741034 {"wTS": "1596440623", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440623-010-18741034 +139871133219774464 10000 1993-08-03T15:43:44.300 10000 1993-08-03T15:43:44.300 201 456678 false 1993-08-03T15:43:43 010-18741040 {"wTS": "1596440623", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440623-010-18741040 +139871133760839680 10000 1993-08-03T15:43:44.429 10000 1993-08-03T15:43:44.429 201 456678 false 1993-08-03T15:43:43 010-18741022 {"wTS": "1596440623", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440623-010-18741022 +139871133895057408 10000 1993-08-03T15:43:44.461 10000 1993-08-03T15:43:44.461 201 456678 false 1993-08-03T15:43:43 010-18741024 {"wTS": "1596440623", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440623-010-18741024 +139871134603894784 10000 1993-08-03T15:43:44.630 10000 1993-08-03T15:43:44.630 201 456678 false 1993-08-03T15:43:44 010-18741030 {"wTS": "1596440624", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440624-010-18741030 +139871135446949888 10000 1993-08-03T15:43:44.831 10000 1993-08-03T15:43:44.831 201 456678 false 1993-08-03T15:43:44 010-18741028 {"wTS": "1596440624", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440624-010-18741028 +139871135690219520 10000 1993-08-03T15:43:44.889 10000 1993-08-03T15:43:44.889 201 456678 false 1993-08-03T15:43:44 010-18741036 {"wTS": "1596440624", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440624-010-18741036 +139871136432611328 10000 1993-08-03T15:43:45.067 10000 1993-08-03T15:43:45.067 201 456678 false 1993-08-03T15:43:44 010-18741023 {"wTS": "1596440624", "rfId": "010-18741023", "remark": "数据", "weight": 118.114} 710078936338096128 1596440624-010-18741023 +139871138248744960 10000 1993-08-03T15:43:45.499 10000 1993-08-03T15:43:45.499 201 456678 false 1993-08-03T15:43:44 010-18741031 {"wTS": "1596440624", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440624-010-18741031 +139871138403934208 10000 1993-08-03T15:43:45.536 10000 1993-08-03T15:43:45.536 201 456678 false 1993-08-03T15:43:44 010-18741025 {"wTS": "1596440624", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440624-010-18741025 +139871138877890560 10000 1993-08-03T15:43:45.649 10000 1993-08-03T15:43:45.649 201 456678 false 1993-08-03T15:43:44 010-18741026 {"wTS": "1596440624", "rfId": "010-18741026", "remark": "数据", "weight": 147.465} 710078936338096128 1596440624-010-18741026 +139871138970165248 10000 1993-08-03T15:43:45.671 10000 1993-08-03T15:43:45.671 201 456678 false 1993-08-03T15:43:44 010-18741029 {"wTS": "1596440624", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440624-010-18741029 +139871139590922240 10000 1993-08-03T15:43:45.819 10000 1993-08-03T15:43:45.819 201 456678 false 1993-08-03T15:43:44 010-18741037 {"wTS": "1596440624", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440624-010-18741037 +139871257476030464 10000 1993-08-03T15:44:13.925 10000 1993-08-03T15:44:13.925 201 456678 false 1993-08-03T15:44:13 010-18741038 {"wTS": "1596440653", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440653-010-18741038 +139871257740271616 10000 1993-08-03T15:44:13.988 10000 1993-08-03T15:44:13.988 201 456678 false 1993-08-03T15:44:13 010-18741039 {"wTS": "1596440653", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440653-010-18741039 +139871258302308352 10000 1993-08-03T15:44:14.122 10000 1993-08-03T15:44:14.122 201 456678 false 1993-08-03T15:44:13 010-18741035 {"wTS": "1596440653", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440653-010-18741035 +139871259027922944 10000 1993-08-03T15:44:14.295 10000 1993-08-03T15:44:14.295 201 456678 false 1993-08-03T15:44:13 010-18741034 {"wTS": "1596440653", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440653-010-18741034 +139871259682234368 10000 1993-08-03T15:44:14.451 10000 1993-08-03T15:44:14.451 201 456678 false 1993-08-03T15:44:13 010-18741040 {"wTS": "1596440653", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440653-010-18741040 +139871260231688192 10000 1993-08-03T15:44:14.582 10000 1993-08-03T15:44:14.582 201 456678 false 1993-08-03T15:44:13 010-18741024 {"wTS": "1596440653", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440653-010-18741024 +139871260512706560 10000 1993-08-03T15:44:14.649 10000 1993-08-03T15:44:14.649 201 456678 false 1993-08-03T15:44:14 010-18741033 {"wTS": "1596440654", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440654-010-18741033 +139871261041188864 10000 1993-08-03T15:44:14.775 10000 1993-08-03T15:44:14.775 201 456678 false 1993-08-03T15:44:14 010-18741028 {"wTS": "1596440654", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440654-010-18741028 +139871261758414848 10000 1993-08-03T15:44:14.946 10000 1993-08-03T15:44:14.946 201 456678 false 1993-08-03T15:44:14 010-18741030 {"wTS": "1596440654", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440654-010-18741030 +139871262639218688 10000 1993-08-03T15:44:15.156 10000 1993-08-03T15:44:15.156 201 456678 false 1993-08-03T15:44:14 010-18741029 {"wTS": "1596440654", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440654-010-18741029 +139871263805235200 10000 1993-08-03T15:44:15.434 10000 1993-08-03T15:44:15.434 201 456678 false 1993-08-03T15:44:14 010-18741036 {"wTS": "1596440654", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440654-010-18741036 +139871264044310528 10000 1993-08-03T15:44:15.491 10000 1993-08-03T15:44:15.491 201 456678 false 1993-08-03T15:44:14 010-18741027 {"wTS": "1596440654", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440654-010-18741027 +139871264610541568 10000 1993-08-03T15:44:15.626 10000 1993-08-03T15:44:15.627 201 456678 false 1993-08-03T15:44:14 010-18741031 {"wTS": "1596440654", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440654-010-18741031 +139871265361321984 10000 1993-08-03T15:44:15.805 10000 1993-08-03T15:44:15.805 201 456678 false 1993-08-03T15:44:14 010-18741037 {"wTS": "1596440654", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440654-010-18741037 +139871265893998592 10000 1993-08-03T15:44:15.932 10000 1993-08-03T15:44:15.932 201 456678 false 1993-08-03T15:44:14 010-18741026 {"wTS": "1596440654", "rfId": "010-18741026", "remark": "数据", "weight": 147.465} 710078936338096128 1596440654-010-18741026 +139871266917408768 10000 1993-08-03T15:44:16.176 10000 1993-08-03T15:44:16.176 201 456678 false 1993-08-03T15:44:14 010-18741025 {"wTS": "1596440654", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440654-010-18741025 +139871267198427136 10000 1993-08-03T15:44:16.243 10000 1993-08-03T15:44:16.243 201 456678 false 1993-08-03T15:44:14 010-18741021 {"wTS": "1596440654", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440654-010-18741021 +139871267999539200 10000 1993-08-03T15:44:16.434 10000 1993-08-03T15:44:16.434 201 456678 false 1993-08-03T15:44:14 010-18741032 {"wTS": "1596440654", "rfId": "010-18741032", "remark": "数据", "weight": 118.863} 710078936338096128 1596440654-010-18741032 +139871383804272640 10000 1993-08-03T15:44:44.044 10000 1993-08-03T15:44:44.044 201 456678 false 1993-08-03T15:44:43 010-18741039 {"wTS": "1596440683", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440683-010-18741039 +139871383946878976 10000 1993-08-03T15:44:44.078 10000 1993-08-03T15:44:44.078 201 456678 false 1993-08-03T15:44:43 010-18741038 {"wTS": "1596440683", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440683-010-18741038 +139871384173371392 10000 1993-08-03T15:44:44.132 10000 1993-08-03T15:44:44.132 201 456678 false 1993-08-03T15:44:43 010-18741035 {"wTS": "1596440683", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440683-010-18741035 +139871384362115072 10000 1993-08-03T15:44:44.177 10000 1993-08-03T15:44:44.177 201 456678 false 1993-08-03T15:44:43 010-18741022 {"wTS": "1596440683", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440683-010-18741022 +139871385318416384 10000 1993-08-03T15:44:44.405 10000 1993-08-03T15:44:44.405 201 456678 false 1993-08-03T15:44:43 010-18741034 {"wTS": "1596440683", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440683-010-18741034 +139871385553297408 10000 1993-08-03T15:44:44.461 10000 1993-08-03T15:44:44.461 201 456678 false 1993-08-03T15:44:43 010-18741040 {"wTS": "1596440683", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440683-010-18741040 +139871385851092992 10000 1993-08-03T15:44:44.532 10000 1993-08-03T15:44:44.532 201 456678 false 1993-08-03T15:44:44 010-18741024 {"wTS": "1596440684", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440684-010-18741024 +139871386111139840 10000 1993-08-03T15:44:44.594 10000 1993-08-03T15:44:44.594 201 456678 false 1993-08-03T15:44:44 010-18741033 {"wTS": "1596440684", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440684-010-18741033 +139871387046469632 10000 1993-08-03T15:44:44.817 10000 1993-08-03T15:44:44.817 201 456678 false 1993-08-03T15:44:44 010-18741030 {"wTS": "1596440684", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440684-010-18741030 +139871388103434240 10000 1993-08-03T15:44:45.069 10000 1993-08-03T15:44:45.069 201 456678 false 1993-08-03T15:44:44 010-18741028 {"wTS": "1596440684", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440684-010-18741028 +139871388552224768 10000 1993-08-03T15:44:45.176 10000 1993-08-03T15:44:45.176 201 456678 false 1993-08-03T15:44:44 010-18741036 {"wTS": "1596440684", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440684-010-18741036 +139871388929712128 10000 1993-08-03T15:44:45.266 10000 1993-08-03T15:44:45.266 201 456678 false 1993-08-03T15:44:44 010-18741027 {"wTS": "1596440684", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440684-010-18741027 +139871390653571072 10000 1993-08-03T15:44:45.677 10000 1993-08-03T15:44:45.677 201 456678 false 1993-08-03T15:44:44 010-18741021 {"wTS": "1596440684", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440684-010-18741021 +139871390666153984 10000 1993-08-03T15:44:45.680 10000 1993-08-03T15:44:45.680 201 456678 false 1993-08-03T15:44:44 010-18741023 {"wTS": "1596440684", "rfId": "010-18741023", "remark": "数据", "weight": 118.114} 710078936338096128 1596440684-010-18741023 +139871391165276160 10000 1993-08-03T15:44:45.799 10000 1993-08-03T15:44:45.799 201 456678 false 1993-08-03T15:44:44 010-18741029 {"wTS": "1596440684", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440684-010-18741029 +139871392822026240 10000 1993-08-03T15:44:46.194 10000 1993-08-03T15:44:46.194 201 456678 false 1993-08-03T15:44:44 010-18741037 {"wTS": "1596440684", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440684-010-18741037 +139871509830524928 10000 1993-08-03T15:45:14.091 10000 1993-08-03T15:45:14.091 201 456678 false 1993-08-03T15:45:13 010-18741039 {"wTS": "1596440713", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440713-010-18741039 +139871510983958528 10000 1993-08-03T15:45:14.366 10000 1993-08-03T15:45:14.366 201 456678 false 1993-08-03T15:45:13 010-18741040 {"wTS": "1596440713", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440713-010-18741040 +139871511017512960 10000 1993-08-03T15:45:14.374 10000 1993-08-03T15:45:14.374 201 456678 false 1993-08-03T15:45:13 010-18741035 {"wTS": "1596440713", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440713-010-18741035 +139871511042678784 10000 1993-08-03T15:45:14.380 10000 1993-08-03T15:45:14.380 201 456678 false 1993-08-03T15:45:13 010-18741022 {"wTS": "1596440713", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440713-010-18741022 +139871511072038912 10000 1993-08-03T15:45:14.387 10000 1993-08-03T15:45:14.387 201 456678 false 1993-08-03T15:45:14 010-18741024 {"wTS": "1596440714", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440714-010-18741024 +139871511088816128 10000 1993-08-03T15:45:14.391 10000 1993-08-03T15:45:14.391 201 456678 false 1993-08-03T15:45:13 010-18741034 {"wTS": "1596440713", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440713-010-18741034 +139871513341157376 10000 1993-08-03T15:45:14.928 10000 1993-08-03T15:45:14.928 201 456678 false 1993-08-03T15:45:14 010-18741033 {"wTS": "1596440714", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440714-010-18741033 +139871513706061824 10000 1993-08-03T15:45:15.015 10000 1993-08-03T15:45:15.015 201 456678 false 1993-08-03T15:45:14 010-18741031 {"wTS": "1596440714", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440714-010-18741031 +139871515257954304 10000 1993-08-03T15:45:15.385 10000 1993-08-03T15:45:15.385 201 456678 false 1993-08-03T15:45:14 010-18741028 {"wTS": "1596440714", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440714-010-18741028 +139871516579160064 10000 1993-08-03T15:45:15.700 10000 1993-08-03T15:45:15.700 201 456678 false 1993-08-03T15:45:14 010-18741030 {"wTS": "1596440714", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440714-010-18741030 +139871517271220224 10000 1993-08-03T15:45:15.865 10000 1993-08-03T15:45:15.865 201 456678 false 1993-08-03T15:45:14 010-18741027 {"wTS": "1596440714", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440714-010-18741027 +139871517942308864 10000 1993-08-03T15:45:16.025 10000 1993-08-03T15:45:16.025 201 456678 false 1993-08-03T15:45:14 010-18741032 {"wTS": "1596440714", "rfId": "010-18741032", "remark": "数据", "weight": 118.863} 710078936338096128 1596440714-010-18741032 +139871518294630400 10000 1993-08-03T15:45:16.109 10000 1993-08-03T15:45:16.109 201 456678 false 1993-08-03T15:45:14 010-18741025 {"wTS": "1596440714", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440714-010-18741025 +139871519984934912 10000 1993-08-03T15:45:16.512 10000 1993-08-03T15:45:16.512 201 456678 false 1993-08-03T15:45:14 010-18741026 {"wTS": "1596440714", "rfId": "010-18741026", "remark": "数据", "weight": 147.465} 710078936338096128 1596440714-010-18741026 +139871520580526080 10000 1993-08-03T15:45:16.654 10000 1993-08-03T15:45:16.654 201 456678 false 1993-08-03T15:45:14 010-18741023 {"wTS": "1596440714", "rfId": "010-18741023", "remark": "数据", "weight": 118.114} 710078936338096128 1596440714-010-18741023 +139871520614080512 10000 1993-08-03T15:45:16.662 10000 1993-08-03T15:45:16.662 201 456678 false 1993-08-03T15:45:14 010-18741021 {"wTS": "1596440714", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440714-010-18741021 +139871635961634816 10000 1993-08-03T15:45:44.163 10000 1993-08-03T15:45:44.163 201 456678 false 1993-08-03T15:45:43 010-18741039 {"wTS": "1596440743", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440743-010-18741039 +139871636230070272 10000 1993-08-03T15:45:44.227 10000 1993-08-03T15:45:44.227 201 456678 false 1993-08-03T15:45:43 010-18741038 {"wTS": "1596440743", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440743-010-18741038 +139871637765185536 10000 1993-08-03T15:45:44.593 10000 1993-08-03T15:45:44.593 201 456678 false 1993-08-03T15:45:43 010-18741022 {"wTS": "1596440743", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440743-010-18741022 +139871637777768448 10000 1993-08-03T15:45:44.596 10000 1993-08-03T15:45:44.597 201 456678 false 1993-08-03T15:45:43 010-18741034 {"wTS": "1596440743", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440743-010-18741034 +139871637794545664 10000 1993-08-03T15:45:44.600 10000 1993-08-03T15:45:44.600 201 456678 false 1993-08-03T15:45:44 010-18741024 {"wTS": "1596440744", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440744-010-18741024 +139871637811322880 10000 1993-08-03T15:45:44.604 10000 1993-08-03T15:45:44.604 201 456678 false 1993-08-03T15:45:44 010-18741040 {"wTS": "1596440744", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440744-010-18741040 +139871637844877312 10000 1993-08-03T15:45:44.612 10000 1993-08-03T15:45:44.612 201 456678 false 1993-08-03T15:45:43 010-18741035 {"wTS": "1596440743", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440743-010-18741035 +139871639900086272 10000 1993-08-03T15:45:45.102 10000 1993-08-03T15:45:45.102 201 456678 false 1993-08-03T15:45:44 010-18741031 {"wTS": "1596440744", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440744-010-18741031 +139871641212903424 10000 1993-08-03T15:45:45.415 10000 1993-08-03T15:45:45.415 201 456678 false 1993-08-03T15:45:44 010-18741036 {"wTS": "1596440744", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440744-010-18741036 +139871641628139520 10000 1993-08-03T15:45:45.514 10000 1993-08-03T15:45:45.514 201 456678 false 1993-08-03T15:45:44 010-18741025 {"wTS": "1596440744", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440744-010-18741025 +139871641854631936 10000 1993-08-03T15:45:45.568 10000 1993-08-03T15:45:45.568 201 456678 false 1993-08-03T15:45:44 010-18741033 {"wTS": "1596440744", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440744-010-18741033 +139871643645599744 10000 1993-08-03T15:45:45.995 10000 1993-08-03T15:45:45.995 201 456678 false 1993-08-03T15:45:44 010-18741029 {"wTS": "1596440744", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440744-010-18741029 +139871643964366848 10000 1993-08-03T15:45:46.071 10000 1993-08-03T15:45:46.071 201 456678 false 1993-08-03T15:45:44 010-18741027 {"wTS": "1596440744", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440744-010-18741027 +139871644929056768 10000 1993-08-03T15:45:46.301 10000 1993-08-03T15:45:46.301 201 456678 false 1993-08-03T15:45:44 010-18741037 {"wTS": "1596440744", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440744-010-18741037 +139871645679837184 10000 1993-08-03T15:45:46.480 10000 1993-08-03T15:45:46.480 201 456678 false 1993-08-03T15:45:44 010-18741021 {"wTS": "1596440744", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440744-010-18741021 +139871645889552384 10000 1993-08-03T15:45:46.530 10000 1993-08-03T15:45:46.530 201 456678 false 1993-08-03T15:45:44 010-18741032 {"wTS": "1596440744", "rfId": "010-18741032", "remark": "数据", "weight": 118.863} 710078936338096128 1596440744-010-18741032 +139871762545729536 10000 1993-08-03T15:46:14.343 10000 1993-08-03T15:46:14.343 201 456678 false 1993-08-03T15:46:13 010-18741039 {"wTS": "1596440773", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440773-010-18741039 +139871762562506752 10000 1993-08-03T15:46:14.347 10000 1993-08-03T15:46:14.347 201 456678 false 1993-08-03T15:46:13 010-18741038 {"wTS": "1596440773", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440773-010-18741038 +139871762843525120 10000 1993-08-03T15:46:14.414 10000 1993-08-03T15:46:14.414 201 456678 false 1993-08-03T15:46:14 010-18741022 {"wTS": "1596440774", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440774-010-18741022 +139871763732717568 10000 1993-08-03T15:46:14.626 10000 1993-08-03T15:46:14.626 201 456678 false 1993-08-03T15:46:14 010-18741034 {"wTS": "1596440774", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440774-010-18741034 +139871764470915072 10000 1993-08-03T15:46:14.802 10000 1993-08-03T15:46:14.802 201 456678 false 1993-08-03T15:46:14 010-18741024 {"wTS": "1596440774", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440774-010-18741024 +139871765951504384 10000 1993-08-03T15:46:15.155 10000 1993-08-03T15:46:15.155 201 456678 false 1993-08-03T15:46:14 010-18741036 {"wTS": "1596440774", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440774-010-18741036 +139871766014418944 10000 1993-08-03T15:46:15.170 10000 1993-08-03T15:46:15.170 201 456678 false 1993-08-03T15:46:14 010-18741028 {"wTS": "1596440774", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440774-010-18741028 +139871766459015168 10000 1993-08-03T15:46:15.276 10000 1993-08-03T15:46:15.276 201 456678 false 1993-08-03T15:46:14 010-18741032 {"wTS": "1596440774", "rfId": "010-18741032", "remark": "数据", "weight": 118.863} 710078936338096128 1596440774-010-18741032 +139871767213989888 10000 1993-08-03T15:46:15.456 10000 1993-08-03T15:46:15.457 201 456678 false 1993-08-03T15:46:14 010-18741031 {"wTS": "1596440774", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440774-010-18741031 +139871768082210816 10000 1993-08-03T15:46:15.663 10000 1993-08-03T15:46:15.663 201 456678 false 1993-08-03T15:46:14 010-18741037 {"wTS": "1596440774", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440774-010-18741037 +139871768556167168 10000 1993-08-03T15:46:15.776 10000 1993-08-03T15:46:15.776 201 456678 false 1993-08-03T15:46:14 010-18741033 {"wTS": "1596440774", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440774-010-18741033 +139871769566994432 10000 1993-08-03T15:46:16.017 10000 1993-08-03T15:46:16.017 201 456678 false 1993-08-03T15:46:14 010-18741030 {"wTS": "1596440774", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440774-010-18741030 +139871771605426176 10000 1993-08-03T15:46:16.503 10000 1993-08-03T15:46:16.503 201 456678 false 1993-08-03T15:46:14 010-18741026 {"wTS": "1596440774", "rfId": "010-18741026", "remark": "数据", "weight": 147.465} 710078936338096128 1596440774-010-18741026 +139871771618009088 10000 1993-08-03T15:46:16.506 10000 1993-08-03T15:46:16.506 201 456678 false 1993-08-03T15:46:14 010-18741025 {"wTS": "1596440774", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440774-010-18741025 +139871772301680640 10000 1993-08-03T15:46:16.669 10000 1993-08-03T15:46:16.669 201 456678 false 1993-08-03T15:46:14 010-18741029 {"wTS": "1596440774", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440774-010-18741029 +139871772809191424 10000 1993-08-03T15:46:16.790 10000 1993-08-03T15:46:16.790 201 456678 false 1993-08-03T15:46:14 010-18741023 {"wTS": "1596440774", "rfId": "010-18741023", "remark": "数据", "weight": 118.114} 710078936338096128 1596440774-010-18741023 +139871888001556480 10000 1993-08-03T15:46:44.254 10000 1993-08-03T15:46:44.254 201 456678 false 1993-08-03T15:46:43 010-18741039 {"wTS": "1596440803", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440803-010-18741039 +139871888240631808 10000 1993-08-03T15:46:44.311 10000 1993-08-03T15:46:44.311 201 456678 false 1993-08-03T15:46:43 010-18741038 {"wTS": "1596440803", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440803-010-18741038 +139871888370655232 10000 1993-08-03T15:46:44.342 10000 1993-08-03T15:46:44.342 201 456678 false 1993-08-03T15:46:43 010-18741035 {"wTS": "1596440803", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440803-010-18741035 +139871889373093888 10000 1993-08-03T15:46:44.581 10000 1993-08-03T15:46:44.581 201 456678 false 1993-08-03T15:46:44 010-18741040 {"wTS": "1596440804", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440804-010-18741040 +139871889549254656 10000 1993-08-03T15:46:44.623 10000 1993-08-03T15:46:44.623 201 456678 false 1993-08-03T15:46:44 010-18741022 {"wTS": "1596440804", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440804-010-18741022 +139871889691860992 10000 1993-08-03T15:46:44.657 10000 1993-08-03T15:46:44.657 201 456678 false 1993-08-03T15:46:44 010-18741034 {"wTS": "1596440804", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440804-010-18741034 +139871890451030016 10000 1993-08-03T15:46:44.838 10000 1993-08-03T15:46:44.838 201 456678 false 1993-08-03T15:46:44 010-18741024 {"wTS": "1596440804", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440804-010-18741024 +139871892028088320 10000 1993-08-03T15:46:45.214 10000 1993-08-03T15:46:45.214 201 456678 false 1993-08-03T15:46:44 010-18741028 {"wTS": "1596440804", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440804-010-18741028 +139871893294768128 10000 1993-08-03T15:46:45.516 10000 1993-08-03T15:46:45.516 201 456678 false 1993-08-03T15:46:44 010-18741027 {"wTS": "1596440804", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440804-010-18741027 +139871893517066240 10000 1993-08-03T15:46:45.569 10000 1993-08-03T15:46:45.569 201 456678 false 1993-08-03T15:46:44 010-18741021 {"wTS": "1596440804", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440804-010-18741021 +139871894364315648 10000 1993-08-03T15:46:45.771 10000 1993-08-03T15:46:45.771 201 456678 false 1993-08-03T15:46:44 010-18741032 {"wTS": "1596440804", "rfId": "010-18741032", "remark": "数据", "weight": 118.863} 710078936338096128 1596440804-010-18741032 +139872014090723328 10000 1993-08-03T15:47:14.316 10000 1993-08-03T15:47:14.316 201 456678 false 1993-08-03T15:47:13 010-18741039 {"wTS": "1596440833", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440833-010-18741039 +139872014187192320 10000 1993-08-03T15:47:14.339 10000 1993-08-03T15:47:14.339 201 456678 false 1993-08-03T15:47:13 010-18741038 {"wTS": "1596440833", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440833-010-18741038 +139872014229135360 10000 1993-08-03T15:47:14.349 10000 1993-08-03T15:47:14.349 201 456678 false 1993-08-03T15:47:14 010-18741035 {"wTS": "1596440834", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440834-010-18741035 +139872015739084800 10000 1993-08-03T15:47:14.709 10000 1993-08-03T15:47:14.709 201 456678 false 1993-08-03T15:47:14 010-18741040 {"wTS": "1596440834", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440834-010-18741040 +139872015860719616 10000 1993-08-03T15:47:14.738 10000 1993-08-03T15:47:14.738 201 456678 false 1993-08-03T15:47:14 010-18741034 {"wTS": "1596440834", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440834-010-18741034 +139872016007520256 10000 1993-08-03T15:47:14.773 10000 1993-08-03T15:47:14.773 201 456678 false 1993-08-03T15:47:14 010-18741024 {"wTS": "1596440834", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440834-010-18741024 +139872016183681024 10000 1993-08-03T15:47:14.815 10000 1993-08-03T15:47:14.816 201 456678 false 1993-08-03T15:47:14 010-18741022 {"wTS": "1596440834", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440834-010-18741022 +139872018347941888 10000 1993-08-03T15:47:15.331 10000 1993-08-03T15:47:15.331 201 456678 false 1993-08-03T15:47:14 010-18741031 {"wTS": "1596440834", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440834-010-18741031 +139872018700263424 10000 1993-08-03T15:47:15.415 10000 1993-08-03T15:47:15.415 201 456678 false 1993-08-03T15:47:14 010-18741036 {"wTS": "1596440834", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440834-010-18741036 +139872019648176128 10000 1993-08-03T15:47:15.641 10000 1993-08-03T15:47:15.641 201 456678 false 1993-08-03T15:47:14 010-18741033 {"wTS": "1596440834", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440834-010-18741033 +139872020088578048 10000 1993-08-03T15:47:15.746 10000 1993-08-03T15:47:15.746 201 456678 false 1993-08-03T15:47:14 010-18741030 {"wTS": "1596440834", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440834-010-18741030 +139872020520591360 10000 1993-08-03T15:47:15.849 10000 1993-08-03T15:47:15.849 201 456678 false 1993-08-03T15:47:14 010-18741028 {"wTS": "1596440834", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440834-010-18741028 +139872021506252800 10000 1993-08-03T15:47:16.084 10000 1993-08-03T15:47:16.084 201 456678 false 1993-08-03T15:47:14 010-18741029 {"wTS": "1596440834", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440834-010-18741029 +139872022290587648 10000 1993-08-03T15:47:16.271 10000 1993-08-03T15:47:16.271 201 456678 false 1993-08-03T15:47:14 010-18741021 {"wTS": "1596440834", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440834-010-18741021 +139872022869401600 10000 1993-08-03T15:47:16.409 10000 1993-08-03T15:47:16.409 201 456678 false 1993-08-03T15:47:14 010-18741025 {"wTS": "1596440834", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440834-010-18741025 +139872023464992768 10000 1993-08-03T15:47:16.551 10000 1993-08-03T15:47:16.551 201 456678 false 1993-08-03T15:47:14 010-18741023 {"wTS": "1596440834", "rfId": "010-18741023", "remark": "数据", "weight": 118.114} 710078936338096128 1596440834-010-18741023 +139872023523713024 10000 1993-08-03T15:47:16.566 10000 1993-08-03T15:47:16.566 201 456678 false 1993-08-03T15:47:14 010-18741026 {"wTS": "1596440834", "rfId": "010-18741026", "remark": "数据", "weight": 147.465} 710078936338096128 1596440834-010-18741026 +139872024970747904 10000 1993-08-03T15:47:16.910 10000 1993-08-03T15:47:16.910 201 456678 false 1993-08-03T15:47:14 010-18741037 {"wTS": "1596440834", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440834-010-18741037 +139872140372828160 10000 1993-08-03T15:47:44.424 10000 1993-08-03T15:47:44.425 201 456678 false 1993-08-03T15:47:43 010-18741038 {"wTS": "1596440863", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440863-010-18741038 +139872140435742720 10000 1993-08-03T15:47:44.439 10000 1993-08-03T15:47:44.439 201 456678 false 1993-08-03T15:47:44 010-18741035 {"wTS": "1596440864", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440864-010-18741035 +139872140939059200 10000 1993-08-03T15:47:44.559 10000 1993-08-03T15:47:44.559 201 456678 false 1993-08-03T15:47:43 010-18741039 {"wTS": "1596440863", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440863-010-18741039 +139872142331568128 10000 1993-08-03T15:47:44.891 10000 1993-08-03T15:47:44.891 201 456678 false 1993-08-03T15:47:44 010-18741040 {"wTS": "1596440864", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440864-010-18741040 +139872142365122560 10000 1993-08-03T15:47:44.899 10000 1993-08-03T15:47:44.899 201 456678 false 1993-08-03T15:47:44 010-18741034 {"wTS": "1596440864", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440864-010-18741034 +139872142381899776 10000 1993-08-03T15:47:44.903 10000 1993-08-03T15:47:44.903 201 456678 false 1993-08-03T15:47:44 010-18741024 {"wTS": "1596440864", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440864-010-18741024 +139872142407065600 10000 1993-08-03T15:47:44.909 10000 1993-08-03T15:47:44.909 201 456678 false 1993-08-03T15:47:44 010-18741022 {"wTS": "1596440864", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440864-010-18741022 +139872144193839104 10000 1993-08-03T15:47:45.335 10000 1993-08-03T15:47:45.335 201 456678 false 1993-08-03T15:47:44 010-18741031 {"wTS": "1596440864", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440864-010-18741031 +139872145619902464 10000 1993-08-03T15:47:45.675 10000 1993-08-03T15:47:45.675 201 456678 false 1993-08-03T15:47:44 010-18741036 {"wTS": "1596440864", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440864-010-18741036 +139872145930280960 10000 1993-08-03T15:47:45.749 10000 1993-08-03T15:47:45.749 201 456678 false 1993-08-03T15:47:44 010-18741029 {"wTS": "1596440864", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440864-010-18741029 +139872147243098112 10000 1993-08-03T15:47:46.062 10000 1993-08-03T15:47:46.062 201 456678 false 1993-08-03T15:47:44 010-18741030 {"wTS": "1596440864", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440864-010-18741030 +139872147276652544 10000 1993-08-03T15:47:46.070 10000 1993-08-03T15:47:46.070 201 456678 false 1993-08-03T15:47:44 010-18741028 {"wTS": "1596440864", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440864-010-18741028 +139872147285041152 10000 1993-08-03T15:47:46.072 10000 1993-08-03T15:47:46.072 201 456678 false 1993-08-03T15:47:44 010-18741027 {"wTS": "1596440864", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440864-010-18741027 +139872147398281100 10000 1993-08-03T15:47:46.099 10000 1993-08-03T15:47:46.099 201 456678 false 1993-08-03T15:47:44 010-18741023 {"wTS": "1596440864", "rfId": "010-18741023", "remark": "数据", "weight": 118.114} 710078936338096128 1596440864-010-18741023 +139872147620585472 10000 1993-08-03T15:47:46.152 10000 1993-08-03T15:47:46.152 201 456678 false 1993-08-03T15:47:44 010-18741026 {"wTS": "1596440864", "rfId": "010-18741026", "remark": "数据", "weight": 147.465} 710078936338096128 1596440864-010-18741026 +139872149973590016 10000 1993-08-03T15:47:46.713 10000 1993-08-03T15:47:46.713 201 456678 false 1993-08-03T15:47:44 010-18741025 {"wTS": "1596440864", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440864-010-18741025 +139872151349321728 10000 1993-08-03T15:47:47.041 10000 1993-08-03T15:47:47.041 201 456678 false 1993-08-03T15:47:44 010-18741032 {"wTS": "1596440864", "rfId": "010-18741032", "remark": "数据", "weight": 118.863} 710078936338096128 1596440864-010-18741032 +139872151533871104 10000 1993-08-03T15:47:47.085 10000 1993-08-03T15:47:47.085 201 456678 false 1993-08-03T15:47:44 010-18741021 {"wTS": "1596440864", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440864-010-18741021 +139872151705837568 10000 1993-08-03T15:47:47.126 10000 1993-08-03T15:47:47.126 201 456678 false 1993-08-03T15:47:45 010-18741037 {"wTS": "1596440865", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440865-010-18741037 +139872266231308288 10000 1993-08-03T15:48:14.431 10000 1993-08-03T15:48:14.431 201 456678 false 1993-08-03T15:48:14 010-18741038 {"wTS": "1596440894", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440894-010-18741038 +139872266407469056 10000 1993-08-03T15:48:14.473 10000 1993-08-03T15:48:14.473 201 456678 false 1993-08-03T15:48:14 010-18741035 {"wTS": "1596440894", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440894-010-18741035 +139872266491355136 10000 1993-08-03T15:48:14.493 10000 1993-08-03T15:48:14.493 201 456678 false 1993-08-03T15:48:14 010-18741039 {"wTS": "1596440894", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440894-010-18741039 +139872267909029888 10000 1993-08-03T15:48:14.831 10000 1993-08-03T15:48:14.831 201 456678 false 1993-08-03T15:48:14 010-18741040 {"wTS": "1596440894", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440894-010-18741040 +139872268039053312 10000 1993-08-03T15:48:14.862 10000 1993-08-03T15:48:14.862 201 456678 false 1993-08-03T15:48:14 010-18741024 {"wTS": "1596440894", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440894-010-18741024 +139872269435756544 10000 1993-08-03T15:48:15.195 10000 1993-08-03T15:48:15.195 201 456678 false 1993-08-03T15:48:14 010-18741031 {"wTS": "1596440894", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440894-010-18741031 +139872269532225536 10000 1993-08-03T15:48:15.218 10000 1993-08-03T15:48:15.218 201 456678 false 1993-08-03T15:48:14 010-18741036 {"wTS": "1596440894", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440894-010-18741036 +139872269859381248 10000 1993-08-03T15:48:15.296 10000 1993-08-03T15:48:15.296 201 456678 false 1993-08-03T15:48:14 010-18741030 {"wTS": "1596440894", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440894-010-18741030 +139872270924734464 10000 1993-08-03T15:48:15.550 10000 1993-08-03T15:48:15.550 201 456678 false 1993-08-03T15:48:14 010-18741033 {"wTS": "1596440894", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440894-010-18741033 +139872272392740864 10000 1993-08-03T15:48:15.900 10000 1993-08-03T15:48:15.900 201 456678 false 1993-08-03T15:48:14 010-18741028 {"wTS": "1596440894", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440894-010-18741028 +139872272791199744 10000 1993-08-03T15:48:15.995 10000 1993-08-03T15:48:15.995 201 456678 false 1993-08-03T15:48:14 010-18741027 {"wTS": "1596440894", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440894-010-18741027 +139872274309537792 10000 1993-08-03T15:48:16.357 10000 1993-08-03T15:48:16.357 201 456678 false 1993-08-03T15:48:14 010-18741021 {"wTS": "1596440894", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440894-010-18741021 +139872392240783360 10000 1993-08-03T15:48:44.474 10000 1993-08-03T15:48:44.474 201 456678 false 1993-08-03T15:48:44 010-18741039 {"wTS": "1596440924", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440924-010-18741039 +139872392354029568 10000 1993-08-03T15:48:44.501 10000 1993-08-03T15:48:44.501 201 456678 false 1993-08-03T15:48:44 010-18741035 {"wTS": "1596440924", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440924-010-18741035 +139872392484052992 10000 1993-08-03T15:48:44.532 10000 1993-08-03T15:48:44.532 201 456678 false 1993-08-03T15:48:44 010-18741038 {"wTS": "1596440924", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440924-010-18741038 +139872393570377728 10000 1993-08-03T15:48:44.791 10000 1993-08-03T15:48:44.791 201 456678 false 1993-08-03T15:48:44 010-18741034 {"wTS": "1596440924", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440924-010-18741034 +139872393742344192 10000 1993-08-03T15:48:44.832 10000 1993-08-03T15:48:44.832 201 456678 false 1993-08-03T15:48:44 010-18741022 {"wTS": "1596440924", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440924-010-18741022 +139872394040139776 10000 1993-08-03T15:48:44.903 10000 1993-08-03T15:48:44.904 201 456678 false 1993-08-03T15:48:44 010-18741040 {"wTS": "1596440924", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440924-010-18741040 +139872394543456256 10000 1993-08-03T15:48:45.023 10000 1993-08-03T15:48:45.023 201 456678 false 1993-08-03T15:48:44 010-18741024 {"wTS": "1596440924", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440924-010-18741024 +139872396456058880 10000 1993-08-03T15:48:45.479 10000 1993-08-03T15:48:45.479 201 456678 false 1993-08-03T15:48:44 010-18741031 {"wTS": "1596440924", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440924-010-18741031 +139872398083448832 10000 1993-08-03T15:48:45.867 10000 1993-08-03T15:48:45.867 201 456678 false 1993-08-03T15:48:44 010-18741036 {"wTS": "1596440924", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440924-010-18741036 +139872398150557696 10000 1993-08-03T15:48:45.883 10000 1993-08-03T15:48:45.883 201 456678 false 1993-08-03T15:48:44 010-18741027 {"wTS": "1596440924", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440924-010-18741027 +139872398276386816 10000 1993-08-03T15:48:45.913 10000 1993-08-03T15:48:45.913 201 456678 false 1993-08-03T15:48:44 010-18741029 {"wTS": "1596440924", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440924-010-18741029 +139872398356078592 10000 1993-08-03T15:48:45.932 10000 1993-08-03T15:48:45.932 201 456678 false 1993-08-03T15:48:44 010-18741023 {"wTS": "1596440924", "rfId": "010-18741023", "remark": "数据", "weight": 118.114} 710078936338096128 1596440924-010-18741023 +139872400310624256 10000 1993-08-03T15:48:46.398 10000 1993-08-03T15:48:46.398 201 456678 false 1993-08-03T15:48:44 010-18741026 {"wTS": "1596440924", "rfId": "010-18741026", "remark": "数据", "weight": 147.465} 710078936338096128 1596440924-010-18741026 +139872402147729408 10000 1993-08-03T15:48:46.836 10000 1993-08-03T15:48:46.836 201 456678 false 1993-08-03T15:48:45 010-18741025 {"wTS": "1596440925", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440925-010-18741025 +13987240259651987 10000 1993-08-03T15:48:46.943 10000 1993-08-03T15:48:46.943 201 456678 false 1993-08-03T15:48:45 010-18741032 {"wTS": "1596440925", "rfId": "010-18741032", "remark": "数据", "weight": 118.863} 710078936338096128 1596440925-010-18741032 +139872403179528192 10000 1993-08-03T15:48:47.082 10000 1993-08-03T15:48:47.082 201 456678 false 1993-08-03T15:48:45 010-18741021 {"wTS": "1596440925", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440925-010-18741021 +139872403720593408 10000 1993-08-03T15:48:47.211 10000 1993-08-03T15:48:47.211 201 456678 false 1993-08-03T15:48:45 010-18741037 {"wTS": "1596440925", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440925-010-18741037 +139872404051943424 10000 1993-08-03T15:48:47.290 10000 1993-08-03T15:48:47.290 201 456678 false 1993-08-03T15:48:44 010-18741028 {"wTS": "1596440924", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440924-010-18741028 +139872404689477632 10000 1993-08-03T15:48:47.442 10000 1993-08-03T15:48:47.442 201 456678 false 1993-08-03T15:48:44 010-18741030 {"wTS": "1596440924", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440924-010-18741030 +139872518397059072 10000 1993-08-03T15:49:14.552 10000 1993-08-03T15:49:14.552 201 456678 false 1993-08-03T15:49:14 010-18741038 {"wTS": "1596440954", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440954-010-18741038 +139872518405447680 10000 1993-08-03T15:49:14.554 10000 1993-08-03T15:49:14.554 201 456678 false 1993-08-03T15:49:14 010-18741035 {"wTS": "1596440954", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440954-010-18741035 +139872518539665408 10000 1993-08-03T15:49:14.586 10000 1993-08-03T15:49:14.586 201 456678 false 1993-08-03T15:49:14 010-18741039 {"wTS": "1596440954", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440954-010-18741039 +139872519579852800 10000 1993-08-03T15:49:14.834 10000 1993-08-03T15:49:14.834 201 456678 false 1993-08-03T15:49:14 010-18741040 {"wTS": "1596440954", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440954-010-18741040 +139872520058003456 10000 1993-08-03T15:49:14.948 10000 1993-08-03T15:49:14.948 201 456678 false 1993-08-03T15:49:14 010-18741034 {"wTS": "1596440954", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440954-010-18741034 +139872520221581312 10000 1993-08-03T15:49:14.987 10000 1993-08-03T15:49:14.987 201 456678 false 1993-08-03T15:49:14 010-18741024 {"wTS": "1596440954", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440954-010-18741024 +139872520645206016 10000 1993-08-03T15:49:15.088 10000 1993-08-03T15:49:15.088 201 456678 false 1993-08-03T15:49:14 010-18741022 {"wTS": "1596440954", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440954-010-18741022 +139872521467289600 10000 1993-08-03T15:49:15.284 10000 1993-08-03T15:49:15.284 201 456678 false 1993-08-03T15:49:14 010-18741033 {"wTS": "1596440954", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440954-010-18741033 +139872523287617536 10000 1993-08-03T15:49:15.718 10000 1993-08-03T15:49:15.718 201 456678 false 1993-08-03T15:49:14 010-18741027 {"wTS": "1596440954", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440954-010-18741027 +139872524629794816 10000 1993-08-03T15:49:16.038 10000 1993-08-03T15:49:16.038 201 456678 false 1993-08-03T15:49:14 010-18741029 {"wTS": "1596440954", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440954-010-18741029 +139872524915007488 10000 1993-08-03T15:49:16.106 10000 1993-08-03T15:49:16.106 201 456678 false 1993-08-03T15:49:14 010-18741036 {"wTS": "1596440954", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440954-010-18741036 +139872525456072704 10000 1993-08-03T15:49:16.235 10000 1993-08-03T15:49:16.235 201 456678 false 1993-08-03T15:49:14 010-18741023 {"wTS": "1596440954", "rfId": "010-18741023", "remark": "数据", "weight": 118.114} 710078936338096128 1596440954-010-18741023 +139872527263817728 10000 1993-08-03T15:49:16.666 10000 1993-08-03T15:49:16.666 201 456678 false 1993-08-03T15:49:15 010-18741026 {"wTS": "1596440955", "rfId": "010-18741026", "remark": "数据", "weight": 147.465} 710078936338096128 1596440955-010-18741026 +139872527469338624 10000 1993-08-03T15:49:16.715 10000 1993-08-03T15:49:16.715 201 456678 false 1993-08-03T15:49:15 010-18741021 {"wTS": "1596440955", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440955-010-18741021 +139872528224313344 10000 1993-08-03T15:49:16.895 10000 1993-08-03T15:49:16.895 201 456678 false 1993-08-03T15:49:15 010-18741037 {"wTS": "1596440955", "rfId": "010-18741037", "remark": "数据", "weight": 114.952} 710078936338096128 1596440955-010-18741037 +139872644595277824 10000 1993-08-03T15:49:44.640 10000 1993-08-03T15:49:44.640 201 456678 false 1993-08-03T15:49:44 010-18741039 {"wTS": "1596440984", "rfId": "010-18741039", "remark": "数据", "weight": 152.171} 710078936338096128 1596440984-010-18741039 +139872644674969600 10000 1993-08-03T15:49:44.659 10000 1993-08-03T15:49:44.659 201 456678 false 1993-08-03T15:49:44 010-18741035 {"wTS": "1596440984", "rfId": "010-18741035", "remark": "数据", "weight": 158.451} 710078936338096128 1596440984-010-18741035 +139872644922433536 10000 1993-08-03T15:49:44.718 10000 1993-08-03T15:49:44.718 201 456678 false 1993-08-03T15:49:44 010-18741038 {"wTS": "1596440984", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596440984-010-18741038 +139872645878734848 10000 1993-08-03T15:49:44.946 10000 1993-08-03T15:49:44.946 201 456678 false 1993-08-03T15:49:44 010-18741040 {"wTS": "1596440984", "rfId": "010-18741040", "remark": "数据", "weight": 145.513} 710078936338096128 1596440984-010-18741040 +139872646386245632 10000 1993-08-03T15:49:45.067 10000 1993-08-03T15:49:45.067 201 456678 false 1993-08-03T15:49:44 010-18741034 {"wTS": "1596440984", "rfId": "010-18741034", "remark": "数据", "weight": 110.069} 710078936338096128 1596440984-010-18741034 +139872646591766528 10000 1993-08-03T15:49:45.116 10000 1993-08-03T15:49:45.116 201 456678 false 1993-08-03T15:49:44 010-18741022 {"wTS": "1596440984", "rfId": "010-18741022", "remark": "数据", "weight": 104.797} 710078936338096128 1596440984-010-18741022 +139872647241883648 10000 1993-08-03T15:49:45.271 10000 1993-08-03T15:49:45.271 201 456678 false 1993-08-03T15:49:44 010-18741024 {"wTS": "1596440984", "rfId": "010-18741024", "remark": "数据", "weight": 62.852} 710078936338096128 1596440984-010-18741024 +139872648152047616 10000 1993-08-03T15:49:45.488 10000 1993-08-03T15:49:45.488 201 456678 false 1993-08-03T15:49:44 010-18741033 {"wTS": "1596440984", "rfId": "010-18741033", "remark": "数据", "weight": 119.771} 710078936338096128 1596440984-010-18741033 +139872649997541376 10000 1993-08-03T15:49:45.928 10000 1993-08-03T15:49:45.928 201 456678 false 1993-08-03T15:49:44 010-18741031 {"wTS": "1596440984", "rfId": "010-18741031", "remark": "数据", "weight": 102.374} 710078936338096128 1596440984-010-18741031 +139872650000171200 10000 1993-08-03T15:49:46.309 10000 1993-08-03T15:49:46.309 201 456678 false 1993-08-03T15:49:45 010-18741027 {"wTS": "1596440985", "rfId": "010-18741027", "remark": "数据", "weight": 89.6} 710078936338096128 1596440985-010-18741027 +139872651008368640 10000 1993-08-03T15:49:46.169 10000 1993-08-03T15:49:46.169 201 456678 false 1993-08-03T15:49:44 010-18741030 {"wTS": "1596440984", "rfId": "010-18741030", "remark": "数据", "weight": 137.051} 710078936338096128 1596440984-010-18741030 +139872653399121920 10000 1993-08-03T15:49:46.739 10000 1993-08-03T15:49:46.739 201 456678 false 1993-08-03T15:49:44 010-18741028 {"wTS": "1596440984", "rfId": "010-18741028", "remark": "数据", "weight": 62.048} 710078936338096128 1596440984-010-18741028 +139872653420093440 10000 1993-08-03T15:49:46.744 10000 1993-08-03T15:49:46.744 201 456678 false 1993-08-03T15:49:44 010-18741036 {"wTS": "1596440984", "rfId": "010-18741036", "remark": "数据", "weight": 141.555} 710078936338096128 1596440984-010-18741036 +139872654166679552 10000 1993-08-03T15:49:46.923 10000 1993-08-03T15:49:46.923 201 456678 false 1993-08-03T15:49:45 010-18741029 {"wTS": "1596440985", "rfId": "010-18741029", "remark": "数据", "weight": 123.022} 710078936338096128 1596440985-010-18741029 +139872655022317568 10000 1993-08-03T15:49:47.126 10000 1993-08-03T15:49:47.126 201 456678 false 1993-08-03T15:49:45 010-18741025 {"wTS": "1596440985", "rfId": "010-18741025", "remark": "数据", "weight": 110.27} 710078936338096128 1596440985-010-18741025 +139872655416582144 10000 1993-08-03T15:49:47.220 10000 1993-08-03T15:49:47.220 201 456678 false 1993-08-03T15:49:45 010-18741032 {"wTS": "1596440985", "rfId": "010-18741032", "remark": "数据", "weight": 118.863} 710078936338096128 1596440985-010-18741032 +139872656439992320 10000 1993-08-03T15:49:47.464 10000 1993-08-03T15:49:47.464 201 456678 false 1993-08-03T15:49:45 010-18741021 {"wTS": "1596440985", "rfId": "010-18741021", "remark": "数据", "weight": 169.033} 710078936338096128 1596440985-010-18741021 +139872770659278848 10000 1993-08-03T15:50:14.696 10000 1993-08-03T15:50:14.696 201 456678 false 1993-08-03T15:50:14 010-18741038 {"wTS": "1596441014", "rfId": "010-18741038", "remark": "数据", "weight": 154.728} 710078936338096128 1596441014-010-18741038 +149906320110972801 10000 2021-12-24T08:19:59.449 10000 2021-12-24T08:19:59.449 201 123321 false 2021-12-24T08:19:59 65322-002867696000 {"wTS": "1598833199", "rfId": "65322-002867696000", "remark": "数据", "weight": 57.5} 1233211234567891011 1598833199-65322-002867696000 +185561918179860480 10000 1993-12-07T17:42:36.328 10000 1993-12-07T17:42:36.328 201 123321 false 1993-12-07T17:42:35 0410-137110012 {"wTS": "1607334155", "rfId": "0410-137110012", "remark": "数据", "weight": 560.0} 1233211234567891011 1607334155-0410-137110012 +185562285613473792 10000 1993-12-07T17:44:03.931 10000 1993-12-07T17:44:03.931 201 123321 false 1993-12-07T17:44:03 0410-137110012 {"wTS": "1607334243", "rfId": "0410-137110012", "remark": "数据", "weight": 580.0} 1233211234567891011 1607334243-0410-137110012 +185564316508053504 10000 1993-12-07T17:52:08.134 10000 1993-12-07T17:52:08.134 201 123321 false 1993-12-07T17:52:07 0410-137110012 {"wTS": "1607334727", "rfId": "0410-137110012", "remark": "数据", "weight": 480.0} 1233211234567891011 1607334727-0410-137110012 +864565750682841088 10000 2021-07-13T17:55:37.817 10000 2021-07-13T17:55:37.817 201 123321 false 2021-07-13T17:55:22 000-000000000000 {"wTS": "1626170122", "rfId": "000-000000000000", "remark": "数据", "weight": 78.0} 720303223645310976 1626170122-000-000000000000 +864566846897745920 10000 2021-07-13T17:59:59.175 10000 2021-07-13T17:59:59.175 201 123321 false 2021-07-13T17:59:30 000-000000000000 {"wTS": "1626170370", "rfId": "000-000000000000", "remark": "数据", "weight": 77.1} 720303223645310976 1626170370-000-000000000000 +864568598468456448 10000 2021-07-13T18:06:56.782 10000 2021-07-13T18:06:56.782 201 123321 false 2021-07-13T18:04:44 000-000000000000 {"wTS": "1626170684", "rfId": "000-000000000000", "remark": "数据", "weight": 77.7} 720303223645310976 1626170684-000-000000000000 +864568901754384384 10000 2021-07-13T18:08:09.091 10000 2021-07-13T18:08:09.091 201 123321 false 2021-07-13T18:07:29 000-000000000000 {"wTS": "1626170849", "rfId": "000-000000000000", "remark": "数据", "weight": 2000.0} 720303223645310976 1626170849-000-000000000000 +864568950307647488 10000 2021-07-13T18:08:20.667 10000 2021-07-13T18:08:20.667 201 123321 false 2021-07-13T18:08:06 000-000000000000 {"wTS": "1626170886", "rfId": "000-000000000000", "remark": "数据", "weight": 2000.0} 720303223645310976 1626170886-000-000000000000 +864577514845728768 10000 2021-07-13T18:42:22.612 10000 2021-07-13T18:42:22.612 201 123321 false 2021-07-13T18:41:05 000-000000000000 {"wTS": "1626172865", "rfId": "000-000000000000", "remark": "数据", "weight": 30.7} 720303223645310976 1626172865-000-000000000000 +864585483348045824 10000 2021-07-13T19:14:02.451 10000 2021-07-13T19:14:02.451 201 123321 false 2021-07-13T19:13:48 999-020052022634 {"wTS": "1626174828", "rfId": "999-020052022634", "remark": "数据", "weight": 2000.0} 720303223645310976 1626174828-999-020052022634 +864595543231000576 10000 2021-07-13T19:54:00.914 10000 2021-07-13T19:54:00.914 201 123321 false 2021-07-13T19:53:56 000-000000000000 {"wTS": "1626177236", "rfId": "000-000000000000", "remark": "数据", "weight": 2000.0} 720303223645310976 1626177236-000-000000000000 +864595601850593280 10000 2021-07-13T19:54:14.890 10000 2021-07-13T19:54:14.890 201 123321 false 2021-07-13T19:54:05 999-020052022634 {"wTS": "1626177245", "rfId": "999-020052022634", "remark": "数据", "weight": 43.7} 720303223645310976 1626177245-999-020052022634 +864595723665764352 10000 2021-07-13T19:54:43.933 10000 2021-07-13T19:54:43.933 201 123321 false 2021-07-13T19:54:39 999-020052022634 {"wTS": "1626177279", "rfId": "999-020052022634", "remark": "数据", "weight": 2000.0} 720303223645310976 1626177279-999-020052022634 +864595868411195392 10000 2021-07-13T19:55:18.443 10000 2021-07-13T19:55:18.443 201 123321 false 2021-07-13T19:55:14 999-020052022634 {"wTS": "1626177314", "rfId": "999-020052022634", "remark": "数据", "weight": 128.1} 720303223645310976 1626177314-999-020052022634 +864596000837955584 10000 2021-07-13T19:55:50.016 10000 2021-07-13T19:55:50.016 201 123321 false 2021-07-13T19:55:46 999-020052022634 {"wTS": "1626177346", "rfId": "999-020052022634", "remark": "数据", "weight": 2000.0} 720303223645310976 1626177346-999-020052022634 +864596477000511488 10000 2021-07-13T19:57:43.542 10000 2021-07-13T19:57:43.542 201 123321 false 2021-07-13T19:57:39 999-020052022634 {"wTS": "1626177459", "rfId": "999-020052022634", "remark": "数据", "weight": 2000.0} 720303223645310976 1626177459-999-020052022634 +864598919201124352 10000 2021-07-13T20:07:25.808 10000 2021-07-13T20:07:25.808 201 123321 false 2021-07-13T20:06:56 999-020052022634 {"wTS": "1626178016", "rfId": "999-020052022634", "remark": "数据", "weight": 2000.0} 720303223645310976 1626178016-999-020052022634 +864598949525942272 10000 2021-07-13T20:07:33.038 10000 2021-07-13T20:07:33.038 201 123321 false 2021-07-13T20:07:28 999-020052022634 {"wTS": "1626178048", "rfId": "999-020052022634", "remark": "数据", "weight": 2000.0} 720303223645310976 1626178048-999-020052022634 +864599123581169664 10000 2021-07-13T20:08:14.536 10000 2021-07-13T20:08:14.536 201 123321 false 2021-07-13T20:08:11 999-020052022634 {"wTS": "1626178091", "rfId": "999-020052022634", "remark": "数据", "weight": 2000.0} 720303223645310976 1626178091-999-020052022634 +864599252606349312 10000 2021-07-13T20:08:45.299 10000 2021-07-13T20:08:45.299 201 123321 false 2021-07-13T20:08:41 999-020052022634 {"wTS": "1626178121", "rfId": "999-020052022634", "remark": "数据", "weight": 2000.0} 720303223645310976 1626178121-999-020052022634 +864599388354998272 10000 2021-07-13T20:09:17.663 10000 2021-07-13T20:09:17.663 201 123321 false 2021-07-13T20:09:13 999-020052022634 {"wTS": "1626178153", "rfId": "999-020052022634", "remark": "数据", "weight": 31.2} 720303223645310976 1626178153-999-020052022634 +864599520311996416 10000 2021-07-13T20:09:49.124 10000 2021-07-13T20:09:49.124 201 123321 false 2021-07-13T20:09:45 999-020052022634 {"wTS": "1626178185", "rfId": "999-020052022634", "remark": "数据", "weight": 31.4} 720303223645310976 1626178185-999-020052022634 +864599651631460352 10000 2021-07-13T20:10:20.433 10000 2021-07-13T20:10:20.433 201 123321 false 2021-07-13T20:10:16 999-020052022634 {"wTS": "1626178216", "rfId": "999-020052022634", "remark": "数据", "weight": 95.7} 720303223645310976 1626178216-999-020052022634 +864599775870939136 10000 2021-07-13T20:10:50.054 10000 2021-07-13T20:10:50.054 201 123321 false 2021-07-13T20:10:45 999-020052022634 {"wTS": "1626178245", "rfId": "999-020052022634", "remark": "数据", "weight": 31.3} 720303223645310976 1626178245-999-020052022634 +925010014523199488 10000 2021-12-27T12:59:33.317 10000 2021-12-27T12:59:33.317 201 123321 false 2021-12-27T13:20:21 984-000100748724 {"wTS": "1640582421", "rfId": "984-000100748724", "remark": "数据", "weight": 0.123} 1233211234567891011 1640582421-984-000100748724 +972179152830517248 10000 2022-05-06T16:52:52.250 10000 2022-05-06T16:52:52.250 202 123321 false 2022-05-06T16:52:49 0410-137110005 {"wTS": "1651827169", "rfId": "0410-137110005", "remark": "", "weight": 200.0} 1233211234567891011 1651827169-0410-137110005 +972179288327507968 10000 2022-05-06T16:53:24.555 10000 2022-05-06T16:53:24.555 202 123321 false 2022-05-06T16:53:22 0410-137110005 {"wTS": "1651827202", "rfId": "0410-137110005", "remark": "", "weight": 100.0} 1233211234567891011 1651827202-0410-137110005 +972182670182240256 10000 2022-05-06T17:06:50.852 10000 2022-05-06T17:06:50.852 201 123321 false 2022-05-06T17:06:47 984-000100744406 {"wTS": "1651828007", "rfId": "984-000100744406", "remark": "数据", "weight": 1.119} 1233211234567891011 1651828007-984-000100744406 +972185021504868352 10000 2022-05-06T17:16:11.451 10000 2022-05-06T17:16:11.451 201 123321 false 2022-05-06T17:16:08 984-000100744406 {"wTS": "1651828568", "rfId": "984-000100744406", "remark": "数据", "weight": 11.19} 1233211234567891011 1651828568-984-000100744406 +972186411757260800 10000 2022-05-06T17:21:42.913 10000 2022-05-06T17:21:42.913 201 123321 false 2022-05-06T17:21:39 984-000100744406 {"wTS": "1651828899", "rfId": "984-000100744406", "remark": "数据", "weight": 111.9} 1233211234567891011 1651828899-984-000100744406 +972187393287311360 10000 2022-05-06T17:25:36.928 10000 2022-05-06T17:25:36.928 201 123321 false 2022-05-06T17:25:33 984-000100744406 {"wTS": "1651829133", "rfId": "984-000100744406", "remark": "数据", "weight": 121.9} 1233211234567891011 1651829133-984-000100744406 -- !test11 -- \N \N \N \N \N @@ -2126,9 +2126,9 @@ true abc def 2022-10-11 1.234 1 2 99 2022-10-22T10:59:59 34.123 234 bcd -- !test12 -- -1 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id":1} -2 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:ff 0000001010 0000001010 2 {"id":1} -3 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:ff 0000001010 0001010 3 {"id":1} +1 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} +2 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:ff 0000001010 0000001010 2 {"id": 1} +3 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:ff 0000001010 0001010 3 {"id": 1} -- !test13 -- 1 (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> @@ -2149,23 +2149,23 @@ true abc def 2022-10-11 1.234 1 2 99 2022-10-22T10:59:59 34.123 2023-06-16T12:34:56.123 2023-06-16T12:34:56.123 -- !json_test -- -1 json {"stringKey":"stringValue","integerKey":12345,"floatKey":123.45,"booleanKey":true,"nullKey":null,"arrayKey":["element1",2,false,null,{"nestedKey":"nestedValue"}],"objectKey":{"nestedStringKey":"nestedStringValue","nestedIntegerKey":67890}} +1 json {\n "stringKey": "stringValue",\n "integerKey": 12345,\n "floatKey": 123.45,\n "booleanKey": true,\n "nullKey": null,\n "arrayKey": ["element1", 2, false, null, {"nestedKey": "nestedValue"}],\n "objectKey": {\n "nestedStringKey": "nestedStringValue",\n "nestedIntegerKey": 67890\n }\n} 2 json2 \N -- !jsonb_test -- -1 jsonb {"nullKey":null,"arrayKey":["element1",2,false,null,{"nestedKey":"nestedValue"}],"floatKey":123.45,"objectKey":{"nestedStringKey":"nestedStringValue","nestedIntegerKey":67890},"stringKey":"stringValue","booleanKey":true,"integerKey":12345} +1 jsonb {"nullKey": null, "arrayKey": ["element1", 2, false, null, {"nestedKey": "nestedValue"}], "floatKey": 123.45, "objectKey": {"nestedStringKey": "nestedStringValue", "nestedIntegerKey": 67890}, "stringKey": "stringValue", "booleanKey": true, "integerKey": 12345} 2 jsonb2 \N -- !filter1 -- -1 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id":1} -2 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:ff 0000001010 0000001010 2 {"id":1} -3 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:ff 0000001010 0001010 3 {"id":1} +1 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} +2 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:ff 0000001010 0000001010 2 {"id": 1} +3 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:ff 0000001010 0001010 3 {"id": 1} -- !filter2 -- -1 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id":1} +1 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} -- !filter3 -- -1 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id":1} +1 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} -- !partition_1_0 -- 1 GZ @@ -2226,11 +2226,11 @@ doris3 20 doris3 20 -- !select_all_types -- -1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id":1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> +1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> 2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N -- !ctas -- -1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id":1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> +1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> 2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N -- !ctas_desc -- @@ -2247,7 +2247,7 @@ decimal_value DECIMAL(10, 3) Yes false \N NONE id INT No true \N inet_value TEXT Yes false \N NONE int_value INT Yes false \N NONE -jsonb_value JSON Yes false \N NONE +jsonb_value TEXT Yes false \N NONE line_value TEXT Yes false \N NONE lseg_value TEXT Yes false \N NONE macaddr_value TEXT Yes false \N NONE From 41672f2b92bf999fbf9e177cee6e979d9af2f102 Mon Sep 17 00:00:00 2001 From: Pxl Date: Thu, 18 Jan 2024 10:40:13 +0800 Subject: [PATCH 078/200] [Feature](materialized-view) support some case unmached to materialized-view (#30036) same column appears in key and value like select id,count(id) group by id; complex expr in sum select sum(if(xxx)); --- fe/fe-core/src/main/cup/sql_parser.cup | 5 ++ .../doris/analysis/BinaryPredicate.java | 4 + .../SelectMaterializedIndexWithAggregate.java | 9 ++- .../doris/analysis/CancelExportStmtTest.java | 10 +-- .../doris/analysis/CancelLoadStmtTest.java | 10 +-- .../doris/analysis/DataDescriptionTest.java | 16 ++-- .../apache/doris/analysis/DeleteStmtTest.java | 4 +- .../apache/doris/analysis/SelectStmtTest.java | 36 ++++----- .../doris/analysis/ShowAlterStmtTest.java | 4 +- .../doris/analysis/ShowBackupStmtTest.java | 2 +- .../doris/analysis/ShowExportStmtTest.java | 4 +- .../doris/analysis/ShowLoadStmtTest.java | 4 +- .../analysis/ShowPartitionsStmtTest.java | 4 +- .../doris/analysis/ShowResourcesStmtTest.java | 2 +- .../apache/doris/analysis/SqlModeTest.java | 4 +- .../apache/doris/catalog/CreateViewTest.java | 2 +- .../org/apache/doris/planner/PlannerTest.java | 6 +- .../apache/doris/planner/QueryPlanTest.java | 74 +++++++++---------- .../apache/doris/planner/RepeatNodeTest.java | 2 +- .../doris/planner/TableFunctionPlanTest.java | 8 +- .../org/apache/doris/policy/PolicyTest.java | 20 ++--- .../apache/doris/qe/OlapQueryCacheTest.java | 46 ++++++------ .../ExtractCommonFactorsRuleFunctionTest.java | 6 +- .../data/ddl_p0/test_create_view.out | 2 +- .../testAggQueryOnAggMV1.out | 10 +++ .../performance_p0/redundant_conjuncts.out | 4 +- .../test_show_create_table_and_views.out | 2 +- .../jdbc/test_clickhouse_jdbc_catalog.groovy | 4 +- .../jdbc/test_mysql_jdbc_catalog.groovy | 6 +- .../test_mysql_jdbc_catalog_nereids.groovy | 2 +- .../jdbc/test_oracle_jdbc_catalog.groovy | 2 +- .../testAggQueryOnAggMV1.groovy | 14 ++++ .../infer_predicate/infer_predicate.groovy | 12 +-- 33 files changed, 187 insertions(+), 153 deletions(-) diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index aaadfb4f563233..a782100ef23f1c 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -1066,6 +1066,11 @@ import_column_descs ::= columns.add(column); RESULT = columns; :} + | import_column_descs:columns COMMA LPAREN import_column_desc:column RPAREN + {: + columns.add(column); + RESULT = columns; + :} ; import_column_desc ::= diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/BinaryPredicate.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/BinaryPredicate.java index 74f3035285c244..bb30a46b3e9539 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/BinaryPredicate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/BinaryPredicate.java @@ -159,6 +159,7 @@ public boolean isUnequivalence() { // for restoring public BinaryPredicate() { super(); + printSqlInParens = true; } public BinaryPredicate(Operator op, Expr e1, Expr e2) { @@ -169,6 +170,7 @@ public BinaryPredicate(Operator op, Expr e1, Expr e2) { children.add(e1); Preconditions.checkNotNull(e2); children.add(e2); + printSqlInParens = true; } public BinaryPredicate(Operator op, Expr e1, Expr e2, Type retType, NullableMode nullableMode) { @@ -181,6 +183,7 @@ public BinaryPredicate(Operator op, Expr e1, Expr e2, Type retType, NullableMode children.add(e2); fn = new Function(new FunctionName(op.name), Lists.newArrayList(e1.getType(), e2.getType()), retType, false, true, nullableMode); + printSqlInParens = true; } protected BinaryPredicate(BinaryPredicate other) { @@ -188,6 +191,7 @@ protected BinaryPredicate(BinaryPredicate other) { op = other.op; slotIsleft = other.slotIsleft; isInferred = other.isInferred; + printSqlInParens = true; } public boolean isInferred() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithAggregate.java index ae627523e36d2c..c0f70caf8510ad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithAggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithAggregate.java @@ -1103,7 +1103,7 @@ private AggRewriteResult rewriteAgg(MaterializedIndex index, } if (isInputSlotsContainsNone( predicates.stream().filter(e -> !indexConjuncts.contains(e.toSql())).collect(Collectors.toList()), - slotsToReplace) && isInputSlotsContainsNone(groupingExprs, slotsToReplace)) { + slotsToReplace)) { ImmutableSet newRequiredSlots = requiredScanOutput.stream() .map(slot -> (Slot) ExpressionUtils.replace(slot, slotMap)).collect(ImmutableSet.toImmutableSet()); return new AggRewriteResult(index, true, newRequiredSlots, exprRewriteMap); @@ -1522,8 +1522,7 @@ public Expression visitSum(Sum sum, RewriteContext context) { if (result != sum) { return result; } - Optional slotOpt = ExpressionUtils.extractSlotOrCastOnSlot(sum.child(0)); - if (!sum.isDistinct() && slotOpt.isPresent()) { + if (!sum.isDistinct()) { Expression expr = castIfNeed(sum.child(), BigIntType.INSTANCE); String sumColumn = normalizeName(CreateMaterializedViewStmt.mvColumnBuilder(AggregateType.SUM, CreateMaterializedViewStmt.mvColumnBuilder(expr.toSql()))); @@ -1532,7 +1531,9 @@ public Expression visitSum(Sum sum, RewriteContext context) { Slot sumSlot = context.checkContext.scan.getOutputByIndex(context.checkContext.index).stream() .filter(s -> sumColumn.equalsIgnoreCase(normalizeName(s.getName()))).findFirst() .orElseThrow(() -> new AnalysisException("cannot find sum slot when select mv")); - context.exprRewriteMap.slotMap.put(slotOpt.get(), sumSlot); + for (Slot slot : sum.child().getInputSlots()) { + context.exprRewriteMap.slotMap.put(slot, sumSlot); + } context.exprRewriteMap.projectExprMap.put(sum.child(), sumSlot); Sum newSum = new Sum(sumSlot); context.exprRewriteMap.aggFuncMap.put(sum, newSum); diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/CancelExportStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/CancelExportStmtTest.java index c814630d830267..f2cf353760096c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/CancelExportStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/CancelExportStmtTest.java @@ -63,7 +63,7 @@ public void testNormal() throws UserException { labelStringLiteral); CancelExportStmt stmt = new CancelExportStmt(null, labelBinaryPredicate); stmt.analyze(analyzer); - Assertions.assertEquals("CANCEL EXPORT FROM testDb WHERE `label` = 'doris_test_label'", + Assertions.assertEquals("CANCEL EXPORT FROM testDb WHERE (`label` = 'doris_test_label')", stmt.toString()); SlotRef labelSlotRefUpper = new SlotRef(null, "LABEL"); @@ -71,7 +71,7 @@ public void testNormal() throws UserException { labelStringLiteral); CancelExportStmt stmtUpper = new CancelExportStmt(null, labelBinaryPredicateUpper); stmtUpper.analyze(analyzer); - Assertions.assertEquals("CANCEL EXPORT FROM testDb WHERE `LABEL` = 'doris_test_label'", + Assertions.assertEquals("CANCEL EXPORT FROM testDb WHERE (`LABEL` = 'doris_test_label')", stmtUpper.toString()); StringLiteral stateStringLiteral = new StringLiteral("PENDING"); @@ -79,7 +79,7 @@ public void testNormal() throws UserException { stateStringLiteral); stmt = new CancelExportStmt(null, stateBinaryPredicate); stmt.analyze(analyzer); - Assertions.assertEquals("CANCEL EXPORT FROM testDb WHERE `state` = 'PENDING'", stmt.toString()); + Assertions.assertEquals("CANCEL EXPORT FROM testDb WHERE (`state` = 'PENDING')", stmt.toString()); LikePredicate labelLikePredicate = new LikePredicate(LikePredicate.Operator.LIKE, labelSlotRef, labelStringLiteral); @@ -93,7 +93,7 @@ public void testNormal() throws UserException { stmt = new CancelExportStmt(null, compoundAndPredicate); stmt.analyze(analyzer); Assertions.assertEquals( - "CANCEL EXPORT FROM testDb WHERE `label` = 'doris_test_label' AND `state` = 'PENDING'", + "CANCEL EXPORT FROM testDb WHERE (`label` = 'doris_test_label') AND (`state` = 'PENDING')", stmt.toString()); CompoundPredicate compoundOrPredicate = new CompoundPredicate(Operator.OR, labelBinaryPredicate, @@ -101,7 +101,7 @@ public void testNormal() throws UserException { stmt = new CancelExportStmt(null, compoundOrPredicate); stmt.analyze(analyzer); Assertions.assertEquals( - "CANCEL EXPORT FROM testDb WHERE `label` = 'doris_test_label' OR `state` = 'PENDING'", + "CANCEL EXPORT FROM testDb WHERE (`label` = 'doris_test_label') OR (`state` = 'PENDING')", stmt.toString()); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/CancelLoadStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/CancelLoadStmtTest.java index d9758ab07bf0a1..f51ac74c342956 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/CancelLoadStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/CancelLoadStmtTest.java @@ -64,7 +64,7 @@ public void testNormal() throws UserException { labelStringLiteral); CancelLoadStmt stmt = new CancelLoadStmt(null, labelBinaryPredicate); stmt.analyze(analyzer); - Assertions.assertEquals("CANCEL LOAD FROM testDb WHERE `label` = 'doris_test_label'", + Assertions.assertEquals("CANCEL LOAD FROM testDb WHERE (`label` = 'doris_test_label')", stmt.toString()); SlotRef labelSlotRefUpper = new SlotRef(null, "LABEL"); @@ -72,7 +72,7 @@ public void testNormal() throws UserException { labelStringLiteral); CancelLoadStmt stmtUpper = new CancelLoadStmt(null, labelBinaryPredicateUpper); stmtUpper.analyze(analyzer); - Assertions.assertEquals("CANCEL LOAD FROM testDb WHERE `LABEL` = 'doris_test_label'", + Assertions.assertEquals("CANCEL LOAD FROM testDb WHERE (`LABEL` = 'doris_test_label')", stmtUpper.toString()); StringLiteral stateStringLiteral = new StringLiteral("LOADING"); @@ -80,7 +80,7 @@ public void testNormal() throws UserException { stateStringLiteral); stmt = new CancelLoadStmt(null, stateBinaryPredicate); stmt.analyze(analyzer); - Assertions.assertEquals("CANCEL LOAD FROM testDb WHERE `state` = 'LOADING'", stmt.toString()); + Assertions.assertEquals("CANCEL LOAD FROM testDb WHERE (`state` = 'LOADING')", stmt.toString()); LikePredicate labelLikePredicate = new LikePredicate(LikePredicate.Operator.LIKE, labelSlotRef, labelStringLiteral); @@ -94,7 +94,7 @@ public void testNormal() throws UserException { stmt = new CancelLoadStmt(null, compoundAndPredicate); stmt.analyze(analyzer); Assertions.assertEquals( - "CANCEL LOAD FROM testDb WHERE `label` = 'doris_test_label' AND `state` = 'LOADING'", + "CANCEL LOAD FROM testDb WHERE (`label` = 'doris_test_label') AND (`state` = 'LOADING')", stmt.toString()); CompoundPredicate compoundOrPredicate = new CompoundPredicate(Operator.OR, labelBinaryPredicate, @@ -102,7 +102,7 @@ public void testNormal() throws UserException { stmt = new CancelLoadStmt(null, compoundOrPredicate); stmt.analyze(analyzer); Assertions.assertEquals( - "CANCEL LOAD FROM testDb WHERE `label` = 'doris_test_label' OR `state` = 'LOADING'", + "CANCEL LOAD FROM testDb WHERE (`label` = 'doris_test_label') OR (`state` = 'LOADING')", stmt.toString()); // test match diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/DataDescriptionTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/DataDescriptionTest.java index c5a56a96c68251..b6a0aa2233c5cc 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/DataDescriptionTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/DataDescriptionTest.java @@ -127,9 +127,9 @@ public void testNormal() throws AnalysisException { desc = new DataDescription("testTable", null, Lists.newArrayList("abc.txt"), Lists.newArrayList("col1", "col2"), new Separator(","), "csv", null, false, null, null, whereExpr, LoadTask.MergeType.MERGE, whereExpr, null, null); desc.analyze("testDb"); - Assert.assertEquals("MERGE DATA INFILE ('abc.txt') INTO TABLE testTable COLUMNS TERMINATED BY ',' FORMAT AS 'csv' (col1, col2) WHERE 1 = 1 DELETE ON 1 = 1", desc.toString()); - Assert.assertEquals("1 = 1", desc.getWhereExpr().toSql()); - Assert.assertEquals("1 = 1", desc.getDeleteCondition().toSql()); + Assert.assertEquals("MERGE DATA INFILE ('abc.txt') INTO TABLE testTable COLUMNS TERMINATED BY ',' FORMAT AS 'csv' (col1, col2) WHERE (1 = 1) DELETE ON (1 = 1)", desc.toString()); + Assert.assertEquals("(1 = 1)", desc.getWhereExpr().toSql()); + Assert.assertEquals("(1 = 1)", desc.getDeleteCondition().toSql()); Assert.assertEquals(",", desc.getColumnSeparator()); desc = new DataDescription("testTable", null, Lists.newArrayList("abc.txt", "bcd.txt"), @@ -168,7 +168,7 @@ public void testNormal() throws AnalysisException { .newArrayList((Expr) predicate)); desc.analyze("testDb"); String sql = "APPEND DATA INFILE ('abc.txt') INTO TABLE testTable PARTITIONS (p1, p2) (k2, k3)" - + " SET (`k1` = alignment_timestamp('day', `k2`))"; + + " SET ((`k1` = alignment_timestamp('day', `k2`)))"; Assert.assertEquals(sql, desc.toString()); // replace_value func @@ -183,7 +183,7 @@ public void testNormal() throws AnalysisException { false, Lists.newArrayList((Expr) predicate)); desc.analyze("testDb"); sql = "APPEND DATA INFILE ('abc.txt') INTO TABLE testTable PARTITIONS (p1, p2) (k2, k3)" - + " SET (`k1` = replace_value('-', '10'))"; + + " SET ((`k1` = replace_value('-', '10')))"; Assert.assertEquals(sql, desc.toString()); // replace_value null @@ -198,7 +198,7 @@ public void testNormal() throws AnalysisException { .newArrayList((Expr) predicate)); desc.analyze("testDb"); sql = "APPEND DATA INFILE ('abc.txt') INTO TABLE testTable PARTITIONS (p1, p2) (k2, k3)" - + " SET (`k1` = replace_value('', NULL))"; + + " SET ((`k1` = replace_value('', NULL)))"; Assert.assertEquals(sql, desc.toString()); // data from table and set bitmap_dict @@ -210,7 +210,7 @@ public void testNormal() throws AnalysisException { "testHiveTable", false, Lists.newArrayList(predicate), null, LoadTask.MergeType.APPEND, null, null); desc.analyze("testDb"); - sql = "APPEND DATA FROM TABLE testHiveTable INTO TABLE testTable PARTITIONS (p1, p2) SET (`k1` = bitmap_dict(`k2`))"; + sql = "APPEND DATA FROM TABLE testHiveTable INTO TABLE testTable PARTITIONS (p1, p2) SET ((`k1` = bitmap_dict(`k2`)))"; Assert.assertEquals(sql, desc.toSql()); Map properties = Maps.newHashMap(); @@ -409,7 +409,7 @@ public void testMysqlLoadData() throws AnalysisException { + "COLUMNS TERMINATED BY '010203' " + "LINES TERMINATED BY '040506' " + "(k1, k2, v1) " - + "SET (`k1` = bitmap_dict('day', `k2`))"; + + "SET ((`k1` = bitmap_dict('day', `k2`)))"; Assert.assertEquals(sql, desc.toSql()); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/DeleteStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/DeleteStmtTest.java index 2bd53e8d3e6a70..809ead85fe496d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/DeleteStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/DeleteStmtTest.java @@ -60,11 +60,11 @@ public void getMethodTest() { Assert.assertEquals("testDb", deleteStmt.getDbName()); Assert.assertEquals("testTbl", deleteStmt.getTableName()); Assert.assertEquals(Lists.newArrayList("partition"), deleteStmt.getPartitionNames()); - Assert.assertEquals("DELETE FROM `testDb`.`testTbl` PARTITION (partition) WHERE `k1` = 'abc'", + Assert.assertEquals("DELETE FROM `testDb`.`testTbl` PARTITION (partition) WHERE (`k1` = 'abc')", deleteStmt.toSql()); deleteStmt = new DeleteStmt(new TableName(internalCtl, "testDb", "testTbl"), null, wherePredicate); - Assert.assertEquals("DELETE FROM `testDb`.`testTbl` WHERE `k1` = 'abc'", + Assert.assertEquals("DELETE FROM `testDb`.`testTbl` WHERE (`k1` = 'abc')", deleteStmt.toSql()); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/SelectStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/SelectStmtTest.java index 331ee4eb060fb0..76bbdec5fdb2d1 100755 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/SelectStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/SelectStmtTest.java @@ -301,9 +301,9 @@ public void testDeduplicateOrs() throws Exception { String commonExpr2 = "`t3`.`k3` = `t1`.`k3`"; String commonExpr3 = "`t1`.`k1` = `t5`.`k1`"; String commonExpr4 = "t5`.`k2` = 'United States'"; - String betweenExpanded1 = "CAST(CAST(`t1`.`k4` AS DECIMALV3(12, 2)) AS INT) >= 100 AND CAST(CAST(`t1`.`k4` AS DECIMALV3(12, 2)) AS INT) <= 150"; - String betweenExpanded2 = "CAST(CAST(`t1`.`k4` AS DECIMALV3(12, 2)) AS INT) >= 50 AND CAST(CAST(`t1`.`k4` AS DECIMALV3(12, 2)) AS INT) <= 100"; - String betweenExpanded3 = "`t1`.`k4` >= 50 AND `t1`.`k4` <= 250"; + String betweenExpanded1 = "(CAST(CAST(`t1`.`k4` AS DECIMALV3(12, 2)) AS INT) >= 100) AND (CAST(CAST(`t1`.`k4` AS DECIMALV3(12, 2)) AS INT) <= 150)"; + String betweenExpanded2 = "(CAST(CAST(`t1`.`k4` AS DECIMALV3(12, 2)) AS INT) >= 50) AND (CAST(CAST(`t1`.`k4` AS DECIMALV3(12, 2)) AS INT) <= 100)"; + String betweenExpanded3 = "(`t1`.`k4` >= 50) AND (`t1`.`k4` <= 250)"; String rewrittenSql = stmt.toSql(); Assert.assertTrue(rewrittenSql.contains(commonExpr1)); @@ -347,17 +347,17 @@ public void testDeduplicateOrs() throws Exception { SelectStmt stmt2 = (SelectStmt) UtFrameUtils.parseAndAnalyzeStmt(sql2, ctx); stmt2.rewriteExprs(new Analyzer(ctx.getEnv(), ctx).getExprRewriter()); String fragment3 = - "(((`t1`.`k4` >= 50 AND `t1`.`k4` <= 300) AND `t2`.`k2` IN ('United States', 'United States1') " + "((((`t1`.`k4` >= 50) AND (`t1`.`k4` <= 300)) AND `t2`.`k2` IN ('United States', 'United States1') " + "AND `t2`.`k3` IN ('CO', 'IL', 'MN', 'OH', 'MT', 'NM', 'TX', 'MO', 'MI')) " - + "AND `t1`.`k1` = `t2`.`k3` AND `t2`.`k2` = 'United States' " - + "AND `t2`.`k3` IN ('CO', 'IL', 'MN') AND `t1`.`k4` >= 100 AND `t1`.`k4` <= 200 " + + "AND (`t1`.`k1` = `t2`.`k3`) AND (`t2`.`k2` = 'United States') " + + "AND `t2`.`k3` IN ('CO', 'IL', 'MN') AND (`t1`.`k4` >= 100) AND (`t1`.`k4` <= 200) " + "OR " - + "`t1`.`k1` = `t2`.`k1` AND `t2`.`k2` = 'United States1' " - + "AND `t2`.`k3` IN ('OH', 'MT', 'NM') AND `t1`.`k4` >= 150 AND `t1`.`k4` <= 300 " + + "(`t1`.`k1` = `t2`.`k1`) AND (`t2`.`k2` = 'United States1') " + + "AND `t2`.`k3` IN ('OH', 'MT', 'NM') AND (`t1`.`k4` >= 150) AND (`t1`.`k4` <= 300) " + "OR " - + "`t1`.`k1` = `t2`.`k1` AND `t2`.`k2` = 'United States' " + + "(`t1`.`k1` = `t2`.`k1`) AND (`t2`.`k2` = 'United States') " + "AND `t2`.`k3` IN ('TX', 'MO', 'MI') " - + "AND `t1`.`k4` >= 50 AND `t1`.`k4` <= 250)"; + + "AND (`t1`.`k4` >= 50) AND (`t1`.`k4` <= 250))"; Assert.assertTrue(stmt2.toSql().contains(fragment3)); String sql3 = "select\n" @@ -370,7 +370,7 @@ public void testDeduplicateOrs() throws Exception { SelectStmt stmt3 = (SelectStmt) UtFrameUtils.parseAndAnalyzeStmt(sql3, ctx); stmt3.rewriteExprs(new Analyzer(ctx.getEnv(), ctx).getExprRewriter()); Assert.assertFalse( - stmt3.toSql().contains("`t1`.`k1` = `t2`.`k3` OR `t1`.`k1` = `t2`.`k3` OR" + " `t1`.`k1` = `t2`.`k3`")); + stmt3.toSql().contains("(`t1`.`k1` = `t2`.`k3`) OR (`t1`.`k1` = `t2`.`k3`) OR" + " (`t1`.`k1` = `t2`.`k3`)")); String sql4 = "select\n" + " avg(t1.k4)\n" @@ -381,7 +381,7 @@ public void testDeduplicateOrs() throws Exception { + " t1.k1 = t2.k2 or t1.k1 = t2.k3 or t1.k1 = t2.k3"; SelectStmt stmt4 = (SelectStmt) UtFrameUtils.parseAndAnalyzeStmt(sql4, ctx); stmt4.rewriteExprs(new Analyzer(ctx.getEnv(), ctx).getExprRewriter()); - Assert.assertTrue(stmt4.toSql().contains("`t1`.`k1` = `t2`.`k2` OR `t1`.`k1` = `t2`.`k3`")); + Assert.assertTrue(stmt4.toSql().contains("(`t1`.`k1` = `t2`.`k2`) OR (`t1`.`k1` = `t2`.`k3`)")); String sql5 = "select\n" + " avg(t1.k4)\n" @@ -435,7 +435,7 @@ public void testDeduplicateOrs() throws Exception { SelectStmt stmt9 = (SelectStmt) UtFrameUtils.parseAndAnalyzeStmt(sql9, ctx); stmt9.rewriteExprs(new Analyzer(ctx.getEnv(), ctx).getExprRewriter()); Assert.assertTrue( - stmt9.toSql().contains("`k1` = 'shutdown' AND `k4` < 1 OR `k1` = 'switchOff' AND `k4` >= 1")); + stmt9.toSql().contains("(`k1` = 'shutdown') AND (`k4` < 1) OR (`k1` = 'switchOff') AND (`k4` >= 1)")); } @Test @@ -528,21 +528,21 @@ public void testDeleteSign() throws Exception { String sql1 = "SELECT /*+ SET_VAR(enable_nereids_planner=true, ENABLE_FALLBACK_TO_ORIGINAL_PLANNER=false) */ * FROM db1.table1 LEFT ANTI JOIN db1.table2 ON db1.table1.siteid = db1.table2.siteid;"; String explain = dorisAssert.query(sql1).explainQuery(); Assert.assertTrue(explain - .contains("PREDICATES: __DORIS_DELETE_SIGN__ = 0")); + .contains("__DORIS_DELETE_SIGN__ = 0")); Assert.assertFalse(explain.contains("other predicates:")); String sql2 = "SELECT /*+ SET_VAR(enable_nereids_planner=false) */ * FROM db1.table1 JOIN db1.table2 ON db1.table1.siteid = db1.table2.siteid;"; explain = dorisAssert.query(sql2).explainQuery(); Assert.assertTrue(explain - .contains("PREDICATES: `db1`.`table1`.`__DORIS_DELETE_SIGN__` = 0")); + .contains("`db1`.`table1`.`__DORIS_DELETE_SIGN__` = 0")); Assert.assertTrue(explain - .contains("PREDICATES: `db1`.`table2`.`__DORIS_DELETE_SIGN__` = 0")); + .contains("`db1`.`table2`.`__DORIS_DELETE_SIGN__` = 0")); Assert.assertFalse(explain.contains("other predicates:")); String sql3 = "SELECT /*+ SET_VAR(enable_nereids_planner=false) */ * FROM db1.table1"; Assert.assertTrue(dorisAssert.query(sql3).explainQuery() - .contains("PREDICATES: `db1`.`table1`.`__DORIS_DELETE_SIGN__` = 0")); + .contains("`db1`.`table1`.`__DORIS_DELETE_SIGN__` = 0")); String sql4 = " SELECT /*+ SET_VAR(enable_nereids_planner=false) */ * FROM db1.table1 table2"; Assert.assertTrue(dorisAssert.query(sql4).explainQuery() - .contains("PREDICATES: `table2`.`__DORIS_DELETE_SIGN__` = 0")); + .contains("`table2`.`__DORIS_DELETE_SIGN__` = 0")); new MockUp() { @Mock public boolean showHiddenColumns() { diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowAlterStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowAlterStmtTest.java index 8d0050b3a729d7..4bcefd6b3f1aaf 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowAlterStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowAlterStmtTest.java @@ -78,7 +78,7 @@ public void testAlterStmt2() throws UserException, AnalysisException { ShowAlterStmt stmt = new ShowAlterStmt(ShowAlterStmt.AlterType.COLUMN, null, binaryPredicate, null, new LimitElement(1, 2)); stmt.analyzeSyntax(analyzer); - Assert.assertEquals("SHOW ALTER TABLE COLUMN FROM `testDb` WHERE `TableName` = \'abc\' LIMIT 1, 2", + Assert.assertEquals("SHOW ALTER TABLE COLUMN FROM `testDb` WHERE (`TableName` = \'abc\') LIMIT 1, 2", stmt.toString()); } @@ -89,7 +89,7 @@ public void testAlterStmt3() throws UserException, AnalysisException { BinaryPredicate binaryPredicate = new BinaryPredicate(Operator.EQ, slotRef, stringLiteral); ShowAlterStmt stmt = new ShowAlterStmt(ShowAlterStmt.AlterType.COLUMN, null, binaryPredicate, null, null); stmt.analyzeSyntax(analyzer); - Assert.assertEquals("SHOW ALTER TABLE COLUMN FROM `testDb` WHERE `CreateTime` = \'2019-12-04 00:00:00\'", + Assert.assertEquals("SHOW ALTER TABLE COLUMN FROM `testDb` WHERE (`CreateTime` = \'2019-12-04 00:00:00\')", stmt.toString()); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowBackupStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowBackupStmtTest.java index 6d340e5d6f1bf0..d3bf61a307e140 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowBackupStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowBackupStmtTest.java @@ -78,7 +78,7 @@ public boolean checkDbPriv(ConnectContext ctx, String qualifiedDb, PrivPredicate new BinaryPredicate(BinaryPredicate.Operator.EQ, new SlotRef(new TableName("a.b.c"), "snapshotname"), new StringLiteral("FINISHED"))); stmt.analyze(analyzer); - Assertions.assertEquals(stmt.toSql(), "SHOW BACKUP WHERE `a`.`b`.`c`.`snapshotname` = 'FINISHED'"); + Assertions.assertEquals(stmt.toSql(), "SHOW BACKUP WHERE (`a`.`b`.`c`.`snapshotname` = 'FINISHED')"); stmt = new ShowBackupStmt("", new LikePredicate(Operator.LIKE, new SlotRef(new TableName("a.b.c"), "snapshotname"), diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowExportStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowExportStmtTest.java index a3a9fc72207428..ed8c33fb9e429d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowExportStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowExportStmtTest.java @@ -53,7 +53,7 @@ public void testWhere() throws UserException { stmt = new ShowExportStmt(null, binaryPredicate, null, new LimitElement(10)); stmt.analyze(analyzer); - Assert.assertEquals("SHOW EXPORT FROM `testDb` WHERE `label` = 'abc' LIMIT 10", stmt.toString()); + Assert.assertEquals("SHOW EXPORT FROM `testDb` WHERE (`label` = 'abc') LIMIT 10", stmt.toString()); Assert.assertFalse(stmt.isLabelUseLike()); StringLiteral stringLiteralLike = new StringLiteral("ab%"); @@ -67,7 +67,7 @@ public void testWhere() throws UserException { BinaryPredicate statePredicate = new BinaryPredicate(Operator.EQ, new SlotRef(null, "state"), new StringLiteral("PENDING")); stmt = new ShowExportStmt(null, statePredicate, null, new LimitElement(10)); stmt.analyze(analyzer); - Assert.assertEquals("SHOW EXPORT FROM `testDb` WHERE `state` = 'PENDING' LIMIT 10", stmt.toString()); + Assert.assertEquals("SHOW EXPORT FROM `testDb` WHERE (`state` = 'PENDING') LIMIT 10", stmt.toString()); } @Test diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowLoadStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowLoadStmtTest.java index c943dcf5736ab6..df90d1df71207c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowLoadStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowLoadStmtTest.java @@ -99,7 +99,7 @@ public void testWhere() throws UserException, AnalysisException { BinaryPredicate binaryPredicate = new BinaryPredicate(Operator.EQ, slotRef, stringLiteral); stmt = new ShowLoadStmt(null, binaryPredicate, null, new LimitElement(10)); stmt.analyze(analyzer); - Assert.assertEquals("SHOW LOAD FROM `testDb` WHERE `label` = \'abc\' LIMIT 10", stmt.toString()); + Assert.assertEquals("SHOW LOAD FROM `testDb` WHERE (`label` = \'abc\') LIMIT 10", stmt.toString()); StringLiteral stringLiteralLike = new StringLiteral("ab%"); LikePredicate likePredicate = new LikePredicate(LikePredicate.Operator.LIKE, slotRef, stringLiteralLike); @@ -111,7 +111,7 @@ public void testWhere() throws UserException, AnalysisException { BinaryPredicate statePredicate = new BinaryPredicate(Operator.EQ, new SlotRef(null, "state"), new StringLiteral("PENDING")); stmt = new ShowLoadStmt(null, statePredicate, null, new LimitElement(10)); stmt.analyze(analyzer); - Assert.assertEquals("SHOW LOAD FROM `testDb` WHERE `state` = \'PENDING\' LIMIT 10", stmt.toString()); + Assert.assertEquals("SHOW LOAD FROM `testDb` WHERE (`state` = \'PENDING\') LIMIT 10", stmt.toString()); } @Test diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowPartitionsStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowPartitionsStmtTest.java index 8dd0221e00a113..0a25b2dde3547d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowPartitionsStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowPartitionsStmtTest.java @@ -100,7 +100,7 @@ public void testShowPartitionsStmtWithBinaryPredicate() throws UserException { BinaryPredicate binaryPredicate = new BinaryPredicate(BinaryPredicate.Operator.GT, slotRef, stringLiteral); ShowPartitionsStmt stmt = new ShowPartitionsStmt(new TableName(internalCtl, "testDb", "testTable"), binaryPredicate, null, null, false); stmt.analyzeImpl(analyzer); - Assert.assertEquals("SHOW PARTITIONS FROM `testDb`.`testTable` WHERE `LastConsistencyCheckTime` > '2019-12-22 10:22:11'", stmt.toString()); + Assert.assertEquals("SHOW PARTITIONS FROM `testDb`.`testTable` WHERE (`LastConsistencyCheckTime` > '2019-12-22 10:22:11')", stmt.toString()); } @Test @@ -259,7 +259,7 @@ List data() { ShowPartitionsStmt stmt = new ShowPartitionsStmt(new TableName("hms", "hms", "testTable"), binaryPredicate, null, null, false); Assertions.assertEquals(stmt.toSql(), "SHOW PARTITIONS FROM `hms`.`testTable` " - + "WHERE `PartitionName` = 'part=part1'"); + + "WHERE (`PartitionName` = 'part=part1')"); } } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowResourcesStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowResourcesStmtTest.java index 16315614066c44..50cf5505ef798b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowResourcesStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowResourcesStmtTest.java @@ -67,7 +67,7 @@ public void testWhere() throws UserException, AnalysisException { BinaryPredicate binaryPredicate = new BinaryPredicate(Operator.EQ, slotRef, stringLiteral); stmt = new ShowResourcesStmt(binaryPredicate, null, new LimitElement(10)); stmt.analyze(analyzer); - Assert.assertEquals("SHOW RESOURCES WHERE `name` = \'abc\' LIMIT 10", stmt.toString()); + Assert.assertEquals("SHOW RESOURCES WHERE (`name` = \'abc\') LIMIT 10", stmt.toString()); LikePredicate likePredicate = new LikePredicate(org.apache.doris.analysis.LikePredicate.Operator.LIKE, slotRef, stringLiteral); diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/SqlModeTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/SqlModeTest.java index 9e0b8c2a460078..d70dfc826f0346 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/SqlModeTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/SqlModeTest.java @@ -41,7 +41,7 @@ public void testScannerConstructor() { } catch (Exception e) { Assert.fail(e.getMessage()); } - Assert.assertEquals("SELECT * FROM `db1`.`tbl1` WHERE `name` = 'BILL GATES'", selectStmt.toSql()); + Assert.assertEquals("SELECT * FROM `db1`.`tbl1` WHERE (`name` = 'BILL GATES')", selectStmt.toSql()); parser = new SqlParser(new SqlScanner(new StringReader(stmt), SqlModeHelper.MODE_DEFAULT)); try { @@ -49,7 +49,7 @@ public void testScannerConstructor() { } catch (Exception e) { Assert.fail(e.getMessage()); } - Assert.assertEquals("SELECT * FROM `db1`.`tbl1` WHERE `name` = 'BILL GATES'", selectStmt.toSql()); + Assert.assertEquals("SELECT * FROM `db1`.`tbl1` WHERE (`name` = 'BILL GATES')", selectStmt.toSql()); } @Test diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateViewTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateViewTest.java index b7945e0b6dedff..14acbcb0d23466 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateViewTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateViewTest.java @@ -185,7 +185,7 @@ public void testAlterView() throws Exception { alter1 = (View) db.getTableOrDdlException("alter1"); Assert.assertEquals( "WITH test1_cte(w1, w2) AS (SELECT `k1`, `k2` FROM `test`.`tbl1`) " - + "SELECT `w1` AS `c1`, sum(`w2`) AS `c2` FROM `test1_cte` WHERE `w1` > 10 GROUP BY `w1` " + + "SELECT `w1` AS `c1`, sum(`w2`) AS `c2` FROM `test1_cte` WHERE (`w1` > 10) GROUP BY `w1` " + "ORDER BY `w1` ASC NULLS FIRST", alter1.getInlineViewDef()); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/PlannerTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/PlannerTest.java index 493053b3b1c268..41f21110330478 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/planner/PlannerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/PlannerTest.java @@ -437,7 +437,7 @@ public void testPushDownPredicateOnGroupingSetAggregate() throws Exception { stmtExecutor.execute(); Planner planner = stmtExecutor.planner(); String plan = planner.getExplainString(new ExplainOptions(false, false, false)); - Assertions.assertTrue(plan.contains("PREDICATES: `k1` = 1\n")); + Assertions.assertTrue(plan.contains("`k1` = 1")); } @Test @@ -459,7 +459,7 @@ public void testPushDownPredicateOnNormalAggregate() throws Exception { stmtExecutor.execute(); Planner planner = stmtExecutor.planner(); String plan = planner.getExplainString(new ExplainOptions(false, false, false)); - Assertions.assertTrue(plan.contains("PREDICATES: `k1` = 1 AND `k2` = 1\n")); + Assertions.assertTrue(plan.contains("(`k1` = 1) AND (`k2` = 1)")); } @Test @@ -471,7 +471,7 @@ public void testPushDownPredicateOnWindowFunction() throws Exception { stmtExecutor.execute(); Planner planner = stmtExecutor.planner(); String plan = planner.getExplainString(new ExplainOptions(false, false, false)); - Assertions.assertTrue(plan.contains("PREDICATES: `k1` = 1\n")); + Assertions.assertTrue(plan.contains("`k1` = 1")); } @Test diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java index 3f7aaa70991565..c36622268ba86c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java @@ -493,7 +493,7 @@ public void testBitmapQuery() throws Exception { assertSQLPlanOrErrorMsgContains( "select count(*) from test.bitmap_table where id2 = 1;", - "Unsupported bitmap type in expression: `id2` = 1" + "Unsupported bitmap type in expression: (`id2` = 1)" ); } @@ -547,7 +547,7 @@ public void testHLLTypeQuery() throws Exception { assertSQLPlanOrErrorMsgContains( "select count(*) from test.hll_table where id2 = 1", - "Hll type dose not support operand: `id2` = 1" + "Hll type dose not support operand: (`id2` = 1)" ); } @@ -732,8 +732,8 @@ public void testJoinPredicateTransitivity() throws Exception { + "left join join2 on join1.id = join2.id\n" + "and join1.id > 1;"; String explainString = getSQLPlanOrErrorMsg("explain " + sql); - Assert.assertTrue(explainString.contains("other join predicates: > 1")); - Assert.assertFalse(explainString.contains("PREDICATES: `join1`.`id` > 1")); + Assert.assertTrue(explainString.contains(" > 1")); + Assert.assertFalse(explainString.contains("`join1`.`id` > 1")); /* // test left join: right table where predicate. @@ -752,8 +752,8 @@ public void testJoinPredicateTransitivity() throws Exception { + "left join join2 on join1.id = join2.id\n" + "and join2.id > 1;"; explainString = getSQLPlanOrErrorMsg("explain " + sql); - Assert.assertTrue(explainString.contains("PREDICATES: `join2`.`id` > 1")); - Assert.assertFalse(explainString.contains("PREDICATES: `join1`.`id` > 1")); + Assert.assertTrue(explainString.contains("`join2`.`id` > 1")); + Assert.assertFalse(explainString.contains("`join1`.`id` > 1")); /* // test inner join: left table where predicate, both push down left table and right table @@ -770,8 +770,8 @@ public void testJoinPredicateTransitivity() throws Exception { + "join join2 on join1.id = join2.id\n" + "and join1.id > 1;"; explainString = getSQLPlanOrErrorMsg("explain " + sql); - Assert.assertTrue(explainString.contains("PREDICATES: `join1`.`id` > 1")); - Assert.assertTrue(explainString.contains("PREDICATES: `join2`.`id` > 1")); + Assert.assertTrue(explainString.contains("`join1`.`id` > 1")); + Assert.assertTrue(explainString.contains("`join2`.`id` > 1")); /* // test inner join: right table where predicate, both push down left table and right table @@ -788,8 +788,8 @@ public void testJoinPredicateTransitivity() throws Exception { + "join join2 on join1.id = join2.id\n" + "and 1 < join2.id;"; explainString = getSQLPlanOrErrorMsg("explain " + sql); - Assert.assertTrue(explainString.contains("PREDICATES: `join1`.`id` > 1")); - Assert.assertTrue(explainString.contains("PREDICATES: `join2`.`id` > 1")); + Assert.assertTrue(explainString.contains("`join1`.`id` > 1")); + Assert.assertTrue(explainString.contains("`join2`.`id` > 1")); sql = "select /*+ SET_VAR(enable_nereids_planner=false) */ *\n from join1\n" + "join join2 on join1.id = join2.value\n" @@ -803,31 +803,31 @@ public void testJoinPredicateTransitivity() throws Exception { + "left anti join join2 on join1.id = join2.id\n" + "and join2.id > 1;"; explainString = getSQLPlanOrErrorMsg("explain " + sql); - Assert.assertTrue(explainString.contains("PREDICATES: `join2`.`id` > 1")); - Assert.assertFalse(explainString.contains("PREDICATES: `join1`.`id` > 1")); + Assert.assertTrue(explainString.contains("`join2`.`id` > 1")); + Assert.assertFalse(explainString.contains("`join1`.`id` > 1")); // test semi join, right table join predicate, only push to right table sql = "select /*+ SET_VAR(enable_nereids_planner=false) */ *\n from join1\n" + "left semi join join2 on join1.id = join2.id\n" + "and join2.id > 1;"; explainString = getSQLPlanOrErrorMsg("explain " + sql); - Assert.assertTrue(explainString.contains("PREDICATES: `join2`.`id` > 1")); - Assert.assertTrue(explainString.contains("PREDICATES: `join1`.`id` > 1")); + Assert.assertTrue(explainString.contains("`join2`.`id` > 1")); + Assert.assertTrue(explainString.contains("`join1`.`id` > 1")); // test anti join, left table join predicate, left table couldn't push down sql = "select /*+ SET_VAR(enable_nereids_planner=false) */ *\n from join1\n" + "left anti join join2 on join1.id = join2.id\n" + "and join1.id > 1;"; explainString = getSQLPlanOrErrorMsg("explain " + sql); - Assert.assertTrue(explainString.contains("other join predicates: > 1")); - Assert.assertFalse(explainString.contains("PREDICATES: `join1`.`id` > 1")); + Assert.assertTrue(explainString.contains(" > 1")); + Assert.assertFalse(explainString.contains("`join1`.`id` > 1")); // test semi join, left table join predicate, only push to left table sql = "select /*+ SET_VAR(enable_nereids_planner=false) */ *\n from join1\n" + "left semi join join2 on join1.id = join2.id\n" + "and join1.id > 1;"; explainString = getSQLPlanOrErrorMsg("explain " + sql); - Assert.assertTrue(explainString.contains("PREDICATES: `join1`.`id` > 1")); + Assert.assertTrue(explainString.contains("`join1`.`id` > 1")); /* // test anti join, left table where predicate, only push to left table @@ -1036,15 +1036,15 @@ public void testConstInPartitionPrune() throws Exception { public void testOrCompoundPredicateFold() throws Exception { String queryStr = "explain select /*+ SET_VAR(enable_nereids_planner=false) */ * from baseall where (k1 > 1) or (k1 > 1 and k2 < 1)"; String explainString = getSQLPlanOrErrorMsg(queryStr); - Assert.assertTrue(explainString.contains("PREDICATES: (`k1` > 1)\n")); + Assert.assertTrue(explainString.contains("`k1` > 1")); queryStr = "explain select /*+ SET_VAR(enable_nereids_planner=false) */ * from baseall where (k1 > 1 and k2 < 1) or (k1 > 1)"; explainString = getSQLPlanOrErrorMsg(queryStr); - Assert.assertTrue(explainString.contains("PREDICATES: `k1` > 1\n")); + Assert.assertTrue(explainString.contains("`k1` > 1")); queryStr = "explain select /*+ SET_VAR(enable_nereids_planner=false) */ * from baseall where (k1 > 1) or (k1 > 1)"; explainString = getSQLPlanOrErrorMsg(queryStr); - Assert.assertTrue(explainString.contains("PREDICATES: (`k1` > 1)\n")); + Assert.assertTrue(explainString.contains("`k1` > 1")); } @Test @@ -1677,7 +1677,7 @@ public void testFromUnixTimeRewrite() throws Exception { //default format String sql = "select /*+ SET_VAR(enable_nereids_planner=false) */ * from test1 where from_unixtime(query_time) > '2021-03-02 10:01:28'"; String explainString = getSQLPlanOrErrorMsg("EXPLAIN " + sql); - Assert.assertTrue(explainString.contains("PREDICATES: `query_time` <= 253402271999 AND `query_time` > 1614650488")); + Assert.assertTrue(explainString.contains("(`query_time` <= 253402271999) AND (`query_time` > 1614650488)")); } @Disabled @@ -1811,7 +1811,7 @@ public void testCompoundPredicateWriteRule() throws Exception { // false or e ==> e String sql1 = "select /*+ SET_VAR(enable_nereids_planner=false) */ * from test.test1 where 2=-2 OR query_time=0;"; String explainString1 = getSQLPlanOrErrorMsg("EXPLAIN " + sql1); - Assert.assertTrue(explainString1.contains("PREDICATES: `query_time` = 0")); + Assert.assertTrue(explainString1.contains("`query_time` = 0")); //true or e ==> true String sql2 = "select /*+ SET_VAR(enable_nereids_planner=false) */ * from test.test1 where -5=-5 OR query_time=0;"; @@ -1826,18 +1826,18 @@ public void testCompoundPredicateWriteRule() throws Exception { //e or false ==> e String sql4 = "select /*+ SET_VAR(enable_nereids_planner=false) */ * from test.test1 where -5!=-5 OR query_time=0;"; String explainString4 = getSQLPlanOrErrorMsg("EXPLAIN " + sql4); - Assert.assertTrue(explainString4.contains("PREDICATES: `query_time` = 0")); + Assert.assertTrue(explainString4.contains("`query_time` = 0")); // true and e ==> e String sql5 = "select /*+ SET_VAR(enable_nereids_planner=false) */ * from test.test1 where -5=-5 AND query_time=0;"; String explainString5 = getSQLPlanOrErrorMsg("EXPLAIN " + sql5); - Assert.assertTrue(explainString5.contains("PREDICATES: `query_time` = 0")); + Assert.assertTrue(explainString5.contains("`query_time` = 0")); // e and true ==> e String sql6 = "select /*+ SET_VAR(enable_nereids_planner=false) */ * from test.test1 where query_time=0 AND -5=-5;"; String explainString6 = getSQLPlanOrErrorMsg("EXPLAIN " + sql6); - Assert.assertTrue(explainString6.contains("PREDICATES: `query_time` = 0")); + Assert.assertTrue(explainString6.contains("`query_time` = 0")); // false and e ==> false String sql7 = "select /*+ SET_VAR(enable_nereids_planner=false) */ * from test.test1 where -5!=-5 AND query_time=0;"; @@ -1852,12 +1852,12 @@ public void testCompoundPredicateWriteRule() throws Exception { // (false or expr1) and (false or expr2) ==> expr1 and expr2 String sql9 = "select /*+ SET_VAR(enable_nereids_planner=false) */ * from test.test1 where (-2=2 or query_time=2) and (-2=2 or stmt_id=2);"; String explainString9 = getSQLPlanOrErrorMsg("EXPLAIN " + sql9); - Assert.assertTrue(explainString9.contains("PREDICATES: `query_time` = 2 AND `stmt_id` = 2")); + Assert.assertTrue(explainString9.contains("(`query_time` = 2) AND (`stmt_id` = 2)")); // false or (expr and true) ==> expr String sql10 = "select /*+ SET_VAR(enable_nereids_planner=false) */ * from test.test1 where (2=-2) OR (query_time=0 AND 1=1);"; String explainString10 = getSQLPlanOrErrorMsg("EXPLAIN " + sql10); - Assert.assertTrue(explainString10.contains("PREDICATES: `query_time` = 0")); + Assert.assertTrue(explainString10.contains("`query_time` = 0")); } @Test @@ -1885,11 +1885,11 @@ public void testOutfile() throws Exception { + " \"max_file_size\" = \"500MB\" );"; String explainStr = getSQLPlanOrErrorMsg("EXPLAIN " + sql); if (Config.enable_date_conversion) { - Assert.assertTrue(explainStr.contains("PREDICATES: `date` >= '2021-10-07' AND" - + " `date` <= '2021-10-11'")); + Assert.assertTrue(explainStr.contains("(`date` >= '2021-10-07') AND" + + " (`date` <= '2021-10-11')")); } else { - Assert.assertTrue(explainStr.contains("PREDICATES: `date` >= '2021-10-07 00:00:00' AND" - + " `date` <= '2021-10-11 00:00:00'")); + Assert.assertTrue(explainStr.contains("(`date` >= '2021-10-07 00:00:00') AND" + + " (`date` <= '2021-10-11 00:00:00')")); } } @@ -2237,7 +2237,7 @@ public void testRewriteOrToIn() throws Exception { sql = "SELECT /*+ SET_VAR(enable_nereids_planner=false) */ * from test1 where (query_time = 1 or query_time = 2 or scan_bytes = 2) and scan_bytes in (2, 3)"; explainString = UtFrameUtils.getSQLPlanOrErrorMsg(connectContext, "EXPLAIN " + sql); - Assert.assertTrue(explainString.contains("PREDICATES: `query_time` IN (1, 2) OR `scan_bytes` = 2 AND `scan_bytes` IN (2, 3)\n")); + Assert.assertTrue(explainString.contains("PREDICATES: `query_time` IN (1, 2) OR (`scan_bytes` = 2) AND `scan_bytes` IN (2, 3)\n")); sql = "SELECT /*+ SET_VAR(enable_nereids_planner=false) */ * from test1 where (query_time = 1 or query_time = 2) and (scan_bytes = 2 or scan_bytes = 3)"; explainString = UtFrameUtils.getSQLPlanOrErrorMsg(connectContext, "EXPLAIN " + sql); @@ -2254,7 +2254,7 @@ public void testRewriteOrToIn() throws Exception { connectContext.getSessionVariable().setRewriteOrToInPredicateThreshold(100); sql = "SELECT /*+ SET_VAR(enable_nereids_planner=false) */ * from test1 where query_time = 1 or query_time = 2 or query_time in (3, 4)"; explainString = UtFrameUtils.getSQLPlanOrErrorMsg(connectContext, "EXPLAIN " + sql); - Assert.assertTrue(explainString.contains("PREDICATES: `query_time` = 1 OR `query_time` = 2 OR `query_time` IN (3, 4)\n")); + Assert.assertTrue(explainString.contains("PREDICATES: (`query_time` = 1) OR (`query_time` = 2) OR `query_time` IN (3, 4)\n")); connectContext.getSessionVariable().setRewriteOrToInPredicateThreshold(2); sql = "SELECT /*+ SET_VAR(enable_nereids_planner=false) */ * from test1 where (query_time = 1 or query_time = 2) and query_time in (3, 4)"; @@ -2264,7 +2264,7 @@ public void testRewriteOrToIn() throws Exception { //test we can handle `!=` and `not in` sql = "select /*+ SET_VAR(enable_nereids_planner=false) */ * from test1 where (query_time = 1 or query_time = 2 or query_time!= 3 or query_time not in (5, 6))"; explainString = UtFrameUtils.getSQLPlanOrErrorMsg(connectContext, "EXPLAIN " + sql); - Assert.assertTrue(explainString.contains("PREDICATES: `query_time` IN (1, 2) OR `query_time` != 3 OR `query_time` NOT IN (5, 6)\n")); + Assert.assertTrue(explainString.contains("PREDICATES: `query_time` IN (1, 2) OR (`query_time` != 3) OR `query_time` NOT IN (5, 6)\n")); //test we can handle merge 2 or more columns sql = "select /*+ SET_VAR(enable_nereids_planner=false) */ * from test1 where (query_time = 1 or query_time = 2 or scan_rows = 3 or scan_rows = 4)"; @@ -2284,8 +2284,8 @@ public void testRewriteOrToIn() throws Exception { + " or (db not in ('x', 'y')) "; explainString = UtFrameUtils.getSQLPlanOrErrorMsg(connectContext, "EXPLAIN " + sql); Assert.assertTrue(explainString.contains( - "PREDICATES: `query_id` = `client_ip` " - + "AND (`stmt_id` IN (1, 2, 3) OR `user` = 'abc' AND `state` IN ('a', 'b', 'c', 'd')) " + "PREDICATES: (`query_id` = `client_ip`) " + + "AND (`stmt_id` IN (1, 2, 3) OR (`user` = 'abc') AND `state` IN ('a', 'b', 'c', 'd')) " + "OR (`db` NOT IN ('x', 'y'))\n")); //ExtractCommonFactorsRule may generate more expr, test the rewriteOrToIn applied on generated exprs @@ -2293,7 +2293,7 @@ public void testRewriteOrToIn() throws Exception { explainString = UtFrameUtils.getSQLPlanOrErrorMsg(connectContext, "EXPLAIN " + sql); Assert.assertTrue(explainString.contains( "PREDICATES: `state` IN ('a', 'b') AND `stmt_id` IN (1, 2) AND" - + " `stmt_id` = 1 AND `state` = 'a' OR `stmt_id` = 2 AND `state` = 'b'\n" + + " (`stmt_id` = 1) AND (`state` = 'a') OR (`stmt_id` = 2) AND (`state` = 'b')\n" )); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/RepeatNodeTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/RepeatNodeTest.java index a4a09d0870d612..9a9b1a3b8c5fec 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/planner/RepeatNodeTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/RepeatNodeTest.java @@ -62,7 +62,7 @@ public void testExpr() throws Exception { String explainString1 = getSQLPlanOrErrorMsg("explain " + sql1); System.out.println(explainString1); Assertions.assertTrue(explainString1.contains( - "output slots: `if(`c`.`id` > 0, 1, 0)`, ``p`.`name``, ``c`.`cost``, ``GROUPING_ID``")); + "output slots: `if((`c`.`id` > 0), 1, 0)`, ``p`.`name``, ``c`.`cost``, ``GROUPING_ID``")); String sql2 = "select /*+ SET_VAR(enable_nereids_planner=false) */ (id + 1) id_, name, sum(cost) from mycost group by grouping sets((id_, name),());"; String explainString2 = getSQLPlanOrErrorMsg("explain " + sql2); diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/TableFunctionPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/TableFunctionPlanTest.java index 2c39a09390b831..a9698080219df8 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/planner/TableFunctionPlanTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/TableFunctionPlanTest.java @@ -132,7 +132,7 @@ public void whereExplodeColumn() throws Exception { Assert.assertTrue(UtFrameUtils.checkPlanResultContainsNode(explainString, 1, "TABLE FUNCTION NODE")); Assert.assertTrue( explainString.contains("table function: explode_split(`db1`.`tbl1`.`k2`, ',')")); - Assert.assertTrue(explainString.contains("PREDICATES: `e1` = '1'")); + Assert.assertTrue(explainString.contains("`e1` = '1'")); Assert.assertTrue(explainString.contains("tuple ids: 0 1")); Assert.assertTrue(explainString.contains("TupleDescriptor{id=1, tbl=tmp, byteSize=32}")); Assert.assertTrue(explainString.contains("SlotDescriptor{id=1, col=e1, colUniqueId=-1, type=VARCHAR")); @@ -153,7 +153,7 @@ public void whereNormalColumn() throws Exception { Assert.assertTrue(explainString.contains("TupleDescriptor{id=1, tbl=tmp, byteSize=32}")); Assert.assertTrue(explainString.contains("SlotDescriptor{id=1, col=e1, colUniqueId=-1, type=VARCHAR")); Assert.assertTrue(UtFrameUtils.checkPlanResultContainsNode(explainString, 0, "OlapScanNode")); - Assert.assertTrue(explainString.contains("PREDICATES: `k1` = 1")); + Assert.assertTrue(explainString.contains("`k1` = 1")); } /* Case6 multi lateral view @@ -508,8 +508,8 @@ public void testLateralViewWithWhere() throws Exception { + " where k1 in (select k2 from db1.table_for_view);"; String explainString = UtFrameUtils.getSQLPlanOrErrorMsg(ctx, sql, true); Assert.assertTrue(explainString.contains("join op: LEFT SEMI JOIN(BROADCAST)")); - Assert.assertTrue(explainString.contains("equal join conjunct: `k1` = `k2`")); - Assert.assertTrue(!explainString.contains("equal join conjunct: `k2` = `k2`")); + Assert.assertTrue(explainString.contains("`k1` = `k2`")); + Assert.assertTrue(!explainString.contains("`k2` = `k2`")); } @Test diff --git a/fe/fe-core/src/test/java/org/apache/doris/policy/PolicyTest.java b/fe/fe-core/src/test/java/org/apache/doris/policy/PolicyTest.java index 6c1b24d531ed65..3169a57159f307 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/policy/PolicyTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/policy/PolicyTest.java @@ -269,7 +269,7 @@ public void testMergeFilter() throws Exception { createPolicy("CREATE ROW POLICY test_row_policy4 ON test.table1 AS PERMISSIVE TO test_policy USING (k2 = 1)"); String queryStr = "EXPLAIN select /*+ SET_VAR(enable_nereids_planner=false) */ * from test.table1"; String explainString = getSQLPlanOrErrorMsg(queryStr); - Assertions.assertTrue(explainString.contains("`k1` = 1 AND `k2` = 1 AND `k2` = 2 OR `k2` = 1")); + Assertions.assertTrue(explainString.contains("(`k1` = 1) AND (`k2` = 1) AND (`k2` = 2) OR (`k2` = 1)")); dropPolicy("DROP ROW POLICY test_row_policy1 ON test.table1"); dropPolicy("DROP ROW POLICY test_row_policy2 ON test.table1"); dropPolicy("DROP ROW POLICY test_row_policy3 ON test.table1"); @@ -283,7 +283,7 @@ public void testMergeFilterNereidsPlanner() throws Exception { createPolicy("CREATE ROW POLICY test_row_policy4 ON test.table1 AS PERMISSIVE TO test_policy USING (k2 = 1)"); String queryStr = "EXPLAIN select * from test.table1"; String explainString = getSQLPlanOrErrorMsg(queryStr); - Assertions.assertTrue(explainString.contains("k2[#1] IN (1, 2) AND k1[#0] = 1")); + Assertions.assertTrue(explainString.contains("k2[#1] IN (1, 2) AND (k1[#0] = 1)")); dropPolicy("DROP ROW POLICY test_row_policy1 ON test.table1"); dropPolicy("DROP ROW POLICY test_row_policy3 ON test.table1"); dropPolicy("DROP ROW POLICY test_row_policy4 ON test.table1"); @@ -295,16 +295,16 @@ public void testComplexSql() throws Exception { createPolicy("CREATE ROW POLICY test_row_policy2 ON test.table1 AS RESTRICTIVE TO test_policy USING (k2 = 1)"); String joinSql = "select /*+ SET_VAR(enable_nereids_planner=false) */ * from table1 join table2 on table1.k1=table2.k1"; - Assertions.assertTrue(getSQLPlanOrErrorMsg(joinSql).contains("PREDICATES: `k1` = 1 AND `k2` = 1")); + Assertions.assertTrue(getSQLPlanOrErrorMsg(joinSql).contains("(`k1` = 1) AND (`k2` = 1)")); String unionSql = "select /*+ SET_VAR(enable_nereids_planner=false) */ * from table1 union select * from table2"; - Assertions.assertTrue(getSQLPlanOrErrorMsg(unionSql).contains("PREDICATES: `k1` = 1 AND `k2` = 1")); + Assertions.assertTrue(getSQLPlanOrErrorMsg(unionSql).contains("(`k1` = 1) AND (`k2` = 1)")); String subQuerySql = "select /*+ SET_VAR(enable_nereids_planner=false) */ * from table2 where k1 in (select k1 from table1)"; - Assertions.assertTrue(getSQLPlanOrErrorMsg(subQuerySql).contains("PREDICATES: `k1` = 1 AND `k2` = 1")); + Assertions.assertTrue(getSQLPlanOrErrorMsg(subQuerySql).contains("(`k1` = 1) AND (`k2` = 1)")); String aliasSql = "select /*+ SET_VAR(enable_nereids_planner=false) */ * from table1 t1 join table2 t2 on t1.k1=t2.k1"; - Assertions.assertTrue(getSQLPlanOrErrorMsg(aliasSql).contains("PREDICATES: `t1`.`k1` = 1 AND `t1`.`k2` = 1")); + Assertions.assertTrue(getSQLPlanOrErrorMsg(aliasSql).contains("(`t1`.`k1` = 1) AND (`t1`.`k2` = 1)")); dropPolicy("DROP ROW POLICY test_row_policy1 ON test.table1"); dropPolicy("DROP ROW POLICY test_row_policy2 ON test.table1"); } @@ -314,13 +314,13 @@ public void testComplexSqlNereidsPlanner() throws Exception { createPolicy("CREATE ROW POLICY test_row_policy1 ON test.table1 AS RESTRICTIVE TO test_policy USING (k1 = 1)"); createPolicy("CREATE ROW POLICY test_row_policy2 ON test.table1 AS RESTRICTIVE TO test_policy USING (k2 = 1)"); String joinSql = "select * from table1 join table2 on table1.k1=table2.k1"; - Assertions.assertTrue(getSQLPlanOrErrorMsg(joinSql).contains("PREDICATES: k1 = 1 AND k2 = 1")); + Assertions.assertTrue(getSQLPlanOrErrorMsg(joinSql).contains("PREDICATES: (k1 = 1) AND (k2 = 1)")); String unionSql = "select * from table1 union select * from table2"; - Assertions.assertTrue(getSQLPlanOrErrorMsg(unionSql).contains("PREDICATES: k1 = 1 AND k2 = 1")); + Assertions.assertTrue(getSQLPlanOrErrorMsg(unionSql).contains("PREDICATES: (k1 = 1) AND (k2 = 1)")); String subQuerySql = "select * from table2 where k1 in (select k1 from table1)"; - Assertions.assertTrue(getSQLPlanOrErrorMsg(subQuerySql).contains("PREDICATES: k1 = 1 AND k2 = 1")); + Assertions.assertTrue(getSQLPlanOrErrorMsg(subQuerySql).contains("PREDICATES: (k1 = 1) AND (k2 = 1)")); String aliasSql = "select * from table1 t1 join table2 t2 on t1.k1=t2.k1"; - Assertions.assertTrue(getSQLPlanOrErrorMsg(aliasSql).contains("PREDICATES: k1 = 1 AND k2 = 1")); + Assertions.assertTrue(getSQLPlanOrErrorMsg(aliasSql).contains("PREDICATES: (k1 = 1) AND (k2 = 1)")); dropPolicy("DROP ROW POLICY test_row_policy1 ON test.table1"); dropPolicy("DROP ROW POLICY test_row_policy2 ON test.table1"); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/OlapQueryCacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/OlapQueryCacheTest.java index 2b0d7162fed5a7..182596c3b63d1c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/qe/OlapQueryCacheTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/qe/OlapQueryCacheTest.java @@ -671,7 +671,7 @@ public void testPartitionIntTypeSql() throws Exception { cache.rewriteSelectStmt(newRangeList); sql = ca.getRewriteStmt().getWhereClause().toSql(); - Assert.assertEquals(sql, "`date` >= 20200114 AND `date` <= 20200115"); + Assert.assertEquals(sql, "(`date` >= 20200114) AND (`date` <= 20200115)"); } catch (Exception e) { LOG.warn("ex={}", e); Assert.fail(e.getMessage()); @@ -713,7 +713,7 @@ public void testSimpleCacheSql() throws Exception { hitRange = range.buildDiskPartitionRange(newRangeList); cache.rewriteSelectStmt(newRangeList); sql = ca.getRewriteStmt().getWhereClause().toSql(); - Assert.assertEquals(sql, "`eventdate` >= '2020-01-14' AND `eventdate` <= '2020-01-15'"); + Assert.assertEquals(sql, "(`eventdate` >= '2020-01-14') AND (`eventdate` <= '2020-01-15')"); } catch (Exception e) { LOG.warn("ex={}", e); Assert.fail(e.getMessage()); @@ -856,7 +856,7 @@ public void testUpdatePartition() throws Exception { cache.rewriteSelectStmt(newRangeList); sql = ca.getRewriteStmt().getWhereClause().toSql(); - Assert.assertEquals(sql, "`eventdate` >= '2020-01-13' AND `eventdate` <= '2020-01-15'"); + Assert.assertEquals(sql, "(`eventdate` >= '2020-01-13') AND (`eventdate` <= '2020-01-15')"); List updateRangeList = range.buildUpdatePartitionRange(); Assert.assertEquals(updateRangeList.size(), 2); @@ -886,7 +886,7 @@ public void testRewriteMultiPredicate1() throws Exception { cache.rewriteSelectStmt(null); LOG.warn("Nokey multi={}", cache.getNokeyStmt().getWhereClause().toSql()); - Assert.assertEquals(cache.getNokeyStmt().getWhereClause().toSql(), "`eventid` = 1"); + Assert.assertEquals(cache.getNokeyStmt().getWhereClause().toSql(), "(`eventid` = 1)"); PartitionRange range = cache.getPartitionRange(); boolean flag = range.analytics(); @@ -904,7 +904,7 @@ public void testRewriteMultiPredicate1() throws Exception { cache.rewriteSelectStmt(newRangeList); sql = ca.getRewriteStmt().getWhereClause().toSql(); LOG.warn("MultiPredicate={}", sql); - Assert.assertEquals(sql, "`eventdate` > '2020-01-13' AND `eventdate` < '2020-01-16' AND `eventid` = 1"); + Assert.assertEquals(sql, "(`eventdate` > '2020-01-13') AND (`eventdate` < '2020-01-16') AND (`eventid` = 1)"); } catch (Exception e) { LOG.warn("multi ex={}", e); Assert.fail(e.getMessage()); @@ -929,7 +929,7 @@ public void testRewriteJoin() throws Exception { PartitionCache cache = (PartitionCache) ca.getCache(); cache.rewriteSelectStmt(null); LOG.warn("Join nokey={}", cache.getNokeyStmt().getWhereClause().toSql()); - Assert.assertEquals(cache.getNokeyStmt().getWhereClause().toSql(), "`eventid` = 1"); + Assert.assertEquals(cache.getNokeyStmt().getWhereClause().toSql(), "(`eventid` = 1)"); PartitionRange range = cache.getPartitionRange(); boolean flag = range.analytics(); @@ -947,8 +947,8 @@ public void testRewriteJoin() throws Exception { cache.rewriteSelectStmt(newRangeList); sql = ca.getRewriteStmt().getWhereClause().toSql(); LOG.warn("Join rewrite={}", sql); - Assert.assertEquals(sql, "`appevent`.`eventdate` >= '2020-01-14'" - + " AND `appevent`.`eventdate` <= '2020-01-15' AND `eventid` = 1"); + Assert.assertEquals(sql, "(`appevent`.`eventdate` >= '2020-01-14')" + + " AND (`appevent`.`eventdate` <= '2020-01-15') AND (`eventid` = 1)"); } catch (Exception e) { LOG.warn("Join ex={}", e); Assert.fail(e.getMessage()); @@ -976,7 +976,7 @@ public void testSubSelect() throws Exception { Assert.assertEquals(cache.getNokeyStmt().toSql(), "SELECT `eventdate` AS `eventdate`, sum(`pv`) AS `sum(``pv``)` " + "FROM (SELECT `eventdate` AS `eventdate`, count(`userid`) AS `pv` " - + "FROM `testDb`.`appevent` WHERE `eventid` = 1 GROUP BY `eventdate`) tbl " + + "FROM `testDb`.`appevent` WHERE (`eventid` = 1) GROUP BY `eventdate`) tbl " + "GROUP BY `eventdate`"); PartitionRange range = cache.getPartitionRange(); @@ -998,8 +998,8 @@ public void testSubSelect() throws Exception { Assert.assertEquals(sql, "SELECT `eventdate` AS `eventdate`, sum(`pv`) AS `sum(``pv``)` " + "FROM (SELECT `eventdate` AS `eventdate`, count(`userid`) AS `pv` " - + "FROM `testDb`.`appevent` WHERE `eventdate` > '2020-01-13' " - + "AND `eventdate` < '2020-01-16' AND `eventid` = 1 GROUP BY `eventdate`) tbl " + + "FROM `testDb`.`appevent` WHERE (`eventdate` > '2020-01-13') " + + "AND (`eventdate` < '2020-01-16') AND (`eventid` = 1) GROUP BY `eventdate`) tbl " + "GROUP BY `eventdate`"); } catch (Exception e) { LOG.warn("sub ex={}", e); @@ -1051,8 +1051,8 @@ public void testSqlCacheKey() { SqlCache sqlCache = (SqlCache) ca.getCache(); String cacheKey = sqlCache.getSqlWithViewStmt(); Assert.assertEquals(cacheKey, "SELECT `eventdate` AS `eventdate`, count(`userid`) " - + "AS `count(``userid``)` FROM `testDb`.`appevent` WHERE `eventdate` >= '2020-01-12' " - + "AND `eventdate` <= '2020-01-14' GROUP BY `eventdate`|"); + + "AS `count(``userid``)` FROM `testDb`.`appevent` WHERE (`eventdate` >= '2020-01-12') " + + "AND (`eventdate` <= '2020-01-14') GROUP BY `eventdate`|"); Assert.assertEquals(selectedPartitionIds.size(), sqlCache.getSumOfPartitionNum()); } @@ -1090,8 +1090,8 @@ public void testSqlCacheKeyWithView() { Assert.assertEquals(cacheKey, "SELECT `testDb`.`view1`.`eventdate` AS `eventdate`, " + "`testDb`.`view1`.`__count_1` AS `__count_1` FROM `testDb`.`view1`|" + "SELECT `eventdate` AS `eventdate`, count(`userid`) AS `__count_1` FROM " - + "`testDb`.`appevent` WHERE `eventdate` >= '2020-01-12' AND " - + "`eventdate` <= '2020-01-14' GROUP BY `eventdate`"); + + "`testDb`.`appevent` WHERE (`eventdate` >= '2020-01-12') AND " + + "(`eventdate` <= '2020-01-14') GROUP BY `eventdate`"); Assert.assertEquals(selectedPartitionIds.size(), sqlCache.getSumOfPartitionNum()); } @@ -1109,7 +1109,7 @@ public void testSqlCacheKeyWithViewForNereids() { String cacheKey = sqlCache.getSqlWithViewStmt(); Assert.assertEquals(cacheKey, "SELECT * from testDb.view1|SELECT `eventdate` AS `eventdate`, " + "count(`userid`) AS `__count_1` FROM `testDb`.`appevent` " - + "WHERE `eventdate` >= '2020-01-12' AND `eventdate` <= '2020-01-14' GROUP BY `eventdate`"); + + "WHERE (`eventdate` >= '2020-01-12') AND (`eventdate` <= '2020-01-14') GROUP BY `eventdate`"); Assert.assertEquals(selectedPartitionIds.size(), sqlCache.getSumOfPartitionNum()); } @@ -1135,7 +1135,7 @@ public void testSqlCacheKeyWithSubSelectView() { Assert.assertEquals(cacheKey, "SELECT `origin`.`eventdate` AS `eventdate`, " + "`origin`.`userid` AS `userid` FROM (SELECT `view2`.`eventdate` AS `eventdate`, " + "`view2`.`userid` AS `userid` FROM `testDb`.`view2` view2 " - + "WHERE `view2`.`eventdate` >= '2020-01-12' AND `view2`.`eventdate` <= '2020-01-14') origin|" + + "WHERE (`view2`.`eventdate` >= '2020-01-12') AND (`view2`.`eventdate` <= '2020-01-14')) origin|" + "SELECT `eventdate` AS `eventdate`, `userid` AS `userid` FROM `testDb`.`appevent`"); Assert.assertEquals(selectedPartitionIds.size(), sqlCache.getSumOfPartitionNum()); } @@ -1187,8 +1187,8 @@ public void testPartitionCacheKeyWithView() { Assert.assertEquals(cache.getSqlWithViewStmt(), "SELECT `testDb`.`view3`.`eventdate` " + "AS `eventdate`, `testDb`.`view3`.`__count_1` AS `__count_1` " + "FROM `testDb`.`view3`|SELECT `eventdate` AS `eventdate`, count(`userid`) " - + "AS `__count_1` FROM `testDb`.`appevent` WHERE `eventdate` >= '2020-01-12' " - + "AND `eventdate` <= '2020-01-15' GROUP BY `eventdate`"); + + "AS `__count_1` FROM `testDb`.`appevent` WHERE (`eventdate` >= '2020-01-12') " + + "AND (`eventdate` <= '2020-01-15') GROUP BY `eventdate`"); } catch (Exception e) { LOG.warn("ex={}", e); Assert.fail(e.getMessage()); @@ -1243,7 +1243,7 @@ public void testSqlCacheKeyWithNestedView() { Assert.assertEquals(cacheKey, "SELECT `testDb`.`view4`.`eventdate` AS `eventdate`, " + "`testDb`.`view4`.`__count_1` AS `__count_1` FROM `testDb`.`view4`|" + "SELECT `eventdate` AS `eventdate`, count(`userid`) AS `__count_1` FROM `testDb`.`view2` " - + "WHERE `eventdate` >= '2020-01-12' AND `eventdate` <= '2020-01-14' GROUP BY `eventdate`|" + + "WHERE (`eventdate` >= '2020-01-12') AND (`eventdate` <= '2020-01-14') GROUP BY `eventdate`|" + "SELECT `eventdate` AS `eventdate`, `userid` AS `userid` FROM `testDb`.`appevent`"); Assert.assertEquals(selectedPartitionIds.size(), sqlCache.getSumOfPartitionNum()); } @@ -1261,8 +1261,8 @@ public void testSqlCacheKeyWithNestedViewForNereids() { SqlCache sqlCache = (SqlCache) ca.getCache(); String cacheKey = sqlCache.getSqlWithViewStmt(); Assert.assertEquals(cacheKey, "SELECT * from testDb.view4|SELECT `eventdate` AS `eventdate`, " - + "count(`userid`) AS `__count_1` FROM `testDb`.`view2` WHERE `eventdate` >= '2020-01-12' " - + "AND `eventdate` <= '2020-01-14' GROUP BY `eventdate`|SELECT `eventdate` AS `eventdate`, " + + "count(`userid`) AS `__count_1` FROM `testDb`.`view2` WHERE (`eventdate` >= '2020-01-12') " + + "AND (`eventdate` <= '2020-01-14') GROUP BY `eventdate`|SELECT `eventdate` AS `eventdate`, " + "`userid` AS `userid` FROM `testDb`.`appevent`"); Assert.assertEquals(selectedPartitionIds.size(), sqlCache.getSumOfPartitionNum()); } @@ -1326,7 +1326,7 @@ public void testNotExistPartitionSql() { cache.rewriteSelectStmt(newRangeList); sql = ca.getRewriteStmt().getWhereClause().toSql(); - Assert.assertEquals(sql, "`date` >= 20200114 AND `date` <= 20200115"); + Assert.assertEquals(sql, "(`date` >= 20200114) AND (`date` <= 20200115)"); } catch (Exception e) { LOG.warn("ex={}", e); Assert.fail(e.getMessage()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/rewrite/ExtractCommonFactorsRuleFunctionTest.java b/fe/fe-core/src/test/java/org/apache/doris/rewrite/ExtractCommonFactorsRuleFunctionTest.java index 1fcffc4fa5cd6d..c2f31ab90a3fca 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/rewrite/ExtractCommonFactorsRuleFunctionTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/rewrite/ExtractCommonFactorsRuleFunctionTest.java @@ -100,7 +100,7 @@ public void testCommonFactors() throws Exception { public void testWideCommonFactorsWithOrPredicate() throws Exception { String query = "select /*+ SET_VAR(enable_nereids_planner=false) */ * from tb1 where tb1.k1 > 1000 or tb1.k1 < 200 or tb1.k1 = 300"; String planString = dorisAssert.query(query).explainQuery(); - Assert.assertTrue(planString.contains("PREDICATES: `tb1`.`k1` = 300 OR `tb1`.`k1` > 1000 OR `tb1`.`k1` < 200")); + Assert.assertTrue(planString.contains("(`tb1`.`k1` = 300) OR (`tb1`.`k1` > 1000) OR (`tb1`.`k1` < 200)")); } @Test @@ -259,8 +259,8 @@ public void testComplexQuery() throws Exception { Assert.assertTrue(planString.contains("`l_partkey` = `p_partkey`")); Assert.assertTrue(planString.contains("`l_shipmode` IN ('AIR', 'AIR REG')")); Assert.assertTrue(planString.contains("`l_shipinstruct` = 'DELIVER IN PERSON'")); - Assert.assertTrue(planString.contains("`l_quantity` >= 9 AND `l_quantity` <= 19 " - + "OR `l_quantity` >= 20 AND `l_quantity` <= 36")); + Assert.assertTrue(planString.contains("(`l_quantity` >= 9) AND (`l_quantity` <= 19) " + + "OR (`l_quantity` >= 20) AND (`l_quantity` <= 36)")); Assert.assertTrue(planString.contains("`p_size` >= 1")); Assert.assertTrue(planString.contains("`p_brand` IN ('Brand#11', 'Brand#21', 'Brand#32')")); Assert.assertTrue(planString.contains("`p_size` <= 15")); diff --git a/regression-test/data/ddl_p0/test_create_view.out b/regression-test/data/ddl_p0/test_create_view.out index 239fafc025ef3a..7ee1123fd46b90 100644 --- a/regression-test/data/ddl_p0/test_create_view.out +++ b/regression-test/data/ddl_p0/test_create_view.out @@ -25,5 +25,5 @@ 3 [-1, 20, 0] [0, 1, 0] -- !test_view_6 -- -v1 CREATE VIEW `v1` COMMENT 'VIEW' AS SELECT `error_code` AS `error_code`, 1 AS `__literal_1`, 'string' AS `__literal_2`, now() AS `__now_3`, dayofyear(`op_time`) AS `__dayofyear_4`, CAST(`source` AS BIGINT) AS `__cast_expr_5`, min(`timestamp`) OVER (ORDER BY `op_time` DESC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) AS `__min_6`, 1 > 2 AS `__binary_predicate_7`, (2 + 3) AS `__arithmetic_expr_8`, 1 IN (1, 2, 3, 4) AS `__in_predicate_9`, `remark` LIKE '%like' AS `__like_predicate_10`, CASE WHEN `remark` = 's' THEN 1 ELSE 2 END AS `__case_expr_11`, (TRUE | FALSE) AS `__arithmetic_expr_12` FROM `regression_test_ddl_p0`.`view_column_name_test`; +v1 CREATE VIEW `v1` COMMENT 'VIEW' AS SELECT `error_code` AS `error_code`, 1 AS `__literal_1`, 'string' AS `__literal_2`, now() AS `__now_3`, dayofyear(`op_time`) AS `__dayofyear_4`, CAST(`source` AS BIGINT) AS `__cast_expr_5`, min(`timestamp`) OVER (ORDER BY `op_time` DESC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) AS `__min_6`, (1 > 2) AS `__binary_predicate_7`, (2 + 3) AS `__arithmetic_expr_8`, 1 IN (1, 2, 3, 4) AS `__in_predicate_9`, `remark` LIKE '%like' AS `__like_predicate_10`, CASE WHEN (`remark` = 's') THEN 1 ELSE 2 END AS `__case_expr_11`, (TRUE | FALSE) AS `__arithmetic_expr_12` FROM `regression_test_ddl_p0`.`view_column_name_test`; diff --git a/regression-test/data/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.out b/regression-test/data/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.out index e64409afb2137d..cd7da97e1a793c 100644 --- a/regression-test/data/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.out +++ b/regression-test/data/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.out @@ -13,3 +13,13 @@ -- !select_mv -- 7 +-- !select_mv -- +1 2 +2 1 +3 1 + +-- !select_mv -- +1 2 +2 2 +3 3 + diff --git a/regression-test/data/performance_p0/redundant_conjuncts.out b/regression-test/data/performance_p0/redundant_conjuncts.out index 9df35ef6b2d692..e4cd154e44ee75 100644 --- a/regression-test/data/performance_p0/redundant_conjuncts.out +++ b/regression-test/data/performance_p0/redundant_conjuncts.out @@ -12,7 +12,7 @@ PLAN FRAGMENT 0 0:VOlapScanNode TABLE: regression_test_performance_p0.redundant_conjuncts(redundant_conjuncts), PREAGGREGATION: OFF. Reason: No AggregateInfo - PREDICATES: `k1` = 1 + PREDICATES: (`k1` = 1) partitions=0/1 (), tablets=0/0, tabletList= cardinality=0, avgRowSize=8.0, numNodes=1 pushAggOp=NONE @@ -30,7 +30,7 @@ PLAN FRAGMENT 0 0:VOlapScanNode TABLE: regression_test_performance_p0.redundant_conjuncts(redundant_conjuncts), PREAGGREGATION: OFF. Reason: No AggregateInfo - PREDICATES: `k1` = 1 OR `k1` = 2 + PREDICATES: (`k1` = 1) OR (`k1` = 2) partitions=0/1 (), tablets=0/0, tabletList= cardinality=0, avgRowSize=8.0, numNodes=1 pushAggOp=NONE diff --git a/regression-test/data/show_p0/test_show_create_table_and_views.out b/regression-test/data/show_p0/test_show_create_table_and_views.out index 95af034fcc4759..2df41e35d235df 100644 --- a/regression-test/data/show_p0/test_show_create_table_and_views.out +++ b/regression-test/data/show_p0/test_show_create_table_and_views.out @@ -24,7 +24,7 @@ show_create_table_and_views_table CREATE TABLE `show_create_table_and_views_tabl 3 1 -- !show -- -show_create_table_and_views_view CREATE VIEW `show_create_table_and_views_view` COMMENT 'VIEW' AS SELECT `user_id` AS `user_id`, `cost` AS `cost` FROM `show_create_table_and_views_db`.`show_create_table_and_views_table` WHERE `good_id` = 2; utf8 utf8_general_ci +show_create_table_and_views_view CREATE VIEW `show_create_table_and_views_view` COMMENT 'VIEW' AS SELECT `user_id` AS `user_id`, `cost` AS `cost` FROM `show_create_table_and_views_db`.`show_create_table_and_views_table` WHERE (`good_id` = 2); utf8 utf8_general_ci -- !select -- 1 47 diff --git a/regression-test/suites/external_table_p0/jdbc/test_clickhouse_jdbc_catalog.groovy b/regression-test/suites/external_table_p0/jdbc/test_clickhouse_jdbc_catalog.groovy index 9948c49d24a55b..4b8aee3abe7d84 100644 --- a/regression-test/suites/external_table_p0/jdbc/test_clickhouse_jdbc_catalog.groovy +++ b/regression-test/suites/external_table_p0/jdbc/test_clickhouse_jdbc_catalog.groovy @@ -87,7 +87,7 @@ suite("test_clickhouse_jdbc_catalog", "p0,external,clickhouse,external_docker,ex order_qt_func_push """select * from ts where from_unixtime(ts,'yyyyMMdd') >= '2022-01-01';""" explain { sql("select * from ts where from_unixtime(ts,'yyyyMMdd') >= '2022-01-01';") - contains """QUERY: SELECT "id", "ts" FROM "doris_test"."ts" WHERE (FROM_UNIXTIME("ts", '%Y%m%d') >= '2022-01-01')""" + contains """QUERY: SELECT "id", "ts" FROM "doris_test"."ts" WHERE ((FROM_UNIXTIME("ts", '%Y%m%d') >= '2022-01-01'))""" } explain { sql("select * from ts where nvl(ts,null) >= '2022-01-01';") @@ -96,7 +96,7 @@ suite("test_clickhouse_jdbc_catalog", "p0,external,clickhouse,external_docker,ex order_qt_func_push2 """select * from ts where ts <= unix_timestamp(from_unixtime(ts,'yyyyMMdd'));""" explain { sql("select * from ts where ts <= unix_timestamp(from_unixtime(ts,'yyyy-MM-dd'));") - contains """QUERY: SELECT "id", "ts" FROM "doris_test"."ts" WHERE ("ts" <= toUnixTimestamp(FROM_UNIXTIME("ts", '%Y-%m-%d')))""" + contains """QUERY: SELECT "id", "ts" FROM "doris_test"."ts" WHERE (("ts" <= toUnixTimestamp(FROM_UNIXTIME("ts", '%Y-%m-%d'))))""" } order_qt_dt_with_tz """ select * from dt_with_tz order by id; """ diff --git a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy index 86afd749e56eae..3457ae2f6f8119 100644 --- a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy +++ b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy @@ -407,18 +407,18 @@ suite("test_mysql_jdbc_catalog", "p0,external,mysql,external_docker,external_doc explain { sql ("select k6, k8 from test1 where nvl(k6, null) = 1;") - contains "QUERY: SELECT `k6`, `k8` FROM `doris_test`.`test1` WHERE (ifnull(`k6`, NULL) = 1)" + contains "QUERY: SELECT `k6`, `k8` FROM `doris_test`.`test1` WHERE ((ifnull(`k6`, NULL) = 1))" } explain { sql ("select k6, k8 from test1 where nvl(nvl(k6, null),null) = 1;") - contains "QUERY: SELECT `k6`, `k8` FROM `doris_test`.`test1` WHERE (ifnull(ifnull(`k6`, NULL), NULL) = 1)" + contains "QUERY: SELECT `k6`, `k8` FROM `doris_test`.`test1` WHERE ((ifnull(ifnull(`k6`, NULL), NULL) = 1))" } sql """ set enable_ext_func_pred_pushdown = "false"; """ explain { sql ("select k6, k8 from test1 where nvl(k6, null) = 1 and k8 = 1;") - contains "QUERY: SELECT `k6`, `k8` FROM `doris_test`.`test1` WHERE (`k8` = 1)" + contains "QUERY: SELECT `k6`, `k8` FROM `doris_test`.`test1` WHERE ((`k8` = 1))" } sql """ set enable_ext_func_pred_pushdown = "true"; """ } finally { diff --git a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.groovy b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.groovy index c63314b186f069..24aac183eb861b 100644 --- a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.groovy +++ b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.groovy @@ -82,7 +82,7 @@ suite("test_mysql_jdbc_catalog_nereids", "p0,external,mysql,external_docker,exte explain { sql("""select id from ${ex_tb0} where id = 111;""") - contains "WHERE (`id` = 111)" + contains "WHERE ((`id` = 111))" } qt_ex_tb0_where """select id from ${ex_tb0} where id = 111;""" order_qt_ex_tb0 """ select id, name from ${ex_tb0} order by id; """ diff --git a/regression-test/suites/external_table_p0/jdbc/test_oracle_jdbc_catalog.groovy b/regression-test/suites/external_table_p0/jdbc/test_oracle_jdbc_catalog.groovy index 75ce7175df51ca..59e4a506d81246 100644 --- a/regression-test/suites/external_table_p0/jdbc/test_oracle_jdbc_catalog.groovy +++ b/regression-test/suites/external_table_p0/jdbc/test_oracle_jdbc_catalog.groovy @@ -137,7 +137,7 @@ suite("test_oracle_jdbc_catalog", "p0,external,oracle,external_docker,external_d // test nvl explain { sql("SELECT * FROM STUDENT WHERE nvl(score, 0) < 95;") - contains """SELECT "ID", "NAME", "AGE", "SCORE" FROM "DORIS_TEST"."STUDENT" WHERE (nvl("SCORE", 0.0) < 95.0)""" + contains """SELECT "ID", "NAME", "AGE", "SCORE" FROM "DORIS_TEST"."STUDENT" WHERE ((nvl("SCORE", 0.0) < 95.0))""" } // for old planner diff --git a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.groovy b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.groovy index 044b92425ec0de..1b7bc10a39c68f 100644 --- a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.groovy +++ b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.groovy @@ -38,6 +38,8 @@ suite ("testAggQueryOnAggMV1") { createMV("create materialized view emps_mv as select deptno, sum(salary), max(commission) from emps group by deptno ;") + createMV("create materialized view emps_mv_count_key as select deptno, count(deptno) from emps group by deptno;") + createMV("create materialized view emps_mv_if as select deptno, sum(if(empid = 1, empid, salary)) from emps group by deptno;") sql """insert into emps values("2020-01-01",1,"a",1,1,1);""" @@ -59,4 +61,16 @@ suite ("testAggQueryOnAggMV1") { contains "(emps_mv)" } qt_select_mv "select sum(salary) as salary from emps;" + + explain { + sql("select deptno, count(deptno) from emps group by deptno order by deptno;") + contains "(emps_mv_count_key)" + } + qt_select_mv "select deptno, count(deptno) from emps group by deptno order by deptno;" + + explain { + sql("select deptno, sum(if(empid = 1, empid, salary)) from emps group by deptno;") + contains "(emps_mv_if)" + } + qt_select_mv "select deptno, sum(if(empid = 1, empid, salary)) from emps group by deptno order by deptno;" } \ No newline at end of file diff --git a/regression-test/suites/nereids_p0/infer_predicate/infer_predicate.groovy b/regression-test/suites/nereids_p0/infer_predicate/infer_predicate.groovy index 55645ed8ea0950..afa323719379f7 100644 --- a/regression-test/suites/nereids_p0/infer_predicate/infer_predicate.groovy +++ b/regression-test/suites/nereids_p0/infer_predicate/infer_predicate.groovy @@ -31,28 +31,28 @@ suite("test_infer_predicate") { explain { sql "select * from infer_tb1 inner join infer_tb2 where infer_tb2.k1 = infer_tb1.k2 and infer_tb2.k1 = 1;" - contains "PREDICATES: k2" + contains "PREDICATES: (k2" } explain { sql "select * from infer_tb1 inner join infer_tb2 where infer_tb1.k2 = infer_tb2.k1 and infer_tb2.k1 = 1;" - contains "PREDICATES: k2" + contains "PREDICATES: (k2" } explain { sql "select * from infer_tb1 inner join infer_tb2 where cast(infer_tb2.k4 as int) = infer_tb1.k2 and infer_tb2.k4 = 1;" - contains "PREDICATES: CAST(k2" + contains "PREDICATES: (CAST(k2" } explain { sql "select * from infer_tb1 inner join infer_tb3 where infer_tb3.k1 = infer_tb1.k2 and infer_tb3.k1 = '123';" - notContains "PREDICATES: k2" + notContains "PREDICATES: (k2" } explain { sql "select * from infer_tb1 left join infer_tb2 on infer_tb1.k1 = infer_tb2.k3 left join infer_tb3 on " + "infer_tb2.k3 = infer_tb3.k2 where infer_tb1.k1 = 1;" - contains "PREDICATES: k3" - contains "PREDICATES: k2" + contains "PREDICATES: (k3" + contains "PREDICATES: (k2" } } From 4f4e2074a1d986246e686b23a68cdd092b3d1fac Mon Sep 17 00:00:00 2001 From: plat1ko Date: Thu, 18 Jan 2024 10:44:04 +0800 Subject: [PATCH 079/200] [feature](merge-cloud) Add CloudTablet (#30045) --- be/src/cloud/cloud_meta_mgr.cpp | 26 +- be/src/cloud/cloud_meta_mgr.h | 11 +- be/src/cloud/cloud_storage_engine.h | 39 ++ be/src/cloud/cloud_tablet.cpp | 441 ++++++++++++++++++++ be/src/cloud/cloud_tablet.h | 124 ++++++ be/src/common/status.h | 3 +- be/src/olap/base_tablet.cpp | 40 ++ be/src/olap/base_tablet.h | 19 +- be/src/olap/rowset/rowset.cpp | 9 + be/src/olap/rowset/rowset.h | 2 + be/src/olap/schema_change.cpp | 7 +- be/src/olap/tablet.cpp | 104 ++--- be/src/olap/tablet.h | 25 +- be/src/olap/tablet_meta.cpp | 7 + be/src/olap/tablet_meta.h | 6 + be/src/vec/exec/scan/new_olap_scan_node.cpp | 16 +- be/src/vec/exec/scan/new_olap_scanner.cpp | 15 +- 17 files changed, 752 insertions(+), 142 deletions(-) create mode 100644 be/src/cloud/cloud_storage_engine.h create mode 100644 be/src/cloud/cloud_tablet.cpp create mode 100644 be/src/cloud/cloud_tablet.h diff --git a/be/src/cloud/cloud_meta_mgr.cpp b/be/src/cloud/cloud_meta_mgr.cpp index cbe6ab8ae24112..d6eb54e5c4190f 100644 --- a/be/src/cloud/cloud_meta_mgr.cpp +++ b/be/src/cloud/cloud_meta_mgr.cpp @@ -29,6 +29,7 @@ #include #include +#include "cloud/cloud_tablet.h" #include "cloud/config.h" #include "cloud/pb_convert.h" #include "common/logging.h" @@ -38,7 +39,6 @@ #include "gen_cpp/olap_file.pb.h" #include "olap/olap_common.h" #include "olap/rowset/rowset_factory.h" -#include "olap/tablet.h" #include "olap/tablet_meta.h" #include "runtime/stream_load/stream_load_context.h" #include "util/network_util.h" @@ -270,12 +270,12 @@ Status CloudMetaMgr::get_tablet_meta(int64_t tablet_id, TabletMetaSharedPtr* tab return Status::OK(); } -Status CloudMetaMgr::sync_tablet_rowsets(Tablet* tablet, bool warmup_delta_data) { +Status CloudMetaMgr::sync_tablet_rowsets(CloudTablet* tablet, bool warmup_delta_data) { return Status::NotSupported("CloudMetaMgr::sync_tablet_rowsets is not implemented"); } Status CloudMetaMgr::sync_tablet_delete_bitmap( - Tablet* tablet, int64_t old_max_version, + CloudTablet* tablet, int64_t old_max_version, const google::protobuf::RepeatedPtrField& rs_metas, const TabletStatsPB& stats, const TabletIndexPB& idx, DeleteBitmap* delete_bitmap) { return Status::NotSupported("CloudMetaMgr::sync_tablet_delete_bitmap is not implemented"); @@ -425,15 +425,15 @@ Status CloudMetaMgr::update_tablet_schema(int64_t tablet_id, const TabletSchema& return Status::OK(); } -Status CloudMetaMgr::update_delete_bitmap(const Tablet* tablet, int64_t lock_id, int64_t initiator, - DeleteBitmap* delete_bitmap) { - VLOG_DEBUG << "update_delete_bitmap , tablet_id: " << tablet->tablet_id(); +Status CloudMetaMgr::update_delete_bitmap(const CloudTablet& tablet, int64_t lock_id, + int64_t initiator, DeleteBitmap* delete_bitmap) { + VLOG_DEBUG << "update_delete_bitmap , tablet_id: " << tablet.tablet_id(); UpdateDeleteBitmapRequest req; UpdateDeleteBitmapResponse res; req.set_cloud_unique_id(config::cloud_unique_id); - req.set_table_id(tablet->table_id()); - req.set_partition_id(tablet->partition_id()); - req.set_tablet_id(tablet->tablet_id()); + req.set_table_id(tablet.table_id()); + req.set_partition_id(tablet.partition_id()); + req.set_tablet_id(tablet.tablet_id()); req.set_lock_id(lock_id); req.set_initiator(initiator); for (auto iter = delete_bitmap->delete_bitmap.begin(); @@ -451,18 +451,18 @@ Status CloudMetaMgr::update_delete_bitmap(const Tablet* tablet, int64_t lock_id, if (res.status().code() == MetaServiceCode::LOCK_EXPIRED) { return Status::Error( "lock expired when update delete bitmap, tablet_id: {}, lock_id: {}", - tablet->tablet_id(), lock_id); + tablet.tablet_id(), lock_id); } return st; } -Status CloudMetaMgr::get_delete_bitmap_update_lock(const Tablet* tablet, int64_t lock_id, +Status CloudMetaMgr::get_delete_bitmap_update_lock(const CloudTablet& tablet, int64_t lock_id, int64_t initiator) { - VLOG_DEBUG << "get_delete_bitmap_update_lock , tablet_id: " << tablet->tablet_id(); + VLOG_DEBUG << "get_delete_bitmap_update_lock , tablet_id: " << tablet.tablet_id(); GetDeleteBitmapUpdateLockRequest req; GetDeleteBitmapUpdateLockResponse res; req.set_cloud_unique_id(config::cloud_unique_id); - req.set_table_id(tablet->table_id()); + req.set_table_id(tablet.table_id()); req.set_lock_id(lock_id); req.set_initiator(initiator); req.set_expiration(10); // 10s expiration time for compaction and schema_change diff --git a/be/src/cloud/cloud_meta_mgr.h b/be/src/cloud/cloud_meta_mgr.h index 6557a6eab8a1db..af5b048b2f06c0 100644 --- a/be/src/cloud/cloud_meta_mgr.h +++ b/be/src/cloud/cloud_meta_mgr.h @@ -29,7 +29,7 @@ namespace doris { class DeleteBitmap; class StreamLoadContext; -class Tablet; +class CloudTablet; class TabletMeta; class TabletSchema; class RowsetMeta; @@ -51,7 +51,7 @@ class CloudMetaMgr { Status get_tablet_meta(int64_t tablet_id, std::shared_ptr* tablet_meta); - Status sync_tablet_rowsets(Tablet* tablet, bool warmup_delta_data = false); + Status sync_tablet_rowsets(CloudTablet* tablet, bool warmup_delta_data = false); Status prepare_rowset(const RowsetMeta& rs_meta, bool is_tmp, std::shared_ptr* existed_rs_meta = nullptr); @@ -79,14 +79,15 @@ class CloudMetaMgr { Status update_tablet_schema(int64_t tablet_id, const TabletSchema& tablet_schema); - Status update_delete_bitmap(const Tablet* tablet, int64_t lock_id, int64_t initiator, + Status update_delete_bitmap(const CloudTablet& tablet, int64_t lock_id, int64_t initiator, DeleteBitmap* delete_bitmap); - Status get_delete_bitmap_update_lock(const Tablet* tablet, int64_t lock_id, int64_t initiator); + Status get_delete_bitmap_update_lock(const CloudTablet& tablet, int64_t lock_id, + int64_t initiator); private: Status sync_tablet_delete_bitmap( - Tablet* tablet, int64_t old_max_version, + CloudTablet* tablet, int64_t old_max_version, const google::protobuf::RepeatedPtrField& rs_metas, const TabletStatsPB& stas, const TabletIndexPB& idx, DeleteBitmap* delete_bitmap); }; diff --git a/be/src/cloud/cloud_storage_engine.h b/be/src/cloud/cloud_storage_engine.h new file mode 100644 index 00000000000000..87e3ed52d39041 --- /dev/null +++ b/be/src/cloud/cloud_storage_engine.h @@ -0,0 +1,39 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include + +namespace doris { +namespace cloud { +class CloudMetaMgr; +} + +class CloudStorageEngine { +public: + CloudStorageEngine(); + + ~CloudStorageEngine(); + + cloud::CloudMetaMgr& meta_mgr() { return *_meta_mgr; } + +private: + std::unique_ptr _meta_mgr; +}; + +} // namespace doris diff --git a/be/src/cloud/cloud_tablet.cpp b/be/src/cloud/cloud_tablet.cpp new file mode 100644 index 00000000000000..03670df78d10d3 --- /dev/null +++ b/be/src/cloud/cloud_tablet.cpp @@ -0,0 +1,441 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "cloud/cloud_tablet.h" + +#include +#include +#include +#include +#include + +#include "cloud/cloud_meta_mgr.h" +#include "cloud/cloud_storage_engine.h" +#include "io/cache/block/block_file_cache_factory.h" +#include "olap/rowset/rowset.h" +#include "olap/rowset/rowset_writer.h" +#include "olap/rowset/segment_v2/inverted_index_desc.h" + +namespace doris { +using namespace ErrorCode; + +CloudTablet::CloudTablet(CloudStorageEngine& engine, TabletMetaSharedPtr tablet_meta) + : BaseTablet(std::move(tablet_meta)), _engine(engine) {} + +CloudTablet::~CloudTablet() = default; + +bool CloudTablet::exceed_version_limit(int32_t limit) { + return _approximate_num_rowsets.load(std::memory_order_relaxed) > limit; +} + +Status CloudTablet::capture_rs_readers(const Version& spec_version, + std::vector* rs_splits, + bool skip_missing_version) { + Versions version_path; + std::shared_lock rlock(_meta_lock); + auto st = _timestamped_version_tracker.capture_consistent_versions(spec_version, &version_path); + if (!st.ok()) { + rlock.unlock(); // avoid logging in lock range + // Check no missed versions or req version is merged + auto missed_versions = calc_missed_versions(spec_version.second); + if (missed_versions.empty()) { + st.set_code(VERSION_ALREADY_MERGED); // Reset error code + } + st.append(" tablet_id=" + std::to_string(tablet_id())); + // clang-format off + LOG(WARNING) << st << '\n' << [this]() { std::string json; get_compaction_status(&json); return json; }(); + // clang-format on + return st; + } + VLOG_DEBUG << "capture consitent versions: " << version_path; + return capture_rs_readers_unlocked(version_path, rs_splits); +} + +// for example: +// [0-4][5-5][8-8][9-9][13-13] +// if spec_version = 12, it will return [6-7],[10-12] +Versions CloudTablet::calc_missed_versions(int64_t spec_version) { + DCHECK(spec_version > 0) << "invalid spec_version: " << spec_version; + + Versions missed_versions; + Versions existing_versions; + { + std::shared_lock rdlock(_meta_lock); + for (const auto& rs : _tablet_meta->all_rs_metas()) { + existing_versions.emplace_back(rs->version()); + } + } + + // sort the existing versions in ascending order + std::sort(existing_versions.begin(), existing_versions.end(), + [](const Version& a, const Version& b) { + // simple because 2 versions are certainly not overlapping + return a.first < b.first; + }); + + auto min_version = existing_versions.front().first; + if (min_version > 0) { + missed_versions.emplace_back(0, std::min(spec_version, min_version - 1)); + } + for (auto it = existing_versions.begin(); it != existing_versions.end() - 1; ++it) { + auto prev_v = it->second; + if (prev_v >= spec_version) { + return missed_versions; + } + auto next_v = (it + 1)->first; + if (next_v > prev_v + 1) { + // there is a hole between versions + missed_versions.emplace_back(prev_v + 1, std::min(spec_version, next_v - 1)); + } + } + auto max_version = existing_versions.back().second; + if (max_version < spec_version) { + missed_versions.emplace_back(max_version + 1, spec_version); + } + return missed_versions; +} + +Status CloudTablet::sync_meta() { + // TODO(lightman): FileCache + return Status::NotSupported("CloudTablet::sync_meta is not implemented"); +} + +// There are only two tablet_states RUNNING and NOT_READY in cloud mode +// This function will erase the tablet from `CloudTabletMgr` when it can't find this tablet in MS. +Status CloudTablet::sync_rowsets(int64_t query_version, bool warmup_delta_data) { + RETURN_IF_ERROR(sync_if_not_running()); + + if (query_version > 0) { + std::shared_lock rlock(_meta_lock); + if (_max_version >= query_version) { + return Status::OK(); + } + } + + // serially execute sync to reduce unnecessary network overhead + std::lock_guard lock(_sync_meta_lock); + if (query_version > 0) { + std::shared_lock rlock(_meta_lock); + if (_max_version >= query_version) { + return Status::OK(); + } + } + + auto st = _engine.meta_mgr().sync_tablet_rowsets(this, warmup_delta_data); + if (st.is()) { + recycle_cached_data(); + } + return st; +} + +// Sync tablet meta and all rowset meta if not running. +// This could happen when BE didn't finish schema change job and another BE committed this schema change job. +// It should be a quite rare situation. +Status CloudTablet::sync_if_not_running() { + if (tablet_state() == TABLET_RUNNING) { + return Status::OK(); + } + + // Serially execute sync to reduce unnecessary network overhead + std::lock_guard lock(_sync_meta_lock); + + { + std::shared_lock rlock(_meta_lock); + if (tablet_state() == TABLET_RUNNING) { + return Status::OK(); + } + } + + TabletMetaSharedPtr tablet_meta; + auto st = _engine.meta_mgr().get_tablet_meta(tablet_id(), &tablet_meta); + if (!st.ok()) { + if (st.is()) { + recycle_cached_data(); + } + return st; + } + + if (tablet_meta->tablet_state() != TABLET_RUNNING) [[unlikely]] { + // MoW may go to here when load while schema change + return Status::Error("invalid tablet state {}. tablet_id={}", + tablet_meta->tablet_state(), tablet_id()); + } + + TimestampedVersionTracker empty_tracker; + { + std::lock_guard wlock(_meta_lock); + RETURN_IF_ERROR(set_tablet_state(TABLET_RUNNING)); + _rs_version_map.clear(); + _stale_rs_version_map.clear(); + std::swap(_timestamped_version_tracker, empty_tracker); + _tablet_meta->clear_rowsets(); + _tablet_meta->clear_stale_rowset(); + _max_version = -1; + } + + st = _engine.meta_mgr().sync_tablet_rowsets(this); + if (st.is()) { + recycle_cached_data(); + } + return st; +} + +void CloudTablet::add_rowsets(std::vector to_add, bool version_overlap, + std::unique_lock& meta_lock, + bool warmup_delta_data) { + if (to_add.empty()) { + return; + } + + auto add_rowsets_directly = [=, this](std::vector& rowsets) { + for (auto& rs : rowsets) { + _rs_version_map.emplace(rs->version(), rs); + _timestamped_version_tracker.add_version(rs->version()); + _max_version = std::max(rs->end_version(), _max_version); + update_base_size(*rs); + } + _tablet_meta->add_rowsets_unchecked(rowsets); + // TODO(plat1ko): Warmup delta rowset data in background + }; + + if (!version_overlap) { + add_rowsets_directly(to_add); + return; + } + + // Filter out existed rowsets + auto remove_it = + std::remove_if(to_add.begin(), to_add.end(), [this](const RowsetSharedPtr& rs) { + if (auto find_it = _rs_version_map.find(rs->version()); + find_it == _rs_version_map.end()) { + return false; + } else if (find_it->second->rowset_id() == rs->rowset_id()) { + return true; // Same rowset + } + + // If version of rowset in `to_add` is equal to rowset in tablet but rowset_id is not equal, + // replace existed rowset with `to_add` rowset. This may occur when: + // 1. schema change converts rowsets which have been double written to new tablet + // 2. cumu compaction picks single overlapping input rowset to perform compaction + _tablet_meta->delete_rs_meta_by_version(rs->version(), nullptr); + _rs_version_map[rs->version()] = rs; + _tablet_meta->add_rowsets_unchecked({rs}); + update_base_size(*rs); + return true; + }); + + to_add.erase(remove_it, to_add.end()); + + // delete rowsets with overlapped version + std::vector to_add_directly; + for (auto& to_add_rs : to_add) { + // delete rowsets with overlapped version + std::vector to_delete; + Version to_add_v = to_add_rs->version(); + // if start_version > max_version, we can skip checking overlap here. + if (to_add_v.first > _max_version) { + // if start_version > max_version, we can skip checking overlap here. + to_add_directly.push_back(to_add_rs); + } else { + to_add_directly.push_back(to_add_rs); + for (auto& [v, rs] : _rs_version_map) { + if (to_add_v.contains(v)) { + to_delete.push_back(rs); + } + } + delete_rowsets(to_delete, meta_lock); + } + } + + add_rowsets_directly(to_add_directly); +} + +void CloudTablet::delete_rowsets(const std::vector& to_delete, + std::unique_lock&) { + if (to_delete.empty()) { + return; + } + std::vector rs_metas; + rs_metas.reserve(to_delete.size()); + for (auto&& rs : to_delete) { + rs_metas.push_back(rs->rowset_meta()); + _stale_rs_version_map[rs->version()] = rs; + } + _timestamped_version_tracker.add_stale_path_version(rs_metas); + for (auto&& rs : to_delete) { + _rs_version_map.erase(rs->version()); + } + + _tablet_meta->modify_rs_metas({}, rs_metas, false); +} + +int CloudTablet::delete_expired_stale_rowsets() { + std::vector expired_rowsets; + int64_t expired_stale_sweep_endtime = + ::time(nullptr) - config::tablet_rowset_stale_sweep_time_sec; + { + std::unique_lock wlock(_meta_lock); + + std::vector path_ids; + // capture the path version to delete + _timestamped_version_tracker.capture_expired_paths(expired_stale_sweep_endtime, &path_ids); + + if (path_ids.empty()) { + return 0; + } + + for (int64_t path_id : path_ids) { + // delete stale versions in version graph + auto version_path = _timestamped_version_tracker.fetch_and_delete_path_by_id(path_id); + for (auto& v_ts : version_path->timestamped_versions()) { + auto rs_it = _stale_rs_version_map.find(v_ts->version()); + if (rs_it != _stale_rs_version_map.end()) { + expired_rowsets.push_back(rs_it->second); + _stale_rs_version_map.erase(rs_it); + } else { + LOG(WARNING) << "cannot find stale rowset " << v_ts->version() << " in tablet " + << tablet_id(); + // clang-format off + DCHECK(false) << [this, &wlock]() { wlock.unlock(); std::string json; get_compaction_status(&json); return json; }(); + // clang-format on + } + _tablet_meta->delete_stale_rs_meta_by_version(v_ts->version()); + VLOG_DEBUG << "delete stale rowset " << v_ts->version(); + } + } + _reconstruct_version_tracker_if_necessary(); + } + recycle_cached_data(expired_rowsets); + return expired_rowsets.size(); +} + +void CloudTablet::update_base_size(const Rowset& rs) { + // Define base rowset as the rowset of version [2-x] + if (rs.start_version() == 2) { + _base_size = rs.data_disk_size(); + } +} + +void CloudTablet::recycle_cached_data() { + // TODO(plat1ko) +} + +void CloudTablet::recycle_cached_data(const std::vector& rowsets) { + // TODO(plat1ko) +} + +void CloudTablet::reset_approximate_stats(int64_t num_rowsets, int64_t num_segments, + int64_t num_rows, int64_t data_size) { + _approximate_num_rowsets.store(num_rowsets, std::memory_order_relaxed); + _approximate_num_segments.store(num_segments, std::memory_order_relaxed); + _approximate_num_rows.store(num_rows, std::memory_order_relaxed); + _approximate_data_size.store(data_size, std::memory_order_relaxed); + int64_t cumu_num_deltas = 0; + int64_t cumu_num_rowsets = 0; + auto cp = _cumulative_point.load(std::memory_order_relaxed); + for (auto& [v, r] : _rs_version_map) { + if (v.second < cp) { + continue; + } + + cumu_num_deltas += r->is_segments_overlapping() ? r->num_segments() : 1; + ++cumu_num_rowsets; + } + _approximate_cumu_num_rowsets.store(cumu_num_rowsets, std::memory_order_relaxed); + _approximate_cumu_num_deltas.store(cumu_num_deltas, std::memory_order_relaxed); +} + +Result> CloudTablet::create_rowset_writer( + RowsetWriterContext& context, bool vertical) { + return ResultError( + Status::NotSupported("CloudTablet::create_rowset_writer is not implemented")); +} + +// return a json string to show the compaction status of this tablet +void CloudTablet::get_compaction_status(std::string* json_result) { + rapidjson::Document root; + root.SetObject(); + + rapidjson::Document path_arr; + path_arr.SetArray(); + + std::vector rowsets; + std::vector stale_rowsets; + { + std::shared_lock rdlock(_meta_lock); + rowsets.reserve(_rs_version_map.size()); + for (auto& it : _rs_version_map) { + rowsets.push_back(it.second); + } + stale_rowsets.reserve(_stale_rs_version_map.size()); + for (auto& it : _stale_rs_version_map) { + stale_rowsets.push_back(it.second); + } + } + std::sort(rowsets.begin(), rowsets.end(), Rowset::comparator); + std::sort(stale_rowsets.begin(), stale_rowsets.end(), Rowset::comparator); + + // get snapshot version path json_doc + _timestamped_version_tracker.get_stale_version_path_json_doc(path_arr); + root.AddMember("cumulative point", _cumulative_point.load(), root.GetAllocator()); + + // print all rowsets' version as an array + rapidjson::Document versions_arr; + rapidjson::Document missing_versions_arr; + versions_arr.SetArray(); + missing_versions_arr.SetArray(); + int64_t last_version = -1; + for (auto& rowset : rowsets) { + const Version& ver = rowset->version(); + if (ver.first != last_version + 1) { + rapidjson::Value miss_value; + miss_value.SetString(fmt::format("[{}-{}]", last_version + 1, ver.first - 1).c_str(), + missing_versions_arr.GetAllocator()); + missing_versions_arr.PushBack(miss_value, missing_versions_arr.GetAllocator()); + } + rapidjson::Value value; + std::string version_str = rowset->get_rowset_info_str(); + value.SetString(version_str.c_str(), version_str.length(), versions_arr.GetAllocator()); + versions_arr.PushBack(value, versions_arr.GetAllocator()); + last_version = ver.second; + } + root.AddMember("rowsets", versions_arr, root.GetAllocator()); + root.AddMember("missing_rowsets", missing_versions_arr, root.GetAllocator()); + + // print all stale rowsets' version as an array + rapidjson::Document stale_versions_arr; + stale_versions_arr.SetArray(); + for (auto& rowset : stale_rowsets) { + rapidjson::Value value; + std::string version_str = rowset->get_rowset_info_str(); + value.SetString(version_str.c_str(), version_str.length(), + stale_versions_arr.GetAllocator()); + stale_versions_arr.PushBack(value, stale_versions_arr.GetAllocator()); + } + root.AddMember("stale_rowsets", stale_versions_arr, root.GetAllocator()); + + // add stale version rowsets + root.AddMember("stale version path", path_arr, root.GetAllocator()); + + // to json string + rapidjson::StringBuffer strbuf; + rapidjson::PrettyWriter writer(strbuf); + root.Accept(writer); + *json_result = std::string(strbuf.GetString()); +} + +} // namespace doris diff --git a/be/src/cloud/cloud_tablet.h b/be/src/cloud/cloud_tablet.h new file mode 100644 index 00000000000000..537c8fe134d13f --- /dev/null +++ b/be/src/cloud/cloud_tablet.h @@ -0,0 +1,124 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include + +#include "olap/base_tablet.h" +#include "olap/version_graph.h" + +namespace doris { + +class CloudStorageEngine; + +class CloudTablet final : public BaseTablet { +public: + CloudTablet(CloudStorageEngine& engine, TabletMetaSharedPtr tablet_meta); + + ~CloudTablet() override; + + bool exceed_version_limit(int32_t limit) override; + + Result> create_rowset_writer(RowsetWriterContext& context, + bool vertical) override; + + Status capture_rs_readers(const Version& spec_version, std::vector* rs_splits, + bool skip_missing_version) override; + + size_t tablet_footprint() override { + return _approximate_data_size.load(std::memory_order_relaxed); + } + + // clang-format off + int64_t fetch_add_approximate_num_rowsets (int64_t x) { return _approximate_num_rowsets .fetch_add(x, std::memory_order_relaxed); } + int64_t fetch_add_approximate_num_segments(int64_t x) { return _approximate_num_segments.fetch_add(x, std::memory_order_relaxed); } + int64_t fetch_add_approximate_num_rows (int64_t x) { return _approximate_num_rows .fetch_add(x, std::memory_order_relaxed); } + int64_t fetch_add_approximate_data_size (int64_t x) { return _approximate_data_size .fetch_add(x, std::memory_order_relaxed); } + int64_t fetch_add_approximate_cumu_num_rowsets (int64_t x) { return _approximate_cumu_num_rowsets.fetch_add(x, std::memory_order_relaxed); } + int64_t fetch_add_approximate_cumu_num_deltas (int64_t x) { return _approximate_cumu_num_deltas.fetch_add(x, std::memory_order_relaxed); } + // clang-format on + + // meta lock must be held when calling this function + void reset_approximate_stats(int64_t num_rowsets, int64_t num_segments, int64_t num_rows, + int64_t data_size); + + // return a json string to show the compaction status of this tablet + void get_compaction_status(std::string* json_result); + + // Synchronize the rowsets from meta service. + // If tablet state is not `TABLET_RUNNING`, sync tablet meta and all visible rowsets. + // If `query_version` > 0 and local max_version of the tablet >= `query_version`, do nothing. + // If 'need_download_data_async' is true, it means that we need to download the new version + // rowsets datas async. + Status sync_rowsets(int64_t query_version = -1, bool warmup_delta_data = false); + + // Synchronize the tablet meta from meta service. + Status sync_meta(); + + // If `version_overlap` is true, function will delete rowsets with overlapped version in this tablet. + // If 'warmup_delta_data' is true, download the new version rowset data in background. + // MUST hold EXCLUSIVE `_meta_lock`. + // If 'need_download_data_async' is true, it means that we need to download the new version + // rowsets datas async. + void add_rowsets(std::vector to_add, bool version_overlap, + std::unique_lock& meta_lock, + bool warmup_delta_data = false); + + // MUST hold EXCLUSIVE `_meta_lock`. + void delete_rowsets(const std::vector& to_delete, + std::unique_lock& meta_lock); + + // When the tablet is dropped, we need to recycle cached data: + // 1. The data in file cache + // 2. The memory in tablet cache + void recycle_cached_data(); + + void recycle_cached_data(const std::vector& rowsets); + + // Return number of deleted stale rowsets + int delete_expired_stale_rowsets(); + +private: + Versions calc_missed_versions(int64_t spec_version); + + // FIXME(plat1ko): No need to record base size if rowsets are ordered by version + void update_base_size(const Rowset& rs); + + Status sync_if_not_running(); + + CloudStorageEngine& _engine; + + // this mutex MUST ONLY be used when sync meta + bthread::Mutex _sync_meta_lock; + + std::atomic _cumulative_point {-1}; + std::atomic _approximate_num_rowsets {-1}; + std::atomic _approximate_num_segments {-1}; + std::atomic _approximate_num_rows {-1}; + std::atomic _approximate_data_size {-1}; + std::atomic _approximate_cumu_num_rowsets {-1}; + // Number of sorted arrays (e.g. for rowset with N segments, if rowset is overlapping, delta is N, otherwise 1) after cumu point + std::atomic _approximate_cumu_num_deltas {-1}; + + [[maybe_unused]] int64_t _base_compaction_cnt = 0; + [[maybe_unused]] int64_t _cumulative_compaction_cnt = 0; + int64_t _max_version = -1; + int64_t _base_size = 0; +}; + +} // namespace doris diff --git a/be/src/common/status.h b/be/src/common/status.h index 5199e86d120da2..2bec1c397e8e89 100644 --- a/be/src/common/status.h +++ b/be/src/common/status.h @@ -274,7 +274,8 @@ namespace ErrorCode { E(INVERTED_INDEX_COMPACTION_ERROR, -6010, false); \ E(KEY_NOT_FOUND, -7000, false); \ E(KEY_ALREADY_EXISTS, -7001, false); \ - E(ENTRY_NOT_FOUND, -7002, false); + E(ENTRY_NOT_FOUND, -7002, false); \ + E(INVALID_TABLET_STATE, -7211, false); // Define constexpr int error_code_name = error_code_value #define M(NAME, ERRORCODE, ENABLESTACKTRACE) constexpr int NAME = ERRORCODE; diff --git a/be/src/olap/base_tablet.cpp b/be/src/olap/base_tablet.cpp index 43f514906cf02e..18445cb17a67da 100644 --- a/be/src/olap/base_tablet.cpp +++ b/be/src/olap/base_tablet.cpp @@ -19,6 +19,8 @@ #include +#include "olap/rowset/rowset.h" +#include "olap/rowset/rowset_reader.h" #include "olap/tablet_fwd.h" #include "olap/tablet_schema_cache.h" #include "util/doris_metrics.h" @@ -79,4 +81,42 @@ Status BaseTablet::update_by_least_common_schema(const TabletSchemaSPtr& update_ return Status::OK(); } +Status BaseTablet::capture_rs_readers_unlocked(const std::vector& version_path, + std::vector* rs_splits) const { + DCHECK(rs_splits != nullptr && rs_splits->empty()); + for (auto version : version_path) { + auto it = _rs_version_map.find(version); + if (it == _rs_version_map.end()) { + VLOG_NOTICE << "fail to find Rowset in rs_version for version. tablet=" << tablet_id() + << ", version='" << version.first << "-" << version.second; + + it = _stale_rs_version_map.find(version); + if (it == _stale_rs_version_map.end()) { + return Status::Error( + "fail to find Rowset in stale_rs_version for version. tablet={}, " + "version={}-{}", + tablet_id(), version.first, version.second); + } + } + RowsetReaderSharedPtr rs_reader; + auto res = it->second->create_reader(&rs_reader); + if (!res.ok()) { + return Status::Error( + "failed to create reader for rowset:{}", it->second->rowset_id().to_string()); + } + rs_splits->push_back(RowSetSplits(std::move(rs_reader))); + } + return Status::OK(); +} + +bool BaseTablet::_reconstruct_version_tracker_if_necessary() { + double orphan_vertex_ratio = _timestamped_version_tracker.get_orphan_vertex_ratio(); + if (orphan_vertex_ratio >= config::tablet_version_graph_orphan_vertex_ratio) { + _timestamped_version_tracker.construct_versioned_tracker( + _tablet_meta->all_rs_metas(), _tablet_meta->all_stale_rs_metas()); + return true; + } + return false; +} + } /* namespace doris */ diff --git a/be/src/olap/base_tablet.h b/be/src/olap/base_tablet.h index 2fa494b420aab3..bb327b3953298c 100644 --- a/be/src/olap/base_tablet.h +++ b/be/src/olap/base_tablet.h @@ -24,6 +24,7 @@ #include "common/status.h" #include "olap/tablet_fwd.h" #include "olap/tablet_meta.h" +#include "olap/version_graph.h" #include "util/metrics.h" namespace doris { @@ -72,19 +73,33 @@ class BaseTablet { return _max_version_schema; } - virtual bool exceed_version_limit(int32_t limit) const = 0; + virtual bool exceed_version_limit(int32_t limit) = 0; virtual Result> create_rowset_writer(RowsetWriterContext& context, bool vertical) = 0; virtual Status capture_rs_readers(const Version& spec_version, std::vector* rs_splits, - bool skip_missing_version) const = 0; + bool skip_missing_version) = 0; virtual size_t tablet_footprint() = 0; + // MUST hold shared meta lock + Status capture_rs_readers_unlocked(const std::vector& version_path, + std::vector* rs_splits) const; + protected: + bool _reconstruct_version_tracker_if_necessary(); + mutable std::shared_mutex _meta_lock; + TimestampedVersionTracker _timestamped_version_tracker; + // After version 0.13, all newly created rowsets are saved in _rs_version_map. + // And if rowset being compacted, the old rowsetis will be saved in _stale_rs_version_map; + std::unordered_map _rs_version_map; + // This variable _stale_rs_version_map is used to record these rowsets which are be compacted. + // These _stale rowsets are been removed when rowsets' pathVersion is expired, + // this policy is judged and computed by TimestampedVersionTracker. + std::unordered_map _stale_rs_version_map; const TabletMetaSharedPtr _tablet_meta; TabletSchemaSPtr _max_version_schema; diff --git a/be/src/olap/rowset/rowset.cpp b/be/src/olap/rowset/rowset.cpp index b3e6b1ca9e0a1c..d09fce730e5395 100644 --- a/be/src/olap/rowset/rowset.cpp +++ b/be/src/olap/rowset/rowset.cpp @@ -92,4 +92,13 @@ void Rowset::merge_rowset_meta(const RowsetMetaSharedPtr& other) { } } +std::string Rowset::get_rowset_info_str() { + std::string disk_size = PrettyPrinter::print( + static_cast(_rowset_meta->total_disk_size()), TUnit::BYTES); + return fmt::format("[{}-{}] {} {} {} {} {}", start_version(), end_version(), num_segments(), + _rowset_meta->has_delete_predicate() ? "DELETE" : "DATA", + SegmentsOverlapPB_Name(_rowset_meta->segments_overlap()), + rowset_id().to_string(), disk_size); +} + } // namespace doris diff --git a/be/src/olap/rowset/rowset.h b/be/src/olap/rowset/rowset.h index a2a275f2eac32e..a1355a81198a97 100644 --- a/be/src/olap/rowset/rowset.h +++ b/be/src/olap/rowset/rowset.h @@ -302,6 +302,8 @@ class Rowset : public std::enable_shared_from_this { // set skip index compaction next time void set_skip_index_compaction(int32_t column_id) { skip_index_compaction.insert(column_id); } + std::string get_rowset_info_str(); + protected: friend class RowsetFactory; diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp index 66e6ffcf2b22c2..327afd6a8c41de 100644 --- a/be/src/olap/schema_change.cpp +++ b/be/src/olap/schema_change.cpp @@ -833,7 +833,8 @@ Status SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletReqV2& } // acquire data sources correspond to history versions - RETURN_IF_ERROR(base_tablet->capture_rs_readers(versions_to_be_changed, &rs_splits)); + RETURN_IF_ERROR( + base_tablet->capture_rs_readers_unlocked(versions_to_be_changed, &rs_splits)); if (rs_splits.empty()) { res = Status::Error( "fail to acquire all data sources. version_num={}, data_source_num={}", @@ -985,8 +986,8 @@ Status SchemaChangeHandler::_get_versions_to_be_changed( } *max_rowset = rowset; - RETURN_IF_ERROR(base_tablet->capture_consistent_versions(Version(0, rowset->version().second), - versions_to_be_changed, false, false)); + RETURN_IF_ERROR(base_tablet->capture_consistent_versions_unlocked( + Version(0, rowset->version().second), versions_to_be_changed, false, false)); return Status::OK(); } diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index 720cd5b2350b72..50ec23040967ab 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -757,7 +757,8 @@ void Tablet::delete_expired_stale_rowset() { Version test_version = Version(0, lastest_delta->end_version()); stale_version_path_map[*path_id_iter] = version_path; - Status status = capture_consistent_versions(test_version, nullptr, false, false); + Status status = + capture_consistent_versions_unlocked(test_version, nullptr, false, false); // 1. When there is no consistent versions, we must reconstruct the tracker. if (!status.ok()) { // 2. fetch missing version after delete @@ -867,19 +868,9 @@ void Tablet::delete_expired_stale_rowset() { #endif } -bool Tablet::_reconstruct_version_tracker_if_necessary() { - double orphan_vertex_ratio = _timestamped_version_tracker.get_orphan_vertex_ratio(); - if (orphan_vertex_ratio >= config::tablet_version_graph_orphan_vertex_ratio) { - _timestamped_version_tracker.construct_versioned_tracker( - _tablet_meta->all_rs_metas(), _tablet_meta->all_stale_rs_metas()); - return true; - } - return false; -} - -Status Tablet::capture_consistent_versions(const Version& spec_version, - std::vector* version_path, - bool skip_missing_version, bool quiet) const { +Status Tablet::capture_consistent_versions_unlocked(const Version& spec_version, + std::vector* version_path, + bool skip_missing_version, bool quiet) const { Status status = _timestamped_version_tracker.capture_consistent_versions(spec_version, version_path); if (!status.ok() && !quiet) { @@ -914,10 +905,10 @@ Status Tablet::capture_consistent_versions(const Version& spec_version, Status Tablet::check_version_integrity(const Version& version, bool quiet) { std::shared_lock rdlock(_meta_lock); - return capture_consistent_versions(version, nullptr, false, quiet); + return capture_consistent_versions_unlocked(version, nullptr, false, quiet); } -bool Tablet::exceed_version_limit(int32_t limit) const { +bool Tablet::exceed_version_limit(int32_t limit) { if (_tablet_meta->version_count() > limit) { exceed_version_limit_counter << 1; return true; @@ -947,7 +938,8 @@ void Tablet::acquire_version_and_rowsets( Status Tablet::capture_consistent_rowsets(const Version& spec_version, std::vector* rowsets) const { std::vector version_path; - RETURN_IF_ERROR(capture_consistent_versions(spec_version, &version_path, false, false)); + RETURN_IF_ERROR( + capture_consistent_versions_unlocked(spec_version, &version_path, false, false)); RETURN_IF_ERROR(_capture_consistent_rowsets_unlocked(version_path, rowsets)); return Status::OK(); } @@ -984,39 +976,12 @@ Status Tablet::_capture_consistent_rowsets_unlocked(const std::vector& } Status Tablet::capture_rs_readers(const Version& spec_version, std::vector* rs_splits, - bool skip_missing_version) const { + bool skip_missing_version) { + std::shared_lock rlock(_meta_lock); std::vector version_path; - RETURN_IF_ERROR( - capture_consistent_versions(spec_version, &version_path, skip_missing_version, false)); - RETURN_IF_ERROR(capture_rs_readers(version_path, rs_splits)); - return Status::OK(); -} - -Status Tablet::capture_rs_readers(const std::vector& version_path, - std::vector* rs_splits) const { - DCHECK(rs_splits != nullptr && rs_splits->empty()); - for (auto version : version_path) { - auto it = _rs_version_map.find(version); - if (it == _rs_version_map.end()) { - VLOG_NOTICE << "fail to find Rowset in rs_version for version. tablet=" << tablet_id() - << ", version='" << version.first << "-" << version.second; - - it = _stale_rs_version_map.find(version); - if (it == _stale_rs_version_map.end()) { - return Status::Error( - "fail to find Rowset in stale_rs_version for version. tablet={}, " - "version={}-{}", - tablet_id(), version.first, version.second); - } - } - RowsetReaderSharedPtr rs_reader; - auto res = it->second->create_reader(&rs_reader); - if (!res.ok()) { - return Status::Error( - "failed to create reader for rowset:{}", it->second->rowset_id().to_string()); - } - rs_splits->push_back(RowSetSplits(std::move(rs_reader))); - } + RETURN_IF_ERROR(capture_consistent_versions_unlocked(spec_version, &version_path, + skip_missing_version, false)); + RETURN_IF_ERROR(capture_rs_readers_unlocked(version_path, rs_splits)); return Status::OK(); } @@ -1419,16 +1384,6 @@ std::vector Tablet::pick_candidate_rowsets_to_build_inverted_in return candidate_rowsets; } -std::string Tablet::_get_rowset_info_str(RowsetSharedPtr rowset, bool delete_flag) { - const Version& ver = rowset->version(); - std::string disk_size = PrettyPrinter::print( - static_cast(rowset->rowset_meta()->total_disk_size()), TUnit::BYTES); - return strings::Substitute("[$0-$1] $2 $3 $4 $5 $6", ver.first, ver.second, - rowset->num_segments(), (delete_flag ? "DELETE" : "DATA"), - SegmentsOverlapPB_Name(rowset->rowset_meta()->segments_overlap()), - rowset->rowset_id().to_string(), disk_size); -} - // For http compaction action void Tablet::get_compaction_status(std::string* json_result) { rapidjson::Document root; @@ -1523,17 +1478,16 @@ void Tablet::get_compaction_status(std::string* json_result) { versions_arr.SetArray(); missing_versions_arr.SetArray(); int64_t last_version = -1; - for (int i = 0; i < rowsets.size(); ++i) { - const Version& ver = rowsets[i]->version(); + for (auto& rowset : rowsets) { + const Version& ver = rowset->version(); if (ver.first != last_version + 1) { rapidjson::Value miss_value; - miss_value.SetString( - strings::Substitute("[$0-$1]", last_version + 1, ver.first - 1).c_str(), - missing_versions_arr.GetAllocator()); + miss_value.SetString(fmt::format("[{}-{}]", last_version + 1, ver.first - 1).c_str(), + missing_versions_arr.GetAllocator()); missing_versions_arr.PushBack(miss_value, missing_versions_arr.GetAllocator()); } rapidjson::Value value; - std::string version_str = _get_rowset_info_str(rowsets[i], delete_flags[i]); + std::string version_str = rowset->get_rowset_info_str(); value.SetString(version_str.c_str(), version_str.length(), versions_arr.GetAllocator()); versions_arr.PushBack(value, versions_arr.GetAllocator()); last_version = ver.second; @@ -1544,15 +1498,9 @@ void Tablet::get_compaction_status(std::string* json_result) { // print all stale rowsets' version as an array rapidjson::Document stale_versions_arr; stale_versions_arr.SetArray(); - for (int i = 0; i < stale_rowsets.size(); ++i) { - const Version& ver = stale_rowsets[i]->version(); + for (auto& rowset : stale_rowsets) { rapidjson::Value value; - std::string disk_size = PrettyPrinter::print( - static_cast(stale_rowsets[i]->rowset_meta()->total_disk_size()), - TUnit::BYTES); - std::string version_str = strings::Substitute( - "[$0-$1] $2 $3 $4", ver.first, ver.second, stale_rowsets[i]->num_segments(), - stale_rowsets[i]->rowset_id().to_string(), disk_size); + std::string version_str = rowset->get_rowset_info_str(); value.SetString(version_str.c_str(), version_str.length(), stale_versions_arr.GetAllocator()); stale_versions_arr.PushBack(value, stale_versions_arr.GetAllocator()); @@ -3449,8 +3397,8 @@ Status Tablet::check_rowid_conversion( Status Tablet::all_rs_id(int64_t max_version, RowsetIdUnorderedSet* rowset_ids) const { // Ensure that the obtained versions of rowsets are continuous std::vector version_path; - RETURN_IF_ERROR( - capture_consistent_versions(Version(0, max_version), &version_path, false, false)); + RETURN_IF_ERROR(capture_consistent_versions_unlocked(Version(0, max_version), &version_path, + false, false)); for (auto& ver : version_path) { if (ver.second == 1) { // [0-1] rowset is empty for each tablet, skip it @@ -3719,8 +3667,7 @@ Status Tablet::check_delete_bitmap_correctness(DeleteBitmapPtr delete_bitmap, in if (rowsets != nullptr) { for (const auto& rowset : *rowsets) { rapidjson::Value value; - std::string version_str = - _get_rowset_info_str(rowset, rowset->rowset_meta()->has_delete_predicate()); + std::string version_str = rowset->get_rowset_info_str(); value.SetString(version_str.c_str(), version_str.length(), required_rowsets_arr.GetAllocator()); required_rowsets_arr.PushBack(value, required_rowsets_arr.GetAllocator()); @@ -3733,8 +3680,7 @@ Status Tablet::check_delete_bitmap_correctness(DeleteBitmapPtr delete_bitmap, in } for (const auto& rowset : rowsets) { rapidjson::Value value; - std::string version_str = - _get_rowset_info_str(rowset, rowset->rowset_meta()->has_delete_predicate()); + std::string version_str = rowset->get_rowset_info_str(); value.SetString(version_str.c_str(), version_str.length(), required_rowsets_arr.GetAllocator()); required_rowsets_arr.PushBack(value, required_rowsets_arr.GetAllocator()); diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h index d6ad0285233f38..d953d8fce4fcb0 100644 --- a/be/src/olap/tablet.h +++ b/be/src/olap/tablet.h @@ -123,7 +123,7 @@ class Tablet final : public BaseTablet { size_t num_rows(); int version_count() const; - bool exceed_version_limit(int32_t limit) const override; + bool exceed_version_limit(int32_t limit) override; uint64_t segment_count() const; Version max_version() const; Version max_version_unlocked() const; @@ -170,9 +170,9 @@ class Tablet final : public BaseTablet { // Given spec_version, find a continuous version path and store it in version_path. // If quiet is true, then only "does this path exist" is returned. // If skip_missing_version is true, return ok even there are missing versions. - Status capture_consistent_versions(const Version& spec_version, - std::vector* version_path, - bool skip_missing_version, bool quiet) const; + Status capture_consistent_versions_unlocked(const Version& spec_version, + std::vector* version_path, + bool skip_missing_version, bool quiet) const; // if quiet is true, no error log will be printed if there are missing versions Status check_version_integrity(const Version& version, bool quiet = false); bool check_version_exist(const Version& version) const; @@ -183,10 +183,7 @@ class Tablet final : public BaseTablet { std::vector* rowsets) const; // If skip_missing_version is true, skip versions if they are missing. Status capture_rs_readers(const Version& spec_version, std::vector* rs_splits, - bool skip_missing_version) const override; - - Status capture_rs_readers(const std::vector& version_path, - std::vector* rs_splits) const; + bool skip_missing_version) override; // meta lock std::shared_mutex& get_header_lock() { return _meta_lock; } @@ -583,9 +580,6 @@ class Tablet final : public BaseTablet { std::shared_ptr cumulative_compaction_policy); uint32_t _calc_base_compaction_score() const; - // When the proportion of empty edges in the adjacency matrix used to represent the version graph - // in the version tracker is greater than the threshold, rebuild the version tracker - bool _reconstruct_version_tracker_if_necessary(); void _init_context_common_fields(RowsetWriterContext& context); void _rowset_ids_difference(const RowsetIdUnorderedSet& cur, const RowsetIdUnorderedSet& pre, @@ -607,7 +601,6 @@ class Tablet final : public BaseTablet { //////////////////////////////////////////////////////////////////////////// void _remove_sentinel_mark_from_delete_bitmap(DeleteBitmapPtr delete_bitmap); - std::string _get_rowset_info_str(RowsetSharedPtr rowset, bool delete_flag); public: static const int64_t K_INVALID_CUMULATIVE_POINT = -1; @@ -615,7 +608,6 @@ class Tablet final : public BaseTablet { private: StorageEngine& _engine; DataDir* _data_dir = nullptr; - TimestampedVersionTracker _timestamped_version_tracker; DorisCallOnce _init_once; // meta store lock is used for prevent 2 threads do checkpoint concurrently @@ -634,13 +626,6 @@ class Tablet final : public BaseTablet { // during publish_txn, which might take hundreds of milliseconds mutable std::mutex _rowset_update_lock; - // After version 0.13, all newly created rowsets are saved in _rs_version_map. - // And if rowset being compacted, the old rowsetis will be saved in _stale_rs_version_map; - std::unordered_map _rs_version_map; - // This variable _stale_rs_version_map is used to record these rowsets which are be compacted. - // These _stale rowsets are been removed when rowsets' pathVersion is expired, - // this policy is judged and computed by TimestampedVersionTracker. - std::unordered_map _stale_rs_version_map; // if this tablet is broken, set to true. default is false std::atomic _is_bad; // timestamp of last cumu compaction failure diff --git a/be/src/olap/tablet_meta.cpp b/be/src/olap/tablet_meta.cpp index c8a0387082c9a2..f14622081d91f8 100644 --- a/be/src/olap/tablet_meta.cpp +++ b/be/src/olap/tablet_meta.cpp @@ -36,6 +36,7 @@ #include "olap/file_header.h" #include "olap/olap_common.h" #include "olap/olap_define.h" +#include "olap/rowset/rowset.h" #include "olap/tablet_meta_manager.h" #include "olap/utils.h" #include "util/debug_points.h" @@ -727,6 +728,12 @@ Status TabletMeta::add_rs_meta(const RowsetMetaSharedPtr& rs_meta) { return Status::OK(); } +void TabletMeta::add_rowsets_unchecked(const std::vector& to_add) { + for (const auto& rs : to_add) { + _rs_metas.push_back(rs->rowset_meta()); + } +} + void TabletMeta::delete_rs_meta_by_version(const Version& version, std::vector* deleted_rs_metas) { auto it = _rs_metas.begin(); diff --git a/be/src/olap/tablet_meta.h b/be/src/olap/tablet_meta.h index db7912a452c083..094bb21507d5c1 100644 --- a/be/src/olap/tablet_meta.h +++ b/be/src/olap/tablet_meta.h @@ -202,6 +202,12 @@ class TabletMeta { // used for after tablet cloned to clear stale rowset void clear_stale_rowset() { _stale_rs_metas.clear(); } + void clear_rowsets() { _rs_metas.clear(); } + + // MUST hold EXCLUSIVE `_meta_lock` in belonged Tablet + // `to_add` MUST NOT have overlapped version with `_rs_metas` in tablet meta. + void add_rowsets_unchecked(const std::vector& to_add); + bool all_beta() const; int64_t storage_policy_id() const { return _storage_policy_id; } diff --git a/be/src/vec/exec/scan/new_olap_scan_node.cpp b/be/src/vec/exec/scan/new_olap_scan_node.cpp index e1f39f2948b314..f7fe7f813f231c 100644 --- a/be/src/vec/exec/scan/new_olap_scan_node.cpp +++ b/be/src/vec/exec/scan/new_olap_scan_node.cpp @@ -528,16 +528,12 @@ Status NewOlapScanNode::_init_scanners(std::list* scanners) { if (_shared_scan_opt) { auto& read_source = tablets_read_source.emplace_back(); - { - std::shared_lock rdlock(tablet->get_header_lock()); - auto st = - tablet->capture_rs_readers({0, version}, &read_source.rs_splits, false); - if (!st.ok()) { - LOG(WARNING) << "fail to init reader.res=" << st; - return Status::InternalError( - "failed to initialize storage reader. tablet_id={} : {}", - tablet->tablet_id(), st.to_string()); - } + auto st = tablet->capture_rs_readers({0, version}, &read_source.rs_splits, false); + if (!st.ok()) { + LOG(WARNING) << "fail to init reader.res=" << st; + return Status::InternalError( + "failed to initialize storage reader. tablet_id={} : {}", + tablet->tablet_id(), st.to_string()); } if (!_state->skip_delete_predicate()) { read_source.fill_delete_predicates(); diff --git a/be/src/vec/exec/scan/new_olap_scanner.cpp b/be/src/vec/exec/scan/new_olap_scanner.cpp index bc15cf7207fb80..20938ecb8ba2ce 100644 --- a/be/src/vec/exec/scan/new_olap_scanner.cpp +++ b/be/src/vec/exec/scan/new_olap_scanner.cpp @@ -182,15 +182,12 @@ Status NewOlapScanner::init() { // to prevent this case: when there are lots of olap scanners to run for example 10000 // the rowsets maybe compacted when the last olap scanner starts ReadSource read_source; - { - std::shared_lock rdlock(tablet->get_header_lock()); - auto st = tablet->capture_rs_readers(_tablet_reader_params.version, - &read_source.rs_splits, - _state->skip_missing_version()); - if (!st.ok()) { - LOG(WARNING) << "fail to init reader.res=" << st; - return st; - } + auto st = tablet->capture_rs_readers(_tablet_reader_params.version, + &read_source.rs_splits, + _state->skip_missing_version()); + if (!st.ok()) { + LOG(WARNING) << "fail to init reader.res=" << st; + return st; } if (!_state->skip_delete_predicate()) { read_source.fill_delete_predicates(); From 6ab8a0784d275aef39acb8d6940c45d29436f2c0 Mon Sep 17 00:00:00 2001 From: walter Date: Thu, 18 Jan 2024 11:42:35 +0800 Subject: [PATCH 080/200] [feature](merge-cloud) Publish meta service and recycler service (#30054) Co-authored-by: plat1ko Co-authored-by: Gavin Chou Co-authored-by: Xiaocc <598887962@qq.com> Co-authored-by: deardeng <565620795@qq.com> Co-authored-by: meiyi Co-authored-by: Luwei <814383175@qq.com> Co-authored-by: Xin Liao Co-authored-by: Lei Zhang <1091517373@qq.com> Co-authored-by: Lightman <31928846+Lchangliang@users.noreply.github.com> Co-authored-by: YueW <45946325+Tanya-W@users.noreply.github.com> Co-authored-by: HHoflittlefish777 <77738092+HHoflittlefish777@users.noreply.github.com> Co-authored-by: Siyang Tang <82279870+TangSiyang2001@users.noreply.github.com> Co-authored-by: bobhan1 Co-authored-by: Kidd <107781942+k-i-d-d@users.noreply.github.com> Co-authored-by: Tanya-W Co-authored-by: panDing19 <56944854+panDing19@users.noreply.github.com> Co-authored-by: AlexYue Co-authored-by: abmdocrt --- .github/workflows/code-checks.yml | 11 +- .gitignore | 5 + build.sh | 54 +- cloud/CMakeLists.txt | 495 ++ cloud/cmake/cppcheck.cmake | 25 + cloud/cmake/thirdparty.cmake | 162 + cloud/conf/doris_cloud.conf | 47 + cloud/conf/fdb.cluster | 18 + cloud/conf/lsan_suppression.conf | 20 + cloud/script/run_all_tests.sh | 124 + cloud/script/start.sh | 84 + cloud/script/stop.sh | 35 + cloud/src/common/CMakeLists.txt | 26 + cloud/src/common/arg_parser.h | 224 + cloud/src/common/bvars.cpp | 180 + cloud/src/common/bvars.h | 237 + cloud/src/common/config.h | 148 + cloud/src/common/configbase.cpp | 331 ++ cloud/src/common/configbase.h | 173 + cloud/src/common/encryption_util.cpp | 836 +++ cloud/src/common/encryption_util.h | 108 + cloud/src/common/jemalloc_hook.cpp | 131 + cloud/src/common/kms.cpp | 165 + cloud/src/common/kms.h | 119 + cloud/src/common/logging.cpp | 135 + cloud/src/common/logging.h | 96 + cloud/src/common/metric.cpp | 272 + cloud/src/common/metric.h | 52 + cloud/src/common/simple_sync_queue.h | 123 + cloud/src/common/simple_thread_pool.h | 164 + cloud/src/common/stopwatch.h | 74 + cloud/src/common/string_util.h | 47 + cloud/src/common/sync_point.cpp | 240 + cloud/src/common/sync_point.h | 215 + cloud/src/common/util.cpp | 345 ++ cloud/src/common/util.h | 138 + cloud/src/gen-cpp/CMakeLists.txt | 18 + cloud/src/main.cpp | 335 ++ cloud/src/meta-service/CMakeLists.txt | 27 + cloud/src/meta-service/codec.cpp | 133 + cloud/src/meta-service/codec.h | 78 + cloud/src/meta-service/doris_txn.cpp | 65 + cloud/src/meta-service/doris_txn.h | 33 + cloud/src/meta-service/http_encode_key.cpp | 268 + cloud/src/meta-service/keys.cpp | 485 ++ cloud/src/meta-service/keys.h | 276 + cloud/src/meta-service/mem_txn_kv.cpp | 449 ++ cloud/src/meta-service/mem_txn_kv.h | 246 + cloud/src/meta-service/meta_server.cpp | 196 + cloud/src/meta-service/meta_server.h | 96 + cloud/src/meta-service/meta_service.cpp | 1884 ++++++ cloud/src/meta-service/meta_service.h | 642 ++ cloud/src/meta-service/meta_service_helper.h | 195 + cloud/src/meta-service/meta_service_http.cpp | 517 ++ cloud/src/meta-service/meta_service_http.h | 77 + cloud/src/meta-service/meta_service_job.cpp | 1229 ++++ .../meta-service/meta_service_partition.cpp | 590 ++ .../meta-service/meta_service_resource.cpp | 2980 ++++++++++ .../src/meta-service/meta_service_schema.cpp | 122 + cloud/src/meta-service/meta_service_schema.h | 33 + .../meta_service_tablet_stats.cpp | 130 + .../meta-service/meta_service_tablet_stats.h | 49 + cloud/src/meta-service/meta_service_txn.cpp | 1756 ++++++ cloud/src/meta-service/txn_kv.cpp | 544 ++ cloud/src/meta-service/txn_kv.h | 463 ++ cloud/src/meta-service/txn_kv_error.h | 79 + cloud/src/rate-limiter/CMakeLists.txt | 12 + cloud/src/rate-limiter/rate_limiter.cpp | 113 + cloud/src/rate-limiter/rate_limiter.h | 86 + cloud/src/recycler/CMakeLists.txt | 23 + cloud/src/recycler/checker.cpp | 626 ++ cloud/src/recycler/checker.h | 97 + cloud/src/recycler/meta_checker.cpp | 445 ++ cloud/src/recycler/meta_checker.h | 56 + cloud/src/recycler/recycler.cpp | 2363 ++++++++ cloud/src/recycler/recycler.h | 200 + cloud/src/recycler/recycler_service.cpp | 408 ++ cloud/src/recycler/recycler_service.h | 53 + cloud/src/recycler/s3_accessor.cpp | 424 ++ cloud/src/recycler/s3_accessor.h | 138 + cloud/src/recycler/util.cpp | 224 + cloud/src/recycler/util.h | 72 + cloud/src/recycler/white_black_list.cpp | 44 + cloud/src/recycler/white_black_list.h | 37 + cloud/src/resource-manager/CMakeLists.txt | 12 + .../src/resource-manager/resource_manager.cpp | 981 ++++ cloud/src/resource-manager/resource_manager.h | 148 + cloud/src/s3_accessor_test.cpp | 827 +++ cloud/test/CMakeLists.txt | 121 + cloud/test/codec_test.cpp | 202 + cloud/test/doris_txn_test.cpp | 100 + cloud/test/encryption_test.cpp | 371 ++ cloud/test/fdb_injection_test.cpp | 838 +++ cloud/test/fdb_metric_example.json | 3058 ++++++++++ cloud/test/http_encode_key_test.cpp | 537 ++ cloud/test/keys_test.cpp | 988 ++++ cloud/test/log_test.cpp | 110 + cloud/test/mem_txn_kv_test.cpp | 940 +++ cloud/test/meta_server_test.cpp | 219 + cloud/test/meta_service_http_test.cpp | 1347 +++++ cloud/test/meta_service_job_test.cpp | 2191 +++++++ cloud/test/meta_service_test.cpp | 5213 +++++++++++++++++ cloud/test/metric_test.cpp | 175 + cloud/test/mock_accessor.h | 142 + cloud/test/mock_resource_manager.h | 77 + cloud/test/rate_limiter_test.cpp | 112 + cloud/test/recycler_test.cpp | 2422 ++++++++ cloud/test/resource_test.cpp | 463 ++ cloud/test/s3_accessor_test.cpp | 829 +++ cloud/test/schema_kv_test.cpp | 606 ++ cloud/test/stopwatch_test.cpp | 51 + cloud/test/sync_point_test.cpp | 340 ++ cloud/test/txn_kv_test.cpp | 547 ++ gensrc/script/gen_build_version.sh | 64 +- run-cloud-ut.sh | 202 + thirdparty/build-thirdparty.sh | 40 + thirdparty/vars.sh | 21 + 117 files changed, 49361 insertions(+), 8 deletions(-) create mode 100644 cloud/CMakeLists.txt create mode 100644 cloud/cmake/cppcheck.cmake create mode 100644 cloud/cmake/thirdparty.cmake create mode 100644 cloud/conf/doris_cloud.conf create mode 100644 cloud/conf/fdb.cluster create mode 100644 cloud/conf/lsan_suppression.conf create mode 100644 cloud/script/run_all_tests.sh create mode 100644 cloud/script/start.sh create mode 100644 cloud/script/stop.sh create mode 100644 cloud/src/common/CMakeLists.txt create mode 100644 cloud/src/common/arg_parser.h create mode 100644 cloud/src/common/bvars.cpp create mode 100644 cloud/src/common/bvars.h create mode 100644 cloud/src/common/config.h create mode 100644 cloud/src/common/configbase.cpp create mode 100644 cloud/src/common/configbase.h create mode 100644 cloud/src/common/encryption_util.cpp create mode 100644 cloud/src/common/encryption_util.h create mode 100644 cloud/src/common/jemalloc_hook.cpp create mode 100644 cloud/src/common/kms.cpp create mode 100644 cloud/src/common/kms.h create mode 100644 cloud/src/common/logging.cpp create mode 100644 cloud/src/common/logging.h create mode 100644 cloud/src/common/metric.cpp create mode 100644 cloud/src/common/metric.h create mode 100644 cloud/src/common/simple_sync_queue.h create mode 100644 cloud/src/common/simple_thread_pool.h create mode 100644 cloud/src/common/stopwatch.h create mode 100644 cloud/src/common/string_util.h create mode 100644 cloud/src/common/sync_point.cpp create mode 100644 cloud/src/common/sync_point.h create mode 100644 cloud/src/common/util.cpp create mode 100644 cloud/src/common/util.h create mode 100644 cloud/src/gen-cpp/CMakeLists.txt create mode 100644 cloud/src/main.cpp create mode 100644 cloud/src/meta-service/CMakeLists.txt create mode 100644 cloud/src/meta-service/codec.cpp create mode 100644 cloud/src/meta-service/codec.h create mode 100644 cloud/src/meta-service/doris_txn.cpp create mode 100644 cloud/src/meta-service/doris_txn.h create mode 100644 cloud/src/meta-service/http_encode_key.cpp create mode 100644 cloud/src/meta-service/keys.cpp create mode 100644 cloud/src/meta-service/keys.h create mode 100644 cloud/src/meta-service/mem_txn_kv.cpp create mode 100644 cloud/src/meta-service/mem_txn_kv.h create mode 100644 cloud/src/meta-service/meta_server.cpp create mode 100644 cloud/src/meta-service/meta_server.h create mode 100644 cloud/src/meta-service/meta_service.cpp create mode 100644 cloud/src/meta-service/meta_service.h create mode 100644 cloud/src/meta-service/meta_service_helper.h create mode 100644 cloud/src/meta-service/meta_service_http.cpp create mode 100644 cloud/src/meta-service/meta_service_http.h create mode 100644 cloud/src/meta-service/meta_service_job.cpp create mode 100644 cloud/src/meta-service/meta_service_partition.cpp create mode 100644 cloud/src/meta-service/meta_service_resource.cpp create mode 100644 cloud/src/meta-service/meta_service_schema.cpp create mode 100644 cloud/src/meta-service/meta_service_schema.h create mode 100644 cloud/src/meta-service/meta_service_tablet_stats.cpp create mode 100644 cloud/src/meta-service/meta_service_tablet_stats.h create mode 100644 cloud/src/meta-service/meta_service_txn.cpp create mode 100644 cloud/src/meta-service/txn_kv.cpp create mode 100644 cloud/src/meta-service/txn_kv.h create mode 100644 cloud/src/meta-service/txn_kv_error.h create mode 100644 cloud/src/rate-limiter/CMakeLists.txt create mode 100644 cloud/src/rate-limiter/rate_limiter.cpp create mode 100644 cloud/src/rate-limiter/rate_limiter.h create mode 100644 cloud/src/recycler/CMakeLists.txt create mode 100644 cloud/src/recycler/checker.cpp create mode 100644 cloud/src/recycler/checker.h create mode 100644 cloud/src/recycler/meta_checker.cpp create mode 100644 cloud/src/recycler/meta_checker.h create mode 100644 cloud/src/recycler/recycler.cpp create mode 100644 cloud/src/recycler/recycler.h create mode 100644 cloud/src/recycler/recycler_service.cpp create mode 100644 cloud/src/recycler/recycler_service.h create mode 100644 cloud/src/recycler/s3_accessor.cpp create mode 100644 cloud/src/recycler/s3_accessor.h create mode 100644 cloud/src/recycler/util.cpp create mode 100644 cloud/src/recycler/util.h create mode 100644 cloud/src/recycler/white_black_list.cpp create mode 100644 cloud/src/recycler/white_black_list.h create mode 100644 cloud/src/resource-manager/CMakeLists.txt create mode 100644 cloud/src/resource-manager/resource_manager.cpp create mode 100644 cloud/src/resource-manager/resource_manager.h create mode 100644 cloud/src/s3_accessor_test.cpp create mode 100644 cloud/test/CMakeLists.txt create mode 100644 cloud/test/codec_test.cpp create mode 100644 cloud/test/doris_txn_test.cpp create mode 100644 cloud/test/encryption_test.cpp create mode 100644 cloud/test/fdb_injection_test.cpp create mode 100644 cloud/test/fdb_metric_example.json create mode 100644 cloud/test/http_encode_key_test.cpp create mode 100644 cloud/test/keys_test.cpp create mode 100644 cloud/test/log_test.cpp create mode 100644 cloud/test/mem_txn_kv_test.cpp create mode 100644 cloud/test/meta_server_test.cpp create mode 100644 cloud/test/meta_service_http_test.cpp create mode 100644 cloud/test/meta_service_job_test.cpp create mode 100644 cloud/test/meta_service_test.cpp create mode 100644 cloud/test/metric_test.cpp create mode 100644 cloud/test/mock_accessor.h create mode 100644 cloud/test/mock_resource_manager.h create mode 100644 cloud/test/rate_limiter_test.cpp create mode 100644 cloud/test/recycler_test.cpp create mode 100644 cloud/test/resource_test.cpp create mode 100644 cloud/test/s3_accessor_test.cpp create mode 100644 cloud/test/schema_kv_test.cpp create mode 100644 cloud/test/stopwatch_test.cpp create mode 100644 cloud/test/sync_point_test.cpp create mode 100644 cloud/test/txn_kv_test.cpp create mode 100755 run-cloud-ut.sh diff --git a/.github/workflows/code-checks.yml b/.github/workflows/code-checks.yml index 9e314bfb6dfc25..59a05a39fb76e4 100644 --- a/.github/workflows/code-checks.yml +++ b/.github/workflows/code-checks.yml @@ -79,15 +79,16 @@ jobs: id: filter with: filters: | - be_changes: + cpp_changes: - 'be/**' + - 'cloud/**' - 'gensrc/proto/**' - 'gensrc/thrift/**' - name: Generate compile_commands.json id: generate run: | - if [[ "${{ steps.filter.outputs.be_changes }}" == 'true' ]]; then + if [[ "${{ steps.filter.outputs.cpp_changes }}" == 'true' ]]; then export DEFAULT_DIR='/opt/doris' mkdir "${DEFAULT_DIR}" @@ -104,14 +105,14 @@ jobs: popd export PATH="${DEFAULT_DIR}/ldb-toolchain/bin/:$(pwd)/thirdparty/installed/bin/:${PATH}" - DISABLE_JAVA_UDF=ON DORIS_TOOLCHAIN=clang ENABLE_PCH=OFF OUTPUT_BE_BINARY=0 ./build.sh --be + DISABLE_JAVA_UDF=ON DORIS_TOOLCHAIN=clang ENABLE_PCH=OFF OUTPUT_BE_BINARY=0 ./build.sh --be --cloud fi - echo "should_check=${{ steps.filter.outputs.be_changes }}" >>${GITHUB_OUTPUT} + echo "should_check=${{ steps.filter.outputs.cpp_changes }}" >>${GITHUB_OUTPUT} - name: Upload uses: actions/upload-artifact@v3 - if: ${{ steps.filter.outputs.be_changes == 'true' }} + if: ${{ steps.filter.outputs.cpp_changes == 'true' }} with: name: compile_commands path: ./be/build_Release/compile_commands.json diff --git a/.gitignore b/.gitignore index 9a35bbe225822d..a8ad35b55d502e 100644 --- a/.gitignore +++ b/.gitignore @@ -99,6 +99,11 @@ be/.devcontainer/ be/src/apache-orc/ zoneinfo/ +# Cloud +cloud/build*/ +cloud/cmake-build*/ +cloud/ut_build*/ + ## tools tools/ssb-tools/ssb-data/ tools/ssb-tools/ssb-dbgen/ diff --git a/build.sh b/build.sh index 3b819929d00d39..86c869339983b7 100755 --- a/build.sh +++ b/build.sh @@ -42,6 +42,7 @@ Usage: $0 --fe build Frontend and Spark DPP application. Default ON. --be build Backend. Default ON. --meta-tool build Backend meta tool. Default OFF. + --cloud build Cloud. Default OFF. --index-tool build Backend inverted index tool. Default OFF. --broker build Broker. Default ON. --audit build audit loader. Default ON. @@ -61,6 +62,7 @@ Usage: $0 $0 build all $0 --be build Backend $0 --meta-tool build Backend meta tool + $0 --cloud build Cloud $0 --index-tool build Backend inverted index tool $0 --fe --clean clean and build Frontend and Spark Dpp application $0 --fe --be --clean clean and build Frontend, Spark Dpp application and Backend @@ -117,6 +119,7 @@ if ! OPTS="$(getopt \ -o '' \ -l 'fe' \ -l 'be' \ + -l 'cloud' \ -l 'broker' \ -l 'audit' \ -l 'meta-tool' \ @@ -138,6 +141,7 @@ eval set -- "${OPTS}" PARALLEL="$(($(nproc) / 4 + 1))" BUILD_FE=0 BUILD_BE=0 +BUILD_CLOUD=0 BUILD_BROKER=0 BUILD_AUDIT=0 BUILD_META_TOOL='OFF' @@ -154,6 +158,7 @@ if [[ "$#" == 1 ]]; then # default BUILD_FE=1 BUILD_BE=1 + BUILD_BROKER=1 BUILD_AUDIT=1 BUILD_META_TOOL='OFF' @@ -177,6 +182,10 @@ else BUILD_BE_JAVA_EXTENSIONS=1 shift ;; + --cloud) + BUILD_CLOUD=1 + shift + ;; --broker) BUILD_BROKER=1 shift @@ -244,6 +253,7 @@ else if [[ "${PARAMETER_COUNT}" -eq 3 ]] && [[ "${PARAMETER_FLAG}" -eq 1 ]]; then BUILD_FE=1 BUILD_BE=1 + BUILD_CLOUD=1 BUILD_BROKER=1 BUILD_AUDIT=1 BUILD_META_TOOL='ON' @@ -307,7 +317,7 @@ update_submodule() { fi } -if [[ "${CLEAN}" -eq 1 && "${BUILD_BE}" -eq 0 && "${BUILD_FE}" -eq 0 && "${BUILD_SPARK_DPP}" -eq 0 ]]; then +if [[ "${CLEAN}" -eq 1 && "${BUILD_BE}" -eq 0 && "${BUILD_FE}" -eq 0 && "${BUILD_SPARK_DPP}" -eq 0 && ${BUILD_CLOUD} -eq 0 ]]; then clean_gensrc clean_be clean_fe @@ -415,6 +425,7 @@ fi echo "Get params: BUILD_FE -- ${BUILD_FE} BUILD_BE -- ${BUILD_BE} + BUILD_CLOUD -- ${BUILD_CLOUD} BUILD_BROKER -- ${BUILD_BROKER} BUILD_AUDIT -- ${BUILD_AUDIT} BUILD_META_TOOL -- ${BUILD_META_TOOL} @@ -541,6 +552,42 @@ if [[ "${BUILD_BE}" -eq 1 ]]; then cd "${DORIS_HOME}" fi +# Clean and build cloud +if [[ "${BUILD_CLOUD}" -eq 1 ]]; then + if [[ -e "${DORIS_HOME}/gensrc/build/gen_cpp/cloud_version.h" ]]; then + rm -f "${DORIS_HOME}/gensrc/build/gen_cpp/cloud_version.h" + fi + CMAKE_BUILD_TYPE="${BUILD_TYPE:-Release}" + echo "Build Cloud: ${CMAKE_BUILD_TYPE}" + CMAKE_BUILD_DIR="${DORIS_HOME}/cloud/build_${CMAKE_BUILD_TYPE}" + if [[ "${CLEAN}" -eq 1 ]]; then + rm -rf "${CMAKE_BUILD_DIR}" + echo "clean cloud" + fi + MAKE_PROGRAM="$(command -v "${BUILD_SYSTEM}")" + echo "-- Make program: ${MAKE_PROGRAM}" + echo "-- Extra cxx flags: ${EXTRA_CXX_FLAGS:-}" + mkdir -p "${CMAKE_BUILD_DIR}" + cd "${CMAKE_BUILD_DIR}" + "${CMAKE_CMD}" -G "${GENERATOR}" \ + -DCMAKE_MAKE_PROGRAM="${MAKE_PROGRAM}" \ + -DCMAKE_EXPORT_COMPILE_COMMANDS=ON \ + -DCMAKE_BUILD_TYPE="${CMAKE_BUILD_TYPE}" \ + -DMAKE_TEST=OFF \ + "${CMAKE_USE_CCACHE}" \ + -DUSE_LIBCPP="${USE_LIBCPP}" \ + -DSTRIP_DEBUG_INFO="${STRIP_DEBUG_INFO}" \ + -DUSE_DWARF="${USE_DWARF}" \ + -DUSE_JEMALLOC="${USE_JEMALLOC}" \ + -DEXTRA_CXX_FLAGS="${EXTRA_CXX_FLAGS}" \ + -DBUILD_CHECK_META="${BUILD_CHECK_META:-OFF}" \ + "${DORIS_HOME}/cloud/" + "${BUILD_SYSTEM}" -j "${PARALLEL}" + "${BUILD_SYSTEM}" install + cd "${DORIS_HOME}" + echo "Build cloud done" +fi + if [[ "${BUILD_DOCS}" = "ON" ]]; then # Build docs, should be built before Frontend echo "Build docs" @@ -754,6 +801,11 @@ if [[ "${BUILD_AUDIT}" -eq 1 ]]; then cd "${DORIS_HOME}" fi +if [[ ${BUILD_CLOUD} -eq 1 ]]; then + rm -rf "${DORIS_HOME}/output/ms" + cp -r -p "${DORIS_HOME}/cloud/output" "${DORIS_HOME}/output/ms" +fi + echo "***************************************" echo "Successfully build Doris" echo "***************************************" diff --git a/cloud/CMakeLists.txt b/cloud/CMakeLists.txt new file mode 100644 index 00000000000000..56931d8850aa25 --- /dev/null +++ b/cloud/CMakeLists.txt @@ -0,0 +1,495 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +cmake_minimum_required(VERSION 3.19.2) + +project(doris_cloud CXX C) + +# Write compile_commands.json +set(CMAKE_EXPORT_COMPILE_COMMANDS ON) + +list(APPEND CMAKE_MODULE_PATH "${CMAKE_CURRENT_SOURCE_DIR}/cmake") +include(cppcheck) + +# set platforms +if (CMAKE_SYSTEM_PROCESSOR MATCHES "amd64|x86_64") + set (ARCH_AMD64 1) +endif () +if (CMAKE_SYSTEM_PROCESSOR MATCHES "^(aarch64.*|AARCH64.*)") + set (ARCH_AARCH64 1) +endif () +if (ARCH_AARCH64 OR CMAKE_SYSTEM_PROCESSOR MATCHES "arm") + set (ARCH_ARM 1) +endif () +if (CMAKE_LIBRARY_ARCHITECTURE MATCHES "i386") + set (ARCH_I386 1) +endif () +if ((ARCH_ARM AND NOT ARCH_AARCH64) OR ARCH_I386) + message (FATAL_ERROR "32bit platforms are not supported") +endif () + +if (CMAKE_SYSTEM_PROCESSOR MATCHES "^(ppc64le.*|PPC64LE.*)") + set (ARCH_PPC64LE 1) +endif () + +if (CMAKE_SYSTEM_NAME MATCHES "Linux") + set (OS_LINUX 1) + add_definitions(-D OS_LINUX) +elseif (CMAKE_SYSTEM_NAME MATCHES "Darwin") + set (OS_MACOSX 1) + add_definitions(-D OS_MACOSX) +endif () + +if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") + set (COMPILER_GCC 1) +elseif (CMAKE_CXX_COMPILER_ID MATCHES "Clang") + set (COMPILER_CLANG 1) +endif () + +# set CMAKE_BUILD_TYPE +if (NOT CMAKE_BUILD_TYPE) + set(CMAKE_BUILD_TYPE RELEASE) +endif() + +string(TOUPPER ${CMAKE_BUILD_TYPE} CMAKE_BUILD_TYPE) +message(STATUS "Build type is ${CMAKE_BUILD_TYPE}") + +# set CMAKE_BUILD_TARGET_ARCH +# use `lscpu | grep 'Architecture' | awk '{print $2}'` only support system which language is en_US.UTF-8 +execute_process(COMMAND bash "-c" "uname -m" + OUTPUT_VARIABLE + CMAKE_BUILD_TARGET_ARCH + OUTPUT_STRIP_TRAILING_WHITESPACE) +message(STATUS "Build target arch is ${CMAKE_BUILD_TARGET_ARCH}") + +# Set dirs +set(BASE_DIR "${CMAKE_CURRENT_SOURCE_DIR}") +set(BUILD_DIR "${CMAKE_CURRENT_BINARY_DIR}") +if (NOT DEFINED ENV{DORIS_BRANCH}) + set(THIRDPARTY_DIR "$ENV{DORIS_THIRDPARTY}/installed") +else() + set(THIRDPARTY_DIR "$ENV{DORIS_THIRDPARTY}/installed-$ENV{DORIS_BRANCH}") +endif() +set(GENSRC_DIR "${BASE_DIR}/../gensrc/build/") +set(SRC_DIR "${BASE_DIR}/src/") +set(TEST_DIR "${CMAKE_SOURCE_DIR}/test/") +set(OUTPUT_DIR "${BASE_DIR}/output") +set(THIRDPARTY_SRC "$ENV{DORIS_THIRDPARTY}/src") + +if (APPLE) + set(MAKE_TEST "ON") +else() + option(MAKE_TEST "ON for make unit test or OFF for not" OFF) +endif() +message(STATUS "make test: ${MAKE_TEST}") + +# Check gcc +if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") + if (CMAKE_CXX_COMPILER_VERSION VERSION_LESS "11.0") + message(FATAL_ERROR "Need GCC version at least 11.0") + endif() +endif() + +set(CMAKE_SKIP_RPATH TRUE) + +# Compile generated source if necessary +message(STATUS "build gensrc if necessary") +execute_process(COMMAND make -C ${BASE_DIR}/../gensrc/ + RESULT_VARIABLE MAKE_GENSRC_RESULT) +if(NOT ${MAKE_GENSRC_RESULT} EQUAL 0 AND NOT APPLE) + message(FATAL_ERROR "Failed to build ${BASE_DIR}/../gensrc/") +endif() + +set(GPERFTOOLS_HOME "${THIRDPARTY_DIR}/gperftools") + +# Set all libraries +include(thirdparty) + +find_program(THRIFT_COMPILER thrift ${CMAKE_SOURCE_DIR}/bin) + +# Check if functions are supported in this platform. All flags will generated +# in gensrc/build/common/env_config.h. +# You can check funcion here which depends on platform. Don't forget add this +# to be/src/common/env_config.h.in +include(CheckFunctionExists) +check_function_exists(sched_getcpu HAVE_SCHED_GETCPU) + +# compiler flags that are common across debug/release builds +# -Wall: Enable all warnings. +# -Wno-sign-compare: suppress warnings for comparison between signed and unsigned +# integers +# -pthread: enable multithreaded malloc +# -DBOOST_DATE_TIME_POSIX_TIME_STD_CONFIG: enable nanosecond precision for boost +# -fno-omit-frame-pointers: Keep frame pointer for functions in register +set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -Wall -Wno-sign-compare -pthread -Werror") +set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -fstrict-aliasing -fno-omit-frame-pointer") +set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -std=gnu++17 -D__STDC_FORMAT_MACROS") +set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -DBOOST_DATE_TIME_POSIX_TIME_STD_CONFIG") +set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -DBOOST_SYSTEM_NO_DEPRECATED") +# Enable the cpu and heap profile of brpc +set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -DBRPC_ENABLE_CPU_PROFILER") + +function(TRY_TO_CHANGE_LINKER LINKER_COMMAND LINKER_NAME) + if (CUSTUM_LINKER_COMMAND STREQUAL "ld") + execute_process(COMMAND ${CMAKE_C_COMPILER} -fuse-ld=${LINKER_COMMAND} -Wl,--version ERROR_QUIET OUTPUT_VARIABLE LD_VERSION) + if ("${LD_VERSION}" MATCHES ${LINKER_NAME}) + message("Linker ${LINKER_NAME} is available, change linker to ${LINKER_NAME}") + set(CUSTUM_LINKER_COMMAND "${LINKER_COMMAND}" PARENT_SCOPE) + endif() + endif() +endfunction() + +# In terms of performance, mold> lld> gold> ld +set(CUSTUM_LINKER_COMMAND "ld") +# TODO: mold will link fail on thirdparty brpc now, waiting for investigation. +# TRY_TO_CHANGE_LINKER("mold" "mold") +TRY_TO_CHANGE_LINKER("lld" "LLD") +TRY_TO_CHANGE_LINKER("gold" "GNU gold") +if (NOT CUSTUM_LINKER_COMMAND STREQUAL "ld") + set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=${CUSTUM_LINKER_COMMAND}") +endif() + +if (USE_LIBCPP AND COMPILER_CLANG) + set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -stdlib=libc++ -lstdc++") + add_definitions(-DUSE_LIBCPP) +endif() + +if (COMPILER_GCC) + # Avoid GCC 11 false alarm + # https://stackoverflow.com/questions/67584073/gcc-11-false-array-subscript-is-partly-outside-array-bounds-warning + # https://stackoverflow.com/questions/69426070/gcc-11-order-of-arguments-triggers-false-positive-wstringop-overflow-is-this-bu + set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -Wno-array-bounds -Wno-stringop-overread") + add_compile_options(-Wno-stringop-overflow -fdiagnostics-color=always) +endif () + +if (COMPILER_CLANG) + add_compile_options (-fcolor-diagnostics) + if(MAKE_TEST STREQUAL "OFF") + add_compile_options(-Qunused-arguments) + endif() +endif () + +# Compile with jemalloc. +# Adding the option `USE_JEMALLOC=ON sh build.sh` when compiling can turn on building with jemalloc +if (USE_JEMALLOC) + set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -DUSE_JEMALLOC") +endif() + +if (CMAKE_CXX_COMPILER_VERSION VERSION_GREATER 7.0) + set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -faligned-new") +endif() + +# For any gcc builds: +# -g: Enable symbols for profiler tools. Produce debugging information in the operating system’s native formt +# -Wno-unused-local-typedefs: Do not warn for local typedefs that are unused. +set(CXX_GCC_FLAGS "${CXX_GCC_FLAGS} -ggdb -Wno-unused-local-typedefs") + +if (USE_DWARF) + set(CXX_GCC_FLAGS "${CXX_GCC_FLAGS} -gdwarf-5") +endif() + +# For CMAKE_BUILD_TYPE=Debug +set(CXX_FLAGS_DEBUG "${CXX_GCC_FLAGS} -O0") + +# For CMAKE_BUILD_TYPE=Release +# -O3: Enable all compiler optimizations +# -DNDEBUG: Turn off dchecks/asserts/debug only code. +set(CXX_FLAGS_RELEASE "${CXX_GCC_FLAGS} -O3 -DNDEBUG") +SET(CXX_FLAGS_ASAN "${CXX_GCC_FLAGS} -O1 -fsanitize=address -DADDRESS_SANITIZER") +SET(CXX_FLAGS_LSAN "${CXX_GCC_FLAGS} -O0 -fsanitize=leak -DLEAK_SANITIZER") + +# Set the flags to the undefined behavior sanitizer, also known as "ubsan" +# Turn on sanitizer and debug symbols to get stack traces: +SET(CXX_FLAGS_UBSAN "${CXX_GCC_FLAGS} -O0 -fno-wrapv -fsanitize=undefined") + +# Set the flags to the thread sanitizer, also known as "tsan" +# Turn on sanitizer and debug symbols to get stack traces: +# Use -Wno-builtin-declaration-mismatch to mute warnings like "new declaration ‘__tsan_atomic16 __tsan_atomic16_fetch_nand(..." +# If use -O0 to compile, process may stack overflow when start. https://github.com/apache/doris/issues/8868 +SET(CXX_FLAGS_TSAN "${CXX_GCC_FLAGS} -O1 -fsanitize=thread -DTHREAD_SANITIZER -Wno-missing-declarations") + +# Set compile flags based on the build type. +if ("${CMAKE_BUILD_TYPE}" STREQUAL "DEBUG") + SET(CMAKE_CXX_FLAGS ${CXX_FLAGS_DEBUG}) +elseif ("${CMAKE_BUILD_TYPE}" STREQUAL "RELEASE") + SET(CMAKE_CXX_FLAGS ${CXX_FLAGS_RELEASE}) +elseif ("${CMAKE_BUILD_TYPE}" STREQUAL "ASAN") + SET(CMAKE_CXX_FLAGS "${CXX_FLAGS_ASAN}") +elseif ("${CMAKE_BUILD_TYPE}" STREQUAL "LSAN") + SET(CMAKE_CXX_FLAGS "${CXX_FLAGS_LSAN}") +elseif ("${CMAKE_BUILD_TYPE}" STREQUAL "UBSAN") + SET(CMAKE_CXX_FLAGS "${CXX_FLAGS_UBSAN}") +elseif ("${CMAKE_BUILD_TYPE}" STREQUAL "TSAN") + SET(CMAKE_CXX_FLAGS "${CXX_FLAGS_TSAN}") +else() + message(FATAL_ERROR "Unknown build type: ${CMAKE_BUILD_TYPE}") +endif() + +# Add flags that are common across build types +SET(CMAKE_CXX_FLAGS "${CXX_COMMON_FLAGS} ${CMAKE_CXX_FLAGS} ${EXTRA_CXX_FLAGS}") + +if(BUILD_CHECK_META STREQUAL "ON") + set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -DBUILD_CHECK_META") +endif() + +message(STATUS "Compiler Flags: ${CMAKE_CXX_FLAGS}") + +# Thrift requires these two definitions for some types that we use +add_definitions(-DHAVE_INTTYPES_H -DHAVE_NETINET_IN_H) + +# Set include dirs +include_directories( + ${SRC_DIR}/ + ${TEST_DIR}/ +) + +include_directories( + SYSTEM + ${GENSRC_DIR}/ + ${THIRDPARTY_DIR}/include + ${GPERFTOOLS_HOME}/include +) + +set(WL_START_GROUP "-Wl,--start-group") +set(WL_END_GROUP "-Wl,--end-group") + +# Set Doris libraries +set(DORIS_LINK_LIBS + ${WL_START_GROUP} + ${WL_END_GROUP} +) + +# COMMON_THIRDPARTY are thirdparty dependencies that can run on all platform +# When adding new dependencies, If you don’t know if it can run on all platforms, +# add it here first. +set(COMMON_THIRDPARTY + brpc + leveldb + libz + gflags + glog + jemalloc + pprof + protobuf + thrift + openssl + crypto + rocksdb + zstd + curl + ${AWS_LIBS} + fmt + jsoncpp + libuuid + ali-sdk +) + +set(DORIS_DEPENDENCIES + ${DORIS_DEPENDENCIES} + ${WL_START_GROUP} + ${COMMON_THIRDPARTY} + ${WL_END_GROUP} +) + +# set(DORIS_DEPENDENCIES ${DORIS_DEPENDENCIES} ${WL_END_GROUP}) + +message(STATUS "DORIS_DEPENDENCIES is ${DORIS_DEPENDENCIES}") + +# Add all external dependencies. They should come after the project's libs. +# static link gcc's lib +set(DORIS_LINK_LIBS ${DORIS_LINK_LIBS} + ${WL_START_GROUP} + CloudGen + Common + MetaService + ResourceManager + Recycler + RateLimiter + ${WL_END_GROUP} + ${DORIS_DEPENDENCIES} + -static-libstdc++ + -static-libgcc + -lstdc++fs +) + +if (USE_JEMALLOC) + set(MALLOCLIB jemalloc) +else () + set(MALLOCLIB tcmalloc) +endif() + +if ("${CMAKE_CXX_COMPILER_ID}" STREQUAL "GNU") + set(ASAN_LIBS -static-libasan) + set(LSAN_LIBS -static-liblsan) + set(UBSAN_LIBS -static-libubsan ${MALLOCLIB}) + set(TSAN_LIBS -static-libtsan) +else () + set(UBSAN_LIBS -rtlib=compiler-rt ${MALLOCLIB}) +endif () + +# Add sanitize static link flags or tcmalloc +if ("${CMAKE_BUILD_TYPE}" STREQUAL "DEBUG" OR "${CMAKE_BUILD_TYPE}" STREQUAL "RELEASE") + set(DORIS_LINK_LIBS ${DORIS_LINK_LIBS} ${MALLOCLIB}) +elseif ("${CMAKE_BUILD_TYPE}" STREQUAL "ASAN") + set(DORIS_LINK_LIBS ${DORIS_LINK_LIBS} ${ASAN_LIBS}) +elseif ("${CMAKE_BUILD_TYPE}" STREQUAL "LSAN") + set(DORIS_LINK_LIBS ${DORIS_LINK_LIBS} ${LSAN_LIBS}) +elseif ("${CMAKE_BUILD_TYPE}" STREQUAL "UBSAN") + set(DORIS_LINK_LIBS ${DORIS_LINK_LIBS} ${UBSAN_LIBS}) +elseif ("${CMAKE_BUILD_TYPE}" STREQUAL "TSAN") + set(DORIS_LINK_LIBS ${DORIS_LINK_LIBS} ${TSAN_LIBS}) + add_definitions("-DTHREAD_SANITIZER") +else() + message(FATAL_ERROR "Unknown build type: ${CMAKE_BUILD_TYPE}") +endif() + +set(DORIS_LINK_LIBS ${DORIS_LINK_LIBS} + -lrt -l:libbfd.a -liberty -lc -lm -ldl -pthread +) + +if(BUILD_CHECK_META STREQUAL "ON") + set(DORIS_LINK_LIBS ${DORIS_LINK_LIBS} -lmysqlclient) +endif () + +# Set libraries for test +set (TEST_LINK_LIBS ${DORIS_LINK_LIBS} + ${WL_START_GROUP} + gmock + gtest + ${WL_END_GROUP} +) + +# Only build static libs +set(BUILD_SHARED_LIBS OFF) + +option(ENABLE_CLANG_COVERAGE "coverage option" OFF) +if (ENABLE_CLANG_COVERAGE AND ENABLE_CLANG_COVERAGE STREQUAL ON AND COMPILER_CLANG) + SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fprofile-instr-generate -fcoverage-mapping") +endif () + +if (${MAKE_TEST} STREQUAL "ON") + SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -DUNIT_TEST -fprofile-arcs -ftest-coverage -fno-access-control -DGTEST_USE_OWN_TR1_TUPLE=0") + SET(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fprofile-arcs -ftest-coverage -lgcov") + add_definitions(-DBE_TEST) +endif () + +# Add libs if needed, download to current dir -- ${BUILD_DIR} +set(FDB_LIB "fdb_lib_7_1_23.tar.xz") +file(GLOB RELEASE_FILE_LIST LIST_DIRECTORIES false "/etc/*release*") +execute_process(COMMAND "cat" ${RELEASE_FILE_LIST} + RESULT_VARIABLE CAT_RET_CODE + OUTPUT_VARIABLE CAT_RET_CONTENT) +string(TOUPPER "${CAT_RET_CONTENT}" CAT_RET_CONTENT) +if ("${CAT_RET_CONTENT}" MATCHES "UBUNTU") + message("Ubuntu OS") + SET(OS_RELEASE "Ubuntu") + set(FDB_LIB_URL "https://doris-build.oss-cn-beijing.aliyuncs.com/thirdparty/fdb/ubuntu/") + string(APPEND FDB_LIB_URL "${FDB_LIB}") + set(FDB_LIB_MD5SUM "a00fe45da95cfac4e0caffa274bb2b30") +else() + # If it is not ubuntu, it is regarded as centos by default + message("Centos OS") + SET(OS_RELEASE "Centos") + set(FDB_LIB_URL "https://doris-build.oss-cn-beijing.aliyuncs.com/thirdparty/fdb/centos/") + string(APPEND FDB_LIB_URL "${FDB_LIB}") + set(FDB_LIB_MD5SUM "f9839a564849c0232a351143b4340de0") +endif() + +if (NOT EXISTS "${THIRDPARTY_SRC}/${FDB_LIB}") + file(MAKE_DIRECTORY ${THIRDPARTY_SRC}) + execute_process(COMMAND "curl" "${FDB_LIB_URL}" + "-o" "${THIRDPARTY_SRC}/${FDB_LIB}" "-k" + RESULTS_VARIABLE DOWNLOAD_RET) + if (NOT ${DOWNLOAD_RET} STREQUAL "0") + execute_process(COMMAND "rm" "-rf" "${THIRDPARTY_SRC}/${FDB_LIB}") + message(FATAL_ERROR "Failed to download dependency of fdb ${FDB_LIB_URL}, remove it") + endif () +endif () + +# Add fdb dependencies +add_definitions(-DFDB_API_VERSION=710) +if (NOT EXISTS ${THIRDPARTY_DIR}/include/foundationdb) + execute_process(COMMAND "md5sum" "${THIRDPARTY_SRC}/${FDB_LIB}" + RESULT_VARIABLE MD5SUM_RET_CODE + OUTPUT_VARIABLE MD5SUM_CONTENT) + if (NOT "${MD5SUM_CONTENT}" MATCHES "${FDB_LIB_MD5SUM}") + execute_process(COMMAND "rm" "-rf" "${THIRDPARTY_SRC}/${FDB_LIB}") + message(FATAL_ERROR "${THIRDPARTY_SRC}/${FDB_LIB} md5sum check failed, remove it") + endif () + execute_process(COMMAND "tar" "xf" "${THIRDPARTY_SRC}/${FDB_LIB}" "-C" "${THIRDPARTY_DIR}/") +endif () + +add_subdirectory(${SRC_DIR}/common) +add_subdirectory(${SRC_DIR}/gen-cpp) +add_subdirectory(${SRC_DIR}/meta-service) +add_subdirectory(${SRC_DIR}/recycler) +add_subdirectory(${SRC_DIR}/rate-limiter) +add_subdirectory(${SRC_DIR}/resource-manager) + +if (${MAKE_TEST} STREQUAL "ON") + add_subdirectory(${TEST_DIR}) +endif () + +if (${MAKE_TEST} STREQUAL "OFF") + set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -lfdb_c -L${THIRDPARTY_DIR}/lib") + + add_executable(doris_cloud src/main.cpp) + + # This permits libraries loaded by dlopen to link to the symbols in the program. + set_target_properties(doris_cloud PROPERTIES ENABLE_EXPORTS 1) + target_link_libraries(doris_cloud ${DORIS_LINK_LIBS}) + + install(DIRECTORY DESTINATION ${OUTPUT_DIR}/lib) + install(TARGETS doris_cloud DESTINATION ${OUTPUT_DIR}/lib) + + if ("${STRIP_DEBUG_INFO}" STREQUAL "ON") + add_custom_command(TARGET doris_cloud POST_BUILD + COMMAND ${CMAKE_OBJCOPY} --only-keep-debug $ $.dbg + COMMAND ${CMAKE_STRIP} --strip-debug --strip-unneeded $ + COMMAND ${CMAKE_OBJCOPY} --add-gnu-debuglink=$.dbg $ + ) + + install(DIRECTORY DESTINATION ${OUTPUT_DIR}/lib/debug_info/) + install(FILES $.dbg DESTINATION ${OUTPUT_DIR}/lib/debug_info/) + endif() +endif() + + +message("install files at ${OUTPUT_DIR}") +install(DIRECTORY DESTINATION ${OUTPUT_DIR}) +install(DIRECTORY DESTINATION ${OUTPUT_DIR}/bin) +install(DIRECTORY DESTINATION ${OUTPUT_DIR}/conf) +install(DIRECTORY DESTINATION ${OUTPUT_DIR}/lib) + +install(FILES + ${BASE_DIR}/script/start.sh + ${BASE_DIR}/script/stop.sh + PERMISSIONS OWNER_READ OWNER_WRITE OWNER_EXECUTE + GROUP_READ GROUP_WRITE GROUP_EXECUTE + WORLD_READ WORLD_EXECUTE + DESTINATION ${OUTPUT_DIR}/bin) + +install(FILES + ${BASE_DIR}/conf/doris_cloud.conf + DESTINATION ${OUTPUT_DIR}/conf) + +install(FILES + ${THIRDPARTY_DIR}/lib/libfdb_c.so + PERMISSIONS OWNER_READ OWNER_WRITE OWNER_EXECUTE + GROUP_READ GROUP_WRITE GROUP_EXECUTE + WORLD_READ WORLD_EXECUTE + DESTINATION ${OUTPUT_DIR}/lib) diff --git a/cloud/cmake/cppcheck.cmake b/cloud/cmake/cppcheck.cmake new file mode 100644 index 00000000000000..0426f0e250449b --- /dev/null +++ b/cloud/cmake/cppcheck.cmake @@ -0,0 +1,25 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +find_program(CPPCHECK NAMES cppcheck) +if (NOT CPPCHECK) + message(STATUS "The check program cppcheck is not found") +else() + message(STATUS "Find program: ${CPPCHECK}, enable CMAKE_CXX_CPPCHECK") + set(CMAKE_CXX_CPPCHECK ${CPPCHECK}) + list(APPEND CMAKE_CXX_CPPCHECK "--enable=warning") +endif() + diff --git a/cloud/cmake/thirdparty.cmake b/cloud/cmake/thirdparty.cmake new file mode 100644 index 00000000000000..02c96a76fa6b8a --- /dev/null +++ b/cloud/cmake/thirdparty.cmake @@ -0,0 +1,162 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +add_library(gflags STATIC IMPORTED) +set_target_properties(gflags PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/libgflags.a) + +add_library(glog STATIC IMPORTED) +set_target_properties(glog PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/libglog.a) + +add_library(backtrace STATIC IMPORTED) +set_target_properties(backtrace PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/libbacktrace.a) + +add_library(pprof STATIC IMPORTED) +set_target_properties(pprof PROPERTIES IMPORTED_LOCATION + ${GPERFTOOLS_HOME}/lib/libprofiler.a) + +add_library(tcmalloc STATIC IMPORTED) +set_target_properties(tcmalloc PROPERTIES IMPORTED_LOCATION + ${GPERFTOOLS_HOME}/lib/libtcmalloc.a) + +add_library(protobuf STATIC IMPORTED) +set_target_properties(protobuf PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/libprotobuf.a) + +add_library(protoc STATIC IMPORTED) +set_target_properties(protoc PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/libprotoc.a) + +add_library(gtest STATIC IMPORTED) +set_target_properties(gtest PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/libgtest.a) + +add_library(gtest_main STATIC IMPORTED) +set_target_properties(gtest_main PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/libgtest_main.a) + +add_library(gmock STATIC IMPORTED) +set_target_properties(gmock PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/libgmock.a) + +add_library(thrift STATIC IMPORTED) +set_target_properties(thrift PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/libthrift.a) + +add_library(crypto STATIC IMPORTED) +set_target_properties(crypto PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/libcrypto.a) + +add_library(openssl STATIC IMPORTED) +set_target_properties(openssl PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/libssl.a) + +add_library(jemalloc STATIC IMPORTED) +set_target_properties(jemalloc PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/libjemalloc_doris.a) + +# Required by brpc +add_library(leveldb STATIC IMPORTED) +set_target_properties(leveldb PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/libleveldb.a) + +add_library(brpc STATIC IMPORTED) +set_target_properties(brpc PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libbrpc.a) + +# For local storage mocking +add_library(rocksdb STATIC IMPORTED) +set_target_properties(rocksdb PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/librocksdb.a) + +# Required by google::protobuf +add_library(libz STATIC IMPORTED) +set_target_properties(libz PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/libz.a) + +add_library(curl STATIC IMPORTED) +set_target_properties(curl PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/libcurl.a) + +add_library(zstd STATIC IMPORTED) +set_target_properties(zstd PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libzstd.a) + +add_library(aws-sdk-core STATIC IMPORTED) +set_target_properties(aws-sdk-core PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libaws-cpp-sdk-core.a) + +add_library(aws-sdk-s3 STATIC IMPORTED) +set_target_properties(aws-sdk-s3 PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libaws-cpp-sdk-s3.a) + +add_library(aws-sdk-transfer STATIC IMPORTED) +set_target_properties(aws-sdk-transfer PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libaws-cpp-sdk-transfer.a) + +add_library(aws-sdk-s3-crt STATIC IMPORTED) +set_target_properties(aws-sdk-s3-crt PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libaws-cpp-sdk-s3-crt.a) + +add_library(aws-crt-cpp STATIC IMPORTED) +set_target_properties(aws-crt-cpp PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libaws-crt-cpp.a) + +add_library(aws-c-cal STATIC IMPORTED) +set_target_properties(aws-c-cal PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libaws-c-cal.a) + +add_library(aws-c-auth STATIC IMPORTED) +set_target_properties(aws-c-auth PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libaws-c-auth.a) + +add_library(aws-c-compression STATIC IMPORTED) +set_target_properties(aws-c-compression PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libaws-c-compression.a) + +add_library(aws-c-common STATIC IMPORTED) +set_target_properties(aws-c-common PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libaws-c-common.a) + +add_library(aws-c-event-stream STATIC IMPORTED) +set_target_properties(aws-c-event-stream PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libaws-c-event-stream.a) + +add_library(aws-c-io STATIC IMPORTED) +set_target_properties(aws-c-io PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libaws-c-io.a) + +add_library(aws-c-http STATIC IMPORTED) +set_target_properties(aws-c-http PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libaws-c-http.a) + +add_library(aws-c-mqtt STATIC IMPORTED) +set_target_properties(aws-c-mqtt PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libaws-c-mqtt.a) + +add_library(aws-checksums STATIC IMPORTED) +set_target_properties(aws-checksums PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libaws-checksums.a) + +add_library(aws-c-s3 STATIC IMPORTED) +set_target_properties(aws-c-s3 PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libaws-c-s3.a) + +add_library(aws-s2n STATIC IMPORTED) +set_target_properties(aws-s2n PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libs2n.a) + +add_library(aws-c-sdkutils STATIC IMPORTED) +set_target_properties(aws-c-sdkutils PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libaws-c-sdkutils.a) + +add_library(jsoncpp STATIC IMPORTED) +set_target_properties(jsoncpp PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libjsoncpp.a) + +add_library(libuuid STATIC IMPORTED) +set_target_properties(libuuid PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libuuid.a) + +add_library(ali-sdk STATIC IMPORTED) +set_target_properties(ali-sdk PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/libalibabacloud-sdk-core.a) + +set(AWS_LIBS + aws-sdk-s3 + aws-sdk-core + aws-sdk-transfer + aws-checksums + aws-c-io + aws-c-event-stream + aws-c-common + aws-c-cal + aws-s2n + aws-c-s3 + aws-c-auth + aws-crt-cpp + aws-c-compression + aws-c-http + aws-c-mqtt + aws-c-sdkutils + aws-sdk-s3-crt) + +add_library(fmt STATIC IMPORTED) +set_target_properties(fmt PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/libfmt.a) diff --git a/cloud/conf/doris_cloud.conf b/cloud/conf/doris_cloud.conf new file mode 100644 index 00000000000000..ed52fda56779b6 --- /dev/null +++ b/cloud/conf/doris_cloud.conf @@ -0,0 +1,47 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +# // meta_service +brpc_listen_port = 5000 +brpc_num_threads = -1 +brpc_idle_timeout_sec = 30 +fdb_cluster = xxx:yyy@127.0.0.1:4500 +fdb_cluster_file_path = ./conf/fdb.cluster +http_token = greedisgood9999 + +# // doris txn config +label_keep_max_second = 259200 +expired_txn_scan_key_nums = 1000 + +# // logging +log_dir = ./log/ +# info warn error +log_level = info +log_size_mb = 1024 +log_filenum_quota = 10 +log_immediate_flush = false +# log_verbose_modules = * + +# // recycler config +recycle_interval_seconds = 3600 +retention_seconds = 259200 +recycle_concurrency = 16 +# recycle_whitelist = +# recycle_blacklist = + +# //max stage num +max_num_stages = 40 diff --git a/cloud/conf/fdb.cluster b/cloud/conf/fdb.cluster new file mode 100644 index 00000000000000..a5f812335a3ecd --- /dev/null +++ b/cloud/conf/fdb.cluster @@ -0,0 +1,18 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +# This file is for unit test only. +q2Yz46DO:Jp3HEFVe@127.0.0.1:4500 diff --git a/cloud/conf/lsan_suppression.conf b/cloud/conf/lsan_suppression.conf new file mode 100644 index 00000000000000..4b6fc3127a9d35 --- /dev/null +++ b/cloud/conf/lsan_suppression.conf @@ -0,0 +1,20 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +# This is a known leak. +leak:libfdb_c.so +leak:brpc::Server::StartInternal diff --git a/cloud/script/run_all_tests.sh b/cloud/script/run_all_tests.sh new file mode 100644 index 00000000000000..5a3fc47d39d01a --- /dev/null +++ b/cloud/script/run_all_tests.sh @@ -0,0 +1,124 @@ +#!/bin/bash +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +echo "input params: $*" + +function usage() { + echo "$0 [--fdb ] [--test ] [--filter ]" +} +if ! OPTS=$(getopt -n "$0" -o a:b:c: -l test:,fdb:,filter:,coverage -- "$@"); then + usage + exit 1 +fi +set -eo pipefail +eval set -- "${OPTS}" + +test="" +fdb_conf="" +filter="" +ENABLE_CLANG_COVERAGE="OFF" +if [[ $# != 1 ]]; then + while true; do + case "$1" in + --coverage) + ENABLE_CLANG_COVERAGE="ON" + shift 1 + ;; + --test) + test="$2" + shift 2 + ;; + --fdb) + fdb_conf="$2" + shift 2 + ;; + --filter) + filter="$2" + shift 2 + ;; + --) + shift + break + ;; + *) + usage + exit 1 + ;; + esac + done +fi +set +eo pipefail + +echo "test=${test} fdb_conf=${fdb_conf} filter=${filter}" + +# fdb memory leaks, we don't care the core dump of unit test +# unset ASAN_OPTIONS + +if [[ "${fdb_conf}" != "" ]]; then + echo "update fdb_cluster.conf with \"${fdb_conf}\"" + echo "${fdb_conf}" >fdb.cluster +fi + +# report converage for unittest +# input param is unittest binary file list +function report_coverage() { + local binary_objects=$1 + local profdata="./report/doris_cloud.profdata" + profraw=$(ls ./report/*.profraw) + local binary_objects_options=() + for object in "${binary_objects[@]}"; do + binary_objects_options[${#binary_objects_options[*]}]="-object ${object}" + done + llvm-profdata merge -o "${profdata}" "${profraw}" + llvm-cov show -output-dir=report -format=html \ + -ignore-filename-regex='(.*gensrc/.*)|(.*_test\.cpp$)' \ + -instr-profile="${profdata}" \ + "${binary_objects_options[*]}" +} + +export LSAN_OPTIONS=suppressions=./lsan_suppression.conf +unittest_files=() +for i in *_test; do + [[ -e "${i}" ]] || break + if [[ "${test}" != "" ]]; then + if [[ "${test}" != "${i}" ]]; then + continue + fi + fi + if [[ -x "${i}" ]]; then + echo "========== ${i} ==========" + fdb=$(ldd "${i}" | grep libfdb_c | grep found) + if [[ "${fdb}" != "" ]]; then + patchelf --set-rpath "$(pwd)" "${i}" + fi + + if [[ "${filter}" == "" ]]; then + LLVM_PROFILE_FILE="./report/${i}.profraw" "./${i}" --gtest_print_time=true --gtest_output="xml:${i}.xml" + else + LLVM_PROFILE_FILE="./report/${i}.profraw" "./${i}" --gtest_print_time=true --gtest_output="xml:${i}.xml" --gtest_filter="${filter}" + fi + unittest_files[${#unittest_files[*]}]="${i}" + echo "--------------------------" + fi +done + +if [[ "_${ENABLE_CLANG_COVERAGE}" == "_ON" ]]; then + report_coverage "${unittest_files[*]}" +fi + +# vim: et ts=2 sw=2: diff --git a/cloud/script/start.sh b/cloud/script/start.sh new file mode 100644 index 00000000000000..d1bfb773675f12 --- /dev/null +++ b/cloud/script/start.sh @@ -0,0 +1,84 @@ +#!/usr/bin/bash +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +curdir="$(cd "$(dirname "${BASH_SOURCE[0]}")" &>/dev/null && pwd)" + +if [[ ! -d bin || ! -d conf || ! -d lib ]]; then + echo "$0 must be invoked at the directory which contains bin, conf and lib" + exit 1 +fi + +DORIS_HOME="$( + cd "${curdir}/.." || exit 1 + pwd +)" + +cd "${DORIS_HOME}" || exit 1 + +daemonized=0 +for arg; do + shift + [[ "${arg}" = "--daemonized" ]] && daemonized=1 && continue + [[ "${arg}" = "-daemonized" ]] && daemonized=1 && continue + [[ "${arg}" = "--daemon" ]] && daemonized=1 && continue + set -- "$@" "${arg}" +done +# echo "$@" "daemonized=${daemonized}"} + +process=doris_cloud + +if [[ -f "${DORIS_HOME}/bin/${process}.pid" ]]; then + pid=$(cat "${DORIS_HOME}/bin/${process}.pid") + if [[ "${pid}" != "" ]]; then + if ! pgrep -f "${pid}" 2>&1 | grep doris_cloud >/dev/null 2>&1; then + echo "pid file existed, ${process} have already started, pid=${pid}" + exit 1 + fi + fi + echo "pid file existed but process not alive, remove it, pid=${pid}" + rm -f "${DORIS_HOME}/bin/${process}.pid" +fi + +lib_path="${DORIS_HOME}/lib" +bin="${DORIS_HOME}/lib/doris_cloud" +if ! ldd "${bin}" | grep -Ei 'libfdb_c.*not found' &>/dev/null; then + if ! command -v patchelf &>/dev/null; then + echo "patchelf is needed to launch meta_service" + exit 1 + fi + patchelf --set-rpath "${lib_path}" "${bin}" + ldd "${bin}" +fi + +export JEMALLOC_CONF="percpu_arena:percpu,background_thread:true,metadata_thp:auto,muzzy_decay_ms:30000,dirty_decay_ms:30000,oversize_threshold:0,lg_tcache_max:16,prof:true,prof_prefix:jeprof.out" + +mkdir -p "${DORIS_HOME}/log" +echo "starts ${process} with args: $*" +if [[ "${daemonized}" -eq 1 ]]; then + date >>"${DORIS_HOME}/log/${process}.out" + nohup "${bin}" "$@" >>"${DORIS_HOME}/log/${process}.out" 2>&1 & + # wait for log flush + sleep 1.5 + tail -n10 "${DORIS_HOME}/log/${process}.out" | grep 'working directory' -B1 -A10 + echo "please check process log for more details" + echo "" +else + "${bin}" "$@" +fi + +# vim: et ts=2 sw=2: diff --git a/cloud/script/stop.sh b/cloud/script/stop.sh new file mode 100644 index 00000000000000..b40b4061c398dc --- /dev/null +++ b/cloud/script/stop.sh @@ -0,0 +1,35 @@ +#!/usr/bin/bash +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +curdir="$(cd "$(dirname "${BASH_SOURCE[0]}")" &>/dev/null && pwd)" +DORIS_HOME="$( + cd "${curdir}/.." || exit 1 + pwd +)" + +cd "${DORIS_HOME}" || exit 1 + +process=doris_cloud + +if [[ ! -f "${DORIS_HOME}/bin/${process}.pid" ]]; then + echo "no ${process}.pid found, process may have been stopped" + exit 1 +fi + +pid=$(cat "${DORIS_HOME}/bin/${process}.pid") +kill -2 "${pid}" +rm -f "${DORIS_HOME}/bin/${process}.pid" diff --git a/cloud/src/common/CMakeLists.txt b/cloud/src/common/CMakeLists.txt new file mode 100644 index 00000000000000..166ad5052dadd9 --- /dev/null +++ b/cloud/src/common/CMakeLists.txt @@ -0,0 +1,26 @@ +# where to put generated libraries +set(LIBRARY_OUTPUT_PATH "${BUILD_DIR}/src/common") + +# where to put generated binaries +set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/src/common") + +set(COMMON_FILES + configbase.cpp + sync_point.cpp + util.cpp + logging.cpp + bvars.cpp + encryption_util.cpp + metric.cpp + kms.cpp +) + +if (USE_JEMALLOC) + set(COMMON_FILES ${COMMON_FILES} + jemalloc_hook.cpp + ) +endif() + +add_library(Common STATIC + ${COMMON_FILES} + ) diff --git a/cloud/src/common/arg_parser.h b/cloud/src/common/arg_parser.h new file mode 100644 index 00000000000000..5edae6dc33e7ee --- /dev/null +++ b/cloud/src/common/arg_parser.h @@ -0,0 +1,224 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include + +static_assert(__cplusplus >= 201701, "compiler need to support c++17 or newer"); + +namespace doris::cloud { + +/** + * This class implements arguments parser, it accepts both short '-arg' and + * long '--argument' arguments + * + * simple::ArgParser parser( + * { // define args that will be used in program + * // arg_name default_value comment + * ArgParser::new_arg ("verion" , false , "print version or not") , + * ArgParser::new_arg ("duration", 10086 , "duration for the test in seconds"), + * ArgParser::new_arg("flags" , "abcdef", "flags for testing") , + * ArgParser::new_arg ("pi" , 3.1415 , "pi") , + * }, + * true // skip unknown args if there is any + * ); + * + * // parse/update from args + * parser.parse(argc, argv); + * // to use args, types must be specified when access args + * if (parser.get(version)) print_version(); + * ... + * + */ +class ArgParser { +public: + using arg_map_t = + std::map, std::string>>; + using arg_t = arg_map_t::value_type; + + /** + * Create a key-value pair of argument, used in initialization + * TODO: template arg pack for extension + * + * @param name argument name + * @val argument value, for default + * @comment comment for the argument + * @return an arg_t, internal element of arg_map_t + */ + template + static arg_t new_arg(const std::string& name, const T& val, const std::string& comment = "") { + compile_time_check_type(); + return arg_t {name, {T {val}, comment}}; + } + + /** + * Print given args + */ + static void print(const arg_map_t& args) { + std::cout << "args: "; + for (const auto& i : args) { + std::cout << "{" << i.first << "="; + if (std::get_if(&std::get<0>(i.second))) { + std::cout << std::get(std::get<0>(i.second)); + } else if (std::get_if(&std::get<0>(i.second))) { + std::cout << (std::get(std::get<0>(i.second)) ? "true" : "false"); + } else if (std::get_if(&std::get<0>(i.second))) { + std::cout << std::get(std::get<0>(i.second)); + } else if (std::get_if(&std::get<0>(i.second))) { + std::cout << std::get(std::get<0>(i.second)); + } else { // not defined type + std::cout << "unknown type of " << i.first; + } + std::cout << "; " << std::get<1>(i.second) << "}, "; + } + std::cout << std::endl; + } + + template + constexpr static void compile_time_check_type() { + static_assert(std::is_same_v || std::is_same_v || + std::is_same_v || std::is_same_v, + "only std::string, long, double, and bool are allowed"); + } + + /** + * Initialize and declare arguments + * + * @param args arguments list that predefined + * @param skip_unknown_arg skip the unknown arguments when parse + */ + ArgParser(const arg_map_t& args = {}, bool skip_unknown_arg = false) + : args_(args), skip_unknown_(skip_unknown_arg) {} + + /** + * Parse arguments + * + * @param argc number of arguments to parse + * @param argv arguments to parse, + * e.g. ["--argument=abc", "-v", "-has_xxx=false"] + * @out parse result, if nullptr, internal container will be used + * if out == nullptr and internal container is not initialized + * call to this function does not make sense + * @return error msg that encountered + */ + std::string parse(int argc, char const* const* argv, arg_map_t* out = nullptr) { + arg_map_t ret; + // copy + if (out != nullptr) + ret = *out; + else + ret = args_; + + std::string msg; + std::vector args; + args.reserve(argc); + for (int i = 0; i < argc; ++i) { + args.emplace_back(argv[i]); + auto& arg = args.back(); + auto eq = arg.find('='); + // process boolean flags first + if (eq == std::string::npos) { // "--version" || "-version" || "--help" + auto k = arg.substr(1); + auto it = ret.find(k) == ret.end() ? ret.find(k.substr(1)) : ret.find(k); + if (it != ret.end() && std::get_if(&std::get<0>(it->second))) { + std::get<0>(it->second) = true; + continue; + } + } + if (arg.size() < 4 || arg[0] != '-' || arg[2] == '-' || eq == std::string::npos) { + // std::cerr << "invalid arg: " << arg << std::endl; + msg += "invalid arg: " + arg + "; "; + if (skip_unknown_) + continue; + else + return msg; + } + std::string k = arg.substr(1 + (arg[1] == '-'), eq - 1 - (arg[1] == '-')); + auto p = ret.find(k); + + if (p == ret.end()) { // no such an arg + msg += "arg not supported: " + k + "; "; + if (skip_unknown_) + continue; + else + return msg; + } + + std::string v = arg.substr(eq + 1); + // std::cout << arg << ", k: " << k << ", v: " << v << std::endl; + if (std::get_if(&std::get<0>(p->second))) { + std::get<0>(p->second) = v; + } else if (std::get_if(&std::get<0>(p->second))) { + if (v[0] >= '0' && v[0] <= '9') { + std::get<0>(p->second) = !!(std::stol(v)); + } else if (v.find("true") != std::string::npos && v.size() == 4) { + std::get<0>(p->second) = true; + } else if (v.find("false") != std::string::npos && v.size() == 5) { + std::get<0>(p->second) = false; + } else { + msg += "invalid arg " + arg + ", it should true or false. "; + if (skip_unknown_) + continue; + else + return msg; + } + } else { // number + std::get<0>(p->second) = std::stol(v); + } + } + + if (out != nullptr) + *out = std::move(ret); + else + args_ = std::move(ret); + + return msg; + } + + auto& args() { return args_; } + + void print() { print(args_); } + + // std::remove_cvref_t is available since c++2a + template + auto get(const std::string& name, + const std::remove_reference_t>& default_ = {}) { + typedef std::remove_reference_t> U; + compile_time_check_type(); + auto it = args_.find(name); + // avoid invalid cast + if (it == args_.end() || !std::get_if(&std::get<0>(it->second))) { + return default_; + } + return std::get(std::get<0>(it->second)); + } + +private: + arg_map_t args_; + bool skip_unknown_; +}; + +} // namespace doris::cloud diff --git a/cloud/src/common/bvars.cpp b/cloud/src/common/bvars.cpp new file mode 100644 index 00000000000000..702dea865022b2 --- /dev/null +++ b/cloud/src/common/bvars.cpp @@ -0,0 +1,180 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "common/bvars.h" + +#include +#include + +// meta-service's bvars +BvarLatencyRecorderWithTag g_bvar_ms_begin_txn("ms", "begin_txn"); +BvarLatencyRecorderWithTag g_bvar_ms_precommit_txn("ms", "precommit_txn"); +BvarLatencyRecorderWithTag g_bvar_ms_commit_txn("ms", "commit_txn"); +BvarLatencyRecorderWithTag g_bvar_ms_abort_txn("ms", "abort_txn"); +BvarLatencyRecorderWithTag g_bvar_ms_get_txn("ms", "get_txn"); +BvarLatencyRecorderWithTag g_bvar_ms_get_current_max_txn_id("ms", "get_current_max_txn_id"); +BvarLatencyRecorderWithTag g_bvar_ms_check_txn_conflict("ms", "check_txn_conflict"); +BvarLatencyRecorderWithTag g_bvar_ms_clean_txn_label("ms", "clean_txn_label"); +BvarLatencyRecorderWithTag g_bvar_ms_get_version("ms", "get_version"); +BvarLatencyRecorderWithTag g_bvar_ms_batch_get_version("ms", "batch_get_version"); +BvarLatencyRecorderWithTag g_bvar_ms_create_tablets("ms", "create_tablets"); +BvarLatencyRecorderWithTag g_bvar_ms_update_tablet("ms", "update_tablet"); +BvarLatencyRecorderWithTag g_bvar_ms_update_tablet_schema("ms", "update_tablet_schema"); +BvarLatencyRecorderWithTag g_bvar_ms_get_tablet("ms", "get_tablet"); +BvarLatencyRecorderWithTag g_bvar_ms_prepare_rowset("ms", "prepare_rowset"); +BvarLatencyRecorderWithTag g_bvar_ms_commit_rowset("ms", "commit_rowset"); +BvarLatencyRecorderWithTag g_bvar_ms_update_tmp_rowset("ms", "update_tmp_rowset"); +BvarLatencyRecorderWithTag g_bvar_ms_get_rowset("ms", "get_rowset"); +BvarLatencyRecorderWithTag g_bvar_ms_drop_index("ms", "drop_index"); +BvarLatencyRecorderWithTag g_bvar_ms_prepare_index("ms", "prepare_index"); +BvarLatencyRecorderWithTag g_bvar_ms_commit_index("ms", "commit_index"); +BvarLatencyRecorderWithTag g_bvar_ms_prepare_partition("ms", "prepare_partition"); +BvarLatencyRecorderWithTag g_bvar_ms_commit_partition("ms", "commit_partition"); +BvarLatencyRecorderWithTag g_bvar_ms_drop_partition("ms", "drop_partition"); +BvarLatencyRecorderWithTag g_bvar_ms_get_tablet_stats("ms", "get_tablet_stats"); +BvarLatencyRecorderWithTag g_bvar_ms_get_obj_store_info("ms", "get_obj_store_info"); +BvarLatencyRecorderWithTag g_bvar_ms_alter_obj_store_info("ms", "alter_obj_store_info"); +BvarLatencyRecorderWithTag g_bvar_ms_create_instance("ms", "create_instance"); +BvarLatencyRecorderWithTag g_bvar_ms_alter_instance("ms", "alter_instance"); +BvarLatencyRecorderWithTag g_bvar_ms_alter_cluster("ms", "alter_cluster"); +BvarLatencyRecorderWithTag g_bvar_ms_get_cluster("ms", "get_cluster"); +BvarLatencyRecorderWithTag g_bvar_ms_create_stage("ms", "create_stage"); +BvarLatencyRecorderWithTag g_bvar_ms_get_stage("ms", "get_stage"); +BvarLatencyRecorderWithTag g_bvar_ms_drop_stage("ms", "drop_stage"); +BvarLatencyRecorderWithTag g_bvar_ms_get_iam("ms", "get_iam"); +BvarLatencyRecorderWithTag g_bvar_ms_alter_iam("ms", "alter_iam"); +BvarLatencyRecorderWithTag g_bvar_ms_update_ak_sk("ms", "update_ak_sk"); +BvarLatencyRecorderWithTag g_bvar_ms_alter_ram_user("ms", "alter_ram_user"); +BvarLatencyRecorderWithTag g_bvar_ms_begin_copy("ms", "begin_copy"); +BvarLatencyRecorderWithTag g_bvar_ms_finish_copy("ms", "finish_copy"); +BvarLatencyRecorderWithTag g_bvar_ms_get_copy_job("ms", "get_copy_job"); +BvarLatencyRecorderWithTag g_bvar_ms_get_copy_files("ms", "get_copy_files"); +BvarLatencyRecorderWithTag g_bvar_ms_filter_copy_files("ms", "filter_copy_files"); +BvarLatencyRecorderWithTag g_bvar_ms_update_delete_bitmap("ms", "update_delete_bitmap"); +BvarLatencyRecorderWithTag g_bvar_ms_get_delete_bitmap("ms", "get_delete_bitmap"); +BvarLatencyRecorderWithTag g_bvar_ms_get_delete_bitmap_update_lock("ms", + "get_delete_bitmap_update_lock"); +BvarLatencyRecorderWithTag g_bvar_ms_get_instance("ms", "get_instance"); + +BvarLatencyRecorderWithTag g_bvar_ms_start_tablet_job("ms", "start_tablet_job"); +BvarLatencyRecorderWithTag g_bvar_ms_finish_tablet_job("ms", "finish_tablet_job"); +BvarLatencyRecorderWithTag g_bvar_ms_get_cluster_status("ms", "get_cluster_status"); +BvarLatencyRecorderWithTag g_bvar_ms_set_cluster_status("ms", "set_cluster_status"); + +// txn_kv's bvars +bvar::LatencyRecorder g_bvar_txn_kv_get("txn_kv", "get"); +bvar::LatencyRecorder g_bvar_txn_kv_range_get("txn_kv", "range_get"); +bvar::LatencyRecorder g_bvar_txn_kv_put("txn_kv", "put"); +bvar::LatencyRecorder g_bvar_txn_kv_commit("txn_kv", "commit"); +bvar::LatencyRecorder g_bvar_txn_kv_atomic_set_ver_key("txn_kv", "atomic_set_ver_key"); +bvar::LatencyRecorder g_bvar_txn_kv_atomic_set_ver_value("txn_kv", "atomic_set_ver_value"); +bvar::LatencyRecorder g_bvar_txn_kv_atomic_add("txn_kv", "atomic_add"); +bvar::LatencyRecorder g_bvar_txn_kv_remove("txn_kv", "remove"); +bvar::LatencyRecorder g_bvar_txn_kv_range_remove("txn_kv", "range_remove"); +bvar::LatencyRecorder g_bvar_txn_kv_get_read_version("txn_kv", "get_read_version"); +bvar::LatencyRecorder g_bvar_txn_kv_get_committed_version("txn_kv", "get_committed_version"); +bvar::LatencyRecorder g_bvar_txn_kv_batch_get("txn_kv", "batch_get"); + +bvar::Adder g_bvar_txn_kv_commit_error_counter; +bvar::Window > g_bvar_txn_kv_commit_error_counter_minute( + "txn_kv", "commit_error", &g_bvar_txn_kv_commit_error_counter, 60); + +bvar::Adder g_bvar_txn_kv_commit_conflict_counter; +bvar::Window > g_bvar_txn_kv_commit_conflict_counter_minute( + "txn_kv", "commit_conflict", &g_bvar_txn_kv_commit_conflict_counter, 60); + +const int64_t BVAR_FDB_INVALID_VALUE = -99999999L; +bvar::Status g_bvar_fdb_client_count("fdb_client_count", BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_configuration_coordinators_count( + "fdb_configuration_coordinators_count", BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_configuration_usable_regions("fdb_configuration_usable_regions", + BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_coordinators_unreachable_count( + "fdb_coordinators_unreachable_count", BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_fault_tolerance_count("fdb_fault_tolerance_count", + BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_data_average_partition_size_bytes( + "fdb_data_average_partition_size_bytes", BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_data_log_server_space_bytes("fdb_data_log_server_space_bytes", + BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_data_moving_data_highest_priority( + "fdb_data_moving_data_highest_priority", BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_data_moving_data_in_flight_bytes( + "fdb_data_moving_data_in_flight_bytes", BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_data_moving_data_in_queue_bytes( + "fdb_data_moving_data_in_queue_bytes", BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_data_moving_total_written_bytes( + "fdb_data_moving_total_written_bytes", BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_data_partition_count("fdb_data_partition_count", + BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_data_storage_server_space_bytes( + "fdb_data_storage_server_space_bytes", BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_data_state_min_replicas_remaining( + "fdb_data_state_min_replicas_remaining", BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_data_total_kv_size_bytes("fdb_data_total_kv_size_bytes", + BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_data_total_disk_used_bytes("fdb_data_total_disk_used_bytes", + BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_generation("fdb_generation", BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_incompatible_connections("fdb_incompatible_connections", + BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_latency_probe_transaction_start_ns( + "fdb_latency_probe_transaction_start_ns", BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_latency_probe_commit_ns("fdb_latency_probe_commit_ns", + BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_latency_probe_read_ns("fdb_latency_probe_read_ns", + BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_machines_count("fdb_machines_count", BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_process_count("fdb_process_count", BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_qos_worst_data_lag_storage_server_ns( + "fdb_qos_worst_data_lag_storage_server_ns", BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_qos_worst_durability_lag_storage_server_ns( + "fdb_qos_worst_durability_lag_storage_server_ns", BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_qos_worst_log_server_queue_bytes( + "fdb_qos_worst_log_server_queue_bytes", BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_qos_worst_storage_server_queue_bytes( + "fdb_qos_worst_storage_server_queue_bytes", BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_workload_conflict_rate_hz("fdb_workload_conflict_rate_hz", + BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_workload_location_rate_hz("fdb_workload_location_rate_hz", + BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_workload_keys_read_hz("fdb_workload_keys_read_hz", + BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_workload_read_bytes_hz("fdb_workload_read_bytes_hz", + BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_workload_read_rate_hz("fdb_workload_read_rate_hz", + BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_workload_write_rate_hz("fdb_workload_write_rate_hz", + BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_workload_written_bytes_hz("fdb_workload_written_bytes_hz", + BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_workload_transactions_started_hz( + "fdb_workload_transactions_started_hz", BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_workload_transactions_committed_hz( + "fdb_workload_transactions_committed_hz", BVAR_FDB_INVALID_VALUE); +bvar::Status g_bvar_fdb_workload_transactions_rejected_hz( + "fdb_workload_transactions_rejected_hz", BVAR_FDB_INVALID_VALUE); + +// checker's bvars +BvarStatusWithTag g_bvar_checker_num_scanned("checker", "num_scanned"); +BvarStatusWithTag g_bvar_checker_num_scanned_with_segment("checker", + "num_scanned_with_segment"); +BvarStatusWithTag g_bvar_checker_num_check_failed("checker", "num_check_failed"); +BvarStatusWithTag g_bvar_checker_check_cost_s("checker", "check_cost_seconds"); +BvarStatusWithTag g_bvar_checker_enqueue_cost_s("checker", "enqueue_cost_seconds"); +BvarStatusWithTag g_bvar_checker_last_success_time_ms("checker", "last_success_time_ms"); +BvarStatusWithTag g_bvar_checker_instance_volume("checker", "instance_volume"); diff --git a/cloud/src/common/bvars.h b/cloud/src/common/bvars.h new file mode 100644 index 00000000000000..1c4c4f749b6fe1 --- /dev/null +++ b/cloud/src/common/bvars.h @@ -0,0 +1,237 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include +#include + +#include +#include +#include +#include +#include + +class BvarLatencyRecorderWithTag { +public: + BvarLatencyRecorderWithTag(std::string module, std::string name) + : module_(std::move(module)), name_(std::move(name)) {} + + void put(const std::string& tag, int64_t value) { + std::shared_ptr instance = nullptr; + { + std::lock_guard l(mutex_); + auto it = bvar_map_.find(tag); + if (it == bvar_map_.end()) { + instance = std::make_shared(module_, name_ + "_" + tag); + bvar_map_[tag] = instance; + } else { + instance = it->second; + } + } + (*instance) << value; + } + + std::shared_ptr get(const std::string& tag) { + std::shared_ptr instance = nullptr; + std::lock_guard l(mutex_); + + auto it = bvar_map_.find(tag); + if (it == bvar_map_.end()) { + instance = std::make_shared(module_, name_ + "_" + tag); + bvar_map_[tag] = instance; + return instance; + } + return it->second; + } + + void remove(const std::string& tag) { + std::lock_guard l(mutex_); + bvar_map_.erase(tag); + } + +private: + bthread::Mutex mutex_; + std::string module_; + std::string name_; + std::map> bvar_map_; +}; + +template +class BvarStatusWithTag { +public: + BvarStatusWithTag(std::string module, std::string name) + : module_(std::move(module)), name_(std::move(name)) {} + + void put(const std::string& tag, T value) { + std::shared_ptr> instance = nullptr; + { + std::lock_guard l(mutex_); + auto it = bvar_map_.find(tag); + if (it == bvar_map_.end()) { + instance = std::make_shared>(module_, name_ + "_" + tag, T()); + bvar_map_[tag] = instance; + } else { + instance = it->second; + } + } + (*instance).set_value(value); + } + + std::shared_ptr> get(const std::string& tag) { + std::shared_ptr> instance = nullptr; + std::lock_guard l(mutex_); + + auto it = bvar_map_.find(tag); + if (it == bvar_map_.end()) { + instance = std::make_shared>(module_, name_ + "_" + tag); + bvar_map_[tag] = instance; + return instance; + } + return it->second; + } + + void remove(const std::string& tag) { + std::lock_guard l(mutex_); + bvar_map_.erase(tag); + } + +private: + bthread::Mutex mutex_; + std::string module_; + std::string name_; + std::map>> bvar_map_; +}; + +// meta-service's bvars +extern BvarLatencyRecorderWithTag g_bvar_ms_begin_txn; +extern BvarLatencyRecorderWithTag g_bvar_ms_precommit_txn; +extern BvarLatencyRecorderWithTag g_bvar_ms_commit_txn; +extern BvarLatencyRecorderWithTag g_bvar_ms_abort_txn; +extern BvarLatencyRecorderWithTag g_bvar_ms_get_txn; +extern BvarLatencyRecorderWithTag g_bvar_ms_get_current_max_txn_id; +extern BvarLatencyRecorderWithTag g_bvar_ms_check_txn_conflict; +extern BvarLatencyRecorderWithTag g_bvar_ms_clean_txn_label; +extern BvarLatencyRecorderWithTag g_bvar_ms_get_version; +extern BvarLatencyRecorderWithTag g_bvar_ms_batch_get_version; +extern BvarLatencyRecorderWithTag g_bvar_ms_create_tablets; +extern BvarLatencyRecorderWithTag g_bvar_ms_update_tablet; +extern BvarLatencyRecorderWithTag g_bvar_ms_update_tablet_schema; +extern BvarLatencyRecorderWithTag g_bvar_ms_get_tablet; +extern BvarLatencyRecorderWithTag g_bvar_ms_prepare_rowset; +extern BvarLatencyRecorderWithTag g_bvar_ms_commit_rowset; +extern BvarLatencyRecorderWithTag g_bvar_ms_update_tmp_rowset; +extern BvarLatencyRecorderWithTag g_bvar_ms_get_rowset; +extern BvarLatencyRecorderWithTag g_bvar_ms_drop_index; +extern BvarLatencyRecorderWithTag g_bvar_ms_prepare_index; +extern BvarLatencyRecorderWithTag g_bvar_ms_commit_index; +extern BvarLatencyRecorderWithTag g_bvar_ms_prepare_partition; +extern BvarLatencyRecorderWithTag g_bvar_ms_commit_partition; +extern BvarLatencyRecorderWithTag g_bvar_ms_drop_partition; +extern BvarLatencyRecorderWithTag g_bvar_ms_get_tablet_stats; +extern BvarLatencyRecorderWithTag g_bvar_ms_get_obj_store_info; +extern BvarLatencyRecorderWithTag g_bvar_ms_alter_obj_store_info; +extern BvarLatencyRecorderWithTag g_bvar_ms_create_instance; +extern BvarLatencyRecorderWithTag g_bvar_ms_alter_instance; +extern BvarLatencyRecorderWithTag g_bvar_ms_alter_cluster; +extern BvarLatencyRecorderWithTag g_bvar_ms_get_cluster; +extern BvarLatencyRecorderWithTag g_bvar_ms_create_stage; +extern BvarLatencyRecorderWithTag g_bvar_ms_get_stage; +extern BvarLatencyRecorderWithTag g_bvar_ms_drop_stage; +extern BvarLatencyRecorderWithTag g_bvar_ms_get_iam; +extern BvarLatencyRecorderWithTag g_bvar_ms_update_ak_sk; +extern BvarLatencyRecorderWithTag g_bvar_ms_alter_iam; +extern BvarLatencyRecorderWithTag g_bvar_ms_alter_ram_user; +extern BvarLatencyRecorderWithTag g_bvar_ms_begin_copy; +extern BvarLatencyRecorderWithTag g_bvar_ms_finish_copy; +extern BvarLatencyRecorderWithTag g_bvar_ms_get_copy_job; +extern BvarLatencyRecorderWithTag g_bvar_ms_get_copy_files; +extern BvarLatencyRecorderWithTag g_bvar_ms_filter_copy_files; +extern BvarLatencyRecorderWithTag g_bvar_ms_start_tablet_job; +extern BvarLatencyRecorderWithTag g_bvar_ms_finish_tablet_job; +extern BvarLatencyRecorderWithTag g_bvar_ms_update_delete_bitmap; +extern BvarLatencyRecorderWithTag g_bvar_ms_get_delete_bitmap; +extern BvarLatencyRecorderWithTag g_bvar_ms_get_delete_bitmap_update_lock; +extern BvarLatencyRecorderWithTag g_bvar_ms_get_cluster_status; +extern BvarLatencyRecorderWithTag g_bvar_ms_set_cluster_status; +extern BvarLatencyRecorderWithTag g_bvar_ms_get_instance; + +// txn_kv's bvars +extern bvar::LatencyRecorder g_bvar_txn_kv_get; +extern bvar::LatencyRecorder g_bvar_txn_kv_range_get; +extern bvar::LatencyRecorder g_bvar_txn_kv_put; +extern bvar::LatencyRecorder g_bvar_txn_kv_commit; +extern bvar::LatencyRecorder g_bvar_txn_kv_atomic_set_ver_key; +extern bvar::LatencyRecorder g_bvar_txn_kv_atomic_set_ver_value; +extern bvar::LatencyRecorder g_bvar_txn_kv_atomic_add; +extern bvar::LatencyRecorder g_bvar_txn_kv_remove; +extern bvar::LatencyRecorder g_bvar_txn_kv_range_remove; +extern bvar::LatencyRecorder g_bvar_txn_kv_get_read_version; +extern bvar::LatencyRecorder g_bvar_txn_kv_get_committed_version; +extern bvar::LatencyRecorder g_bvar_txn_kv_batch_get; + +extern bvar::Adder g_bvar_txn_kv_commit_error_counter; +extern bvar::Adder g_bvar_txn_kv_commit_conflict_counter; + +extern const int64_t BVAR_FDB_INVALID_VALUE; +extern bvar::Status g_bvar_fdb_client_count; +extern bvar::Status g_bvar_fdb_configuration_coordinators_count; +extern bvar::Status g_bvar_fdb_configuration_usable_regions; +extern bvar::Status g_bvar_fdb_coordinators_unreachable_count; +extern bvar::Status g_bvar_fdb_fault_tolerance_count; +extern bvar::Status g_bvar_fdb_data_average_partition_size_bytes; +extern bvar::Status g_bvar_fdb_data_log_server_space_bytes; +extern bvar::Status g_bvar_fdb_data_moving_data_highest_priority; +extern bvar::Status g_bvar_fdb_data_moving_data_in_flight_bytes; +extern bvar::Status g_bvar_fdb_data_moving_data_in_queue_bytes; +extern bvar::Status g_bvar_fdb_data_moving_total_written_bytes; +extern bvar::Status g_bvar_fdb_data_partition_count; +extern bvar::Status g_bvar_fdb_data_storage_server_space_bytes; +extern bvar::Status g_bvar_fdb_data_state_min_replicas_remaining; +extern bvar::Status g_bvar_fdb_data_total_kv_size_bytes; +extern bvar::Status g_bvar_fdb_data_total_disk_used_bytes; +extern bvar::Status g_bvar_fdb_generation; +extern bvar::Status g_bvar_fdb_incompatible_connections; +extern bvar::Status g_bvar_fdb_latency_probe_transaction_start_ns; +extern bvar::Status g_bvar_fdb_latency_probe_commit_ns; +extern bvar::Status g_bvar_fdb_latency_probe_read_ns; +extern bvar::Status g_bvar_fdb_machines_count; +extern bvar::Status g_bvar_fdb_process_count; +extern bvar::Status g_bvar_fdb_qos_worst_data_lag_storage_server_ns; +extern bvar::Status g_bvar_fdb_qos_worst_durability_lag_storage_server_ns; +extern bvar::Status g_bvar_fdb_qos_worst_log_server_queue_bytes; +extern bvar::Status g_bvar_fdb_qos_worst_storage_server_queue_bytes; +extern bvar::Status g_bvar_fdb_workload_conflict_rate_hz; +extern bvar::Status g_bvar_fdb_workload_location_rate_hz; +extern bvar::Status g_bvar_fdb_workload_keys_read_hz; +extern bvar::Status g_bvar_fdb_workload_read_bytes_hz; +extern bvar::Status g_bvar_fdb_workload_read_rate_hz; +extern bvar::Status g_bvar_fdb_workload_write_rate_hz; +extern bvar::Status g_bvar_fdb_workload_written_bytes_hz; +extern bvar::Status g_bvar_fdb_workload_transactions_started_hz; +extern bvar::Status g_bvar_fdb_workload_transactions_committed_hz; +extern bvar::Status g_bvar_fdb_workload_transactions_rejected_hz; + +// checker +extern BvarStatusWithTag g_bvar_checker_num_scanned; +extern BvarStatusWithTag g_bvar_checker_num_scanned_with_segment; +extern BvarStatusWithTag g_bvar_checker_num_check_failed; +extern BvarStatusWithTag g_bvar_checker_check_cost_s; +extern BvarStatusWithTag g_bvar_checker_enqueue_cost_s; +extern BvarStatusWithTag g_bvar_checker_last_success_time_ms; +extern BvarStatusWithTag g_bvar_checker_instance_volume; diff --git a/cloud/src/common/config.h b/cloud/src/common/config.h new file mode 100644 index 00000000000000..34136f8b0dd726 --- /dev/null +++ b/cloud/src/common/config.h @@ -0,0 +1,148 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include "configbase.h" + +namespace doris::cloud::config { + +CONF_Int32(brpc_listen_port, "5000"); +CONF_Int32(brpc_num_threads, "-1"); +// Set -1 to disable it. +CONF_Int32(brpc_idle_timeout_sec, "-1"); +CONF_String(hostname, ""); +CONF_String(fdb_cluster, "xxx:yyy@127.0.0.1:4500"); +CONF_String(fdb_cluster_file_path, "./conf/fdb.cluster"); +CONF_String(http_token, "greedisgood9999"); +// use volatile mem kv for test. MUST NOT be `true` in production environment. +CONF_Bool(use_mem_kv, "false"); +CONF_Int32(meta_server_register_interval_ms, "20000"); +CONF_Int32(meta_server_lease_ms, "60000"); + +CONF_Int64(fdb_txn_timeout_ms, "10000"); +CONF_Int64(brpc_max_body_size, "3147483648"); +CONF_Int64(brpc_socket_max_unwritten_bytes, "1073741824"); + +// logging +CONF_String(log_dir, "./log/"); +CONF_String(log_level, "info"); // info warn error fatal +CONF_Int64(log_size_mb, "1024"); +CONF_Int32(log_filenum_quota, "10"); +CONF_Int32(warn_log_filenum_quota, "1"); +CONF_Bool(log_immediate_flush, "false"); +CONF_Strings(log_verbose_modules, ""); // Comma seprated list: a.*,b.* +CONF_Int32(log_verbose_level, "5"); + +// recycler config +CONF_mInt64(recycle_interval_seconds, "3600"); +CONF_mInt64(retention_seconds, "259200"); // 72h +CONF_Int32(recycle_concurrency, "16"); +CONF_Int32(recycle_job_lease_expired_ms, "60000"); +CONF_mInt64(compacted_rowset_retention_seconds, "10800"); // 3h +CONF_mInt64(dropped_index_retention_seconds, "10800"); // 3h +CONF_mInt64(dropped_partition_retention_seconds, "10800"); // 3h +// Which instance should be recycled. If empty, recycle all instances. +CONF_Strings(recycle_whitelist, ""); // Comma seprated list +// These instances will not be recycled, only effective when whitelist is empty. +CONF_Strings(recycle_blacklist, ""); // Comma seprated list +CONF_mInt32(instance_recycler_worker_pool_size, "10"); +CONF_Bool(enable_checker, "false"); +// Currently only used for recycler test +CONF_Bool(enable_inverted_check, "false"); +// interval for scanning instances to do checks and inspections +CONF_mInt32(scan_instances_interval_seconds, "60"); // 1min +// interval for check object +CONF_mInt32(check_object_interval_seconds, "43200"); // 12hours + +CONF_String(test_s3_ak, "ak"); +CONF_String(test_s3_sk, "sk"); +CONF_String(test_s3_endpoint, "endpoint"); +CONF_String(test_s3_region, "region"); +CONF_String(test_s3_bucket, "bucket"); +// CONF_Int64(a, "1073741824"); +// CONF_Bool(b, "true"); + +// txn config +CONF_Int32(label_keep_max_second, "259200"); //3 * 24 * 3600 seconds +CONF_Int32(expired_txn_scan_key_nums, "1000"); + +// Maximum number of version of a tablet. If the version num of a tablet exceed limit, +// the load process will reject new incoming load job of this tablet. +// This is to avoid too many version num. +CONF_Int64(max_tablet_version_num, "2000"); + +// metrics config +CONF_Bool(use_detailed_metrics, "true"); + +// stage num config +CONF_Int32(max_num_stages, "40"); + +// qps limit config + +// limit by each warehouse each rpc +CONF_Int64(default_max_qps_limit, "1000000"); +// limit by each warehouse specific rpc +CONF_String(specific_max_qps_limit, "get_cluster:5000000;begin_txn:5000000"); +CONF_Bool(enable_rate_limit, "true"); +CONF_Int64(bvar_qps_update_second, "5"); + +CONF_Int32(copy_job_max_retention_second, "259200"); //3 * 24 * 3600 seconds +CONF_String(arn_id, ""); +CONF_String(arn_ak, ""); +CONF_String(arn_sk, ""); +CONF_Int64(internal_stage_objects_expire_time_second, "259200"); // 3 * 24 * 3600 seconds + +// format with base64: eg, "cloudcloudcloudcloud" -> "c2VsZWN0ZGJzZWxlY3RkYnNlbGVjdGRic2VsZWN0ZGI=" +CONF_String(encryption_key, "c2VsZWN0ZGJzZWxlY3RkYnNlbGVjdGRic2VsZWN0ZGI="); +CONF_String(encryption_method, "AES_256_ECB"); + +// Temporary configs for upgrade +CONF_mBool(write_schema_kv, "true"); +CONF_mBool(split_tablet_stats, "true"); +CONF_mBool(snapshot_get_tablet_stats, "true"); + +// Value codec version +CONF_mInt16(meta_schema_value_version, 0); + +// For instance check interval +CONF_Int64(reserved_buffer_days, "3"); + +// For recycler to do periodically log to detect alive +CONF_Int32(periodically_log_ms, "5000"); + +// For kms +CONF_Bool(enable_kms, "false"); +CONF_String(kms_info_encryption_key, ""); // encryption_key to encrypt kms sk +CONF_String(kms_info_encryption_method, "AES_256_ECB"); +// kms ak does not need to be encrypted yet, so use plaintext +CONF_String(kms_ak, ""); +// kms sk uses base64-encoded ciphertext +// the plaintext must be encrypted with kms_info_encryption_key and kms_info_encryption_method +CONF_String(kms_sk, ""); +CONF_String(kms_endpoint, ""); +CONF_String(kms_region, ""); +CONF_String(kms_provider, "ali"); // ali/tx/aws/hw, only support ali now +CONF_String(kms_cmk, ""); +// When starting up, add kms data key if is missing, all MS need to be restarted simultaneously. +CONF_Bool(focus_add_kms_data_key, "false"); + +// Whether to retry the retryable errors that returns by the underlying txn store. +CONF_Bool(enable_txn_store_retry, "false"); +CONF_Int32(txn_store_retry_times, "20"); + +} // namespace doris::cloud::config diff --git a/cloud/src/common/configbase.cpp b/cloud/src/common/configbase.cpp new file mode 100644 index 00000000000000..a1ab4853822cf2 --- /dev/null +++ b/cloud/src/common/configbase.cpp @@ -0,0 +1,331 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include +#include +#include +#include +#include +#include +#include +#include + +#define __IN_CONFIGBASE_CPP__ +#include "common/config.h" +#undef __IN_CONFIGBASE_CPP__ + +namespace doris::cloud::config { + +std::map* Register::_s_field_map = nullptr; +std::map>* RegisterConfValidator::_s_field_validator = nullptr; +std::map* full_conf_map = nullptr; + +// trim string +std::string& trim(std::string& s) { + // rtrim + s.erase(std::find_if(s.rbegin(), s.rend(), [](unsigned char c) { return !std::isspace(c); }) + .base(), + s.end()); + // ltrim + s.erase(s.begin(), + std::find_if(s.begin(), s.end(), [](unsigned char c) { return !std::isspace(c); })); + return s; +} + +// split string by '=' +void splitkv(const std::string& s, std::string& k, std::string& v) { + const char sep = '='; + int start = 0; + int end = 0; + if ((end = s.find(sep, start)) != std::string::npos) { + k = s.substr(start, end - start); + v = s.substr(end + 1); + } else { + k = s; + v = ""; + } +} + +// replace env variables +bool replaceenv(std::string& s) { + std::size_t pos = 0; + std::size_t start = 0; + while ((start = s.find("${", pos)) != std::string::npos) { + std::size_t end = s.find("}", start + 2); + if (end == std::string::npos) { + return false; + } + std::string envkey = s.substr(start + 2, end - start - 2); + const char* envval = std::getenv(envkey.c_str()); + if (envval == nullptr) { + return false; + } + s.erase(start, end - start + 1); + s.insert(start, envval); + pos = start + strlen(envval); + } + return true; +} + +bool strtox(const std::string& valstr, bool& retval); +bool strtox(const std::string& valstr, int16_t& retval); +bool strtox(const std::string& valstr, int32_t& retval); +bool strtox(const std::string& valstr, int64_t& retval); +bool strtox(const std::string& valstr, double& retval); +bool strtox(const std::string& valstr, std::string& retval); + +template +bool strtox(const std::string& valstr, std::vector& retval) { + std::stringstream ss(valstr); + std::string item; + T t; + while (std::getline(ss, item, ',')) { + if (!strtox(trim(item), t)) { + return false; + } + retval.push_back(t); + } + return true; +} + +bool strtox(const std::string& valstr, bool& retval) { + if (valstr.compare("true") == 0) { + retval = true; + } else if (valstr.compare("false") == 0) { + retval = false; + } else { + return false; + } + return true; +} + +template +bool strtointeger(const std::string& valstr, T& retval) { + if (valstr.length() == 0) { + return false; // empty-string is only allowed for string type. + } + char* end; + errno = 0; + const char* valcstr = valstr.c_str(); + int64_t ret64 = strtoll(valcstr, &end, 10); + if (errno || end != valcstr + strlen(valcstr)) { + return false; // bad parse + } + T tmp = retval; + retval = static_cast(ret64); + if (retval != ret64) { + retval = tmp; + return false; + } + return true; +} + +bool strtox(const std::string& valstr, int16_t& retval) { + return strtointeger(valstr, retval); +} + +bool strtox(const std::string& valstr, int32_t& retval) { + return strtointeger(valstr, retval); +} + +bool strtox(const std::string& valstr, int64_t& retval) { + return strtointeger(valstr, retval); +} + +bool strtox(const std::string& valstr, double& retval) { + if (valstr.length() == 0) { + return false; // empty-string is only allowed for string type. + } + char* end = nullptr; + errno = 0; + const char* valcstr = valstr.c_str(); + retval = strtod(valcstr, &end); + if (errno || end != valcstr + strlen(valcstr)) { + return false; // bad parse + } + return true; +} + +bool strtox(const std::string& valstr, std::string& retval) { + retval = valstr; + return true; +} + +template +bool convert(const std::string& value, T& retval) { + std::string valstr(value); + trim(valstr); + if (!replaceenv(valstr)) { + return false; + } + return strtox(valstr, retval); +} + +// load conf file +bool Properties::load(const char* conf_file, bool must_exist) { + // if conf_file is null, use the empty props + if (conf_file == nullptr) { + return true; + } + + // open the conf file + std::ifstream input(conf_file); + if (!input.is_open()) { + if (must_exist) { + std::cerr << "config::load() failed to open the file:" << conf_file << std::endl; + return false; + } + return true; + } + + // load properties + std::string line; + std::string key; + std::string value; + line.reserve(512); + while (input) { + // read one line at a time + std::getline(input, line); + + // remove left and right spaces + trim(line); + + // ignore comments + if (line.empty() || line[0] == '#') { + continue; + } + + // read key and value + splitkv(line, key, value); + trim(key); + trim(value); + + // insert into file_conf_map + file_conf_map[key] = value; + } + + // close the conf file + input.close(); + + return true; +} + +template +bool Properties::get_or_default(const char* key, const char* defstr, T& retval, + bool* is_retval_set) const { + const auto& it = file_conf_map.find(std::string(key)); + std::string valstr; + if (it == file_conf_map.end()) { + if (defstr == nullptr) { + // Not found in conf map, and no default value need to be set, just return + *is_retval_set = false; + return true; + } else { + valstr = std::string(defstr); + } + } else { + valstr = it->second; + } + *is_retval_set = true; + return convert(valstr, retval); +} + +void Properties::set(const std::string& key, const std::string& val) { + file_conf_map.emplace(key, val); +} + +void Properties::set_force(const std::string& key, const std::string& val) { + file_conf_map[key] = val; +} + +template +std::ostream& operator<<(std::ostream& out, const std::vector& v) { + size_t last = v.size() - 1; + for (size_t i = 0; i < v.size(); ++i) { + out << v[i]; + if (i != last) { + out << ", "; + } + } + return out; +} + +#define SET_FIELD(FIELD, TYPE, FILL_CONF_MAP, SET_TO_DEFAULT) \ + if (strcmp((FIELD).type, #TYPE) == 0) { \ + TYPE new_value = TYPE(); \ + bool is_newval_set = false; \ + if (!props.get_or_default((FIELD).name, ((SET_TO_DEFAULT) ? (FIELD).defval : nullptr), \ + new_value, &is_newval_set)) { \ + std::cerr << "config field error: " << (FIELD).name << std::endl; \ + return false; \ + } \ + if (!is_newval_set) { \ + continue; \ + } \ + TYPE& ref_conf_value = *reinterpret_cast((FIELD).storage); \ + TYPE old_value = ref_conf_value; \ + ref_conf_value = new_value; \ + if (RegisterConfValidator::_s_field_validator != nullptr) { \ + auto validator = RegisterConfValidator::_s_field_validator->find((FIELD).name); \ + if (validator != RegisterConfValidator::_s_field_validator->end() && \ + !(validator->second)()) { \ + ref_conf_value = old_value; \ + std::cerr << "validate " << (FIELD).name << "=" << new_value << " failed" \ + << std::endl; \ + return false; \ + } \ + } \ + if (FILL_CONF_MAP) { \ + std::ostringstream oss; \ + oss << ref_conf_value; \ + (*full_conf_map)[(FIELD).name] = oss.str(); \ + } \ + continue; \ + } + +// init conf fields +bool init(const char* conf_file, bool fill_conf_map, bool must_exist, bool set_to_default) { + Properties props; + // load properties file + if (!props.load(conf_file, must_exist)) { + return false; + } + // fill full_conf_map ? + if (fill_conf_map && full_conf_map == nullptr) { + full_conf_map = new std::map(); + } + + // set conf fields + for (const auto& it : *Register::_s_field_map) { + SET_FIELD(it.second, bool, fill_conf_map, set_to_default); + SET_FIELD(it.second, int16_t, fill_conf_map, set_to_default); + SET_FIELD(it.second, int32_t, fill_conf_map, set_to_default); + SET_FIELD(it.second, int64_t, fill_conf_map, set_to_default); + SET_FIELD(it.second, double, fill_conf_map, set_to_default); + SET_FIELD(it.second, std::string, fill_conf_map, set_to_default); + SET_FIELD(it.second, std::vector, fill_conf_map, set_to_default); + SET_FIELD(it.second, std::vector, fill_conf_map, set_to_default); + SET_FIELD(it.second, std::vector, fill_conf_map, set_to_default); + SET_FIELD(it.second, std::vector, fill_conf_map, set_to_default); + SET_FIELD(it.second, std::vector, fill_conf_map, set_to_default); + SET_FIELD(it.second, std::vector, fill_conf_map, set_to_default); + } + + return true; +} + +} // namespace doris::cloud::config diff --git a/cloud/src/common/configbase.h b/cloud/src/common/configbase.h new file mode 100644 index 00000000000000..a3b3b70829a924 --- /dev/null +++ b/cloud/src/common/configbase.h @@ -0,0 +1,173 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace doris::cloud::config { + +class Register { +public: + struct Field { + const char* type = nullptr; + const char* name = nullptr; + void* storage = nullptr; + const char* defval = nullptr; + bool valmutable = false; + Field(const char* ftype, const char* fname, void* fstorage, const char* fdefval, + bool fvalmutable) + : type(ftype), + name(fname), + storage(fstorage), + defval(fdefval), + valmutable(fvalmutable) {} + }; + +public: + static std::map* _s_field_map; + +public: + Register(const char* ftype, const char* fname, void* fstorage, const char* fdefval, + bool fvalmutable) { + if (_s_field_map == nullptr) { + _s_field_map = new std::map(); + } + Field field(ftype, fname, fstorage, fdefval, fvalmutable); + _s_field_map->insert(std::make_pair(std::string(fname), field)); + } +}; + +// RegisterConfValidator class is used to store validator function of registered config fields in +// Register::_s_field_map. +// If any validator return false when BE bootstart, the bootstart will be terminated. +// If validator return false when use http API to update some config, the config will not +// be modified and the API will return failure. +class RegisterConfValidator { +public: + // Validator for each config name. + static std::map>* _s_field_validator; + +public: + RegisterConfValidator(const char* fname, const std::function& validator) { + if (_s_field_validator == nullptr) { + _s_field_validator = new std::map>(); + } + // register validator to _s_field_validator + _s_field_validator->insert(std::make_pair(std::string(fname), validator)); + } +}; + +#define DEFINE_FIELD(FIELD_TYPE, FIELD_NAME, FIELD_DEFAULT, VALMUTABLE) \ + FIELD_TYPE FIELD_NAME; \ + static Register reg_##FIELD_NAME(#FIELD_TYPE, #FIELD_NAME, &(FIELD_NAME), FIELD_DEFAULT, \ + VALMUTABLE); + +#define DECLARE_FIELD(FIELD_TYPE, FIELD_NAME) extern FIELD_TYPE FIELD_NAME; + +#define DEFINE_VALIDATOR(FIELD_NAME, VALIDATOR) \ + static auto validator_##FIELD_NAME = VALIDATOR; \ + static RegisterConfValidator reg_validator_##FIELD_NAME( \ + #FIELD_NAME, []() -> bool { return validator_##FIELD_NAME(FIELD_NAME); }); + +#define DECLARE_VALIDATOR(FIELD_NAME) ; + +#ifdef __IN_CONFIGBASE_CPP__ +#define CONF_Bool(name, defaultstr) DEFINE_FIELD(bool, name, defaultstr, false) +#define CONF_Int16(name, defaultstr) DEFINE_FIELD(int16_t, name, defaultstr, false) +#define CONF_Int32(name, defaultstr) DEFINE_FIELD(int32_t, name, defaultstr, false) +#define CONF_Int64(name, defaultstr) DEFINE_FIELD(int64_t, name, defaultstr, false) +#define CONF_Double(name, defaultstr) DEFINE_FIELD(double, name, defaultstr, false) +#define CONF_String(name, defaultstr) DEFINE_FIELD(std::string, name, defaultstr, false) +#define CONF_Bools(name, defaultstr) DEFINE_FIELD(std::vector, name, defaultstr, false) +#define CONF_Int16s(name, defaultstr) DEFINE_FIELD(std::vector, name, defaultstr, false) +#define CONF_Int32s(name, defaultstr) DEFINE_FIELD(std::vector, name, defaultstr, false) +#define CONF_Int64s(name, defaultstr) DEFINE_FIELD(std::vector, name, defaultstr, false) +#define CONF_Doubles(name, defaultstr) DEFINE_FIELD(std::vector, name, defaultstr, false) +#define CONF_Strings(name, defaultstr) \ + DEFINE_FIELD(std::vector, name, defaultstr, false) +#define CONF_mBool(name, defaultstr) DEFINE_FIELD(bool, name, defaultstr, true) +#define CONF_mInt16(name, defaultstr) DEFINE_FIELD(int16_t, name, defaultstr, true) +#define CONF_mInt32(name, defaultstr) DEFINE_FIELD(int32_t, name, defaultstr, true) +#define CONF_mInt64(name, defaultstr) DEFINE_FIELD(int64_t, name, defaultstr, true) +#define CONF_mDouble(name, defaultstr) DEFINE_FIELD(double, name, defaultstr, true) +#define CONF_mString(name, defaultstr) DEFINE_FIELD(std::string, name, defaultstr, true) +#define CONF_Validator(name, validator) DEFINE_VALIDATOR(name, validator) + +#else +#define CONF_Bool(name, defaultstr) DECLARE_FIELD(bool, name) +#define CONF_Int16(name, defaultstr) DECLARE_FIELD(int16_t, name) +#define CONF_Int32(name, defaultstr) DECLARE_FIELD(int32_t, name) +#define CONF_Int64(name, defaultstr) DECLARE_FIELD(int64_t, name) +#define CONF_Double(name, defaultstr) DECLARE_FIELD(double, name) +#define CONF_String(name, defaultstr) DECLARE_FIELD(std::string, name) +#define CONF_Bools(name, defaultstr) DECLARE_FIELD(std::vector, name) +#define CONF_Int16s(name, defaultstr) DECLARE_FIELD(std::vector, name) +#define CONF_Int32s(name, defaultstr) DECLARE_FIELD(std::vector, name) +#define CONF_Int64s(name, defaultstr) DECLARE_FIELD(std::vector, name) +#define CONF_Doubles(name, defaultstr) DECLARE_FIELD(std::vector, name) +#define CONF_Strings(name, defaultstr) DECLARE_FIELD(std::vector, name) +#define CONF_mBool(name, defaultstr) DECLARE_FIELD(bool, name) +#define CONF_mInt16(name, defaultstr) DECLARE_FIELD(int16_t, name) +#define CONF_mInt32(name, defaultstr) DECLARE_FIELD(int32_t, name) +#define CONF_mInt64(name, defaultstr) DECLARE_FIELD(int64_t, name) +#define CONF_mDouble(name, defaultstr) DECLARE_FIELD(double, name) +#define CONF_mString(name, defaultstr) DECLARE_FIELD(std::string, name) +#define CONF_Validator(name, validator) DECLARE_VALIDATOR(name) +#endif + +// configuration properties load from config file. +class Properties { +public: + // load conf from file, if must_exist is true and file does not exist, return false + bool load(const char* conf_file, bool must_exist = true); + + // Find the config value by key from `file_conf_map`. + // If found, set `retval` to the config value, + // or set `retval` to `defstr` + // if retval is not set(in case defstr is nullptr), set is_retval_set to false + template + bool get_or_default(const char* key, const char* defstr, T& retval, bool* is_retval_set) const; + + void set(const std::string& key, const std::string& val); + + void set_force(const std::string& key, const std::string& val); + + // dump props to conf file + bool dump(const std::string& conffile); + +private: + std::map file_conf_map; +}; + +// full configurations. +extern std::map* full_conf_map; + +// Init the config from `conf_file`. +// If fill_conf_map is true, the updated config will also update the `full_conf_map`. +// If must_exist is true and `conf_file` does not exist, this function will return false. +// If set_to_default is true, the config value will be set to default value if not found in `conf_file`. +// Return true if load +bool init(const char* conf_file, bool fill_conf_map = false, bool must_exist = true, + bool set_to_default = true); + +} // namespace doris::cloud::config diff --git a/cloud/src/common/encryption_util.cpp b/cloud/src/common/encryption_util.cpp new file mode 100644 index 00000000000000..e5347cb9ceb082 --- /dev/null +++ b/cloud/src/common/encryption_util.cpp @@ -0,0 +1,836 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "common/encryption_util.h" + +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +#include "common/config.h" +#include "common/kms.h" +#include "common/logging.h" +#include "common/sync_point.h" +#include "common/util.h" +#include "meta-service/keys.h" +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" + +namespace doris::cloud { +namespace config { +extern std::string encryption_key; +}; // namespace config + +enum class EncryptionMode { + AES_128_ECB, + AES_256_ECB, + AES_128_CBC, + AES_256_CBC, + AES_128_CFB, + AES_256_CFB, + AES_128_CFB1, + AES_256_CFB1, + AES_128_CFB8, + AES_256_CFB8, + AES_128_CFB128, + AES_256_CFB128, + AES_128_CTR, + AES_256_CTR, + AES_128_OFB, + AES_256_OFB, +}; + +enum EncryptionState { AES_SUCCESS = 0, AES_BAD_DATA = -1 }; + +class EncryptionUtil { +public: + static int encrypt(EncryptionMode mode, const unsigned char* source, uint32_t source_length, + const unsigned char* key, uint32_t key_length, const char* iv_str, + int iv_input_length, bool padding, unsigned char* encrypt); + + static int decrypt(EncryptionMode mode, const unsigned char* encrypt, uint32_t encrypt_length, + const unsigned char* key, uint32_t key_length, const char* iv_str, + int iv_input_length, bool padding, unsigned char* decrypt_content); +}; + +// aes encrypt/dencrypt +static const int ENCRYPTION_MAX_KEY_LENGTH = 256; + +const EVP_CIPHER* get_evp_type(const EncryptionMode mode) { + switch (mode) { + case EncryptionMode::AES_128_ECB: + return EVP_aes_128_ecb(); + case EncryptionMode::AES_128_CBC: + return EVP_aes_128_cbc(); + case EncryptionMode::AES_128_CFB: + return EVP_aes_128_cfb(); + case EncryptionMode::AES_128_CFB1: + return EVP_aes_128_cfb1(); + case EncryptionMode::AES_128_CFB8: + return EVP_aes_128_cfb8(); + case EncryptionMode::AES_128_CFB128: + return EVP_aes_128_cfb128(); + case EncryptionMode::AES_128_CTR: + return EVP_aes_128_ctr(); + case EncryptionMode::AES_128_OFB: + return EVP_aes_128_ofb(); + case EncryptionMode::AES_256_ECB: + return EVP_aes_256_ecb(); + case EncryptionMode::AES_256_CBC: + return EVP_aes_256_cbc(); + case EncryptionMode::AES_256_CFB: + return EVP_aes_256_cfb(); + case EncryptionMode::AES_256_CFB1: + return EVP_aes_256_cfb1(); + case EncryptionMode::AES_256_CFB8: + return EVP_aes_256_cfb8(); + case EncryptionMode::AES_256_CFB128: + return EVP_aes_256_cfb128(); + case EncryptionMode::AES_256_CTR: + return EVP_aes_256_ctr(); + case EncryptionMode::AES_256_OFB: + return EVP_aes_256_ofb(); + default: + return nullptr; + } +} + +static uint mode_key_sizes[] = { + 128 /* AES_128_ECB */, 256 /* AES_256_ECB */, 128 /* AES_128_CBC */, + 256 /* AES_256_CBC */, 128 /* AES_128_CFB */, 256 /* AES_256_CFB */, + 128 /* AES_128_CFB1 */, 256 /* AES_256_CFB1 */, 128 /* AES_128_CFB8 */, + 256 /* AES_256_CFB8 */, 128 /* AES_128_CFB128 */, 256 /* AES_256_CFB128 */, + 128 /* AES_128_CTR */, 256 /* AES_256_CTR */, 128 /* AES_128_OFB */, + 256 /* AES_256_OFB */, +}; + +static void create_key(const unsigned char* origin_key, uint32_t key_length, uint8_t* encrypt_key, + EncryptionMode mode) { + const uint key_size = mode_key_sizes[int(mode)] / 8; + uint8_t* origin_key_end = ((uint8_t*)origin_key) + key_length; /* origin key boundary*/ + + uint8_t* encrypt_key_end; /* encrypt key boundary */ + encrypt_key_end = encrypt_key + key_size; + + std::memset(encrypt_key, 0, key_size); /* initialize key */ + + uint8_t* ptr; /* Start of the encrypt key*/ + uint8_t* origin_ptr; /* Start of the origin key */ + for (ptr = encrypt_key, origin_ptr = (uint8_t*)origin_key; origin_ptr < origin_key_end; + ptr++, origin_ptr++) { + if (ptr == encrypt_key_end) { + /* loop over origin key until we used all key */ + ptr = encrypt_key; + } + *ptr ^= *origin_ptr; + } +} + +static int do_encrypt(EVP_CIPHER_CTX* cipher_ctx, const EVP_CIPHER* cipher, + const unsigned char* source, uint32_t source_length, + const unsigned char* encrypt_key, const unsigned char* iv, bool padding, + unsigned char* encrypt, int* length_ptr) { + int ret = EVP_EncryptInit(cipher_ctx, cipher, encrypt_key, iv); + if (ret == 0) { + return ret; + } + ret = EVP_CIPHER_CTX_set_padding(cipher_ctx, padding); + if (ret == 0) { + return ret; + } + int u_len = 0; + + ret = EVP_EncryptUpdate(cipher_ctx, encrypt, &u_len, source, source_length); + if (ret == 0) { + return ret; + } + int f_len = 0; + ret = EVP_EncryptFinal(cipher_ctx, encrypt + u_len, &f_len); + *length_ptr = u_len + f_len; + return ret; +} + +int EncryptionUtil::encrypt(EncryptionMode mode, const unsigned char* source, + uint32_t source_length, const unsigned char* key, uint32_t key_length, + const char* iv_str, int iv_input_length, bool padding, + unsigned char* encrypt) { + const EVP_CIPHER* cipher = get_evp_type(mode); + /* The encrypt key to be used for encryption */ + unsigned char encrypt_key[ENCRYPTION_MAX_KEY_LENGTH / 8]; + create_key(key, key_length, encrypt_key, mode); + + int iv_length = EVP_CIPHER_iv_length(cipher); + if (cipher == nullptr || (iv_length > 0 && !iv_str)) { + return AES_BAD_DATA; + } + char* init_vec = nullptr; + std::string iv_default("SELECTDBCLOUD___"); + + if (iv_str) { + init_vec = iv_default.data(); + memcpy(init_vec, iv_str, std::min(iv_input_length, EVP_MAX_IV_LENGTH)); + init_vec[iv_length] = '\0'; + } + EVP_CIPHER_CTX* cipher_ctx = EVP_CIPHER_CTX_new(); + EVP_CIPHER_CTX_reset(cipher_ctx); + int length = 0; + int ret = do_encrypt(cipher_ctx, cipher, source, source_length, encrypt_key, + reinterpret_cast(init_vec), padding, encrypt, &length); + EVP_CIPHER_CTX_free(cipher_ctx); + if (ret == 0) { + ERR_clear_error(); + return AES_BAD_DATA; + } else { + return length; + } +} + +static int do_decrypt(EVP_CIPHER_CTX* cipher_ctx, const EVP_CIPHER* cipher, + const unsigned char* encrypt, uint32_t encrypt_length, + const unsigned char* encrypt_key, const unsigned char* iv, bool padding, + unsigned char* decrypt_content, int* length_ptr) { + int ret = EVP_DecryptInit(cipher_ctx, cipher, encrypt_key, iv); + if (ret == 0) { + return ret; + } + ret = EVP_CIPHER_CTX_set_padding(cipher_ctx, padding); + if (ret == 0) { + return ret; + } + int u_len = 0; + ret = EVP_DecryptUpdate(cipher_ctx, decrypt_content, &u_len, encrypt, encrypt_length); + if (ret == 0) { + return ret; + } + int f_len = 0; + ret = EVP_DecryptFinal_ex(cipher_ctx, decrypt_content + u_len, &f_len); + *length_ptr = u_len + f_len; + return ret; +} + +int EncryptionUtil::decrypt(EncryptionMode mode, const unsigned char* encrypt, + uint32_t encrypt_length, const unsigned char* key, uint32_t key_length, + const char* iv_str, int iv_input_length, bool padding, + unsigned char* decrypt_content) { + const EVP_CIPHER* cipher = get_evp_type(mode); + + /* The encrypt key to be used for decryption */ + unsigned char encrypt_key[ENCRYPTION_MAX_KEY_LENGTH / 8]; + create_key(key, key_length, encrypt_key, mode); + + int iv_length = EVP_CIPHER_iv_length(cipher); + if (cipher == nullptr || (iv_length > 0 && !iv_str)) { + return AES_BAD_DATA; + } + char* init_vec = nullptr; + std::string iv_default("SELECTDBCLOUD___"); + + if (iv_str) { + init_vec = iv_default.data(); + memcpy(init_vec, iv_str, std::min(iv_input_length, EVP_MAX_IV_LENGTH)); + init_vec[iv_length] = '\0'; + } + EVP_CIPHER_CTX* cipher_ctx = EVP_CIPHER_CTX_new(); + EVP_CIPHER_CTX_reset(cipher_ctx); + int length = 0; + int ret = do_decrypt(cipher_ctx, cipher, encrypt, encrypt_length, encrypt_key, + reinterpret_cast(init_vec), padding, decrypt_content, + &length); + EVP_CIPHER_CTX_free(cipher_ctx); + if (ret > 0) { + return length; + } else { + ERR_clear_error(); + return AES_BAD_DATA; + } +} + +// base64 endcode/decode + +static char encoding_table[] = {'A', 'B', 'C', 'D', 'E', 'F', 'G', 'H', 'I', 'J', 'K', 'L', 'M', + 'N', 'O', 'P', 'Q', 'R', 'S', 'T', 'U', 'V', 'W', 'X', 'Y', 'Z', + 'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm', + 'n', 'o', 'p', 'q', 'r', 's', 't', 'u', 'v', 'w', 'x', 'y', 'z', + '0', '1', '2', '3', '4', '5', '6', '7', '8', '9', '+', '/'}; + +static const char base64_pad = '='; + +static short decoding_table[256] = { + -2, -2, -2, -2, -2, -2, -2, -2, -2, -1, -1, -2, -2, -1, -2, -2, -2, -2, -2, -2, -2, -2, + -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -1, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, 62, + -2, -2, -2, 63, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, -2, -2, -2, -2, -2, -2, -2, 0, + 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, + 23, 24, 25, -2, -2, -2, -2, -2, -2, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, + 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, -2, -2, -2, -2, -2, -2, -2, -2, -2, + -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, + -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, + -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, + -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, + -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, + -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2}; + +static int mod_table[] = {0, 2, 1}; + +size_t base64_encode(const unsigned char* data, size_t length, unsigned char* encoded_data) { + size_t output_length = (size_t)(4.0 * ceil((double)length / 3.0)); + + if (encoded_data == nullptr) { + return 0; + } + + for (uint32_t i = 0, j = 0; i < length;) { + uint32_t octet_a = i < length ? data[i++] : 0; + uint32_t octet_b = i < length ? data[i++] : 0; + uint32_t octet_c = i < length ? data[i++] : 0; + uint32_t triple = (octet_a << 0x10) + (octet_b << 0x08) + octet_c; + + encoded_data[j++] = encoding_table[(triple >> 3 * 6) & 0x3F]; + encoded_data[j++] = encoding_table[(triple >> 2 * 6) & 0x3F]; + encoded_data[j++] = encoding_table[(triple >> 1 * 6) & 0x3F]; + encoded_data[j++] = encoding_table[(triple >> 0 * 6) & 0x3F]; + } + + for (int i = 0; i < mod_table[length % 3]; i++) { + encoded_data[output_length - 1 - i] = '='; + } + + return output_length; +} + +size_t base64_decode(const char* data, size_t length, char* decoded_data) { + const char* current = data; + size_t ch = 0; + size_t i = 0; + size_t j = 0; + size_t k = 0; + + // run through the whole string, converting as we go + while ((ch = *current++) != '\0' && length-- > 0) { + if (ch >= 256 || ch < 0) { + return -1; + } + + if (ch == base64_pad) { + if (*current != '=' && (i % 4) == 1) { + return -1; + } + continue; + } + + ch = decoding_table[ch]; + // a space or some other separator character, we simply skip over + if (ch == -1) { + continue; + } else if (ch == -2) { + return -1; + } + + switch (i % 4) { + case 0: + decoded_data[j] = ch << 2; + break; + case 1: + decoded_data[j++] |= ch >> 4; + decoded_data[j] = (ch & 0x0f) << 4; + break; + case 2: + decoded_data[j++] |= ch >> 2; + decoded_data[j] = (ch & 0x03) << 6; + break; + case 3: + decoded_data[j++] |= ch; + break; + default: + break; + } + + i++; + } + + k = j; + /* mop things up if we ended on a boundary */ + if (ch == base64_pad) { + switch (i % 4) { + case 1: + return 0; + case 2: + k++; + case 3: + decoded_data[k] = 0; + default: + break; + } + } + + decoded_data[j] = '\0'; + + return j; +} + +// encrypt/dencrypt with base64 + +static std::unordered_map to_encryption_mode { + {"AES_128_ECB", EncryptionMode::AES_128_ECB}, + {"AES_256_ECB", EncryptionMode::AES_256_ECB}, + {"AES_128_CBC", EncryptionMode::AES_128_CBC}, + {"AES_256_CBC", EncryptionMode::AES_256_CBC}, + {"AES_128_CFB", EncryptionMode::AES_128_CFB}, + {"AES_256_CFB", EncryptionMode::AES_256_CFB}, + {"AES_128_CFB1", EncryptionMode::AES_128_CFB1}, + {"AES_256_CFB1", EncryptionMode::AES_256_CFB1}, + {"AES_128_CFB8", EncryptionMode::AES_128_CFB8}, + {"AES_256_CFB8", EncryptionMode::AES_256_CFB8}, + {"AES_128_CFB128", EncryptionMode::AES_128_CFB128}, + {"AES_256_CFB128", EncryptionMode::AES_256_CFB128}, + {"AES_128_CTR", EncryptionMode::AES_128_CTR}, + {"AES_256_CTR", EncryptionMode::AES_256_CTR}, + {"AES_128_OFB", EncryptionMode::AES_128_OFB}, + {"AES_256_OFB", EncryptionMode::AES_256_OFB}, +}; + +static inline int encrypt_to_base64_impl(std::string_view source, EncryptionMode mode, + const std::string& key, std::string* encrypt) { + /* + * Buffer for ciphertext. Ensure the buffer is long enough for the + * ciphertext which may be longer than the plaintext, depending on the + * algorithm and mode. + */ + int cipher_len = source.length() + 16; + std::string cipher_text(cipher_len, '0'); + int cipher_text_len = EncryptionUtil::encrypt( + mode, (unsigned char*)source.data(), source.length(), (unsigned char*)key.c_str(), + key.length(), nullptr, 0, true, (unsigned char*)cipher_text.data()); + if (cipher_text_len < 0) { + return -1; + } + + int encoded_len = (size_t)(4.0 * ceil(cipher_text_len / 3.0)); + std::string encoded_text(encoded_len, '0'); + int encoded_text_len = base64_encode((unsigned char*)cipher_text.data(), cipher_text_len, + (unsigned char*)encoded_text.data()); + if (encoded_text_len < 0) { + return -1; + } + encrypt->assign((char*)encoded_text.data(), encoded_text_len); + return 0; +} + +static int encrypt_to_base64(std::string_view source, const std::string& encrypt_method, + const std::string& key, std::string* encrypt) { + if (source.empty()) { + *encrypt = ""; + return 0; + } + auto iter = to_encryption_mode.find(encrypt_method); + if (iter == to_encryption_mode.end()) { + return -1; + } + + return encrypt_to_base64_impl(source, iter->second, key, encrypt); +} + +static inline int decrypt_with_base64_impl(std::string_view encrypt, EncryptionMode mode, + const std::string& key, std::string* source) { + // base64 + std::unique_ptr decoded_text(new char[encrypt.length()]); + int decoded_text_len = base64_decode(encrypt.data(), encrypt.length(), decoded_text.get()); + if (decoded_text_len < 0) { + return -1; + } + + std::unique_ptr plain_text(new char[decoded_text_len]); + int plain_text_len = EncryptionUtil::decrypt( + mode, (unsigned char*)decoded_text.get(), decoded_text_len, (unsigned char*)key.c_str(), + key.length(), nullptr, 0, true, (unsigned char*)plain_text.get()); + if (plain_text_len < 0) { + return -1; + } + source->assign(plain_text.get(), plain_text_len); + return 0; +} + +static int decrypt_with_base64(std::string_view encrypt, const std::string& encrypt_method, + const std::string& key, std::string* source) { + if (encrypt.empty()) { + *source = ""; + return 0; + } + auto iter = to_encryption_mode.find(encrypt_method); + if (iter == to_encryption_mode.end()) { + return -1; + } + return decrypt_with_base64_impl(encrypt, iter->second, key, source); +} + +int encrypt_ak_sk(AkSkRef plain_ak_sk, const std::string& encryption_method, + const std::string& encryption_key, AkSkPair* cipher_ak_sk) { + std::string encrypt_ak; + std::string encrypt_sk; + if (encrypt_to_base64(plain_ak_sk.second, encryption_method, encryption_key, &encrypt_sk) != + 0) { + *cipher_ak_sk = {"", ""}; + return -1; + } + *cipher_ak_sk = {std::string(plain_ak_sk.first), std::move(encrypt_sk)}; + return 0; +} + +int decrypt_ak_sk(AkSkRef cipher_ak_sk, const std::string& encryption_method, + const std::string& encryption_key, AkSkPair* plain_ak_sk) { + std::string ak; + std::string sk; + if (decrypt_with_base64(cipher_ak_sk.second, encryption_method, encryption_key, &sk) != 0) { + *plain_ak_sk = {"", ""}; + return -1; + } + *plain_ak_sk = {std::string(cipher_ak_sk.first), std::move(sk)}; + return 0; +} + +int decrypt_ak_sk_helper(std::string_view cipher_ak, std::string_view cipher_sk, + const EncryptionInfoPB& encryption_info, AkSkPair* plain_ak_sk_pair) { + std::string key; + int ret = get_encryption_key_for_ak_sk(encryption_info.key_id(), &key); + { + TEST_SYNC_POINT_CALLBACK("decrypt_ak_sk:get_encryption_key_ret", &ret); + TEST_SYNC_POINT_CALLBACK("decrypt_ak_sk:get_encryption_key", &key); + } + if (ret != 0) { + LOG(WARNING) << "failed to get encryption key version_id: " << encryption_info.key_id(); + return -1; + } + ret = decrypt_ak_sk({cipher_ak, cipher_sk}, encryption_info.encryption_method(), key, + plain_ak_sk_pair); + if (ret != 0) { + LOG(WARNING) << "failed to decrypt"; + return -1; + } + return 0; +} + +/** + * @brief Generates a random root key. If a root key already exists, returns immediately. + * + * @param txn_kv + * @param kms_client + * @param plaintext store the plaintext of the root key + * @param encoded_ciphertext store the base64-encoded ciphertext of the root key. + * @return int 0 for success to generate, 1 for not need to generate, -1 for failure. + */ +static int generate_random_root_key(TxnKv* txn_kv, KmsClient* kms_client, std::string* plaintext, + std::string* encoded_ciphertext) { + /** + * 1. If KMS is enabled, use KMS to generate a new key. + * 2. If KMS is not enabled, try using the encryption_key from the configuration, which must be in Base64 format. + * 3. If no key is found in the configuration, generate a random key in memory. + */ + std::string key = system_meta_service_encryption_key_info_key(); + std::string val; + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG_WARNING("failed to create txn").tag("err", err); + return -1; + } + err = txn->get(key, &val); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + LOG_WARNING("failed to get key of encryption_key_info").tag("err", err); + return -1; + } + + if (err == TxnErrorCode::TXN_OK) { + if (config::enable_kms && config::focus_add_kms_data_key) { + EncryptionKeyInfoPB key_info; + if (!key_info.ParseFromString(val)) { + LOG_WARNING("failed to parse encryption_root_key"); + return -1; + } + for (const auto& item : key_info.items()) { + if (item.has_kms_info()) { + return 1; + } + } + LOG(INFO) << "focus to create kms data key"; + } else { + LOG(INFO) << "not need to generate root key"; + return 1; + } + } + + // 1. use kms to generate a new key + if (config::enable_kms) { + if (kms_client == nullptr) { + LOG_WARNING("no kms client"); + return -1; + } + std::string encoded_root_key_ciphertext; + std::string encoded_root_key_plaintext; + if (kms_client->generate_data_key(&encoded_root_key_ciphertext, + &encoded_root_key_plaintext) != 0) { + LOG_WARNING("failed to generate data key"); + return -1; + } + if (encoded_root_key_ciphertext.empty() || encoded_root_key_plaintext.empty()) { + LOG_WARNING("empty data key generated"); + return -1; + } + + // decode plaintext + std::string root_key_plaintext(encoded_root_key_plaintext.length(), '0'); + int decoded_len = + base64_decode(encoded_root_key_plaintext.c_str(), + encoded_root_key_plaintext.length(), root_key_plaintext.data()); + if (decoded_len < 0) { + LOG_WARNING("failed to decode plaintext of kms"); + return -1; + } + root_key_plaintext.assign(root_key_plaintext.data(), decoded_len); + + *plaintext = std::move(root_key_plaintext); + *encoded_ciphertext = std::move(encoded_root_key_ciphertext); + return 0; + } + + // 2. try using the encryption_key from the configuration + if (!cloud::config::encryption_key.empty()) { + std::string decoded_string(cloud::config::encryption_key.length(), '0'); + int decoded_text_len = + base64_decode(cloud::config::encryption_key.c_str(), + cloud::config::encryption_key.length(), decoded_string.data()); + if (decoded_text_len < 0) { + LOG_WARNING("fail to decode encryption_key in config"); + return -1; + } + decoded_string.assign(decoded_string.data(), decoded_text_len); + *plaintext = std::move(decoded_string); + *encoded_ciphertext = cloud::config::encryption_key; + return 0; + } + + // 3. otherwise, generate a random data key in memory + std::mt19937 rnd(time(nullptr)); + std::uniform_int_distribution dist(std::numeric_limits::min(), + std::numeric_limits::max()); + std::string root_key_plaintext(32, '0'); + for (char& i : root_key_plaintext) { + i = (char)dist(rnd); + } + + // encode in base64 + int key_len = root_key_plaintext.length(); + int encoded_len = (size_t)(4.0 * ceil(key_len / 3.0)); + std::string encoded_text(encoded_len, '0'); + int encoded_text_len = base64_encode((unsigned char*)root_key_plaintext.data(), key_len, + (unsigned char*)encoded_text.data()); + if (encoded_text_len < 0) { + LOG_WARNING("failed to encode encryption_key"); + return -1; + } + std::string encoded_root_key_ciphertext; + encoded_root_key_ciphertext.assign(encoded_text.data(), encoded_text_len); + + *plaintext = std::move(root_key_plaintext); + *encoded_ciphertext = std::move(encoded_root_key_ciphertext); + return 0; +} + +// Todo: Does not need to be locked now, only generated when the process is initialized +std::map global_encryption_key_info_map; // key_id->encryption_key + +static int get_current_root_keys(TxnKv* txn_kv, std::map* keys) { + std::unique_ptr kms_client; + if (config::enable_kms) { + if (config::kms_info_encryption_key.empty() || config::kms_info_encryption_method.empty() || + config::kms_ak.empty() || config::kms_sk.empty()) { + LOG_WARNING("incorrect kms conf") + .tag("encryption_key", config::kms_info_encryption_key) + .tag("encryption_method", config::kms_info_encryption_method) + .tag("ak", config::kms_ak) + .tag("sk", config::kms_sk); + return -1; + } + std::string decoded_encryption_key(config::kms_info_encryption_key.length(), '0'); + int decoded_key_len = cloud::base64_decode(config::kms_info_encryption_key.c_str(), + config::kms_info_encryption_key.length(), + decoded_encryption_key.data()); + decoded_encryption_key.assign(decoded_encryption_key.data(), decoded_key_len); + AkSkPair out; + if (decrypt_ak_sk({config::kms_ak, config::kms_sk}, config::kms_info_encryption_method, + decoded_encryption_key, &out) != 0) { + LOG_WARNING("failed to decrypt kms info"); + return -1; + } + + KmsConf conf {out.first, out.second, config::kms_endpoint, + config::kms_region, config::kms_cmk, config::kms_provider}; + + auto ret = create_kms_client(std::move(conf), &kms_client); + if (ret != 0) { + LOG_WARNING("failed to create kms client").tag("ret", ret); + return -1; + } + ret = kms_client->init(); + if (ret != 0) { + LOG_WARNING("failed to init kms client").tag("ret", ret); + return -1; + } + } + + // To avoid transaction timeouts, it is necessary to first generate a root key + std::string root_key_plaintext; + std::string encoded_root_key_ciphertext; + int ret = generate_random_root_key(txn_kv, kms_client.get(), &root_key_plaintext, + &encoded_root_key_ciphertext); + if (ret == -1) { + LOG_WARNING("failed to generate random root key"); + return -1; + } + + while (true) { + std::string key = system_meta_service_encryption_key_info_key(); + std::string val; + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG_WARNING("failed to create txn").tag("ret", ret); + return -1; + } + err = txn->get(key, &val); + if (ret != 0 && ret != 1) { + LOG_WARNING("failed to get key of encryption_key_info").tag("ret", ret); + return -1; + } + + bool need_to_focus_add_kms_data_key = true; + EncryptionKeyInfoPB key_info; + if (err == TxnErrorCode::TXN_OK) { + if (!key_info.ParseFromString(val)) { + LOG_WARNING("failed to parse encryption_root_key"); + return -1; + } + + LOG_INFO("get server encryption_root_key").tag("key_info", proto_to_json(key_info)); + + for (const auto& item : key_info.items()) { + std::string encoded_root_key_plaintext; + if (item.has_kms_info()) { + need_to_focus_add_kms_data_key = false; + // use kms to decrypt + if (kms_client == nullptr) { + LOG_WARNING("no kms client"); + return -1; + } + if (item.kms_info().endpoint() != kms_client->conf().endpoint || + item.kms_info().region() != kms_client->conf().region) { + LOG_WARNING("kms info is not match") + .tag("kms endpoint", kms_client->conf().endpoint) + .tag("kms region", kms_client->conf().region) + .tag("saved endpoint", item.kms_info().endpoint()) + .tag("saved region", item.kms_info().region()); + return -1; + } + + auto ret = kms_client->decrypt(item.key(), &encoded_root_key_plaintext); + if (ret != 0) { + LOG_WARNING("failed to decrypt encryption_root_key"); + return -1; + } + } else { + encoded_root_key_plaintext = item.key(); // Todo: do not copy + } + + std::string root_key_plaintext(encoded_root_key_plaintext.length(), '0'); + int decoded_text_len = base64_decode(encoded_root_key_plaintext.c_str(), + encoded_root_key_plaintext.length(), + root_key_plaintext.data()); + if (decoded_text_len < 0) { + LOG_WARNING("failed to decode encryption_root_key"); + return -1; + } + root_key_plaintext.assign(root_key_plaintext.data(), decoded_text_len); + keys->insert({item.key_id(), std::move(root_key_plaintext)}); + } + if (config::enable_kms && config::focus_add_kms_data_key && + need_to_focus_add_kms_data_key) { + // Todo: need to restart other ms to update global_encryption_key_info_map now + LOG(INFO) << "focus to add kms data key"; + } else { + return 0; + } + } + + // encryption_root_key not found, need to save a new root key into fdb + if (root_key_plaintext.empty() || encoded_root_key_ciphertext.empty()) { + LOG_WARNING("empty new root key"); + return -1; + } + + int32_t new_key_id = key_info.items().size() + 1; + auto* item = key_info.add_items(); + item->set_key_id(new_key_id); + item->set_key(encoded_root_key_ciphertext); + if (config::enable_kms) { + item->mutable_kms_info()->set_endpoint(config::kms_endpoint); + item->mutable_kms_info()->set_region(config::kms_region); + item->mutable_kms_info()->set_cmk(config::kms_cmk); + } + + val = key_info.SerializeAsString(); + if (val.empty()) { + LOG_WARNING("failed to serialize"); + return -1; + } + txn->put(key, val); + LOG_INFO("put server encryption_key") + .tag("encryption_key", encoded_root_key_ciphertext) + .tag("key_id", new_key_id); + err = txn->commit(); + if (err == TxnErrorCode::TXN_CONFLICT) { + LOG_WARNING("commit encryption_key is conflicted, retry it later"); + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + continue; + } else if (err != TxnErrorCode::TXN_OK) { + LOG_WARNING("failed to commit encryption_key"); + return -1; + } + keys->insert({new_key_id, std::move(root_key_plaintext)}); + return 0; + } + return 0; +} + +int init_global_encryption_key_info_map(TxnKv* txn_kv) { + if (get_current_root_keys(txn_kv, &global_encryption_key_info_map) != 0) { + return -1; + } + DCHECK(!global_encryption_key_info_map.empty()); + return 0; +} + +} // namespace doris::cloud diff --git a/cloud/src/common/encryption_util.h b/cloud/src/common/encryption_util.h new file mode 100644 index 00000000000000..b6cb2fb2fde5f8 --- /dev/null +++ b/cloud/src/common/encryption_util.h @@ -0,0 +1,108 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include + +#include +#include +#include +#include +#include + +namespace doris::cloud { +namespace config { +extern std::string encryption_method; +} // namespace config +class TxnKv; +class EncryptionInfoPB; + +using AkSkPair = std::pair; +using AkSkRef = std::pair; +/** + * @brief Encrypt ak/sk pair + * + * @param plain_ak_sk plain ak/sk pair + * @param encryption_method encryption_method + * @param encryption_key encryption_key + * @param cipher_ak_sk output cipher ak/sk pair in base64 format + * @return 0 for success, negative for error + */ +int encrypt_ak_sk(AkSkRef plain_ak_sk, const std::string& encryption_method, + const std::string& encryption_key, AkSkPair* cipher_ak_sk); + +/** + * @brief Decrypt ak/sk pair + * + * @param cipher_ak_sk cipher ak/sk pair in base64 format + * @param encryption_method encryption_method + * @param encryption_key encryption_key + * @param plain_ak_sk output plain ak/sk pair + * @return 0 for success, negative for error + */ +int decrypt_ak_sk(AkSkRef cipher_ak_sk, const std::string& encryption_method, + const std::string& encryption_key, AkSkPair* plain_ak_sk); + +extern std::map global_encryption_key_info_map; + +// Todo: Should we need to refresh it +int init_global_encryption_key_info_map(TxnKv* txn_kv); + +/** + * @brief Get the encryption key for ak sk by key_id + * + * @param version_id + * @param encryption_key output encryption_key + * @return 0 for success, negative for error + */ +inline static int get_encryption_key_for_ak_sk(int64_t key_id, std::string* encryption_key) { + if (global_encryption_key_info_map.count(key_id)) { + *encryption_key = global_encryption_key_info_map.at(key_id); + return 0; + } + return -1; +} + +int decrypt_ak_sk_helper(std::string_view cipher_ak, std::string_view cipher_sk, + const EncryptionInfoPB& encryption_info, AkSkPair* plain_ak_sk_pair); + +/** + * @brief Get the newest encryption key for ak sk + * + * @param key_id + * @param encryption_key + * @return 0 for success, negative for error + */ +inline static int get_newest_encryption_key_for_ak_sk(int64_t* key_id, + std::string* encryption_key) { + if (global_encryption_key_info_map.empty()) { + return -1; + } + auto it = global_encryption_key_info_map.crbegin(); + *key_id = it->first; + *encryption_key = it->second; + return 0; +} + +inline static const std::string& get_encryption_method_for_ak_sk() { + return config::encryption_method; +} + +size_t base64_decode(const char* data, size_t length, char* decoded_data); + +} // namespace doris::cloud diff --git a/cloud/src/common/jemalloc_hook.cpp b/cloud/src/common/jemalloc_hook.cpp new file mode 100644 index 00000000000000..88380bab41c7b1 --- /dev/null +++ b/cloud/src/common/jemalloc_hook.cpp @@ -0,0 +1,131 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "jemalloc/jemalloc.h" + +#ifndef __THROW +#if __cplusplus +#define __THROW noexcept +#else +#define __THROW +#endif +#endif + +extern "C" { +void* doris_malloc(size_t size) __THROW { + return jemalloc(size); +} + +void doris_free(void* p) __THROW { + jefree(p); +} + +void* doris_realloc(void* p, size_t size) __THROW { + return jerealloc(p, size); +} + +void* doris_calloc(size_t n, size_t size) __THROW { + return jecalloc(n, size); +} + +void doris_cfree(void* ptr) __THROW { + jefree(ptr); +} + +void* doris_memalign(size_t align, size_t size) __THROW { + return jealigned_alloc(align, size); +} + +void* doris_aligned_alloc(size_t align, size_t size) __THROW { + return jealigned_alloc(align, size); +} + +void* doris_valloc(size_t size) __THROW { + return jevalloc(size); +} + +void* doris_pvalloc(size_t size) __THROW { + return jevalloc(size); +} + +int doris_posix_memalign(void** r, size_t align, size_t size) __THROW { + return jeposix_memalign(r, align, size); +} + +size_t doris_malloc_usable_size(void* ptr) __THROW { + return jemalloc_usable_size(ptr); +} + +#ifndef __APPLE__ +#define ALIAS(doris_fn) __attribute__((alias(#doris_fn), used)) +void* malloc(size_t size) __THROW ALIAS(doris_malloc); +void free(void* p) __THROW ALIAS(doris_free); +void* realloc(void* p, size_t size) __THROW ALIAS(doris_realloc); +void* calloc(size_t n, size_t size) __THROW ALIAS(doris_calloc); +void cfree(void* ptr) __THROW ALIAS(doris_cfree); +void* memalign(size_t align, size_t size) __THROW ALIAS(doris_memalign); +void* aligned_alloc(size_t align, size_t size) __THROW ALIAS(doris_aligned_alloc); +void* valloc(size_t size) __THROW ALIAS(doris_valloc); +void* pvalloc(size_t size) __THROW ALIAS(doris_pvalloc); +int posix_memalign(void** r, size_t a, size_t s) __THROW ALIAS(doris_posix_memalign); +size_t malloc_usable_size(void* ptr) __THROW ALIAS(doris_malloc_usable_size); +#else +void* malloc(size_t size) { + return doris_malloc(size); +} + +void free(void* p) { + return doris_free(p); +} + +void* realloc(void* p, size_t size) { + return doris_realloc(p, size); +} + +void* calloc(size_t n, size_t size) { + return doris_calloc(n, size); +} + +void cfree(void* ptr) { + return doris_cfree(ptr); +} + +void* memalign(size_t align, size_t size) { + return doris_memalign(align, size); +} + +void* aligned_alloc(size_t align, size_t size) { + return doris_aligned_alloc(align, size); +} + +void* valloc(size_t size) { + return doris_valloc(size); +} + +void* pvalloc(size_t size) { + return doris_pvalloc(size); +} + +int posix_memalign(void** r, size_t a, size_t s) { + return doris_posix_memalign(r, a, s); +} + +size_t malloc_usable_size(void* ptr) { + return doris_malloc_usable_size(ptr); +} +#endif +} diff --git a/cloud/src/common/kms.cpp b/cloud/src/common/kms.cpp new file mode 100644 index 00000000000000..02290eefc2900c --- /dev/null +++ b/cloud/src/common/kms.cpp @@ -0,0 +1,165 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "common/kms.h" + +#include +#include +#include + +#include + +#include "common/logging.h" +#include "common/sync_point.h" +namespace doris::cloud { + +int create_kms_client(KmsConf&& conf, std::unique_ptr* kms_client) { + if (conf.ak.empty() || conf.sk.empty() || conf.endpoint.empty() || conf.region.empty() || + conf.provider.empty() || conf.cmk.empty()) { + LOG(WARNING) << "incorrect kms conf"; + return -1; + } + // Todo: only support ali cloud now + if (conf.provider != "ali") { + LOG(WARNING) << "only support kms for ali cloud now"; + return -1; + } + *kms_client = std::make_unique(std::move(conf)); + return 0; +} + +AliKmsClient::AliKmsClient(KmsConf&& conf) : KmsClient(std::move(conf)) { + AlibabaCloud::InitializeSdk(); +} + +AliKmsClient::~AliKmsClient() { + AlibabaCloud::ShutdownSdk(); +} + +int AliKmsClient::init() { + AlibabaCloud::ClientConfiguration configuration(conf_.region); + AlibabaCloud::Credentials credential(conf_.ak, conf_.sk); + kms_client_ = std::make_unique(std::move(credential), + std::move(configuration)); + return 0; +} + +int AliKmsClient::encrypt(const std::string& plaintext, std::string* output) { + { + [[maybe_unused]] int ret = -1; + [[maybe_unused]] const auto* p = &plaintext; + TEST_SYNC_POINT_CALLBACK("alikms::encrypt::plaintext", + reinterpret_cast(const_cast(p))); + TEST_SYNC_POINT_CALLBACK("alikms::encrypt::output", output); + TEST_SYNC_POINT_RETURN_WITH_VALUE("alikms::encrypt::ret", &ret); + } + AlibabaCloud::CommonRequest request(AlibabaCloud::CommonRequest::RequestPattern::RpcPattern); + request.setHttpMethod(AlibabaCloud::HttpRequest::Method::Post); + request.setDomain(conf_.endpoint); + request.setVersion("2016-01-20"); + request.setQueryParameter("Action", "Encrypt"); + request.setQueryParameter("KeyId", conf_.cmk); + request.setQueryParameter("Plaintext", plaintext); + + auto response = kms_client_->commonResponse(request); + if (response.isSuccess()) { + Json::Value json; + Json::Reader reader; + if (reader.parse(response.result().payload(), json) && json.isMember("CiphertextBlob")) { + *output = json["CiphertextBlob"].asString(); + return 0; + } else { + LOG(WARNING) << "failed to parse response, response=" << response.result().payload(); + return -1; + } + } else { + LOG(WARNING) << "failed to encrypt data, error=" << response.error().errorMessage() + << " request id=" << response.error().requestId(); + return -1; + } + return 0; +} + +int AliKmsClient::decrypt(const std::string& ciphertext, std::string* output) { + { + [[maybe_unused]] int ret = -1; + TEST_SYNC_POINT_CALLBACK("alikms::decrypt::output", output); + TEST_SYNC_POINT_RETURN_WITH_VALUE("alikms::decrypt::ret", &ret); + } + AlibabaCloud::CommonRequest request(AlibabaCloud::CommonRequest::RequestPattern::RpcPattern); + request.setHttpMethod(AlibabaCloud::HttpRequest::Method::Post); + request.setDomain(conf_.endpoint); + request.setVersion("2016-01-20"); + request.setQueryParameter("Action", "Decrypt"); + request.setQueryParameter("CiphertextBlob", ciphertext); + + auto response = kms_client_->commonResponse(request); + if (response.isSuccess()) { + Json::Value json; + Json::Reader reader; + if (reader.parse(response.result().payload(), json) && json.isMember("Plaintext")) { + *output = json["Plaintext"].asString(); + return 0; + } else { + LOG(WARNING) << "failed to parse response, response=" << response.result().payload(); + return -1; + } + } else { + LOG(WARNING) << "failed to decrypt data, error=" << response.error().errorMessage() + << " request id=" << response.error().requestId(); + return -1; + } + return 0; +} + +int AliKmsClient::generate_data_key(std::string* ciphertext, std::string* plaintext) { + { + [[maybe_unused]] int ret = -1; + TEST_SYNC_POINT_CALLBACK("alikms::generate_data_key::ciphertext", ciphertext); + TEST_SYNC_POINT_CALLBACK("alikms::generate_data_key::plaintext", plaintext); + TEST_SYNC_POINT_RETURN_WITH_VALUE("alikms::generate_data_key::ret", &ret); + } + AlibabaCloud::CommonRequest request(AlibabaCloud::CommonRequest::RequestPattern::RpcPattern); + request.setHttpMethod(AlibabaCloud::HttpRequest::Method::Post); + request.setDomain(conf_.endpoint); + request.setVersion("2016-01-20"); + request.setQueryParameter("Action", "GenerateDataKey"); + request.setQueryParameter("KeySpec", "AES_256"); + request.setQueryParameter("KeyId", "839ff82c-dcb5-4438-a6a4-6ec832443ba8"); + + auto response = kms_client_->commonResponse(request); + if (response.isSuccess()) { + Json::Value json; + Json::Reader reader; + if (reader.parse(response.result().payload(), json) && json.isMember("Plaintext") && + json.isMember("CiphertextBlob")) { + *plaintext = json["Plaintext"].asString(); + *ciphertext = json["CiphertextBlob"].asString(); + return 0; + } else { + LOG(WARNING) << "failed to parse response, response=" << response.result().payload(); + return -1; + } + } else { + LOG(WARNING) << "failed to generate data key, error=" << response.error().errorMessage() + << " request id=" << response.error().requestId(); + return -1; + } + return 0; +} + +} // namespace doris::cloud \ No newline at end of file diff --git a/cloud/src/common/kms.h b/cloud/src/common/kms.h new file mode 100644 index 00000000000000..db5b98d1eeb981 --- /dev/null +++ b/cloud/src/common/kms.h @@ -0,0 +1,119 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include + +#include +#include +#include + +namespace doris::cloud { + +struct KmsConf { + std::string ak; + std::string sk; + std::string endpoint; + std::string region; + std::string cmk; + std::string provider; +}; + +class KmsClient { +public: + KmsClient(KmsConf&& conf) : conf_(std::move(conf)) {} + virtual ~KmsClient() = default; + + const KmsConf& conf() const { return conf_; } + + // returns 0 for success otherwise error + virtual int init() = 0; + + /** + * @brief This function encrypts the plaintext. + * + * @param plaintext The plaintext (base64-encoded) to be encrypted. + * @param output Output the ciphertext (base64-encoded). + * @return int Returns 0 on success and -1 on failure. + */ + virtual int encrypt(const std::string& plaintext, std::string* output) = 0; + + /** + * @brief This function decrypts the ciphertext. + * + * @param ciphertext The ciphertext (base64-encoded) to be decrypted. + * @param output Output the decrypted (base64-encoded) plaintext. + * @return int Returns 0 on success and -1 on failure. + */ + virtual int decrypt(const std::string& ciphertext, std::string* output) = 0; + + /** + * @brief This function generate data key + * + * @param ciphertext return ciphertext (base64-encoded) + * @param plaintext return plaintext (base64-encoded) + * @return int Returns 0 on success and -1 on failure. + */ + virtual int generate_data_key(std::string* ciphertext, std::string* plaintext) = 0; + +protected: + KmsConf conf_; +}; + +int create_kms_client(KmsConf&& conf, std::unique_ptr* kms_client); + +class AliKmsClient : public KmsClient { +public: + explicit AliKmsClient(KmsConf&& conf); + ~AliKmsClient() override; + + // returns 0 for success otherwise error + int init() override; + + /** + * @brief This function encrypts the plaintext. + * + * @param plaintext The plaintext (base64-encoded) to be encrypted. + * @param output Output the ciphertext (base64-encoded). + * @return int Returns 0 on success and -1 on failure. + */ + int encrypt(const std::string& plaintext, std::string* output) override; + + /** + * @brief This function decrypts the ciphertext. + * + * @param ciphertext The ciphertext (base64-encoded) to be decrypted. + * @param output Output the decrypted (base64-encoded) plaintext. + * @return int Returns 0 on success and -1 on failure. + */ + int decrypt(const std::string& ciphertext, std::string* output) override; + + /** + * @brief This function generate data key + * + * @param ciphertext return ciphertext (base64-encoded) + * @param plaintext return plaintext (base64-encoded) + * @return int Returns 0 on success and -1 on failure. + */ + int generate_data_key(std::string* ciphertext, std::string* plaintext) override; + +private: + std::unique_ptr kms_client_; +}; + +} // namespace doris::cloud \ No newline at end of file diff --git a/cloud/src/common/logging.cpp b/cloud/src/common/logging.cpp new file mode 100644 index 00000000000000..67dd53a6810e54 --- /dev/null +++ b/cloud/src/common/logging.cpp @@ -0,0 +1,135 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "logging.h" + +#include +#include +#include +#include + +#include +#include + +#include "config.h" + +namespace doris::cloud { + +static butil::LinkedList* get_annotate_tag_list() { + static std::once_flag log_annotated_tags_key_once; + static bthread_key_t log_annotated_tags_key; + std::call_once( + log_annotated_tags_key_once, + +[](bthread_key_t* key) { + bthread_key_create( + key, +[](void* value) { + delete reinterpret_cast*>(value); + }); + }, + &log_annotated_tags_key); + + auto* tag_list = reinterpret_cast*>( + bthread_getspecific(log_annotated_tags_key)); + if (!tag_list) { + tag_list = new butil::LinkedList(); + bthread_setspecific(log_annotated_tags_key, tag_list); + } + return tag_list; +} + +AnnotateTag::AnnotateTag(default_tag_t, std::string_view key, std::string value) + : key_(key), value_(std::move(value)) { + get_annotate_tag_list()->Append(this); +} + +AnnotateTag::AnnotateTag(std::string_view key, std::string_view value) + : AnnotateTag(default_tag, key, fmt::format("\"{}\"", value)) {} + +AnnotateTag::~AnnotateTag() { + RemoveFromList(); +} + +void AnnotateTag::format_tag_list(std::ostream& stream) { + butil::LinkedList* list = get_annotate_tag_list(); + butil::LinkNode* head = list->head(); + const butil::LinkNode* end = list->end(); + for (; head != end; head = head->next()) { + stream << ' ' << head->value()->key_ << '=' << head->value()->value_; + } +} + +/** + * @param basename the basename of log file + * @return true for success + */ +bool init_glog(const char* basename) { + static std::mutex mtx; + static bool inited = false; + std::lock_guard logging_lock(mtx); + if (inited) return true; + + FLAGS_alsologtostderr = false; + // Don't log to stderr except fatal level + // so fatal log can output to be.out . + FLAGS_stderrthreshold = google::ERROR; + + // Set glog log dir + FLAGS_log_dir = config::log_dir; + // Buffer log messages for at most this many seconds + FLAGS_logbufsecs = 1; + // Set log roll mode + // Candidates: day, hour, size + FLAGS_log_split_method = "size"; + // Sets the maximum log file size (in MB). + FLAGS_max_log_size = config::log_size_mb; + // Set roll num + FLAGS_log_filenum_quota = config::log_filenum_quota; + + // FIXME(walter) update glog patch + // Set warn log roll num + // FLAGS_warn_log_filenum_quota = config::warn_log_filenum_quota; + + // clang-format off + // set log level + std::string& loglevel = config::log_level; + // Can be 0 1 2 3 ... the larger the higher level for logging, + // corrensponding to INFO WARNING ERROR FATAL + // const int GLOG_INFO = 0, GLOG_WARNING = 1, GLOG_ERROR = 2, GLOG_FATAL = 3, NUM_SEVERITIES = 4; + auto tolower = [](std::string s) { for (auto& i : s) i |= 0x20; return s; }; + FLAGS_minloglevel = tolower(loglevel) == "info" ? 0 + : tolower(loglevel) == "warn" ? 1 + : tolower(loglevel) == "error" ? 2 + : tolower(loglevel) == "fatal" ? 3 + : 0; // Default INFO + // clang-format on + + // Log messages at a level <= this flag are buffered. + // Log messages at a higher level are flushed immediately. + FLAGS_logbuflevel = config::log_immediate_flush ? -1 : 0; + + // Set verbose modules + FLAGS_v = -1; + for (auto& i : config::log_verbose_modules) { + if (i.empty()) continue; + google::SetVLOGLevel(i.c_str(), config::log_verbose_level); + } + google::InitGoogleLogging(basename); + inited = true; + return true; +} + +} // namespace doris::cloud diff --git a/cloud/src/common/logging.h b/cloud/src/common/logging.h new file mode 100644 index 00000000000000..422dc814568fe6 --- /dev/null +++ b/cloud/src/common/logging.h @@ -0,0 +1,96 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include +#include + +#include +#include + +namespace doris::cloud { + +bool init_glog(const char* basename); + +/// Wrap a glog stream and tag on the log. usage: +/// LOG_INFO("here is an info for a {} query", query_type).tag("query_id", queryId); +#define LOG_INFO(...) ::doris::cloud::TaggableLogger(LOG(INFO), ##__VA_ARGS__) +#define LOG_WARNING(...) ::doris::cloud::TaggableLogger(LOG(WARNING), ##__VA_ARGS__) +#define LOG_ERROR(...) ::doris::cloud::TaggableLogger(LOG(ERROR), ##__VA_ARGS__) +#define LOG_FATAL(...) ::doris::cloud::TaggableLogger(LOG(FATAL), ##__VA_ARGS__) + +class AnnotateTag final : public butil::LinkNode { + struct default_tag_t {}; + constexpr static default_tag_t default_tag {}; + +public: + template , T>> + AnnotateTag(std::string_view key, T value) + : AnnotateTag(default_tag, key, std::to_string(value)) {} + AnnotateTag(std::string_view key, std::string_view value); + ~AnnotateTag(); + + static void format_tag_list(std::ostream& stream); + + static void* operator new(size_t) = delete; + static void* operator new[](size_t) = delete; + +private: + explicit AnnotateTag(default_tag_t, std::string_view key, std::string value); + + std::string_view key_; + std::string value_; +}; + +class TaggableLogger { +public: + template + TaggableLogger(std::ostream& stream, std::string_view fmt, Args&&... args) : stream_(stream) { + if constexpr (sizeof...(args) == 0) { + stream_ << fmt; + } else { + stream_ << fmt::format(fmt, std::forward(args)...); + } + AnnotateTag::format_tag_list(stream_); + }; + + template + TaggableLogger& tag(std::string_view key, const V& value) { + stream_ << ' ' << key << '='; + if constexpr (std::is_convertible_v) { + stream_ << '"' << value << '"'; + } else { + stream_ << value; + } + return *this; + } + +private: + std::ostream& stream_; +}; + +} // namespace doris::cloud + +// To keep it simple and practical, we don't actually need so many VLOG levels. +// Using `VLOG(${number})` is confusing and hard to desid in most cases, all we +// need is a complementary debug level to glog's default 4 levels of logging. +// "One VLOG level to rule them all!" +#define DEBUG 5 +// VLOG_DEBUG is alias of VLOG(DEBUG) I.O.W VLOG(5) +#define VLOG_DEBUG VLOG(DEBUG) diff --git a/cloud/src/common/metric.cpp b/cloud/src/common/metric.cpp new file mode 100644 index 00000000000000..ee3013b1a237db --- /dev/null +++ b/cloud/src/common/metric.cpp @@ -0,0 +1,272 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "metric.h" + +#include +#include +#include + +#include +#include +#include +#include +#include + +#include "common/bvars.h" +#include "meta-service/txn_kv_error.h" + +namespace doris::cloud { + +// The format of the output is shown in "test/fdb_metric_example.json" +static const std::string FDB_STATUS_KEY = "\xff\xff/status/json"; + +static std::string get_fdb_status(TxnKv* txn_kv) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create_txn, err=" << err; + return ""; + } + std::string status_val; + err = txn->get(FDB_STATUS_KEY, &status_val); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to get FDB_STATUS_KEY, err=" << err; + return ""; + } + return status_val; +} + +// The format of fdb status details: +// +// Configuration: +// Redundancy mode - double +// Storage engine - ssd-2 +// Coordinators - 3 +// Usable Regions - 1 + +// Cluster: +// FoundationDB processes - 15 +// Zones - 3 +// Machines - 3 +// Memory availability - 2.9 GB per process on machine with least available +// >>>>> (WARNING: 4.0 GB recommended) <<<<< +// Retransmissions rate - 3 Hz +// Fault Tolerance - 1 machines +// Server time - 02/16/23 16:48:14 + +// Data: +// Replication health - Healthy +// Moving data - 0.000 GB +// Sum of key-value sizes - 4.317 GB +// Disk space used - 11.493 GB + +// Operating space: +// Storage server - 462.8 GB free on most full server +// Log server - 462.8 GB free on most full server + +// Workload: +// Read rate - 84 Hz +// Write rate - 4 Hz +// Transactions started - 222 Hz +// Transactions committed - 4 Hz +// Conflict rate - 0 Hz + +// Backup and DR: +// Running backups - 0 +// Running DRs - 0 + +static void export_fdb_status_details(const std::string& status_str) { + using namespace rapidjson; + Document document; + try { + document.Parse(status_str.c_str()); + if (document.HasParseError()) { + LOG(WARNING) << "fail to parse status str, err: " + << GetParseError_En(document.GetParseError()); + return; + } + } catch (std::exception& e) { + LOG(WARNING) << "fail to parse status str, err: " << e.what(); + return; + } + + if (!document.HasMember("cluster") || !document.HasMember("client")) { + LOG(WARNING) << "err fdb status details"; + return; + } + auto get_value = [&](const std::vector& v) -> int64_t { + if (v.empty()) return BVAR_FDB_INVALID_VALUE; + auto node = document.FindMember("cluster"); + for (const auto& name : v) { + if (!node->value.HasMember(name)) return BVAR_FDB_INVALID_VALUE; + node = node->value.FindMember(name); + } + if (node->value.IsInt64()) return node->value.GetInt64(); + if (node->value.IsDouble()) return static_cast(node->value.GetDouble()); + if (node->value.IsObject()) return node->value.MemberCount(); + if (node->value.IsArray()) return node->value.Size(); + return BVAR_FDB_INVALID_VALUE; + }; + auto get_nanoseconds = [&](const std::vector& v) -> int64_t { + constexpr double NANOSECONDS = 1e9; + auto node = document.FindMember("cluster"); + for (const auto& name : v) { + if (!node->value.HasMember(name)) return BVAR_FDB_INVALID_VALUE; + node = node->value.FindMember(name); + } + if (node->value.IsInt64()) return node->value.GetInt64() * NANOSECONDS; + DCHECK(node->value.IsDouble()); + return static_cast(node->value.GetDouble() * NANOSECONDS); + }; + // Configuration + g_bvar_fdb_configuration_coordinators_count.set_value( + get_value({"configuration", "coordinators_count"})); + g_bvar_fdb_configuration_usable_regions.set_value( + get_value({"configuration", "usable_regions"})); + + // Cluster + g_bvar_fdb_process_count.set_value(get_value({"processes"})); + g_bvar_fdb_machines_count.set_value(get_value({"machines"})); + g_bvar_fdb_fault_tolerance_count.set_value( + get_value({"fault_tolerance", "max_zone_failures_without_losing_data"})); + g_bvar_fdb_generation.set_value(get_value({"generation"})); + g_bvar_fdb_incompatible_connections.set_value(get_value({"incompatible_connections"})); + + // Data/Operating space + g_bvar_fdb_data_average_partition_size_bytes.set_value( + get_value({"data", "average_partition_size_bytes"})); + g_bvar_fdb_data_partition_count.set_value(get_value({"data", "partitions_count"})); + g_bvar_fdb_data_total_disk_used_bytes.set_value(get_value({"data", "total_disk_used_bytes"})); + g_bvar_fdb_data_total_kv_size_bytes.set_value(get_value({"data", "total_kv_size_bytes"})); + g_bvar_fdb_data_log_server_space_bytes.set_value( + get_value({"data", "least_operating_space_bytes_log_server"})); + g_bvar_fdb_data_storage_server_space_bytes.set_value( + get_value({"data", "least_operating_space_bytes_storage_server"})); + g_bvar_fdb_data_moving_data_highest_priority.set_value( + get_value({"data", "moving_data", "highest_priority"})); + g_bvar_fdb_data_moving_data_in_flight_bytes.set_value( + get_value({"data", "moving_data", "in_flight_bytes"})); + g_bvar_fdb_data_moving_data_in_queue_bytes.set_value( + get_value({"data", "moving_data", "in_queue_bytes"})); + g_bvar_fdb_data_moving_total_written_bytes.set_value( + get_value({"data", "moving_data", "total_written_bytes"})); + g_bvar_fdb_data_state_min_replicas_remaining.set_value( + get_value({"data", "state", "min_replicas_remaining"})); + + // Latency probe + g_bvar_fdb_latency_probe_transaction_start_ns.set_value( + get_nanoseconds({"latency_probe", "transaction_start_seconds"})); + g_bvar_fdb_latency_probe_commit_ns.set_value( + get_nanoseconds({"latency_probe", "commit_seconds"})); + g_bvar_fdb_latency_probe_read_ns.set_value(get_nanoseconds({"latency_probe", "read_seconds"})); + + // Workload + g_bvar_fdb_workload_conflict_rate_hz.set_value( + get_value({"workload", "transactions", "conflicted", "hz"})); + g_bvar_fdb_workload_location_rate_hz.set_value( + get_value({"workload", "operations", "location_requests", "hz"})); + g_bvar_fdb_workload_keys_read_hz.set_value(get_value({"workload", "keys", "read", "hz"})); + g_bvar_fdb_workload_read_bytes_hz.set_value(get_value({"workload", "bytes", "read", "hz"})); + g_bvar_fdb_workload_read_rate_hz.set_value( + get_value({"workload", "operations", "reads", "hz"})); + g_bvar_fdb_workload_written_bytes_hz.set_value( + get_value({"workload", "bytes", "written", "hz"})); + g_bvar_fdb_workload_write_rate_hz.set_value( + get_value({"workload", "operations", "writes", "hz"})); + g_bvar_fdb_workload_transactions_started_hz.set_value( + get_value({"workload", "transactions", "started", "hz"})); + g_bvar_fdb_workload_transactions_committed_hz.set_value( + get_value({"workload", "transactions", "committed", "hz"})); + g_bvar_fdb_workload_transactions_rejected_hz.set_value( + get_value({"workload", "transactions", "rejected_for_queued_too_long", "hz"})); + + // QOS + g_bvar_fdb_qos_worst_data_lag_storage_server_ns.set_value( + get_nanoseconds({"qos", "worst_data_lag_storage_server", "seconds"})); + g_bvar_fdb_qos_worst_durability_lag_storage_server_ns.set_value( + get_nanoseconds({"qos", "worst_durability_lag_storage_server", "seconds"})); + g_bvar_fdb_qos_worst_log_server_queue_bytes.set_value( + get_value({"qos", "worst_queue_bytes_log_server"})); + g_bvar_fdb_qos_worst_storage_server_queue_bytes.set_value( + get_value({"qos", "worst_queue_bytes_storage_server"})); + + // Backup and DR + + // Client Count + g_bvar_fdb_client_count.set_value(get_value({"clients", "count"})); + + // Coordinators Unreachable Count + auto unreachable_count = 0; + if (auto node = document.FindMember("client"); node->value.HasMember("coordinators")) { + if (node = node->value.FindMember("coordinators"); node->value.HasMember("coordinators")) { + if (node = node->value.FindMember("coordinators"); node->value.IsArray()) { + for (const auto& c : node->value.GetArray()) { + if (c.HasMember("reachable") && c.FindMember("reachable")->value.IsBool() && + !c.FindMember("reachable")->value.GetBool()) { + ++unreachable_count; + } + } + g_bvar_fdb_coordinators_unreachable_count.set_value(unreachable_count); + } + } + } +} + +void FdbMetricExporter::export_fdb_metrics(TxnKv* txn_kv) { + std::string fdb_status = get_fdb_status(txn_kv); + export_fdb_status_details(fdb_status); +} + +FdbMetricExporter::~FdbMetricExporter() { + stop(); +} + +int FdbMetricExporter::start() { + if (txn_kv_ == nullptr) return -1; + std::unique_lock lock(running_mtx_); + if (running_) { + return 0; + } + + running_ = true; + thread_ = std::make_unique([this] { + while (running_.load(std::memory_order_acquire)) { + export_fdb_metrics(txn_kv_.get()); + std::unique_lock l(running_mtx_); + running_cond_.wait_for(l, std::chrono::milliseconds(sleep_interval_ms_), + [this]() { return !running_.load(std::memory_order_acquire); }); + LOG(INFO) << "finish to collect fdb metric"; + } + }); + return 0; +} + +void FdbMetricExporter::stop() { + { + std::unique_lock lock(running_mtx_); + running_.store(false); + running_cond_.notify_all(); + } + + if (thread_ != nullptr && thread_->joinable()) { + thread_->join(); + thread_.reset(); + } +} + +} // namespace doris::cloud diff --git a/cloud/src/common/metric.h b/cloud/src/common/metric.h new file mode 100644 index 00000000000000..33429a276c6034 --- /dev/null +++ b/cloud/src/common/metric.h @@ -0,0 +1,52 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include +#include +#include +#include +#include + +#include "common/logging.h" +#include "meta-service/txn_kv.h" + +namespace doris::cloud { + +class FdbMetricExporter { +public: + FdbMetricExporter(std::shared_ptr txn_kv) + : txn_kv_(std::move(txn_kv)), running_(false) {} + ~FdbMetricExporter(); + + int start(); + void stop(); + + static void export_fdb_metrics(TxnKv* txn_kv); + +private: + std::shared_ptr txn_kv_; + std::unique_ptr thread_; + std::atomic running_; + std::mutex running_mtx_; + std::condition_variable running_cond_; + int sleep_interval_ms_ = 5000; +}; + +} // namespace doris::cloud \ No newline at end of file diff --git a/cloud/src/common/simple_sync_queue.h b/cloud/src/common/simple_sync_queue.h new file mode 100644 index 00000000000000..b54593dcb37df1 --- /dev/null +++ b/cloud/src/common/simple_sync_queue.h @@ -0,0 +1,123 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace doris::cloud { + +template > class CONT = std::deque> +class SimpleSyncQueue { +public: + SimpleSyncQueue() = default; + explicit SimpleSyncQueue(std::uint32_t max_size) : _max_size(max_size) {} + virtual ~SimpleSyncQueue() { _queue.clear(); } + + void put(const T& t) { + std::unique_lock locker(_mutex); + _not_full.wait(locker, [this]() { return _queue.size() < _max_size; }); + _queue.push_back(std::move(t)); + _not_empty.notify_one(); + } + + bool put_with_timeout(const T& t, int timeout /*in milliseconds*/) { + std::unique_lock locker(_mutex); + if (_not_full.wait_for(locker, std::chrono::milliseconds(timeout), + [this]() { return _queue.size() < _max_size; })) { + _queue.push_back(std::move(t)); + _not_empty.notify_one(); + return true; + } + //timeout + return false; + } + + template + void emplace(Args&&... args) { + std::unique_lock locker(_mutex); + _not_full.wait(locker, [this]() { return _queue.size() < _max_size; }); + _queue.emplace_back(std::forward(args)...); + _not_empty.notify_one(); + } + + template + bool emplace_with_timeout(int timeout /*in milliseconds*/, Args&&... args) { + std::unique_lock locker(_mutex); + if (_not_full.wait_for(locker, std::chrono::milliseconds(timeout), + [this]() { return _queue.size() < _max_size; })) { + _queue.emplace_back(std::forward(args)...); + _not_empty.notify_one(); + return true; + } + //timeout + return false; + } + + void get(T* t) { + std::unique_lock locker(_mutex); + _not_empty.wait(locker, [this]() { return !_queue.empty(); }); + (*t) = std::move(_queue.front()); + _queue.pop_front(); + _not_full.notify_one(); + } + + bool get_with_timeout(T* t, int timeout /*in milliseconds*/) { + std::unique_lock locker(_mutex); + if (_not_empty.wait_for(locker, std::chrono::milliseconds(timeout), + [this]() { return !_queue.empty(); })) { + (*t) = std::move(_queue.front()); + _queue.pop_front(); + _not_full.notify_one(); + return true; + } + //timeout + return false; + } + + void clear() { + std::lock_guard locker(_mutex); + _queue.clear(); + } + + size_t size() { + std::lock_guard locker(_mutex); + return _queue.size(); + } + + bool empty() { + std::lock_guard locker(_mutex); + return _queue.empty(); + } + +private: + CONT _queue; + std::mutex _mutex; + std::condition_variable _not_empty; + std::condition_variable _not_full; + std::uint32_t _max_size = std::numeric_limits::max(); +}; + +} // namespace doris::cloud diff --git a/cloud/src/common/simple_thread_pool.h b/cloud/src/common/simple_thread_pool.h new file mode 100644 index 00000000000000..e18d6787bf7a46 --- /dev/null +++ b/cloud/src/common/simple_thread_pool.h @@ -0,0 +1,164 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include +#include +#include +#include +#include + +#include "simple_sync_queue.h" + +namespace doris::cloud { + +class SimpleThreadPool { +private: + using JobType = std::function; + // a lock based sync queue + std::shared_ptr> _job_queue; + + std::vector _worker_thread_group; // multi thread pool + std::atomic _is_running; + size_t _pool_size; + +public: + SimpleThreadPool(size_t size) : _is_running(false), _pool_size(size) { + _job_queue = std::make_shared>(_pool_size * 2); + } + + SimpleThreadPool() = delete; + + /** + * Submits jobs, if job queue is full it will blocked + * + * @return 0 if succeeded + */ + int submit(JobType f) { + if (!_is_running) { + // not started + return -1; + } + _job_queue->emplace(std::move(f)); + return 0; + } + + /** + * Submits jobs with a limit blocking time + * + * @param timeout max blocking time in milliseconds + * @return 0 if success + * -1 if thread pool not start yet + * -2 if time out + */ + int submit_with_timeout(JobType f, int timeout) { + if (!_is_running) { + // not started + return -1; + } + if (!_job_queue->emplace_with_timeout(timeout, std::move(f))) { + return -2; + } + return 0; + } + + /** + * + * @return always 0 + */ + int start() { + _is_running = true; + _worker_thread_group.clear(); + for (size_t i = 0; i < _pool_size; ++i) { + _worker_thread_group.emplace_back(&SimpleThreadPool::work, this); + } + return 0; + } + + /** + * Stops to get jobs from job queue, the job being done will finish normally + * + * @return 0 if succeed, otherwise non-zero value returned + */ + int stop() { + if (!_is_running) { + // already stopped + return -1; + } + _is_running = false; + for (auto& i : _worker_thread_group) { + if (i.joinable()) { + i.join(); + } + } + return 0; + } + + /** + * Gets size of the pool + * + * @return the thread number(pool size) in the thread pool + */ + size_t size() const { return _pool_size; } + + /** + * Terminates the thread pool immediately, jobs in the queue will not be done + * and the running threads will be detached + * + * @return 0 if succeed, otherwise non-zero value returned + */ + int terminate() { + if (!_is_running) { + // not running + return -1; + } + _is_running = false; + _job_queue->clear(); // abandon all jobs + for (auto& i : _worker_thread_group) { + i.detach(); + } + return 0; + } + + ~SimpleThreadPool() { + if (_is_running) { + stop(); + } + } + +private: + /** + * Working thread + */ + void work() { + while (_is_running || !_job_queue->empty()) { + JobType job; + if (!_job_queue->get_with_timeout(&job, 10)) { + continue; + } + try { + job(); + } catch (...) { + // do nothing + } + } + } +}; + +} // namespace doris::cloud diff --git a/cloud/src/common/stopwatch.h b/cloud/src/common/stopwatch.h new file mode 100644 index 00000000000000..1cbd1bd7a9f38e --- /dev/null +++ b/cloud/src/common/stopwatch.h @@ -0,0 +1,74 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include + +namespace doris::cloud { + +class StopWatch { +public: + StopWatch() { + running_ = true; + start_ = std::chrono::steady_clock::now(); + }; + ~StopWatch() = default; + + void start() { + if (!running_) { + start_ = std::chrono::steady_clock::now(); + running_ = true; + } + } + + void pause() { + if (running_) { + elapsed_ = elapsed_ + (std::chrono::steady_clock::now() - start_); + running_ = false; + } + } + + void resume() { + if (!running_) { + start_ = std::chrono::steady_clock::now(); + running_ = true; + } + } + + void reset() { + start_ = std::chrono::steady_clock::now(); + elapsed_ = std::chrono::steady_clock::duration{0}; + running_ = true; + } + + int64_t elapsed_us() const { + if (!running_) { + return std::chrono::duration_cast(elapsed_).count(); + } + + auto end = std::chrono::steady_clock::now(); + return std::chrono::duration_cast(elapsed_ + (end - start_)).count(); + } + +private: + std::chrono::steady_clock::time_point start_; + std::chrono::steady_clock::duration elapsed_{0}; + bool running_{false}; +}; + +} // namespace doris::cloud \ No newline at end of file diff --git a/cloud/src/common/string_util.h b/cloud/src/common/string_util.h new file mode 100644 index 00000000000000..ea9929a46e11b3 --- /dev/null +++ b/cloud/src/common/string_util.h @@ -0,0 +1,47 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. +#pragma once + +#include +#include + +namespace doris::cloud { + +static inline std::string trim(std::string& str) { + const std::string drop = "/ \t"; + str.erase(str.find_last_not_of(drop) + 1); + return str.erase(0, str.find_first_not_of(drop)); +} + +static inline std::vector split(const std::string& str, const char delim) { + std::vector result; + size_t start = 0; + size_t pos = str.find(delim); + while (pos != std::string::npos) { + if (pos > start) { + result.push_back(str.substr(start, pos - start)); + } + start = pos + 1; + pos = str.find(delim, start); + } + + if (start < str.length()) result.push_back(str.substr(start)); + + return result; +} + +} // namespace doris::cloud diff --git a/cloud/src/common/sync_point.cpp b/cloud/src/common/sync_point.cpp new file mode 100644 index 00000000000000..3f0b0c295b35f3 --- /dev/null +++ b/cloud/src/common/sync_point.cpp @@ -0,0 +1,240 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +// Most code of this file is copied and modified from rocksdb SyncPoint. +// And modified by Gavin (github.com/gavinchou). + +// clang-format off +#include "sync_point.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace doris::cloud { + +struct SyncPoint::Data { // impl +public: + Data() : enabled_(false) { } + virtual ~Data() {} + void process(const std::string& point, void* cb_arg); + void load_dependency(const std::vector& dependencies); + void load_dependency_and_markers( + const std::vector& dependencies, + const std::vector& markers); + bool predecessors_all_cleared(const std::string& point); + void set_call_back(const std::string& point, + const std::function& callback); + void clear_call_back(const std::string& point); + void clear_all_call_backs(); + void enable_processing(); + void disable_processing(); + void clear_trace(); +private: + bool disable_by_marker(const std::string& point, std::thread::id thread_id); +private: + // successor/predecessor map loaded from load_dependency + std::unordered_map> successors_; + std::unordered_map> predecessors_; + std::unordered_map> callbacks_; + std::unordered_map> markers_; + std::unordered_map marked_thread_id_; + std::mutex mutex_; + std::condition_variable cv_; + // sync points that have been passed through + std::unordered_set cleared_points_; + std::atomic enabled_; + int num_callbacks_running_ = 0; +}; + +SyncPoint* SyncPoint::get_instance() { + static SyncPoint sync_point; + return &sync_point; +} +SyncPoint::SyncPoint() : + impl_(new Data) { +} +SyncPoint:: ~SyncPoint() { + delete impl_; +} +void SyncPoint::load_dependency(const std::vector& dependencies) { + impl_->load_dependency(dependencies); +} +void SyncPoint::load_dependency_and_markers( + const std::vector& dependencies, + const std::vector& markers) { + impl_->load_dependency_and_markers(dependencies, markers); +} +void SyncPoint::set_call_back(const std::string& point, + const std::function& callback) { + impl_->set_call_back(point, callback); +} +void SyncPoint::clear_call_back(const std::string& point) { + impl_->clear_call_back(point); +} +void SyncPoint::clear_all_call_backs() { + impl_->clear_all_call_backs(); +} +void SyncPoint::enable_processing() { + impl_->enable_processing(); +} +void SyncPoint::disable_processing() { + impl_->disable_processing(); +} +void SyncPoint::clear_trace() { + impl_->clear_trace(); +} +void SyncPoint::process(const std::string& point, void* cb_arg) { + impl_->process(point, cb_arg); +} + +// ============================================================================= +// SyncPoint implementation +// ============================================================================= + +void SyncPoint::Data::load_dependency( + const std::vector& dependencies) { + std::lock_guard lock(mutex_); + successors_.clear(); + predecessors_.clear(); + cleared_points_.clear(); + for (const auto& dependency : dependencies) { + successors_[dependency.predecessor].push_back(dependency.successor); + predecessors_[dependency.successor].push_back(dependency.predecessor); + } + cv_.notify_all(); +} + +/** + * Markers are also dependency descriptions + */ +void SyncPoint::Data::load_dependency_and_markers( + const std::vector& dependencies, + const std::vector& markers) { + std::lock_guard lock(mutex_); + successors_.clear(); + predecessors_.clear(); + cleared_points_.clear(); + markers_.clear(); + marked_thread_id_.clear(); + for (const auto& dependency : dependencies) { + successors_[dependency.predecessor].push_back(dependency.successor); + predecessors_[dependency.successor].push_back(dependency.predecessor); + } + for (const auto& marker : markers) { + successors_[marker.predecessor].push_back(marker.successor); + predecessors_[marker.successor].push_back(marker.predecessor); + markers_[marker.predecessor].push_back(marker.successor); + } + cv_.notify_all(); +} + +bool SyncPoint::Data::predecessors_all_cleared(const std::string& point) { + for (const auto& pred : predecessors_[point]) { + if (cleared_points_.count(pred) == 0) { + return false; + } + } + return true; +} + +void SyncPoint::Data::clear_call_back(const std::string& point) { + std::unique_lock lock(mutex_); + while (num_callbacks_running_ > 0) { + cv_.wait(lock); + } + callbacks_.erase(point); +} + +void SyncPoint::Data::clear_all_call_backs() { + std::unique_lock lock(mutex_); + while (num_callbacks_running_ > 0) { + cv_.wait(lock); + } + callbacks_.clear(); +} + +void SyncPoint::Data::process(const std::string& point, void* cb_arg) { + if (!enabled_) { + return; + } + std::unique_lock lock(mutex_); + auto thread_id = std::this_thread::get_id(); + auto marker_iter = markers_.find(point); + // if current sync point is a marker + // record it in marked_thread_id_ for all its successors + if (marker_iter != markers_.end()) { + for (auto& marked_point : marker_iter->second) { + marked_thread_id_.emplace(marked_point, thread_id); + } + } + // if current point is a marker's successor + if (disable_by_marker(point, thread_id)) { + return; + } + while (!predecessors_all_cleared(point)) { + cv_.wait(lock); + if (disable_by_marker(point, thread_id)) { + return; + } + } + auto callback_pair = callbacks_.find(point); + if (callback_pair != callbacks_.end()) { + num_callbacks_running_++; + mutex_.unlock(); + callback_pair->second(cb_arg); + mutex_.lock(); + num_callbacks_running_--; + } + cleared_points_.insert(point); + cv_.notify_all(); +} + +bool SyncPoint::Data::disable_by_marker(const std::string& point, + std::thread::id thread_id) { + auto marked_point_iter = marked_thread_id_.find(point); + return marked_point_iter != marked_thread_id_.end() // is a successor + && thread_id != marked_point_iter->second; +} + +void SyncPoint::Data::set_call_back(const std::string& point, + const std::function& callback) { + std::lock_guard lock(mutex_); + callbacks_[point] = callback; +} + +void SyncPoint::Data::clear_trace() { + std::lock_guard lock(mutex_); + cleared_points_.clear(); +} + +void SyncPoint::Data::enable_processing() { + enabled_ = true; +} + +void SyncPoint::Data::disable_processing() { + enabled_ = false; +} + +} // namespace doris::cloud +// clang-format on diff --git a/cloud/src/common/sync_point.h b/cloud/src/common/sync_point.h new file mode 100644 index 00000000000000..b81ce31a5c1bff --- /dev/null +++ b/cloud/src/common/sync_point.h @@ -0,0 +1,215 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +// Most code of this file is copied and modified from rocksdb SyncPoint. +// And modified by Gavin (github.com/gavinchou). + +#pragma once +// clang-format off +#include +#include +#include +#include + +namespace doris::cloud { + +// This class provides facility to reproduce race conditions deterministically +// in unit tests. +// Developer could specify sync points in the codebase via TEST_SYNC_POINT. +// Each sync point represents a position in the execution stream of a thread. +// In the unit test, 'Happens After' relationship among sync points could be +// setup via SyncPoint::load_dependency, to reproduce a desired interleave of +// threads execution. +// Refer to (DBTest,TransactionLogIteratorRace), for an example use case. +class SyncPoint { +public: + static SyncPoint* get_instance(); + SyncPoint(const SyncPoint&) = delete; + SyncPoint& operator=(const SyncPoint&) = delete; + ~SyncPoint(); + struct SyncPointPair { + std::string predecessor; + std::string successor; + }; + + // call once at the beginning of a test to setup the dependency between + // sync points + // + // Example: + // load_dependency({{"point1", "point2"}, + // {"point2", "point3"}, + // {"point3", "point4"}}); + // + // test case thread thread for object being tested + // | | + // | | + // | \-------------0-------------\ | + // | \-> x sync point1 set in code + // | /----------1----------------/ | + // point2 o <-/ /-> x sync point4 set in code + // | / | + // z / | + // z /---------2-----------/ | there may be nothing + // | / | between point1 point4 + // point3 o --/ | they are for sync + // | | between test case and object + // v v + // + // vertical arrow means the procedure of each thread, the running order will + // be: + // test case thread -> point1 -> point2 -> point3 -> point4 -> object being + // tested + // + // we may do a lot of things between point2 and point3, say, change the + // object's status, call another method, propagate data race and etc. + void load_dependency(const std::vector& dependencies); + + // call once at the beginning of a test to setup the dependency between + // sync points and setup markers indicating the successor is only enabled + // when it is processed on the same thread as the predecessor. + // When adding a marker, it implicitly adds a dependency for the marker pair. + void load_dependency_and_markers( + const std::vector& dependencies, + const std::vector& markers); + + // The argument to the callback is passed through from + // TEST_SYNC_POINT_CALLBACK(); nullptr if TEST_SYNC_POINT or + // TEST_IDX_SYNC_POINT was used. + void set_call_back(const std::string& point, + const std::function& callback); + + // Clear callback function by point + void clear_call_back(const std::string& point); + + // Clear all call back functions. + void clear_all_call_backs(); + + // Enable sync point processing (disabled on startup) + void enable_processing(); + + // Disable sync point processing + void disable_processing(); + + // Remove the execution trace of all sync points + void clear_trace(); + + // Triggered by TEST_SYNC_POINT, blocking execution until all predecessors + // are executed. + // And/or call registered callback function, with argument `cb_arg` + void process(const std::string& point, void* cb_arg = nullptr); + + // TODO: it might be useful to provide a function that blocks until all + // sync points are cleared. + // We want this to be public so we can subclass the implementation + struct Data; + +private: + // Singleton + SyncPoint(); + Data* impl_; // implementation which is hidden in cpp file +}; +} // namespace doris::cloud + +// TEST_SYNC_POINT is no op in release build. +// Turn on this feature by defining the macro +#ifndef UNIT_TEST +# define TEST_SYNC_POINT(x) +# define TEST_IDX_SYNC_POINT(x, index) +# define TEST_SYNC_POINT_CALLBACK(x, y) +# define TEST_SYNC_POINT_RETURN_WITH_VALUE(sync_point_name, ret_val_ptr) +# define TEST_SYNC_POINT_RETURN_WITH_VOID(sync_point_name) +// seldom called +# define INIT_SYNC_POINT_SINGLETONS() +#else +// Use TEST_SYNC_POINT to specify sync points inside code base. +// Sync points can have happens-after depedency on other sync points, +// configured at runtime via SyncPoint::load_dependency. This could be +// utilized to re-produce race conditions between threads. +# define TEST_SYNC_POINT(x) doris::cloud::SyncPoint::get_instance()->process(x) +# define TEST_IDX_SYNC_POINT(x, index) \ + doris::cloud::SyncPoint::get_instance()->process(x + std::to_string(index)) +# define TEST_SYNC_POINT_CALLBACK(x, y) \ + doris::cloud::SyncPoint::get_instance()->process(x, y) +# define INIT_SYNC_POINT_SINGLETONS() \ + (void)doris::cloud::SyncPoint::get_instance(); + +/** + * Inject return points for testing. + * + * Currently we can only insert more points to get context from tested thread + * and process in testing thread, e.g. + * + * tested thread: + * ... + * TEST_SYNC_POINT_CALLBACK("point_ctx", ptr_to_ctx); + * TEST_SYNC_POINT_RETURN_WITH_VALUE("point_ret", ptr_to_ret_val); + * ... + * + * testing thread: + * sync_point->add("point_ctx", [&ctx](void* ptr_to_ctx) { ctx = ptr_to_ctx; }); + * sync_point->add("point_ret", [](void* ptr_to_ret) {...}); + * sync_point->add("point_ret::pred", [&ctx](void* pred) { pred = *ctx ? true : false; }); + * + * See sync_point_test.cpp for more details. + */ +#pragma GCC diagnostic ignored "-Waddress" +# define TEST_SYNC_POINT_RETURN_WITH_VALUE(sync_point_name, ret_val_ptr) \ +static_assert(ret_val_ptr != nullptr, "ret_val_ptr cannot be nullptr");\ +TEST_SYNC_POINT_CALLBACK(sync_point_name, ret_val_ptr); \ +{ \ + bool pred = false; \ + TEST_SYNC_POINT_CALLBACK(sync_point_name"::pred", &pred); \ + if (pred) return *ret_val_ptr; \ +} + +# define TEST_SYNC_POINT_RETURN_WITH_VOID(sync_point_name) \ +{ \ + bool pred = false; \ + TEST_SYNC_POINT_CALLBACK(sync_point_name"::pred", &pred); \ + if (pred) return; \ +} + +#endif // UNIT_TEST + +// TODO: define injection point in production env. +// the `if` expr can be live configure of the application +#define ENABLE_INJECTION_POINT 0 +#ifndef ENABLE_INJECTION_POINT +# define TEST_INJECTION_POINT(x) +# define TEST_IDX_TEST_INJECTION_POINT(x, index) +# define TEST_INJECTION_POINT_CALLBACK(x, y) +# define INIT_INJECTION_POINT_SINGLETONS() +#else +# define TEST_INJECTION_POINT(x) \ + if (ENABLE_INJECTION_POINT) { \ + doris::cloud::SyncPoint::get_instance()->process(x); \ + } +# define TEST_IDX_INJECTION_POINT(x, index) \ + if (ENABLE_INJECTION_POINT) { \ + doris::cloud::SyncPoint::get_instance()->process(x + std::to_string(index)); \ + } +# define TEST_INJECTION_POINT_CALLBACK(x, y) \ + if (ENABLE_INJECTION_POINT) { \ + doris::cloud::SyncPoint::get_instance()->process(x, y); \ + } +# define INIT_INJECTION_POINT_SINGLETONS() \ + if (ENABLE_INJECTION_POINT) { \ + (void)doris::cloud::SyncPoint::get_instance(); \ + } +#endif // ENABLE_INJECTION_POINT + +// clang-format on diff --git a/cloud/src/common/util.cpp b/cloud/src/common/util.cpp new file mode 100644 index 00000000000000..8d1c8fed983c8f --- /dev/null +++ b/cloud/src/common/util.cpp @@ -0,0 +1,345 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +// clang-format off +#include "util.h" + +#include +#include +#include + +// FIXME: we should not rely other modules that may rely on this common module +#include "common/logging.h" +#include "meta-service/keys.h" +#include "meta-service/codec.h" +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" + +#include +#include +#include +#include +// clang-format on + +namespace doris::cloud { + +/** + * This is a naïve implementation of hex, DONOT use it on retical path. + */ +std::string hex(std::string_view str) { + std::stringstream ss; + for (auto& i : str) { + ss << std::hex << std::setw(2) << std::setfill('0') << ((int16_t)i & 0xff); + } + return ss.str(); +} + +/** + * This is a naïve implementation of unhex. + */ +std::string unhex(std::string_view hex_str) { + // clang-format off + const static std::unordered_map table = { + {'0', 0}, {'1', 1}, {'2', 2}, {'3', 3}, {'4', 4}, + {'5', 5}, {'6', 6}, {'7', 7}, {'8', 8}, {'9', 9}, + {'a', 10}, {'b', 11}, {'c', 12}, {'d', 13}, {'e', 14}, {'f', 15}, + {'A', 10}, {'B', 11}, {'C', 12}, {'D', 13}, {'E', 14}, {'F', 15}}; + [[maybe_unused]] static int8_t lut[std::max({'9', 'f', 'F'}) + 1]; + lut[(int)'0'] = 0; lut[(int)'1'] = 1; lut[(int)'2'] = 2; lut[(int)'3'] = 3; lut[(int)'4'] = 4; lut[(int)'5'] = 5; lut[(int)'6'] = 6; lut[(int)'7'] = 7; lut[(int)'8'] = 8; lut[(int)'9'] = 9; + lut[(int)'a'] = 10; lut[(int)'b'] = 11; lut[(int)'c'] = 12; lut[(int)'d'] = 13; lut[(int)'e'] = 14; lut[(int)'f'] = 15; + lut[(int)'A'] = 10; lut[(int)'B'] = 11; lut[(int)'C'] = 12; lut[(int)'D'] = 13; lut[(int)'E'] = 14; lut[(int)'F'] = 15; + // clang-format on + size_t len = hex_str.length(); + len &= ~0x01UL; + std::string buf(len >> 1, '\0'); + for (size_t i = 0; i < len; ++i) { + const auto it = table.find(hex_str[i]); + if (it == table.end()) break; + buf[i >> 1] |= i & 0x1 ? (it->second & 0x0f) : (it->second & 0x0f) << 4; + } + return buf; +} + +static std::string explain_fields(std::string_view text, const std::vector& fields, + const std::vector& pos, bool unicode = false) { + if (fields.size() != pos.size() || fields.size() == 0 || pos.size() == 0) { + return std::string(text.data(), text.size()); + } + size_t last_hyphen_pos = pos.back() + 1; + std::stringstream ss; + std::string blank_line(last_hyphen_pos + 1, ' '); + + // clang-format off + static const std::string hyphen("\xe2\x94\x80"); // ─ e2 94 80 + static const std::string bar ("\xe2\x94\x82"); // │ e2 94 82 + static const std::string angle ("\xe2\x94\x8c"); // ┌ e2 94 8c + static const std::string arrow ("\xe2\x96\xbc"); // ▼ e2 96 bc + // clang-format on + + // Each line with hyphens + for (size_t i = 0; i < fields.size(); ++i) { + std::string line = blank_line; + line[pos[i]] = '/'; + int nbar = i; + for (size_t j = 0; j < i; ++j) { + line[pos[j]] = '|'; + } + int nhyphen = 0; + for (size_t j = pos[i] + 1; j <= last_hyphen_pos; ++j) { + line[j] = '-'; + ++nhyphen; + } + + if (unicode) { + int i = line.size(); + line.resize(line.size() + 2 * (1 /*angle*/ + nbar + nhyphen), ' '); + int j = line.size(); + while (--i >= 0) { + if (line[i] == '-') { + line[--j] = hyphen[2]; + line[--j] = hyphen[1]; + line[--j] = hyphen[0]; + } else if (line[i] == '|') { + line[--j] = bar[2]; + line[--j] = bar[1]; + line[--j] = bar[0]; + } else if (line[i] == '/') { + line[--j] = angle[2]; + line[--j] = angle[1]; + line[--j] = angle[0]; + } else { + --j; + continue; + } + line[i] = i != j ? ' ' : line[i]; // Replace if needed + } + } + + ss << line << " " << i << ". " << fields[i] << "\n"; + } + + // Mark position indicator + std::string line = blank_line; + for (size_t i = 0; i < fields.size(); ++i) { + line[pos[i]] = '|'; + } + + if (unicode) { + int i = line.size(); + line.resize(line.size() + 2 * fields.size(), ' '); + int j = line.size(); + while (--i >= 0) { + if (line[i] != '|') { + --j; + continue; + } + line[--j] = bar[2]; + line[--j] = bar[1]; + line[--j] = bar[0]; + line[i] = i != j ? ' ' : line[i]; // Replace if needed + } + } + + ss << line << "\n"; + + line = blank_line; + for (size_t i = 0; i < fields.size(); ++i) { + line[pos[i]] = 'v'; + } + + if (unicode) { + int i = line.size(); + line.resize(line.size() + 2 * fields.size(), ' '); + int j = line.size(); + while (--i >= 0) { + if (line[i] != 'v') { + --j; + continue; + } + line[--j] = arrow[2]; + line[--j] = arrow[1]; + line[--j] = arrow[0]; + line[i] = i != j ? ' ' : line[i]; // Replace if needed + } + } + + ss << line << "\n"; + + // Original text to explain + ss << text << "\n"; + + return ss.str(); +} + +std::string prettify_key(std::string_view key_hex, bool unicode) { + // Decoded result container + // val tag pos + // .---------------^----------------. .^. .^. + std::vector, int, int>> fields; + std::string unhex_key = unhex(key_hex); + int key_space = unhex_key[0]; + std::string_view key_copy = unhex_key; + key_copy.remove_prefix(1); // Remove the first key space byte + int ret = decode_key(&key_copy, &fields); + if (ret != 0) return ""; + + std::vector fields_str; + std::vector fields_pos; + fields_str.reserve(fields.size() + 1); + fields_pos.reserve(fields.size() + 1); + // Key space byte + fields_str.push_back("key space: " + std::to_string(key_space)); + fields_pos.push_back(0); + + for (auto& i : fields) { + fields_str.emplace_back(std::get<1>(i) == EncodingTag::BYTES_TAG + ? std::get(std::get<0>(i)) + : std::to_string(std::get(std::get<0>(i)))); + fields_pos.push_back((std::get<2>(i) + 1) * 2); + } + + return explain_fields(key_hex, fields_str, fields_pos, unicode); +} + +std::string proto_to_json(const ::google::protobuf::Message& msg, bool add_whitespace) { + std::string json; + google::protobuf::util::JsonPrintOptions opts; + opts.add_whitespace = add_whitespace; + opts.preserve_proto_field_names = true; + google::protobuf::util::MessageToJsonString(msg, &json, opts); + return json; +} + +std::vector split_string(const std::string_view& str, int n) { + std::vector substrings; + + for (size_t i = 0; i < str.size(); i += n) { + substrings.push_back(str.substr(i, n)); + } + + return substrings; +} + +bool ValueBuf::to_pb(google::protobuf::Message* pb) const { + butil::IOBuf merge; + for (auto&& it : iters) { + it->reset(); + while (it->has_next()) { + auto [k, v] = it->next(); + merge.append_user_data((void*)v.data(), v.size(), +[](void*) {}); + } + } + butil::IOBufAsZeroCopyInputStream merge_stream(merge); + return pb->ParseFromZeroCopyStream(&merge_stream); +} + +void ValueBuf::remove(Transaction* txn) const { + for (auto&& it : iters) { + it->reset(); + while (it->has_next()) { + txn->remove(it->next().first); + } + } +} + +TxnErrorCode ValueBuf::get(Transaction* txn, std::string_view key, bool snapshot) { + iters.clear(); + ver = -1; + + std::string begin_key {key}; + std::string end_key {key}; + encode_int64(INT64_MAX, &end_key); + std::unique_ptr it; + TxnErrorCode err = txn->get(begin_key, end_key, &it, snapshot); + if (err != TxnErrorCode::TXN_OK) { + return err; + } + if (!it->has_next()) { + return TxnErrorCode::TXN_KEY_NOT_FOUND; + } + // Extract version + auto [k, _] = it->next(); + if (k.size() == key.size()) { // Old version KV + DCHECK(k == key) << hex(k) << ' ' << hex(key); + DCHECK_EQ(it->size(), 1) << hex(k) << ' ' << hex(key); + ver = 0; + } else { + k.remove_prefix(key.size()); + int64_t suffix; + if (decode_int64(&k, &suffix) != 0) [[unlikely]] { + LOG_WARNING("failed to decode key").tag("key", hex(k)); + return TxnErrorCode::TXN_UNIDENTIFIED_ERROR; + } + ver = suffix >> 56 & 0xff; + } + bool more = it->more(); + if (!more) { + iters.push_back(std::move(it)); + return TxnErrorCode::TXN_OK; + } + begin_key = it->next_begin_key(); + iters.push_back(std::move(it)); + do { + err = txn->get(begin_key, end_key, &it, snapshot); + if (err != TxnErrorCode::TXN_OK) { + return err; + } + more = it->more(); + if (more) { + begin_key = it->next_begin_key(); + } + iters.push_back(std::move(it)); + } while (more); + return TxnErrorCode::TXN_OK; +} + +TxnErrorCode get(Transaction* txn, std::string_view key, ValueBuf* val, bool snapshot) { + return val->get(txn, key, snapshot); +} + +TxnErrorCode key_exists(Transaction* txn, std::string_view key, bool snapshot) { + std::string end_key {key}; + encode_int64(INT64_MAX, &end_key); + std::unique_ptr it; + TxnErrorCode err = txn->get(key, end_key, &it, snapshot, 1); + if (err != TxnErrorCode::TXN_OK) { + return err; + } + return it->has_next() ? TxnErrorCode::TXN_OK : TxnErrorCode::TXN_KEY_NOT_FOUND; +} + +void put(Transaction* txn, std::string_view key, const google::protobuf::Message& pb, uint8_t ver, + size_t split_size) { + std::string value; + bool ret = pb.SerializeToString(&value); // Always success + DCHECK(ret) << hex(key) << ' ' << pb.ShortDebugString(); + put(txn, key, value, ver, split_size); +} + +void put(Transaction* txn, std::string_view key, std::string_view value, uint8_t ver, + size_t split_size) { + auto split_vec = split_string(value, split_size); + int64_t suffix_base = ver; + suffix_base <<= 56; + for (size_t i = 0; i < split_vec.size(); ++i) { + std::string k(key); + encode_int64(suffix_base + i, &k); + txn->put(k, split_vec[i]); + } +} + +} // namespace doris::cloud diff --git a/cloud/src/common/util.h b/cloud/src/common/util.h new file mode 100644 index 00000000000000..de37c2f4d9bf7c --- /dev/null +++ b/cloud/src/common/util.h @@ -0,0 +1,138 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include +#include +#include + +#include "meta-service/txn_kv_error.h" + +namespace google::protobuf { +class Message; +} + +namespace doris::cloud { +class TxnKv; +class Transaction; +class RangeGetIterator; + +std::string hex(std::string_view str); + +std::string unhex(std::string_view str); + +/** + * Prettifies the given key, the first byte must be key space tag, say 0x01, and + * the remaining part must be the output of codec funtion family. + * + * The result is like following: + * + * /------------------------------------------------------------------------------------------------------- 0. key space: 1 + * | /----------------------------------------------------------------------------------------------------- 1. txn + * | | /----------------------------------------------------------------------------------------- 2. instance_id_deadbeef + * | | | /------------------------------------------- 3. txn_label + * | | | | /------------------- 4. 10003 + * | | | | | /- 5. insert_3fd5ad12d0054a9b-8c776d3218c6adb7 + * | | | | | | + * v v v v v v + * 011074786e000110696e7374616e63655f69645f646561646265656600011074786e5f696e646578000112000000000000271310696e736572745f336664356164313264303035346139622d386337373664333231386336616462370001 + * + * @param key_hex encoded key hex string + * @param unicode whether to use unicode (UTF8) to draw line, default false. + * @return the pretty format, empty result if error occurs + */ +std::string prettify_key(std::string_view key_hex, bool unicode = false); + +/** + * Converts proto message to json string + * + * @return empty string if conversion failed + */ +std::string proto_to_json(const ::google::protobuf::Message& msg, bool add_whitespace = false); + +/** + * Supports splitting large values (>100KB) into multiple KVs, with a logical value size of up to the fdb transaction limit (<10MB). + * Supports multi version format parsing of values (which can be any byte sequence format), and can recognize the version of values + * that are forward compatible with older versions of values. + * Key format: + * {origin_key}{suffix: i64} + * suffix (big-endian): + * |Bytes 0 |Bytes 1-5 |Bytes 6-7 | + * |-------------|-------------|-------------| + * |version |dummy |sequence | + */ +struct ValueBuf { + // TODO(plat1ko): Support decompression + [[nodiscard]] bool to_pb(google::protobuf::Message* pb) const; + // TODO: More bool to_xxx(Xxx* xxx) const; + + // Remove all splitted KV in `iters_` via `txn` + void remove(Transaction* txn) const; + + // Get a key, save raw splitted values of the key to `this`, value length may be bigger than 100k + // Return TXN_OK for success get a key, TXN_KEY_NOT_FOUND for key not found, otherwise for error. + TxnErrorCode get(Transaction* txn, std::string_view key, bool snapshot = false); + + std::vector> iters; + int8_t ver {-1}; +}; + +/** + * Get a key, return key's value, value length may be bigger than 100k + * @param txn fdb txn handler + * @param key encode key + * @param val return wrapped raw splitted values of the key + * @param snapshot if true, `key` will not be included in txn conflict detection this time + * @return return TXN_OK for success get a key, TXN_KEY_NOT_FOUND for key not found, otherwise for error. + */ +TxnErrorCode get(Transaction* txn, std::string_view key, ValueBuf* val, bool snapshot = false); + +/** + * Test whether key exists + * @param txn fdb txn handler + * @param key encode key + * @param snapshot if true, `key` will not be included in txn conflict detection this time + * @return TXN_OK for key existed, TXN_KEY_NOT_FOUND for key not found, otherwise for kv error + */ +TxnErrorCode key_exists(Transaction* txn, std::string_view key, bool snapshot = false); + +/** + * Put a KV, it's value may be bigger than 100k + * TODO(plat1ko): Support compression + * @param txn fdb txn handler + * @param key encode key + * @param pb value to save + * @param ver value version + * @param split_size how many byte sized fragments are the value split into + */ +void put(Transaction* txn, std::string_view key, const google::protobuf::Message& pb, uint8_t ver, + size_t split_size = 90 * 1000); + +/** + * Put a KV, it's value may be bigger than 100k + * @param txn fdb txn handler + * @param key encode key + * @param value value to save + * @param ver value version + * @param split_size how many byte sized fragments are the value split into + */ +void put(Transaction* txn, std::string_view key, std::string_view value, uint8_t ver, + size_t split_size = 90 * 1000); + +} // namespace doris::cloud diff --git a/cloud/src/gen-cpp/CMakeLists.txt b/cloud/src/gen-cpp/CMakeLists.txt new file mode 100644 index 00000000000000..596598e1361044 --- /dev/null +++ b/cloud/src/gen-cpp/CMakeLists.txt @@ -0,0 +1,18 @@ +set(LIBRARY_OUTPUT_PATH "${BUILD_DIR}/src/gen_cpp") + +set (GEN_CPP_DIR ${GENSRC_DIR}/gen_cpp) +set(SRC_FILES + ${GEN_CPP_DIR}/cloud.pb.cc + ${GEN_CPP_DIR}/olap_file.pb.cc + ${GEN_CPP_DIR}/types.pb.cc + ${GEN_CPP_DIR}/segment_v2.pb.cc + ${GEN_CPP_DIR}/olap_common.pb.cc +) + +# keep everything in one library, the object files reference +# each other +add_library(CloudGen STATIC ${SRC_FILES}) + +# Setting these files as code-generated lets make clean and incremental builds +# work correctly +set_source_files_properties(${SRC_FILES} PROPERTIES GENERATED TRUE) diff --git a/cloud/src/main.cpp b/cloud/src/main.cpp new file mode 100644 index 00000000000000..9356a3546d03a9 --- /dev/null +++ b/cloud/src/main.cpp @@ -0,0 +1,335 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include +#include // ::open +#include +#include // ::lockf + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "common/arg_parser.h" +#include "common/config.h" +#include "common/encryption_util.h" +#include "common/logging.h" +#include "meta-service/mem_txn_kv.h" +#include "meta-service/meta_server.h" +#include "meta-service/txn_kv.h" +#include "recycler/recycler.h" + +using namespace doris::cloud; + +/** + * Generates a pidfile with given process name + * + * @return an fd holder with auto-storage-lifecycle + */ +std::shared_ptr gen_pidfile(const std::string& process_name) { + std::cerr << "process working directory: " << std::filesystem::current_path() << std::endl; + std::string pid_path = "./bin/" + process_name + ".pid"; + int fd = ::open(pid_path.c_str(), O_RDWR | O_CREAT, S_IRUSR | S_IWUSR | S_IRGRP | S_IROTH); + // clang-format off + std::shared_ptr holder(&fd, // Just pretend to need an address of int + [fd, pid_path](...) { // The real fd is captured + if (fd <= 0) { return; } + [[maybe_unused]] auto x = ::lockf(fd, F_UNLCK, 0); + ::close(fd); + // FIXME: removing the pidfile may result in missing pidfile + // after launching the process... + // std::error_code ec; std::filesystem::remove(pid_path, ec); + }); + // clang-format on + if (::lockf(fd, F_TLOCK, 0) != 0) { + std::cerr << "failed to lock pidfile=" << pid_path << " fd=" << fd << std::endl; + return nullptr; + } + std::fstream pidfile(pid_path, std::ios::out); + if (!pidfile.is_open()) { + std::cerr << "failed to open pid file " << pid_path << std::endl; + return nullptr; + } + pidfile << getpid() << std::endl; + pidfile.close(); + std::cout << "pid=" << getpid() << " written to file=" << pid_path << std::endl; + return holder; +} + +/** + * Prepares extra conf files + */ +std::string prepare_extra_conf_file() { + // If the target file is not empty and `config::fdb_cluster` is empty, use the exists file. + if (config::fdb_cluster.empty()) { + try { + if (std::filesystem::exists(config::fdb_cluster_file_path) && + std::filesystem::file_size(config::fdb_cluster_file_path) > 0) { + return ""; + } + } catch (std::filesystem::filesystem_error& e) { + return fmt::format("prepare_extra_conf_file: {}", e.what()); + } + + return "Please specify the fdb_cluster in doris_cloud.conf"; + } + + std::fstream fdb_cluster_file(config::fdb_cluster_file_path, std::ios::out); + fdb_cluster_file << "# DO NOT EDIT UNLESS YOU KNOW WHAT YOU ARE DOING!\n" + << "# This file is auto-generated with doris_cloud.conf:fdb_cluster.\n" + << "# It is not to be edited by hand.\n" + << config::fdb_cluster; + fdb_cluster_file.close(); + return ""; +} + +// TODO(gavin): support daemon mode +// must be called before pidfile generation and any network resource +// initialization, +// void daemonize(1, 1); // Maybe nohup will do? + +// Arguments +// clang-format off +constexpr static const char* ARG_META_SERVICE = "meta-service"; +constexpr static const char* ARG_RECYCLER = "recycler"; +constexpr static const char* ARG_HELP = "help"; +constexpr static const char* ARG_VERSION = "version"; +constexpr static const char* ARG_CONF = "conf"; +ArgParser args( + { + ArgParser::new_arg(ARG_META_SERVICE, false, "run as meta-service"), + ArgParser::new_arg(ARG_RECYCLER, false, "run as recycler") , + ArgParser::new_arg(ARG_HELP, false, "print help msg") , + ArgParser::new_arg(ARG_VERSION, false, "print version info") , + ArgParser::new_arg(ARG_CONF, "./conf/doris_cloud.conf", "path to conf file") , + } +); +// clang-format on + +static void help() { + args.print(); +} + +static std::string build_info() { + std::stringstream ss; +#if defined(NDEBUG) + ss << "version:{" DORIS_CLOUD_BUILD_VERSION "-release}" +#else + ss << "version:{" DORIS_CLOUD_BUILD_VERSION "-debug}" +#endif + << " code_version:{commit=" DORIS_CLOUD_BUILD_HASH " time=" DORIS_CLOUD_BUILD_VERSION_TIME + "}" + << " build_info:{initiator=" DORIS_CLOUD_BUILD_INITIATOR " build_at=" DORIS_CLOUD_BUILD_TIME + " build_on=" DORIS_CLOUD_BUILD_OS_VERSION "}\n"; + return ss.str(); +} + +// TODO(gavin): add doris cloud role to the metrics name +bvar::Status doris_cloud_version_metrics("doris_cloud_version", [] { + std::stringstream ss; + ss << DORIS_CLOUD_BUILD_VERSION_MAJOR << 0 << DORIS_CLOUD_BUILD_VERSION_MINOR << 0 + << DORIS_CLOUD_BUILD_VERSION_PATCH; + return std::strtoul(ss.str().c_str(), nullptr, 10); +}()); + +namespace brpc { +DECLARE_uint64(max_body_size); +DECLARE_int64(socket_max_unwritten_bytes); +} // namespace brpc + +int main(int argc, char** argv) { + if (argc > 1) { + if (auto ret = args.parse(argc - 1, argv + 1); !ret.empty()) { + std::cerr << ret << std::endl; + help(); + return -1; + } + } + + if (argc < 2 || args.get(ARG_HELP)) { + help(); + return 0; + } + + if (args.get(ARG_VERSION)) { + std::cout << build_info(); + return 0; + } + + // FIXME(gavin): do we need to enable running both MS and recycler within + // single process + if (!(args.get(ARG_META_SERVICE) ^ args.get(ARG_RECYCLER))) { + std::cerr << "only one of --meta-service and --recycler must be specified" << std::endl; + return 1; + } + + // There may be more roles to play + std::string process_name = args.get(ARG_META_SERVICE) ? "meta_service" + : args.get(ARG_RECYCLER) ? "recycler" + : ""; + if (process_name.empty()) { + std::cerr << "failed to determine prcess name with given args" << std::endl; + return 1; + } + + auto pid_file_fd_holder = gen_pidfile("doris_cloud"); + if (pid_file_fd_holder == nullptr) { + return -1; + } + + auto conf_file = args.get(ARG_CONF); + if (!config::init(conf_file.c_str(), true)) { + std::cerr << "failed to init config file, conf=" << conf_file << std::endl; + return -1; + } + + if (auto ret = prepare_extra_conf_file(); !ret.empty()) { + std::cerr << "failed to prepare extra conf file, err=" << ret << std::endl; + return -1; + } + + if (!init_glog(process_name.data())) { + std::cerr << "failed to init glog" << std::endl; + return -1; + } + + // We can invoke glog from now on + + std::string msg; + LOG(INFO) << build_info(); + std::cout << build_info() << std::endl; + + brpc::Server server; + brpc::FLAGS_max_body_size = config::brpc_max_body_size; + brpc::FLAGS_socket_max_unwritten_bytes = config::brpc_socket_max_unwritten_bytes; + + std::shared_ptr txn_kv; + if (config::use_mem_kv) { + // MUST NOT be used in production environment + std::cerr << "use volatile mem kv, please make sure it is not a production environment" + << std::endl; + txn_kv = std::make_shared(); + } else { + txn_kv = std::make_shared(); + } + if (txn_kv == nullptr) { + LOG(WARNING) << "failed to create txn kv, invalid txnkv type"; + return 1; + } + LOG(INFO) << "begin to init txn kv"; + int ret = txn_kv->init(); + if (ret != 0) { + LOG(WARNING) << "failed to init txnkv, ret=" << ret; + return 1; + } + LOG(INFO) << "successfully init txn kv"; + + if (init_global_encryption_key_info_map(txn_kv.get()) != 0) { + LOG(WARNING) << "failed to init global encryption key map"; + return -1; + } + + std::unique_ptr meta_server; + std::unique_ptr recycler; + std::thread periodiccally_log_thread; + std::mutex periodiccally_log_thread_lock; + std::condition_variable periodiccally_log_thread_cv; + std::atomic_bool periodiccally_log_thread_run = true; + + if (args.get(ARG_META_SERVICE)) { + meta_server = std::make_unique(txn_kv); + int ret = meta_server->start(&server); + if (ret != 0) { + msg = "failed to start meta server"; + LOG(ERROR) << msg; + std::cerr << msg << std::endl; + return ret; + } + msg = "meta-service started"; + LOG(INFO) << msg; + std::cout << msg << std::endl; + } else if (args.get(ARG_RECYCLER)) { + recycler = std::make_unique(txn_kv); + int ret = recycler->start(&server); + if (ret != 0) { + msg = "failed to start recycler"; + LOG(ERROR) << msg; + std::cerr << msg << std::endl; + return ret; + } + msg = "recycler started"; + LOG(INFO) << msg; + std::cout << msg << std::endl; + auto periodiccally_log = [&]() { + while (periodiccally_log_thread_run) { + std::unique_lock lck {periodiccally_log_thread_lock}; + periodiccally_log_thread_cv.wait_for( + lck, std::chrono::milliseconds(config::periodically_log_ms)); + LOG(INFO) << "Periodically log for recycler"; + } + }; + periodiccally_log_thread = std::thread {periodiccally_log}; + } else { + std::cerr << "cloud starts without doing anything and exits" << std::endl; + return -1; + } + // start service + brpc::ServerOptions options; + if (config::brpc_idle_timeout_sec != -1) { + options.idle_timeout_sec = config::brpc_idle_timeout_sec; + } + if (config::brpc_num_threads != -1) { + options.num_threads = config::brpc_num_threads; + } + int port = config::brpc_listen_port; + if (server.Start(port, &options) != 0) { + char buf[64]; + LOG(WARNING) << "failed to start brpc, errno=" << errno + << ", errmsg=" << strerror_r(errno, buf, 64) << ", port=" << port; + return -1; + } + LOG(INFO) << "successfully started brpc listening on port=" << port; + + server.RunUntilAskedToQuit(); // Wait for signals + server.ClearServices(); + if (meta_server) { + meta_server->stop(); + } + if (recycler) { + recycler->stop(); + } + + if (periodiccally_log_thread.joinable()) { + { + std::unique_lock lck {periodiccally_log_thread_lock}; + periodiccally_log_thread_run = false; + // immediately notify the log thread to quickly exit in case it block the + // whole procedure + periodiccally_log_thread_cv.notify_one(); + } + periodiccally_log_thread.join(); + } + + return 0; +} diff --git a/cloud/src/meta-service/CMakeLists.txt b/cloud/src/meta-service/CMakeLists.txt new file mode 100644 index 00000000000000..33dd9b6c0ceb7b --- /dev/null +++ b/cloud/src/meta-service/CMakeLists.txt @@ -0,0 +1,27 @@ + + +# where to put generated libraries +set(LIBRARY_OUTPUT_PATH "${BUILD_DIR}/src/meta-service") + +# where to put generated binaries +set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/src/meta-service") + +# set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -lfdb_c -L${THIRDPARTY_DIR}/lib") + +add_library(MetaService + meta_server.cpp + meta_service.cpp + meta_service_http.cpp + meta_service_job.cpp + meta_service_resource.cpp + meta_service_schema.cpp + meta_service_tablet_stats.cpp + meta_service_partition.cpp + meta_service_txn.cpp + txn_kv.cpp + codec.cpp + keys.cpp + doris_txn.cpp + mem_txn_kv.cpp + http_encode_key.cpp +) diff --git a/cloud/src/meta-service/codec.cpp b/cloud/src/meta-service/codec.cpp new file mode 100644 index 00000000000000..fd2085eb8bf1be --- /dev/null +++ b/cloud/src/meta-service/codec.cpp @@ -0,0 +1,133 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "codec.h" + +namespace doris::cloud { + +void encode_bytes(std::string_view bytes, std::string* b) { + // possible_num_0x00 prefix_suffix + b->reserve(b->size() + (bytes.size() / 256 * 2) + 3); + b->push_back(static_cast(EncodingTag::BYTES_TAG)); + std::size_t escape_pos = 0; + std::size_t start_pos = 0; + while (true) { + escape_pos = bytes.find(EncodingTag::BYTE_ESCAPE, escape_pos); + if (escape_pos == std::string::npos) { + break; + } + b->insert(b->end(), bytes.begin() + start_pos, bytes.begin() + escape_pos); + b->push_back(EncodingTag::BYTE_ESCAPE); + b->push_back(EncodingTag::ESCAPED_00); + ++escape_pos; + start_pos = escape_pos; + } + b->insert(b->end(), bytes.begin() + start_pos, bytes.end()); + b->push_back(EncodingTag::BYTE_ESCAPE); + b->push_back(EncodingTag::BYTES_ENDING); +} + +/** + * Decodes byte sequence which is generated with `encode_bytes` + * + * @param in intput for decoding + * @param out output + * @return 0 for success + */ +int decode_bytes(std::string_view* in, std::string* out) { + if (in->at(0) != EncodingTag::BYTES_TAG) return -1; + using byte = unsigned char; + in->remove_prefix(1); // Remove bytes marker + while (true) { + size_t pos = in->find(EncodingTag::BYTE_ESCAPE); + if (pos == std::string::npos) { // At least one should be found + // No EncodingTag::BYTE_ESCAPE found, array without ending + return -2; + } + if ((pos + 1) >= in->size()) { + // Malformed bytes encoding + return -3; + } + byte c = static_cast((*in)[pos + 1]); + if (c == EncodingTag::BYTES_ENDING) { + if (out != nullptr) { + out->append(in->data(), pos); + } + in->remove_prefix(pos + 2); + break; + } else if (c == EncodingTag::ESCAPED_00) { + if (out != nullptr) { + out->append(in->data(), pos + 1); + } + in->remove_prefix(pos + 2); + } else { + // undefined escaping marker + return -4; + } + } + return 0; +} + +/** + * Encodes int64 to 8-byte big endian + * FIXME: use entire 8-bytes + */ +void encode_int64(int64_t val, std::string* b) { + // static_assert(std::endian::little); // Since c++20 + std::string dat(9, '\x00'); + dat[0] = val < 0 ? EncodingTag::NEGATIVE_FIXED_INT_TAG : EncodingTag::POSITIVE_FIXED_INT_TAG; + int64_t& v = *reinterpret_cast(dat.data() + 1); + v = val < 0 ? -val : val; + // clang-format off + // assert: highest bit (sign) is never 1 + v = ((v & 0xffffffff00000000) >> 32) | ((v & 0x00000000ffffffff) << 32); + v = ((v & 0xffff0000ffff0000) >> 16) | ((v & 0x0000ffff0000ffff) << 16); + v = ((v & 0xff00ff00ff00ff00) >> 8) | ((v & 0x00ff00ff00ff00ff) << 8); + // clang-format on + b->reserve(b->size() + dat.size()); + b->insert(b->end(), dat.begin(), dat.end()); +} + +int decode_int64(std::string_view* in, int64_t* val) { + // static_assert(std::endian::little); // Since c++20 + if (in->size() < 9) return -1; // Insufficient length to decode + if (in->at(0) != EncodingTag::NEGATIVE_FIXED_INT_TAG && + in->at(0) != EncodingTag::POSITIVE_FIXED_INT_TAG) { + // Invalid tag + return -2; + } + bool is_negative = in->at(0) == EncodingTag::NEGATIVE_FIXED_INT_TAG; + uint64_t v = *reinterpret_cast(in->data() + 1); + // clang-format off + // assert: highest bit (sign) is never 1 + v = ((v & 0xffffffff00000000) >> 32) | ((v & 0x00000000ffffffff) << 32); + v = ((v & 0xffff0000ffff0000) >> 16) | ((v & 0x0000ffff0000ffff) << 16); + v = ((v & 0xff00ff00ff00ff00) >> 8) | ((v & 0x00ff00ff00ff00ff) << 8); + // clang-format on + + // We haven't used entire 64 bits of unsigned int64 yet, hence we treat + // EncodingTag::NEGATIVE_FIXED_INT_TAG and EncodingTag::POSITIVE_FIXED_INT_TAG + // the same here + *val = static_cast(v); + *val = is_negative ? -*val : *val; + + in->remove_prefix(9); + + return 0; +} + +} // namespace doris::cloud diff --git a/cloud/src/meta-service/codec.h b/cloud/src/meta-service/codec.h new file mode 100644 index 00000000000000..7f64a17176d6ba --- /dev/null +++ b/cloud/src/meta-service/codec.h @@ -0,0 +1,78 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include + +namespace doris::cloud { + +struct EncodingTag { + // Tags for each type + enum Tag : unsigned char { + BYTES_TAG = 0x10, + NEGATIVE_FIXED_INT_TAG = 0x11, + POSITIVE_FIXED_INT_TAG = 0x12, + }; + + // Magic value used for encoding + enum E1 : unsigned char { + BYTE_ESCAPE = 0x00, + BYTES_ENDING = 0x01, + ESCAPED_00 = 0xff, + }; +}; + +/** + * Encodes a byte sequence. Order is preserved. + * + * e.g. + * + * 0xdead00beef => 0x10 dead 00ff beef 0001 + * + * @param bytes byte sequence to encode + * @param b output, result will append to this string + */ +void encode_bytes(std::string_view bytes, std::string* b); + +/** + * Decodes byte sequence which is generated with `encode_bytes` + * + * @param in intput for decoding + * @param out output + * @return 0 for success otherwise error + */ +int decode_bytes(std::string_view* in, std::string* out); + +/** + * Encodes int64 to 8-byte big endian + * Negative 0x11 0000000000000000 + * Positive 0x12 0000000000000000 + * FIXME: use entire 8-bytes + */ +void encode_int64(int64_t val, std::string* b); + +/** + * Decodes byte sequence which is generated with `encode_int64` + * + * @param in intput for decoding + * @param val output + * @return 0 for success otherwise error + */ +int decode_int64(std::string_view* in, int64_t* val); + +} // namespace doris::cloud diff --git a/cloud/src/meta-service/doris_txn.cpp b/cloud/src/meta-service/doris_txn.cpp new file mode 100644 index 00000000000000..7c185bbeb4ad21 --- /dev/null +++ b/cloud/src/meta-service/doris_txn.cpp @@ -0,0 +1,65 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "doris_txn.h" + +namespace doris::cloud { + +int get_txn_id_from_fdb_ts(std::string_view fdb_vts, int64_t* txn_id) { + if (fdb_vts.size() != 10) return 1; // Malformed version timestamp + + // fdb version timestamp is big-endian + // MSB LSB + // 0000000000000000 0000 + // ts seq + // byte addr 0 1 2 3 4 5 6 7 8 9 + int64_t ver = *reinterpret_cast(fdb_vts.data()); + + // static_assert(std::endian::little); // Since c++20 + // Convert big endian to little endian + static auto to_little = [](int64_t v) { + v = ((v & 0xffffffff00000000) >> 32) | ((v & 0x00000000ffffffff) << 32); + v = ((v & 0xffff0000ffff0000) >> 16) | ((v & 0x0000ffff0000ffff) << 16); + v = ((v & 0xff00ff00ff00ff00) >> 8) | ((v & 0x00ff00ff00ff00ff) << 8); + return v; + }; + ver = to_little(ver); + + int64_t seq = *reinterpret_cast(fdb_vts.data() + 2); + seq = to_little(seq); + seq &= 0x000000000000ffff; // Strip off non-seq part + + // CAUTION: DO NOT EVER TOUCH IT!!! UNLESS YOU ARE PREPARED FOR THE DOOM!!! + // CAUTION: DO NOT EVER TOUCH IT!!! UNLESS YOU ARE PREPARED FOR THE DOOM!!! + // CAUTION: DO NOT EVER TOUCH IT!!! UNLESS YOU ARE PREPARED FOR THE DOOM!!! + static constexpr int SEQ_RETAIN_BITS = 10; + + if (seq >= (1L << SEQ_RETAIN_BITS)) { + // seq exceeds the max value + return 2; + } + + // Squeeze seq into ver + seq &= ((1L << SEQ_RETAIN_BITS) - 1L); // Strip off uninterested part + ver <<= SEQ_RETAIN_BITS; + ver |= seq; + + *txn_id = ver; + return 0; +} + +} // namespace doris::cloud diff --git a/cloud/src/meta-service/doris_txn.h b/cloud/src/meta-service/doris_txn.h new file mode 100644 index 00000000000000..9bc9a7617b0fc2 --- /dev/null +++ b/cloud/src/meta-service/doris_txn.h @@ -0,0 +1,33 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include + +namespace doris::cloud { + +/** + * Converts 10-byte fdb version timestamp to 8-byte doris txn id + * + * @param fdb_vts 10 bytes fdb version timestamp + * @param txn_id 8-byte output txn_id for doris + * @return 0 for success otherwise error + */ +int get_txn_id_from_fdb_ts(std::string_view fdb_vts, int64_t* txn_id); + +} // namespace doris::cloud diff --git a/cloud/src/meta-service/http_encode_key.cpp b/cloud/src/meta-service/http_encode_key.cpp new file mode 100644 index 00000000000000..df27ce5a0a5f64 --- /dev/null +++ b/cloud/src/meta-service/http_encode_key.cpp @@ -0,0 +1,268 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include "common/sync_point.h" +#include "common/util.h" +#include "meta-service/doris_txn.h" +#include "meta-service/keys.h" +#include "meta-service/meta_service_http.h" +#include "meta-service/meta_service_schema.h" +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" + +namespace doris::cloud { + +// Initializes a tuple with give params, bind runtime input params to a tuple +// like structure +// TODO(gavin): add type restriction with static_assert or concept +// clang-format off +template +struct KeyInfoSetter { + // Gets the result with input params + R get() { + R r; + if (params.size() != std::tuple_size_v) { return r; } + init(r, std::make_index_sequence>{}); + return r; + } + + // Iterate over all fields and set with value + template + void init(T&& t, std::index_sequence) { + (set_helper(t, params), ...); + } + + // Set the given element in tuple with index `I`. + // The trick is using `if constexpr` to choose compilation code statically + // even though `std::get(t)` is either `string` or `int64_t`. + template + void set_helper(T&& t, const std::vector& params) { + using base_type = typename std::remove_reference_t::base_type; + if constexpr (std::is_same_v, std::string>) { + std::get(t) = *params[I]; + } + if constexpr (std::is_same_v, int64_t>) { + std::get(t) = std::strtoll(params[I]->c_str(), nullptr, 10); + } + } + + std::vector params; +}; +// clang-format on + +using param_type = const std::vector; + +template +static std::string parse(const ValueBuf& buf) { + ProtoType pb; + if (!buf.to_pb(&pb)) return ""; + return proto_to_json(pb); +} + +static std::string parse_txn_label(const ValueBuf& buf) { + std::string value; + if (buf.iters.size() != 1 || buf.iters[0]->size() != 1) { + // FIXME(plat1ko): Hard code as we are confident that the `TxnLabel` length + // will not exceed the splitting threshold + return value; + } + buf.iters[0]->reset(); + auto [k, v] = buf.iters[0]->next(); + if (v.size() < VERSION_STAMP_LEN) { + return value; + } + int64_t txn_id = 0; + if (0 != get_txn_id_from_fdb_ts(v.substr(v.size() - VERSION_STAMP_LEN, v.size()), &txn_id)) { + return value; + } + TxnLabelPB pb; + if (!pb.ParseFromArray(v.data(), v.size() - VERSION_STAMP_LEN)) { + return value; + } + value = proto_to_json(pb); + value += fmt::format("\ntxn_id={}", txn_id); + return value; +} + +static std::string parse_delete_bitmap(const ValueBuf& buf) { + std::stringstream ss; + ss << std::hex << std::setw(2) << std::setfill('0'); + for (auto&& it : buf.iters) { + it->reset(); + while (it->has_next()) { + auto [k, v] = it->next(); + for (auto i : v) ss << ((int16_t)i & 0xff); + } + } + return ss.str(); +} + +static std::string parse_tablet_schema(const ValueBuf& buf) { + doris::TabletSchemaCloudPB pb; + if (!parse_schema_value(buf, &pb)) return ""; + return proto_to_json(pb); +} + +// See keys.h to get all types of key, e.g: MetaRowsetKeyInfo +// key_type -> {{param1, param2 ...}, encoding_func, value_parsing_func} +// clang-format off +static std::unordered_map, std::function, std::function>> param_set { + {"InstanceKey", {{"instance_id"}, [](param_type& p) { return instance_key(KeyInfoSetter{p}.get()); } , parse}} , + {"TxnLabelKey", {{"instance_id", "db_id", "label"}, [](param_type& p) { return txn_label_key(KeyInfoSetter{p}.get()); } , parse_txn_label}} , + {"TxnInfoKey", {{"instance_id", "db_id", "txn_id"}, [](param_type& p) { return txn_info_key(KeyInfoSetter{p}.get()); } , parse}} , + {"TxnIndexKey", {{"instance_id", "txn_id"}, [](param_type& p) { return txn_index_key(KeyInfoSetter{p}.get()); } , parse}} , + {"TxnRunningKey", {{"instance_id", "db_id", "txn_id"}, [](param_type& p) { return txn_running_key(KeyInfoSetter{p}.get()); } , parse}} , + {"VersionKey", {{"instance_id", "db_id", "tbl_id", "partition_id"}, [](param_type& p) { return version_key(KeyInfoSetter{p}.get()); } , parse}} , + {"MetaRowsetKey", {{"instance_id", "tablet_id", "version"}, [](param_type& p) { return meta_rowset_key(KeyInfoSetter{p}.get()); } , parse}} , + {"MetaRowsetTmpKey", {{"instance_id", "txn_id", "tablet_id"}, [](param_type& p) { return meta_rowset_tmp_key(KeyInfoSetter{p}.get()); } , parse}} , + {"MetaTabletKey", {{"instance_id", "table_id", "index_id", "part_id", "tablet_id"}, [](param_type& p) { return meta_tablet_key(KeyInfoSetter{p}.get()); } , parse}} , + {"MetaTabletIdxKey", {{"instance_id", "tablet_id"}, [](param_type& p) { return meta_tablet_idx_key(KeyInfoSetter{p}.get()); } , parse}} , + {"RecycleIndexKey", {{"instance_id", "index_id"}, [](param_type& p) { return recycle_index_key(KeyInfoSetter{p}.get()); } , parse}} , + {"RecyclePartKey", {{"instance_id", "part_id"}, [](param_type& p) { return recycle_partition_key(KeyInfoSetter{p}.get()); } , parse}} , + {"RecycleRowsetKey", {{"instance_id", "tablet_id", "rowset_id"}, [](param_type& p) { return recycle_rowset_key(KeyInfoSetter{p}.get()); } , parse}} , + {"RecycleTxnKey", {{"instance_id", "db_id", "txn_id"}, [](param_type& p) { return recycle_txn_key(KeyInfoSetter{p}.get()); } , parse}} , + {"StatsTabletKey", {{"instance_id", "table_id", "index_id", "part_id", "tablet_id"}, [](param_type& p) { return stats_tablet_key(KeyInfoSetter{p}.get()); } , parse}} , + {"JobTabletKey", {{"instance_id", "table_id", "index_id", "part_id", "tablet_id"}, [](param_type& p) { return job_tablet_key(KeyInfoSetter{p}.get()); } , parse}} , + {"CopyJobKey", {{"instance_id", "stage_id", "table_id", "copy_id", "group_id"}, [](param_type& p) { return copy_job_key(KeyInfoSetter{p}.get()); } , parse}} , + {"CopyFileKey", {{"instance_id", "stage_id", "table_id", "obj_key", "obj_etag"}, [](param_type& p) { return copy_file_key(KeyInfoSetter{p}.get()); } , parse}} , + {"RecycleStageKey", {{"instance_id", "stage_id"}, [](param_type& p) { return recycle_stage_key(KeyInfoSetter{p}.get()); } , parse}} , + {"JobRecycleKey", {{"instance_id"}, [](param_type& p) { return job_check_key(KeyInfoSetter{p}.get()); } , parse}} , + {"MetaSchemaKey", {{"instance_id", "index_id", "schema_version"}, [](param_type& p) { return meta_schema_key(KeyInfoSetter{p}.get()); } , parse_tablet_schema}} , + {"MetaDeleteBitmap", {{"instance_id", "tablet_id", "rowest_id", "version", "seg_id"}, [](param_type& p) { return meta_delete_bitmap_key(KeyInfoSetter{p}.get()); } , parse_delete_bitmap}} , + {"MetaDeleteBitmapUpdateLock", {{"instance_id", "table_id", "partition_id"}, [](param_type& p) { return meta_delete_bitmap_update_lock_key( KeyInfoSetter{p}.get()); }, parse}}, + {"MetaPendingDeleteBitmap", {{"instance_id", "tablet_id"}, [](param_type& p) { return meta_pending_delete_bitmap_key( KeyInfoSetter{p}.get()); } , parse}} , + {"RLJobProgressKey", {{"instance_id", "db_id", "job_id"}, [](param_type& p) { return rl_job_progress_key_info( KeyInfoSetter{p}.get()); } , parse}} , + {"MetaServiceRegistryKey", {std::vector {}, [](param_type& p) { return system_meta_service_registry_key(); } , parse}} , + {"MetaServiceArnInfoKey", {std::vector {}, [](param_type& p) { return system_meta_service_arn_info_key(); } , parse}} , + {"MetaServiceEncryptionKey", {std::vector {}, [](param_type& p) { return system_meta_service_encryption_key_info_key(); } , parse}} , +}; +// clang-format on + +static MetaServiceResponseStatus encode_key(const brpc::URI& uri, std::string& key) { + MetaServiceResponseStatus status; + status.set_code(MetaServiceCode::OK); + std::string_view key_type = http_query(uri, "key_type"); + auto it = param_set.find(key_type); + if (it == param_set.end()) { + status.set_code(MetaServiceCode::INVALID_ARGUMENT); + status.set_msg(fmt::format("key_type not supported: {}", + (key_type.empty() ? "(empty)" : key_type))); + return status; + } + std::remove_cv_t params; + params.reserve(std::get<0>(it->second).size()); + for (auto& i : std::get<0>(it->second)) { + auto p = uri.GetQuery(i.data()); + if (p == nullptr || p->empty()) { + status.set_code(MetaServiceCode::INVALID_ARGUMENT); + status.set_msg(fmt::format("{} is not given or empty", i)); + return status; + } + params.emplace_back(p); + } + key = std::get<1>(it->second)(params); + return status; +} + +HttpResponse process_http_get_value(TxnKv* txn_kv, const brpc::URI& uri) { + std::string key; + if (auto hex_key = http_query(uri, "key"); !hex_key.empty()) { + key = unhex(hex_key); + } else { // Encode key from params + auto st = encode_key(uri, key); + if (st.code() != MetaServiceCode::OK) { + return http_json_reply(st); + } + } + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) [[unlikely]] { + return http_json_reply(MetaServiceCode::KV_TXN_CREATE_ERR, + fmt::format("failed to create txn, err={}", err)); + } + ValueBuf value; + err = cloud::get(txn.get(), key, &value, true); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + // FIXME: Key not found err + return http_json_reply(MetaServiceCode::KV_TXN_GET_ERR, + fmt::format("kv not found, key={}", hex(key))); + } + if (err != TxnErrorCode::TXN_OK) { + return http_json_reply(MetaServiceCode::KV_TXN_GET_ERR, + fmt::format("failed to get kv, key={}", hex(key))); + } + std::string_view key_type = http_query(uri, "key_type"); + auto it = param_set.find(key_type); + if (it == param_set.end()) { + return http_json_reply(MetaServiceCode::INVALID_ARGUMENT, + fmt::format("key_type not supported: {}", + (key_type.empty() ? "(empty)" : key_type))); + } + auto readable_value = std::get<2>(it->second)(value); + if (readable_value.empty()) [[unlikely]] { + return http_json_reply(MetaServiceCode::PROTOBUF_PARSE_ERR, + fmt::format("failed to parse value, key={}", hex(key))); + } + return http_text_reply(MetaServiceCode::OK, "", readable_value); +} + +HttpResponse process_http_encode_key(const brpc::URI& uri) { + std::string key; + auto st = encode_key(uri, key); + if (st.code() != MetaServiceCode::OK) { + return http_json_reply(st); + } + + // Print to ensure + bool unicode = !(uri.GetQuery("unicode") != nullptr && *uri.GetQuery("unicode") == "false"); + + auto hex_key = hex(key); + std::string body = prettify_key(hex_key, unicode); + TEST_SYNC_POINT_CALLBACK("process_http_encode_key::empty_body", &body); + if (body.empty()) { + return http_json_reply(MetaServiceCode::INVALID_ARGUMENT, + "failed to decode encoded key, key=" + hex_key, + "failed to decode key, it may be malformed"); + } + + static auto format_fdb_key = [](const std::string& s) { + std::stringstream r; + for (size_t i = 0; i < s.size(); ++i) { + if (!(i % 2)) r << "\\x"; + r << s[i]; + } + return r.str(); + }; + + return http_text_reply(MetaServiceCode::OK, "", body + format_fdb_key(hex_key) + "\n"); +} + +} // namespace doris::cloud diff --git a/cloud/src/meta-service/keys.cpp b/cloud/src/meta-service/keys.cpp new file mode 100644 index 00000000000000..2ccd1e6f2d168d --- /dev/null +++ b/cloud/src/meta-service/keys.cpp @@ -0,0 +1,485 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "keys.h" + +#include +#include +#include + +#include "codec.h" + +namespace doris::cloud { + +// clang-format off +// Prefix +[[maybe_unused]] static const char* INSTANCE_KEY_PREFIX = "instance"; + +[[maybe_unused]] static const char* TXN_KEY_PREFIX = "txn"; +[[maybe_unused]] static const char* VERSION_KEY_PREFIX = "version"; +[[maybe_unused]] static const char* META_KEY_PREFIX = "meta"; +[[maybe_unused]] static const char* RECYCLE_KEY_PREFIX = "recycle"; +[[maybe_unused]] static const char* STATS_KEY_PREFIX = "stats"; +[[maybe_unused]] static const char* JOB_KEY_PREFIX = "job"; +[[maybe_unused]] static const char* COPY_KEY_PREFIX = "copy"; + +// Infix +[[maybe_unused]] static const char* TXN_KEY_INFIX_LABEL = "txn_label"; +[[maybe_unused]] static const char* TXN_KEY_INFIX_INFO = "txn_info"; +[[maybe_unused]] static const char* TXN_KEY_INFIX_INDEX = "txn_index"; +[[maybe_unused]] static const char* TXN_KEY_INFIX_RUNNING = "txn_running"; + +[[maybe_unused]] static const char* VERSION_KEY_INFIX = "partition"; + +[[maybe_unused]] static const char* META_KEY_INFIX_ROWSET = "rowset"; +[[maybe_unused]] static const char* META_KEY_INFIX_ROWSET_TMP = "rowset_tmp"; +[[maybe_unused]] static const char* META_KEY_INFIX_TABLET = "tablet"; +[[maybe_unused]] static const char* META_KEY_INFIX_TABLET_IDX = "tablet_index"; +[[maybe_unused]] static const char* META_KEY_INFIX_SCHEMA = "schema"; +[[maybe_unused]] static const char* META_KEY_INFIX_DELETE_BITMAP = "delete_bitmap"; +[[maybe_unused]] static const char* META_KEY_INFIX_DELETE_BITMAP_LOCK = "delete_bitmap_lock"; +[[maybe_unused]] static const char* META_KEY_INFIX_DELETE_BITMAP_PENDING = "delete_bitmap_pending"; + +[[maybe_unused]] static const char* RECYCLE_KEY_INFIX_INDEX = "index"; +[[maybe_unused]] static const char* RECYCLE_KEY_INFIX_PART = "partition"; +[[maybe_unused]] static const char* RECYCLE_KEY_TXN = "txn"; + +[[maybe_unused]] static const char* STATS_KEY_INFIX_TABLET = "tablet"; + +[[maybe_unused]] static const char* JOB_KEY_INFIX_TABLET = "tablet"; +[[maybe_unused]] static const char* JOB_KEY_INFIX_RL_PROGRESS = "routine_load_progress"; + +[[maybe_unused]] static const char* COPY_JOB_KEY_INFIX = "job"; +[[maybe_unused]] static const char* COPY_FILE_KEY_INFIX = "loading_file"; +[[maybe_unused]] static const char* STAGE_KEY_INFIX = "stage"; + +// clang-format on + +// clang-format off +template +constexpr static bool is_one_of() { return std::is_same_v; } +/** + * Checks the first type is one of the given types (type collection) + * @param T type to check + * @param U first type in the collection + * @param R the rest types in the collection + */ +template +constexpr static typename std::enable_if_t<0 < sizeof...(R), bool> is_one_of() { + return ((std::is_same_v) || is_one_of()); +} + +template +constexpr static bool not_all_types_distinct() { return std::is_same_v; } +/** + * Checks if there are 2 types are the same in the given type list + */ +template +constexpr static typename std::enable_if_t<0 < sizeof...(R), bool> +not_all_types_distinct() { + // The last part of this expr is a `for` loop + return is_one_of() || is_one_of() || not_all_types_distinct(); +} + +template +struct check_types { + static_assert(is_one_of(), "Invalid key type"); + static_assert(!not_all_types_distinct(), "Type conflict, there are at least 2 types that are identical in the list."); + static constexpr bool value = is_one_of() && !not_all_types_distinct(); +}; +template +inline constexpr bool check_types_v = check_types::value; + +template +static void encode_prefix(const T& t, std::string* key) { + // Input type T must be one of the following, add if needed + static_assert(check_types_v); + + key->push_back(CLOUD_USER_KEY_SPACE01); + // Prefixes for key families + if constexpr (std::is_same_v) { + encode_bytes(INSTANCE_KEY_PREFIX, key); + } else if constexpr (std::is_same_v + || std::is_same_v + || std::is_same_v + || std::is_same_v) { + encode_bytes(TXN_KEY_PREFIX, key); + } else if constexpr (std::is_same_v + || std::is_same_v + || std::is_same_v + || std::is_same_v + || std::is_same_v + || std::is_same_v + || std::is_same_v + || std::is_same_v) { + encode_bytes(META_KEY_PREFIX, key); + } else if constexpr (std::is_same_v) { + encode_bytes(VERSION_KEY_PREFIX, key); + } else if constexpr (std::is_same_v + || std::is_same_v + || std::is_same_v + || std::is_same_v + || std::is_same_v) { + encode_bytes(RECYCLE_KEY_PREFIX, key); + } else if constexpr (std::is_same_v) { + encode_bytes(STATS_KEY_PREFIX, key); + } else if constexpr (std::is_same_v + || std::is_same_v + || std::is_same_v) { + encode_bytes(JOB_KEY_PREFIX, key); + } else if constexpr (std::is_same_v + || std::is_same_v) { + encode_bytes(COPY_KEY_PREFIX, key); + } else { + // This branch mean to be unreachable, add an assert(false) here to + // prevent missing branch match. + // Postpone deduction of static_assert by evaluating sizeof(T) + static_assert(!sizeof(T), "all types must be matched with if constexpr"); + } + encode_bytes(std::get<0>(t), key); // instance_id +} +// clang-format on + +//============================================================================== +// Resource keys +//============================================================================== + +void instance_key(const InstanceKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "instance" ${instance_id} +} + +//============================================================================== +// Transaction keys +//============================================================================== + +std::string txn_key_prefix(std::string_view instance_id) { + std::string out; + encode_prefix(TxnIndexKeyInfo {instance_id, 0}, &out); + return out; +} + +void txn_label_key(const TxnLabelKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "txn" ${instance_id} + encode_bytes(TXN_KEY_INFIX_LABEL, out); // "txn_index" + encode_int64(std::get<1>(in), out); // db_id + encode_bytes(std::get<2>(in), out); // label +} + +void txn_info_key(const TxnInfoKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "txn" ${instance_id} + encode_bytes(TXN_KEY_INFIX_INFO, out); // "txn_info" + encode_int64(std::get<1>(in), out); // db_id + encode_int64(std::get<2>(in), out); // txn_id +} + +void txn_index_key(const TxnIndexKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "txn" ${instance_id} + encode_bytes(TXN_KEY_INFIX_INDEX, out); // "txn_index" + encode_int64(std::get<1>(in), out); // txn_id +} + +void txn_running_key(const TxnRunningKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "txn" ${instance_id} + encode_bytes(TXN_KEY_INFIX_RUNNING, out); // "txn_running" + encode_int64(std::get<1>(in), out); // db_id + encode_int64(std::get<2>(in), out); // txn_id +} + +//============================================================================== +// Version keys +//============================================================================== + +void version_key(const VersionKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "version" ${instance_id} + encode_bytes(VERSION_KEY_INFIX, out); // "partition" + encode_int64(std::get<1>(in), out); // db_id + encode_int64(std::get<2>(in), out); // tbl_id + encode_int64(std::get<3>(in), out); // partition_id +} + +//============================================================================== +// Meta keys +//============================================================================== + +std::string meta_key_prefix(std::string_view instance_id) { + std::string out; + encode_prefix(MetaTabletIdxKeyInfo {instance_id, 0}, &out); + return out; +} + +void meta_rowset_key(const MetaRowsetKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "meta" ${instance_id} + encode_bytes(META_KEY_INFIX_ROWSET, out); // "rowset" + encode_int64(std::get<1>(in), out); // tablet_id + encode_int64(std::get<2>(in), out); // version +} + +void meta_rowset_tmp_key(const MetaRowsetTmpKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "meta" ${instance_id} + encode_bytes(META_KEY_INFIX_ROWSET_TMP, out); // "rowset_tmp" + encode_int64(std::get<1>(in), out); // txn_id + encode_int64(std::get<2>(in), out); // tablet_id +} + +void meta_tablet_key(const MetaTabletKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "meta" ${instance_id} + encode_bytes(META_KEY_INFIX_TABLET, out); // "tablet" + encode_int64(std::get<1>(in), out); // table_id + encode_int64(std::get<2>(in), out); // index_id + encode_int64(std::get<3>(in), out); // partition_id + encode_int64(std::get<4>(in), out); // tablet_id +} + +void meta_tablet_idx_key(const MetaTabletIdxKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "meta" ${instance_id} + encode_bytes(META_KEY_INFIX_TABLET_IDX, out); // "tablet_index" + encode_int64(std::get<1>(in), out); // tablet_id +} + +void meta_schema_key(const MetaSchemaKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "meta" ${instance_id} + encode_bytes(META_KEY_INFIX_SCHEMA, out); // "schema" + encode_int64(std::get<1>(in), out); // index_id + encode_int64(std::get<2>(in), out); // schema_version +} + +void meta_delete_bitmap_key(const MetaDeleteBitmapInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "meta" ${instance_id} + encode_bytes(META_KEY_INFIX_DELETE_BITMAP, out); // "delete_bitmap" + encode_int64(std::get<1>(in), out); // tablet_id + encode_bytes(std::get<2>(in), out); // rowset_id + encode_int64(std::get<3>(in), out); // version + encode_int64(std::get<4>(in), out); // segment_id +} + +void meta_delete_bitmap_update_lock_key(const MetaDeleteBitmapUpdateLockInfo& in, + std::string* out) { + encode_prefix(in, out); // 0x01 "meta" ${instance_id} + encode_bytes(META_KEY_INFIX_DELETE_BITMAP_LOCK, out); // "delete_bitmap_lock" + encode_int64(std::get<1>(in), out); // table_id + encode_int64(std::get<2>(in), out); // partition_id +} + +void meta_pending_delete_bitmap_key(const MetaPendingDeleteBitmapInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "meta" ${instance_id} + encode_bytes(META_KEY_INFIX_DELETE_BITMAP_PENDING, out); // "delete_bitmap_pending" + encode_int64(std::get<1>(in), out); // table_id +} + +//============================================================================== +// Recycle keys +//============================================================================== + +std::string recycle_key_prefix(std::string_view instance_id) { + std::string out; + encode_prefix(RecycleIndexKeyInfo {instance_id, 0}, &out); + return out; +} + +void recycle_index_key(const RecycleIndexKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "recycle" ${instance_id} + encode_bytes(RECYCLE_KEY_INFIX_INDEX, out); // "index" + encode_int64(std::get<1>(in), out); // index_id +} + +void recycle_partition_key(const RecyclePartKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "recycle" ${instance_id} + encode_bytes(RECYCLE_KEY_INFIX_PART, out); // "partition" + encode_int64(std::get<1>(in), out); // partition_id +} + +void recycle_rowset_key(const RecycleRowsetKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "recycle" ${instance_id} + encode_bytes(META_KEY_INFIX_ROWSET, out); // "rowset" + encode_int64(std::get<1>(in), out); // tablet_id + encode_bytes(std::get<2>(in), out); // rowset_id +} + +void recycle_txn_key(const RecycleTxnKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "recycle" ${instance_id} + encode_bytes(RECYCLE_KEY_TXN, out); // "txn" + encode_int64(std::get<1>(in), out); // db_id + encode_int64(std::get<2>(in), out); // txn_id +} + +void recycle_stage_key(const RecycleStageKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "recycle" ${instance_id} + encode_bytes(STAGE_KEY_INFIX, out); // "stage" + encode_bytes(std::get<1>(in), out); // stage_id +} + +//============================================================================== +// Stats keys +//============================================================================== + +void stats_tablet_key(const StatsTabletKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "stats" ${instance_id} + encode_bytes(STATS_KEY_INFIX_TABLET, out); // "tablet" + encode_int64(std::get<1>(in), out); // table_id + encode_int64(std::get<2>(in), out); // index_id + encode_int64(std::get<3>(in), out); // partition_id + encode_int64(std::get<4>(in), out); // tablet_id +} + +void stats_tablet_data_size_key(const StatsTabletKeyInfo& in, std::string* out) { + stats_tablet_key(in, out); + encode_bytes(STATS_KEY_SUFFIX_DATA_SIZE, out); +} +void stats_tablet_num_rows_key(const StatsTabletKeyInfo& in, std::string* out) { + stats_tablet_key(in, out); + encode_bytes(STATS_KEY_SUFFIX_NUM_ROWS, out); +} +void stats_tablet_num_rowsets_key(const StatsTabletKeyInfo& in, std::string* out) { + stats_tablet_key(in, out); + encode_bytes(STATS_KEY_SUFFIX_NUM_ROWSETS, out); +} +void stats_tablet_num_segs_key(const StatsTabletKeyInfo& in, std::string* out) { + stats_tablet_key(in, out); + encode_bytes(STATS_KEY_SUFFIX_NUM_SEGS, out); +} + +//============================================================================== +// Job keys +//============================================================================== + +void job_tablet_key(const JobTabletKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "job" ${instance_id} + encode_bytes(JOB_KEY_INFIX_TABLET, out); // "tablet" + encode_int64(std::get<1>(in), out); // table_id + encode_int64(std::get<2>(in), out); // index_id + encode_int64(std::get<3>(in), out); // partition_id + encode_int64(std::get<4>(in), out); // tablet_id +} + +void job_recycle_key(const JobRecycleKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "job" ${instance_id} + encode_bytes("recycle", out); // "recycle" +} + +void job_check_key(const JobRecycleKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "job" ${instance_id} + encode_bytes("check", out); // "check" +} + +void rl_job_progress_key_info(const RLJobProgressKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "job" ${instance_id} + encode_bytes(JOB_KEY_INFIX_RL_PROGRESS, out); // "routine_load_progress" + encode_int64(std::get<1>(in), out); // db_id + encode_int64(std::get<2>(in), out); // job_id +} + +//============================================================================== +// Copy keys +//============================================================================== + +std::string copy_key_prefix(std::string_view instance_id) { + std::string out; + encode_prefix(CopyJobKeyInfo {instance_id, "", 0, "", 0}, &out); + return out; +} + +void copy_job_key(const CopyJobKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "copy" ${instance_id} + encode_bytes(COPY_JOB_KEY_INFIX, out); // "job" + encode_bytes(std::get<1>(in), out); // stage_id + encode_int64(std::get<2>(in), out); // table_id + encode_bytes(std::get<3>(in), out); // copy_id + encode_int64(std::get<4>(in), out); // group_id +} + +void copy_file_key(const CopyFileKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "copy" ${instance_id} + encode_bytes(COPY_FILE_KEY_INFIX, out); // "loading_file" + encode_bytes(std::get<1>(in), out); // stage_id + encode_int64(std::get<2>(in), out); // table_id + encode_bytes(std::get<3>(in), out); // obj_key + encode_bytes(std::get<4>(in), out); // obj_etag +} + +// 0x02 0:"system" 1:"meta-service" 2:"registry" +std::string system_meta_service_registry_key() { + std::string ret; + ret.push_back(CLOUD_SYS_KEY_SPACE02); + encode_bytes("system", &ret); + encode_bytes("meta-service", &ret); + encode_bytes("registry", &ret); + return ret; +} + +// 0x02 0:"system" 1:"meta-service" 2:"arn_info" +std::string system_meta_service_arn_info_key() { + std::string ret; + ret.push_back(CLOUD_SYS_KEY_SPACE02); + encode_bytes("system", &ret); + encode_bytes("meta-service", &ret); + encode_bytes("arn_info", &ret); + return ret; +} + +// 0x02 0:"system" 1:"meta-service" 2:"encryption_key_info" +std::string system_meta_service_encryption_key_info_key() { + std::string ret; + ret.push_back(CLOUD_SYS_KEY_SPACE02); + encode_bytes("system", &ret); + encode_bytes("meta-service", &ret); + encode_bytes("encryption_key_info", &ret); + return ret; +} + +//============================================================================== +// Other keys +//============================================================================== + +//============================================================================== +// Decode keys +//============================================================================== +int decode_key(std::string_view* in, + std::vector, int, int>>* out) { + int pos = 0; + int last_len = static_cast(in->size()); + while (!in->empty()) { + int ret = 0; + auto tag = in->at(0); + if (tag == EncodingTag::BYTES_TAG) { + std::string str; + ret = decode_bytes(in, &str); + if (ret != 0) return ret; + out->emplace_back(std::move(str), tag, pos); + } else if (tag == EncodingTag::NEGATIVE_FIXED_INT_TAG || + tag == EncodingTag::POSITIVE_FIXED_INT_TAG) { + int64_t v; + ret = decode_int64(in, &v); + if (ret != 0) return ret; + out->emplace_back(v, tag, pos); + } else { + return -1; + } + pos += last_len - in->size(); + last_len = in->size(); + } + return 0; +} + +} // namespace doris::cloud diff --git a/cloud/src/meta-service/keys.h b/cloud/src/meta-service/keys.h new file mode 100644 index 00000000000000..8786d669158663 --- /dev/null +++ b/cloud/src/meta-service/keys.h @@ -0,0 +1,276 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include +#include +#include +#include + +// clang-format off +// Key encoding schemes: +// +// 0x01 "instance" ${instance_id} -> InstanceInfoPB +// +// 0x01 "txn" ${instance_id} "txn_label" ${db_id} ${label} -> TxnLabelPB ${version_timestamp} +// 0x01 "txn" ${instance_id} "txn_info" ${db_id} ${txn_id} -> TxnInfoPB +// 0x01 "txn" ${instance_id} "txn_db_tbl" ${txn_id} -> TxnIndexPB +// 0x01 "txn" ${instance_id} "txn_running" ${db_id} ${txn_id} -> TxnRunningPB +// +// 0x01 "version" ${instance_id} "partition" ${db_id} ${tbl_id} ${partition_id} -> VersionPB +// +// 0x01 "meta" ${instance_id} "rowset" ${tablet_id} ${version} -> RowsetMetaCloudPB +// 0x01 "meta" ${instance_id} "rowset_tmp" ${txn_id} ${tablet_id} -> RowsetMetaCloudPB +// 0x01 "meta" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} -> TabletMetaCloudPB +// 0x01 "meta" ${instance_id} "tablet_index" ${tablet_id} -> TabletIndexPB +// 0x01 "meta" ${instance_id} "schema" ${index_id} ${schema_version} -> TabletSchemaCloudPB +// 0x01 "meta" ${instance_id} "delete_bitmap_lock" ${table_id} ${partition_id} -> DeleteBitmapUpdateLockPB +// 0x01 "meta" ${instance_id} "delete_bitmap_pending" ${table_id} -> PendingDeleteBitmapPB +// 0x01 "meta" ${instance_id} "delete_bitmap" ${tablet_id} ${rowset_id} ${version} ${segment_id} -> roaringbitmap +// +// 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} -> TabletStatsPB +// 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "data_size" -> int64 +// 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "num_rows" -> int64 +// 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "num_rowsets" -> int64 +// 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "num_segs" -> int64 +// +// 0x01 "recycle" ${instance_id} "index" ${index_id} -> RecycleIndexPB +// 0x01 "recycle" ${instance_id} "partition" ${partition_id} -> RecyclePartitionPB +// 0x01 "recycle" ${instance_id} "rowset" ${tablet_id} ${rowset_id} -> RecycleRowsetPB +// 0x01 "recycle" ${instance_id} "txn" ${db_id} ${txn_id} -> RecycleTxnPB +// 0x01 "recycle" ${instance_id} "stage" ${stage_id} -> RecycleStagePB +// +// 0x01 "job" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} -> TabletJobInfoPB +// 0x01 "job" ${instance_id} "recycle" -> JobRecyclePB +// 0x01 "job" ${instance_id} "check" -> JobRecyclePB +// +// 0x02 "system" "meta-service" "registry" -> MetaServiceRegistryPB +// 0x02 "system" "meta-service" "arn_info" -> RamUserPB +// 0x02 "system" "meta-service" "encryption_key_info" -> EncryptionKeyInfoPB +// +// 0x01 "copy" ${instance_id} "job" ${stage_id} ${table_id} ${copy_id} ${group_id} -> CopyJobPB +// 0x01 "copy" ${instance_id} "loading_file" ${stage_id} ${table_id} ${obj_name} ${etag} -> CopyFilePB +// clang-format on + +namespace doris::cloud { + +static const constexpr unsigned char CLOUD_USER_KEY_SPACE01 = 0x01; +static const constexpr unsigned char CLOUD_SYS_KEY_SPACE02 = 0x02; +static constexpr uint32_t VERSION_STAMP_LEN = 10; + +// Suffix +static constexpr std::string_view STATS_KEY_SUFFIX_DATA_SIZE = "data_size"; +static constexpr std::string_view STATS_KEY_SUFFIX_NUM_ROWS = "num_rows"; +static constexpr std::string_view STATS_KEY_SUFFIX_NUM_ROWSETS = "num_rowsets"; +static constexpr std::string_view STATS_KEY_SUFFIX_NUM_SEGS = "num_segs"; + +// clang-format off +/** + * Wraps std::tuple for differnet types even if the underlying type is the same. + * + * @param N for elemination of same underlying types of type alias when we use + * `using` to declare a new type. + * + * @param Base for base tuple, the underlying type + */ +template +struct BasicKeyInfo : Base { + template + BasicKeyInfo(Args&&... args) : Base(std::forward(args)...) {} + constexpr static size_t n = N; + using base_type = Base; +}; + +// ATTN: newly added key must have different type number + +// 0:instance_id +using InstanceKeyInfo = BasicKeyInfo<0 , std::tuple>; + +// 0:instance_id 1:db_id 2:label +using TxnLabelKeyInfo = BasicKeyInfo<1 , std::tuple>; + +// 0:instance_id 1:db_id 2:txn_id +using TxnInfoKeyInfo = BasicKeyInfo<2 , std::tuple>; + +// 0:instance_id 1:txn_id +using TxnIndexKeyInfo = BasicKeyInfo<3 , std::tuple>; + +// 0:instance_id 1:db_id 2:txn_id +using TxnRunningKeyInfo = BasicKeyInfo<5 , std::tuple>; + +// 0:instance_id 1:db_id 2:tbl_id 3:partition_id +using VersionKeyInfo = BasicKeyInfo<6 , std::tuple>; + +// 0:instance_id 1:tablet_id 2:version +using MetaRowsetKeyInfo = BasicKeyInfo<7 , std::tuple>; + +// 0:instance_id 1:txn_id 2:tablet_id +using MetaRowsetTmpKeyInfo = BasicKeyInfo<8 , std::tuple>; + +// 0:instance_id 1:table_id 2:index_id 3:part_id 4:tablet_id +using MetaTabletKeyInfo = BasicKeyInfo<9 , std::tuple>; + +// 0:instance_id 1:tablet_id +using MetaTabletIdxKeyInfo = BasicKeyInfo<10, std::tuple>; + +// 0:instance_id 1:index_id +using RecycleIndexKeyInfo = BasicKeyInfo<11, std::tuple>; + +// 0:instance_id 1:part_id +using RecyclePartKeyInfo = BasicKeyInfo<12, std::tuple>; + +// 0:instance_id 1:tablet_id 2:rowset_id +using RecycleRowsetKeyInfo = BasicKeyInfo<13, std::tuple>; + +// 0:instance_id 1:db_id 2:txn_id +using RecycleTxnKeyInfo = BasicKeyInfo<14, std::tuple>; + +// 0:instance_id 1:table_id 2:index_id 3:part_id 4:tablet_id +using StatsTabletKeyInfo = BasicKeyInfo<15, std::tuple>; + +// 0:instance_id 1:table_id 2:index_id 3:part_id 4:tablet_id +using JobTabletKeyInfo = BasicKeyInfo<16, std::tuple>; + +// 0:instance_id 1:stage_id 2:table_id 3:copy_id 4:group_id +using CopyJobKeyInfo = BasicKeyInfo<17, std::tuple>; + +// 0:instance_id 1:stage_id 2:table_id 3:obj_key 4:obj_etag +using CopyFileKeyInfo = BasicKeyInfo<18, std::tuple>; + +// 0:instance_id 1:stage_id +using RecycleStageKeyInfo = BasicKeyInfo<19, std::tuple>; + +// 0:instance_id +using JobRecycleKeyInfo = BasicKeyInfo<20 , std::tuple>; + +// 0:instance_id 1:index_id 2:schema_version +using MetaSchemaKeyInfo = BasicKeyInfo<21, std::tuple>; + +// 0:instance_id 1:tablet_id 2:rowest_id 3:version 4:seg_id +using MetaDeleteBitmapInfo = BasicKeyInfo<22 , std::tuple>; + +// partition_id of -1 indicates all partitions +// 0:instance_id 1:table_id 2:partition_id +using MetaDeleteBitmapUpdateLockInfo = BasicKeyInfo<23 , std::tuple>; + +// 0:instance_id 1:tablet_id +using MetaPendingDeleteBitmapInfo = BasicKeyInfo<24 , std::tuple>; + +// 0:instance_id 1:db_id 2:job_id +using RLJobProgressKeyInfo = BasicKeyInfo<25, std::tuple>; + +void instance_key(const InstanceKeyInfo& in, std::string* out); +static inline std::string instance_key(const InstanceKeyInfo& in) { std::string s; instance_key(in, &s); return s; } + +std::string txn_key_prefix(std::string_view instance_id); +void txn_label_key(const TxnLabelKeyInfo& in, std::string* out); +void txn_info_key(const TxnInfoKeyInfo& in, std::string* out); +void txn_index_key(const TxnIndexKeyInfo& in, std::string* out); +void txn_running_key(const TxnRunningKeyInfo& in, std::string* out); +static inline std::string txn_label_key(const TxnLabelKeyInfo& in) { std::string s; txn_label_key(in, &s); return s; } +static inline std::string txn_info_key(const TxnInfoKeyInfo& in) { std::string s; txn_info_key(in, &s); return s; } +static inline std::string txn_index_key(const TxnIndexKeyInfo& in) { std::string s; txn_index_key(in, &s); return s; } +static inline std::string txn_running_key(const TxnRunningKeyInfo& in) { std::string s; txn_running_key(in, &s); return s; } + +void version_key(const VersionKeyInfo& in, std::string* out); +static inline std::string version_key(const VersionKeyInfo& in) { std::string s; version_key(in, &s); return s; } + +std::string meta_key_prefix(std::string_view instance_id); +void meta_rowset_key(const MetaRowsetKeyInfo& in, std::string* out); +void meta_rowset_tmp_key(const MetaRowsetTmpKeyInfo& in, std::string* out); +void meta_tablet_idx_key(const MetaTabletIdxKeyInfo& in, std::string* out); +void meta_tablet_key(const MetaTabletKeyInfo& in, std::string* out); +void meta_schema_key(const MetaSchemaKeyInfo& in, std::string* out); +void meta_delete_bitmap_key(const MetaDeleteBitmapInfo& in, std::string* out); +void meta_delete_bitmap_update_lock_key(const MetaDeleteBitmapUpdateLockInfo& in, std::string* out); +void meta_pending_delete_bitmap_key(const MetaPendingDeleteBitmapInfo& in, std::string* out); +static inline std::string meta_rowset_key(const MetaRowsetKeyInfo& in) { std::string s; meta_rowset_key(in, &s); return s; } +static inline std::string meta_rowset_tmp_key(const MetaRowsetTmpKeyInfo& in) { std::string s; meta_rowset_tmp_key(in, &s); return s; } +static inline std::string meta_tablet_idx_key(const MetaTabletIdxKeyInfo& in) { std::string s; meta_tablet_idx_key(in, &s); return s; } +static inline std::string meta_tablet_key(const MetaTabletKeyInfo& in) { std::string s; meta_tablet_key(in, &s); return s; } +static inline std::string meta_schema_key(const MetaSchemaKeyInfo& in) { std::string s; meta_schema_key(in, &s); return s; } +static inline std::string meta_delete_bitmap_key(const MetaDeleteBitmapInfo& in) { std::string s; meta_delete_bitmap_key(in, &s); return s; } +static inline std::string meta_delete_bitmap_update_lock_key(const MetaDeleteBitmapUpdateLockInfo& in) { std::string s; meta_delete_bitmap_update_lock_key(in, &s); return s; } +static inline std::string meta_pending_delete_bitmap_key(const MetaPendingDeleteBitmapInfo& in) { std::string s; meta_pending_delete_bitmap_key(in, &s); return s; } + +std::string recycle_key_prefix(std::string_view instance_id); +void recycle_index_key(const RecycleIndexKeyInfo& in, std::string* out); +void recycle_partition_key(const RecyclePartKeyInfo& in, std::string* out); +void recycle_rowset_key(const RecycleRowsetKeyInfo& in, std::string* out); +void recycle_txn_key(const RecycleTxnKeyInfo& in, std::string* out); +void recycle_stage_key(const RecycleStageKeyInfo& in, std::string* out); +static inline std::string recycle_index_key(const RecycleIndexKeyInfo& in) { std::string s; recycle_index_key(in, &s); return s; } +static inline std::string recycle_partition_key(const RecyclePartKeyInfo& in) { std::string s; recycle_partition_key(in, &s); return s; } +static inline std::string recycle_rowset_key(const RecycleRowsetKeyInfo& in) { std::string s; recycle_rowset_key(in, &s); return s; } +static inline std::string recycle_txn_key(const RecycleTxnKeyInfo& in) { std::string s; recycle_txn_key(in, &s); return s; } +static inline std::string recycle_stage_key(const RecycleStageKeyInfo& in) { std::string s; recycle_stage_key(in, &s); return s; } + +void stats_tablet_key(const StatsTabletKeyInfo& in, std::string* out); +void stats_tablet_data_size_key(const StatsTabletKeyInfo& in, std::string* out); +void stats_tablet_num_rows_key(const StatsTabletKeyInfo& in, std::string* out); +void stats_tablet_num_rowsets_key(const StatsTabletKeyInfo& in, std::string* out); +void stats_tablet_num_segs_key(const StatsTabletKeyInfo& in, std::string* out); +static inline std::string stats_tablet_key(const StatsTabletKeyInfo& in) { std::string s; stats_tablet_key(in, &s); return s; } + +void job_recycle_key(const JobRecycleKeyInfo& in, std::string* out); +void job_check_key(const JobRecycleKeyInfo& in, std::string* out); +static inline std::string job_check_key(const JobRecycleKeyInfo& in) { std::string s; job_check_key(in, &s); return s; } +void job_tablet_key(const JobTabletKeyInfo& in, std::string* out); +static inline std::string job_tablet_key(const JobTabletKeyInfo& in) { std::string s; job_tablet_key(in, &s); return s; } +void rl_job_progress_key_info(const RLJobProgressKeyInfo& in, std::string* out); +static inline std::string rl_job_progress_key_info(const RLJobProgressKeyInfo& in) { std::string s; rl_job_progress_key_info(in, &s); return s; } + +std::string copy_key_prefix(std::string_view instance_id); +void copy_job_key(const CopyJobKeyInfo& in, std::string* out); +void copy_file_key(const CopyFileKeyInfo& in, std::string* out); +[[maybe_unused]] static std::string copy_job_key(const CopyJobKeyInfo& in) { std::string s; copy_job_key(in, &s); return s; } +[[maybe_unused]] static std::string copy_file_key(const CopyFileKeyInfo& in) { std::string s; copy_file_key(in, &s); return s; } + +std::string system_meta_service_registry_key(); +std::string system_meta_service_arn_info_key(); + +// Note: +// This key points to a value (EncryptionKeyInfoPB, the format is below) which stores a set of items, +// and each item represents a group of encrption key. +// The size of each item: 8 Bytes (int64 key_id) + 32 Bytes * 1.3 (256bit key * base64 amplification factor) = 50 Bytes. +// The maximum size kv of fdb can store: 100k/50Bytes = 2048 items +// +// message EncryptionKeyInfoPB { +// message Item { +// optional int64 key_id = 1; +// optional string key = 2; +// } +// repeated Item items = 1; +// } +std::string system_meta_service_encryption_key_info_key(); +// clang-format on +// TODO: add a family of decoding functions if needed + +/** + * Decodes a given key without key space byte (the first byte). + * Note that the input may be partially decode if the return value is non-zero. + * + * @param in input byte stream, successfully decoded part will be consumed + * @param out the vector of each in the input stream + * @return 0 for successful decoding of the entire input, otherwise error. + */ +int decode_key(std::string_view* in, + std::vector, int, int>>* out); + +} // namespace doris::cloud diff --git a/cloud/src/meta-service/mem_txn_kv.cpp b/cloud/src/meta-service/mem_txn_kv.cpp new file mode 100644 index 00000000000000..e10f5dd3d3ed13 --- /dev/null +++ b/cloud/src/meta-service/mem_txn_kv.cpp @@ -0,0 +1,449 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "mem_txn_kv.h" + +#include + +#include +#include +#include +#include +#include +#include +#include + +#include "common/sync_point.h" +#include "meta-service/txn_kv_error.h" +#include "txn_kv.h" + +namespace doris::cloud { + +int MemTxnKv::init() { + return 0; +} + +TxnErrorCode MemTxnKv::create_txn(std::unique_ptr* txn) { + auto t = new memkv::Transaction(this->shared_from_this()); + txn->reset(t); + return TxnErrorCode::TXN_OK; +} + +TxnErrorCode MemTxnKv::get_kv(const std::string& key, std::string* val, int64_t version) { + std::lock_guard l(lock_); + auto it = mem_kv_.find(key); + if (it == mem_kv_.end() || it->second.empty()) { + return TxnErrorCode::TXN_KEY_NOT_FOUND; + } + + for (auto&& entry : it->second) { + if (entry.commit_version <= version) { + if (!entry.value.has_value()) { + return TxnErrorCode::TXN_KEY_NOT_FOUND; + } + *val = *entry.value; + return TxnErrorCode::TXN_OK; + } + } + return TxnErrorCode::TXN_KEY_NOT_FOUND; +} + +TxnErrorCode MemTxnKv::get_kv(const std::string& begin, const std::string& end, int64_t version, + int limit, bool* more, std::map* kv_list) { + if (begin >= end) { + return TxnErrorCode::TXN_OK; + } + + bool use_limit = true; + + if (limit < 0) { + return TxnErrorCode::TXN_UNIDENTIFIED_ERROR; + } + if (limit == 0) { + use_limit = false; + } + + std::unique_lock l(lock_); + + *more = false; + auto begin_iter = mem_kv_.lower_bound(begin); + auto end_iter = mem_kv_.lower_bound(end); + for (; begin_iter != mem_kv_.end() && begin_iter != end_iter; begin_iter++) { + for (auto&& entry : begin_iter->second) { + if (entry.commit_version > version) { + continue; + } + + if (!entry.value.has_value()) { + break; + } + + kv_list->insert_or_assign(begin_iter->first, *entry.value); + limit--; + break; + } + if (use_limit && limit == 0) { + break; + } + } + if (use_limit && limit == 0 && ++begin_iter != end_iter) { + *more = true; + } + return TxnErrorCode::TXN_OK; +} + +TxnErrorCode MemTxnKv::update(const std::set& read_set, + const std::vector& op_list, int64_t read_version, + int64_t* committed_version) { + std::lock_guard l(lock_); + + // check_conflict + for (const auto& k : read_set) { + auto iter = log_kv_.find(k); + if (iter != log_kv_.end()) { + auto log_item = iter->second; + if (log_item.front().commit_version_ > read_version) { + LOG(WARNING) << "commit conflict"; + //keep the same behaviour with fdb. + return TxnErrorCode::TXN_CONFLICT; + } + } + } + + ++committed_version_; + + int16_t seq = 0; + for (const auto& vec : op_list) { + const auto& [op_type, k, v] = vec; + LogItem log_item {op_type, committed_version_, k, v}; + log_kv_[k].push_front(log_item); + switch (op_type) { + case memkv::ModifyOpType::PUT: { + mem_kv_[k].push_front(Version {committed_version_, v}); + break; + } + case memkv::ModifyOpType::ATOMIC_SET_VER_KEY: { + std::string ver_key(k); + gen_version_timestamp(committed_version_, seq, &ver_key); + mem_kv_[ver_key].push_front(Version {committed_version_, v}); + break; + } + case memkv::ModifyOpType::ATOMIC_SET_VER_VAL: { + std::string ver_val(v); + gen_version_timestamp(committed_version_, seq, &ver_val); + mem_kv_[k].push_front(Version {committed_version_, ver_val}); + break; + } + case memkv::ModifyOpType::ATOMIC_ADD: { + std::string org_val; + if (!mem_kv_[k].empty()) { + org_val = mem_kv_[k].front().value.value_or(""); + } + if (org_val.size() != 8) { + org_val.resize(8, '\0'); + } + int64_t res = *(int64_t*)org_val.data() + *(int64_t*)v.data(); + std::memcpy(org_val.data(), &res, 8); + mem_kv_[k].push_front(Version {committed_version_, org_val}); + break; + } + case memkv::ModifyOpType::REMOVE: { + mem_kv_[k].push_front(Version {committed_version_, std::nullopt}); + break; + } + case memkv::ModifyOpType::REMOVE_RANGE: { + auto begin_iter = mem_kv_.lower_bound(k); + auto end_iter = mem_kv_.lower_bound(v); + while (begin_iter != end_iter) { + mem_kv_[begin_iter->first].push_front(Version {committed_version_, std::nullopt}); + begin_iter++; + } + break; + } + default: + break; + } + } + + *committed_version = committed_version_; + return TxnErrorCode::TXN_OK; +} + +int MemTxnKv::gen_version_timestamp(int64_t ver, int16_t seq, std::string* str) { + // Convert litter endian to big endian + static auto to_big_int64 = [](int64_t v) { + v = ((v & 0xffffffff00000000) >> 32) | ((v & 0x00000000ffffffff) << 32); + v = ((v & 0xffff0000ffff0000) >> 16) | ((v & 0x0000ffff0000ffff) << 16); + v = ((v & 0xff00ff00ff00ff00) >> 8) | ((v & 0x00ff00ff00ff00ff) << 8); + return v; + }; + + static auto to_big_int16 = [](int16_t v) { + v = ((v & 0xff00) >> 8) | ((v & 0x00ff) << 8); + return v; + }; + + ver = to_big_int64(ver); + seq = to_big_int16(seq); + + int size = str->size(); + str->resize(size + 10, '\0'); + std::memcpy(str->data() + size, &ver, sizeof(ver)); + std::memcpy(str->data() + size + 8, &seq, sizeof(seq)); + return 0; +} + +int64_t MemTxnKv::get_last_commited_version() { + std::lock_guard l(lock_); + return committed_version_; +} + +int64_t MemTxnKv::get_last_read_version() { + std::lock_guard l(lock_); + read_version_ = committed_version_; + return read_version_; +} + +} // namespace doris::cloud + +namespace doris::cloud::memkv { + +// ============================================================================= +// Impl of Transaction +// ============================================================================= + +Transaction::Transaction(std::shared_ptr kv) : kv_(std::move(kv)) { + std::lock_guard l(lock_); + read_version_ = kv_->committed_version_; +} + +int Transaction::init() { + return 0; +} + +void Transaction::put(std::string_view key, std::string_view val) { + std::lock_guard l(lock_); + std::string k(key.data(), key.size()); + std::string v(val.data(), val.size()); + writes_.insert_or_assign(k, v); + op_list_.emplace_back(ModifyOpType::PUT, k, v); +} + +TxnErrorCode Transaction::get(std::string_view key, std::string* val, bool snapshot) { + std::lock_guard l(lock_); + std::string k(key.data(), key.size()); + // the key set by atomic_xxx can't not be read before the txn is committed. + // if it is read, the txn will not be able to commit. + if (unreadable_keys_.count(k) != 0) { + aborted_ = true; + LOG(WARNING) << "read unreadable key, abort"; + return TxnErrorCode::TXN_UNIDENTIFIED_ERROR; + } + return inner_get(k, val, snapshot); +} + +TxnErrorCode Transaction::get(std::string_view begin, std::string_view end, + std::unique_ptr* iter, bool snapshot, + int limit) { + TEST_SYNC_POINT_CALLBACK("memkv::Transaction::get", &limit); + std::lock_guard l(lock_); + std::string begin_k(begin.data(), begin.size()); + std::string end_k(end.data(), end.size()); + // TODO: figure out what happen if range_get has part of unreadable_keys + if (unreadable_keys_.count(begin_k) != 0) { + aborted_ = true; + LOG(WARNING) << "read unreadable key, abort"; + return TxnErrorCode::TXN_UNIDENTIFIED_ERROR; + } + return inner_get(begin_k, end_k, iter, snapshot, limit); +} + +TxnErrorCode Transaction::inner_get(const std::string& key, std::string* val, bool snapshot) { + // Read your writes. + auto it = writes_.find(key); + if (it != writes_.end()) { + *val = it->second; + return TxnErrorCode::TXN_OK; + } + + if (!snapshot) read_set_.emplace(key); + TxnErrorCode err = kv_->get_kv(key, val, read_version_); + if (err != TxnErrorCode::TXN_OK) { + return err; + } + + for (auto&& [start, end] : remove_ranges_) { + if (start <= key && key < end) { + return TxnErrorCode::TXN_KEY_NOT_FOUND; + } + } + return TxnErrorCode::TXN_OK; +} + +TxnErrorCode Transaction::inner_get(const std::string& begin, const std::string& end, + std::unique_ptr* iter, bool snapshot, + int limit) { + bool more = false; + std::map kv_map; + TxnErrorCode err = kv_->get_kv(begin, end, read_version_, limit, &more, &kv_map); + if (err != TxnErrorCode::TXN_OK) { + return err; + } + + // Overwrite by your writes. + auto pred = [&](const std::pair& val) { + for (auto&& [start, end] : remove_ranges_) { + if (start <= val.first && val.first < end) { + return true; + } + } + return false; + }; + for (auto it = kv_map.begin(), last = kv_map.end(); it != last;) { + if (pred(*it)) { + it = kv_map.erase(it); + } else { + ++it; + } + } + + if (!snapshot) { + for (auto&& [key, _] : kv_map) { + read_set_.insert(key); + } + } + + auto begin_iter = writes_.lower_bound(begin); + auto end_iter = writes_.lower_bound(end); + while (begin_iter != end_iter) { + kv_map.insert_or_assign(begin_iter->first, begin_iter->second); + begin_iter++; + } + + std::vector> kv_list(kv_map.begin(), kv_map.end()); + *iter = std::make_unique(std::move(kv_list), more); + return TxnErrorCode::TXN_OK; +} + +void Transaction::atomic_set_ver_key(std::string_view key_prefix, std::string_view val) { + std::lock_guard l(lock_); + std::string k(key_prefix.data(), key_prefix.size()); + std::string v(val.data(), val.size()); + unreadable_keys_.insert(k); + op_list_.emplace_back(ModifyOpType::ATOMIC_SET_VER_KEY, k, v); +} + +void Transaction::atomic_set_ver_value(std::string_view key, std::string_view value) { + std::lock_guard l(lock_); + std::string k(key.data(), key.size()); + std::string v(value.data(), value.size()); + unreadable_keys_.insert(k); + op_list_.emplace_back(ModifyOpType::ATOMIC_SET_VER_VAL, k, v); +} + +void Transaction::atomic_add(std::string_view key, int64_t to_add) { + std::string k(key.data(), key.size()); + std::string v(sizeof(to_add), '\0'); + memcpy(v.data(), &to_add, sizeof(to_add)); + std::lock_guard l(lock_); + op_list_.emplace_back(ModifyOpType::ATOMIC_ADD, std::move(k), std::move(v)); +} + +void Transaction::remove(std::string_view key) { + std::lock_guard l(lock_); + std::string k(key.data(), key.size()); + writes_.erase(k); + std::string end_key = k; + end_key.push_back(0x0); + remove_ranges_.emplace_back(k, end_key); + op_list_.emplace_back(ModifyOpType::REMOVE, k, ""); +} + +void Transaction::remove(std::string_view begin, std::string_view end) { + std::lock_guard l(lock_); + std::string begin_k(begin.data(), begin.size()); + std::string end_k(end.data(), end.size()); + if (begin_k >= end_k) { + aborted_ = true; + } else { + // ATTN: we do not support read your writes about delete range. + auto begin_iter = writes_.lower_bound(begin_k); + auto end_iter = writes_.lower_bound(end_k); + writes_.erase(begin_iter, end_iter); + remove_ranges_.emplace_back(begin_k, end_k); + op_list_.emplace_back(ModifyOpType::REMOVE_RANGE, begin_k, end_k); + } +} + +TxnErrorCode Transaction::commit() { + std::lock_guard l(lock_); + if (aborted_) { + return TxnErrorCode::TXN_UNIDENTIFIED_ERROR; + } + auto code = kv_->update(read_set_, op_list_, read_version_, &committed_version_); + if (code != TxnErrorCode::TXN_OK) { + return code; + } + commited_ = true; + op_list_.clear(); + read_set_.clear(); + writes_.clear(); + remove_ranges_.clear(); + return TxnErrorCode::TXN_OK; +} + +TxnErrorCode Transaction::get_read_version(int64_t* version) { + std::lock_guard l(lock_); + *version = read_version_; + return TxnErrorCode::TXN_OK; +} + +TxnErrorCode Transaction::get_committed_version(int64_t* version) { + std::lock_guard l(lock_); + if (!commited_) { + return TxnErrorCode::TXN_UNIDENTIFIED_ERROR; + } + *version = committed_version_; + return TxnErrorCode::TXN_OK; +} + +TxnErrorCode Transaction::abort() { + return TxnErrorCode::TXN_OK; +} + +TxnErrorCode Transaction::batch_get(std::vector>* res, + const std::vector& keys, + const BatchGetOptions& opts) { + if (keys.empty()) { + return TxnErrorCode::TXN_OK; + } + std::lock_guard l(lock_); + res->reserve(keys.size()); + for (const auto& k : keys) { + if (unreadable_keys_.count(k) != 0) { + aborted_ = true; + LOG(WARNING) << "read unreadable key, abort"; + return TxnErrorCode::TXN_UNIDENTIFIED_ERROR; + } + std::string val; + auto ret = inner_get(k, &val, opts.snapshot); + ret == TxnErrorCode::TXN_OK ? res->push_back(val) : res->push_back(std::nullopt); + } + return TxnErrorCode::TXN_OK; +} + +} // namespace doris::cloud::memkv diff --git a/cloud/src/meta-service/mem_txn_kv.h b/cloud/src/meta-service/mem_txn_kv.h new file mode 100644 index 00000000000000..e385c414bc5e69 --- /dev/null +++ b/cloud/src/meta-service/mem_txn_kv.h @@ -0,0 +1,246 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "meta-service/txn_kv_error.h" +#include "txn_kv.h" + +namespace doris::cloud { + +namespace memkv { +class Transaction; +enum class ModifyOpType; +} // namespace memkv + +class MemTxnKv : public TxnKv, public std::enable_shared_from_this { + friend class memkv::Transaction; + +public: + MemTxnKv() = default; + ~MemTxnKv() override = default; + + TxnErrorCode create_txn(std::unique_ptr* txn) override; + + int init() override; + + TxnErrorCode get_kv(const std::string& key, std::string* val, int64_t version); + TxnErrorCode get_kv(const std::string& begin, const std::string& end, int64_t version, + int limit, bool* more, std::map* kv_list); + +private: + using OpTuple = std::tuple; + TxnErrorCode update(const std::set& read_set, const std::vector& op_list, + int64_t read_version, int64_t* committed_version); + + int get_kv(std::map* kv, int64_t* version); + + int64_t get_last_commited_version(); + int64_t get_last_read_version(); + + static int gen_version_timestamp(int64_t ver, int16_t seq, std::string* str); + + struct LogItem { + memkv::ModifyOpType op_; + int64_t commit_version_; + + // for get's op: key=key, value="" + // for range get's op: key=begin, value=end + // for atomic_set_ver_key/atomic_set_ver_value's op: key=key, value=value + // for atomic_add's op: key=key, value=to_add + // for remove's op: key=key, value="" + // for range remove's op: key=begin, value=end + std::string key; + std::string value; + }; + + struct Version { + int64_t commit_version; + std::optional value; + }; + + std::map> mem_kv_; + std::unordered_map> log_kv_; + std::mutex lock_; + int64_t committed_version_ = 0; + int64_t read_version_ = 0; +}; + +namespace memkv { + +enum class ModifyOpType { + PUT, + ATOMIC_SET_VER_KEY, + ATOMIC_SET_VER_VAL, + ATOMIC_ADD, + REMOVE, + REMOVE_RANGE +}; + +class Transaction : public cloud::Transaction { +public: + Transaction(std::shared_ptr kv); + + ~Transaction() override = default; + + /** + * + * @return 0 for success otherwise false + */ + int init(); + + void put(std::string_view key, std::string_view val) override; + + using cloud::Transaction::get; + /** + * @param snapshot if true, `key` will not be included in txn conflict detection this time + * @return TXN_OK for success get a key, TXN_KEY_NOT_FOUND for key not found, otherwise for error + */ + TxnErrorCode get(std::string_view key, std::string* val, bool snapshot = false) override; + /** + * Closed-open range + * @param snapshot if true, key range will not be included in txn conflict detection this time + * @param limit if non-zero, indicates the maximum number of key-value pairs to return + * @return TXN_OK for success, otherwise for error + */ + TxnErrorCode get(std::string_view begin, std::string_view end, + std::unique_ptr* iter, bool snapshot = false, + int limit = 10000) override; + + /** + * Put a key-value pair in which key will in the form of + * `key_prefix + versiontimestamp`. `versiontimestamp` is autogenerated by the + * system and it's 10-byte long and encoded in big-endian + * + * @param key_prefix prefix for key convertion, can be zero-length + * @param val value + */ + void atomic_set_ver_key(std::string_view key_prefix, std::string_view val) override; + + /** + * Put a key-value pair in which key will in the form of + * `value + versiontimestamp`. `versiontimestamp` is autogenerated by the + * system and it's 10-byte long and encoded in big-endian + * + * @param key prefix for key convertion, can be zero-length + * @param val value + */ + void atomic_set_ver_value(std::string_view key, std::string_view val) override; + + /** + * Adds a value to database + * @param to_add positive for addition, negative for substraction + */ + void atomic_add(std::string_view key, int64_t to_add) override; + // TODO: min max or and xor cmp_and_clear set_ver_value + + void remove(std::string_view key) override; + + /** + * Remove a closed-open range + */ + void remove(std::string_view begin, std::string_view end) override; + + /** + * + *@return TXN_OK for success otherwise error + */ + TxnErrorCode commit() override; + + TxnErrorCode get_read_version(int64_t* version) override; + TxnErrorCode get_committed_version(int64_t* version) override; + + TxnErrorCode abort() override; + + TxnErrorCode batch_get(std::vector>* res, + const std::vector& keys, + const BatchGetOptions& opts = BatchGetOptions()) override; + +private: + TxnErrorCode inner_get(const std::string& key, std::string* val, bool snapshot); + + TxnErrorCode inner_get(const std::string& begin, const std::string& end, + std::unique_ptr* iter, bool snapshot, + int limit); + + std::shared_ptr kv_ {nullptr}; + bool commited_ = false; + bool aborted_ = false; + std::mutex lock_; + std::set unreadable_keys_; + std::set read_set_; + std::map writes_; + std::list> remove_ranges_; + std::vector> op_list_; + + int64_t committed_version_ = -1; + int64_t read_version_ = -1; +}; + +class RangeGetIterator : public cloud::RangeGetIterator { +public: + RangeGetIterator(std::vector> kvs, bool more) + : kvs_(std::move(kvs)), kvs_size_(kvs_.size()), idx_(0), more_(more) {} + + ~RangeGetIterator() override = default; + + bool has_next() override { return idx_ < kvs_size_; } + + std::pair next() override { + if (idx_ < 0 || idx_ >= kvs_size_) return {}; + auto& kv = kvs_[idx_++]; + return {kv.first, kv.second}; + } + + void seek(size_t pos) override { idx_ = pos; } + + bool more() override { return more_; } + + int size() override { return kvs_size_; } + void reset() override { idx_ = 0; } + + std::string next_begin_key() override { + std::string k; + if (!more()) return k; + auto& key = kvs_[kvs_size_ - 1].first; + k.reserve(key.size() + 1); + k.append(key); + k.push_back('\x00'); + return k; + } + +private: + std::vector> kvs_; + int kvs_size_; + int idx_; + bool more_; +}; + +} // namespace memkv +} // namespace doris::cloud \ No newline at end of file diff --git a/cloud/src/meta-service/meta_server.cpp b/cloud/src/meta-service/meta_server.cpp new file mode 100644 index 00000000000000..775abbf6a37f3c --- /dev/null +++ b/cloud/src/meta-service/meta_server.cpp @@ -0,0 +1,196 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "meta_server.h" + +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#include "common/config.h" +#include "common/metric.h" +#include "common/sync_point.h" +#include "common/util.h" +#include "meta-service/keys.h" +#include "meta-service/meta_service.h" +#include "meta-service/txn_kv_error.h" +#include "rate-limiter/rate_limiter.h" +#include "resource-manager/resource_manager.h" +#include "txn_kv.h" + +namespace doris::cloud { + +MetaServer::MetaServer(std::shared_ptr txn_kv) : txn_kv_(std::move(txn_kv)) {} + +int MetaServer::start(brpc::Server* server) { + DCHECK(server); + auto rc_mgr = std::make_shared(txn_kv_); + int ret = rc_mgr->init(); + TEST_SYNC_POINT_CALLBACK("MetaServer::start:1", &ret); + if (ret != 0) { + LOG(WARNING) << "failed to init resrouce manager, ret=" << ret; + return 1; + } + + // Add server register + server_register_.reset(new MetaServerRegister(txn_kv_)); + ret = server_register_->start(); + TEST_SYNC_POINT_CALLBACK("MetaServer::start:2", &ret); + if (ret != 0) { + LOG(WARNING) << "failed to start server register"; + return -1; + } + + fdb_metric_exporter_.reset(new FdbMetricExporter(txn_kv_)); + ret = fdb_metric_exporter_->start(); + TEST_SYNC_POINT_CALLBACK("MetaServer::start:3", &ret); + if (ret != 0) { + LOG(WARNING) << "failed to start fdb metric exporter"; + return -2; + } + + auto rate_limiter = std::make_shared(); + + // Add service + auto meta_service = std::make_unique(txn_kv_, rc_mgr, rate_limiter); + auto meta_service_proxy = new MetaServiceProxy(std::move(meta_service)); + server->AddService(meta_service_proxy, brpc::SERVER_OWNS_SERVICE); + + return 0; +} + +void MetaServer::stop() { + server_register_->stop(); + fdb_metric_exporter_->stop(); +} + +void MetaServerRegister::prepare_registry(ServiceRegistryPB* reg) { + using namespace std::chrono; + auto now = duration_cast(system_clock::now().time_since_epoch()).count(); + std::string ip = butil::my_ip_cstr(); + int32_t port = config::brpc_listen_port; + std::string id = ip + ":" + std::to_string(port); + ServiceRegistryPB::Item item; + item.set_id(id); + item.set_ip(ip); + item.set_port(port); + item.set_ctime_ms(now); + item.set_mtime_ms(now); + item.set_expiration_time_ms(now + config::meta_server_lease_ms); + if (!config::hostname.empty()) { + item.set_host(config::hostname); + } + + if (!id_.empty() && id_ != id) { + LOG(WARNING) << "server id changed, old=" << id_ << " new=" << id; + id_ = id; + } + + int num_items = reg->items_size(); + ServiceRegistryPB out; + for (int i = 0; i < num_items; ++i) { + const auto& e = reg->items(i); + if (e.expiration_time_ms() < now) continue; + if (e.id() == id) { + item.set_ctime_ms(e.ctime_ms()); + continue; + } + *out.add_items() = e; + } + *out.add_items() = item; + *reg = out; +} + +MetaServerRegister::MetaServerRegister(std::shared_ptr txn_kv) + : running_(0), txn_kv_(std::move(txn_kv)) { + register_thread_.reset(new std::thread([this] { + while (running_.load() == 0) { + LOG(INFO) << "register thread wait for start"; + std::unique_lock l(mtx_); + cv_.wait_for(l, std::chrono::milliseconds(config::meta_server_register_interval_ms)); + } + LOG(INFO) << "register thread begins to run"; + std::mt19937 gen(std::random_device("/dev/urandom")()); + std::uniform_int_distribution rd_len(50, 300); + + while (running_.load() == 1) { + std::string key = system_meta_service_registry_key(); + std::string val; + std::unique_ptr txn; + int tried = 0; + do { + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) break; + err = txn->get(key, &val); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) break; + ServiceRegistryPB reg; + if (err == TxnErrorCode::TXN_OK && !reg.ParseFromString(val)) break; + LOG_EVERY_N(INFO, 100) + << "get server registry, key=" << hex(key) << " reg=" << proto_to_json(reg); + prepare_registry(®); + val = reg.SerializeAsString(); + if (val.empty()) break; + txn->put(key, val); + LOG_EVERY_N(INFO, 100) + << "put server registry, key=" << hex(key) << " reg=" << proto_to_json(reg); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to commit registry, key=" << hex(key) + << " val=" << proto_to_json(reg) << " retry times=" << ++tried; + std::this_thread::sleep_for(std::chrono::milliseconds(rd_len(gen))); + continue; + } + } while (false); + std::unique_lock l(mtx_); + cv_.wait_for(l, std::chrono::milliseconds(config::meta_server_register_interval_ms)); + } + LOG(INFO) << "register thread quits"; + })); +} + +MetaServerRegister::~MetaServerRegister() { + stop(); +} + +int MetaServerRegister::start() { + if (txn_kv_ == nullptr) return -1; + std::unique_lock lock(mtx_); + running_.store(1); + cv_.notify_all(); + return 0; +} + +void MetaServerRegister::stop() { + { + std::unique_lock lock(mtx_); + running_.store(2); + cv_.notify_all(); + } + if (register_thread_ != nullptr && register_thread_->joinable()) { + register_thread_->join(); + register_thread_.reset(); + } +} + +} // namespace doris::cloud diff --git a/cloud/src/meta-service/meta_server.h b/cloud/src/meta-service/meta_server.h new file mode 100644 index 00000000000000..ee44e377cb3ebd --- /dev/null +++ b/cloud/src/meta-service/meta_server.h @@ -0,0 +1,96 @@ +// // Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include + +#include +#include +#include +#include + +#include "common/metric.h" +#include "txn_kv.h" + +namespace doris::cloud { + +class MetaServerRegister; + +class MetaServer { +public: + MetaServer(std::shared_ptr txn_kv); + ~MetaServer() = default; + + /** + * Starts to listen and server + * + * return 0 for success otherwise failure + */ + int start(brpc::Server* server); + + void stop(); + +private: + std::shared_ptr txn_kv_; + std::unique_ptr server_register_; + std::unique_ptr fdb_metric_exporter_; +}; + +class ServiceRegistryPB; + +class MetaServerRegister { +public: + MetaServerRegister(std::shared_ptr txn_kv); + ~MetaServerRegister(); + + /** + * Starts registering + * + * @return 0 on success, otherwise failure. + */ + int start(); + + /** + * Notifies all the threads to quit and stop registering current server. + * TODO(gavin): should we remove the server from the registry list actively + * when we call stop(). + */ + void stop(); + +private: + /** + * Prepares registry with given existing registry. If the server already + * exists in the registry list, update mtime and lease, otherwise create a + * new item for the server in the registry list. + * + * @param reg input and output param + */ + void prepare_registry(ServiceRegistryPB* reg); + +private: + std::unique_ptr register_thread_; + std::atomic running_; + std::mutex mtx_; + std::condition_variable cv_; + std::string id_; + + std::shared_ptr txn_kv_; // Relies on other members, must be the + // first to destruct +}; + +} // namespace doris::cloud diff --git a/cloud/src/meta-service/meta_service.cpp b/cloud/src/meta-service/meta_service.cpp new file mode 100644 index 00000000000000..780b05ae38dfcc --- /dev/null +++ b/cloud/src/meta-service/meta_service.cpp @@ -0,0 +1,1884 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "meta_service.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "common/bvars.h" +#include "common/config.h" +#include "common/encryption_util.h" +#include "common/logging.h" +#include "common/stopwatch.h" +#include "common/string_util.h" +#include "common/sync_point.h" +#include "common/util.h" +#include "meta-service/codec.h" +#include "meta-service/doris_txn.h" +#include "meta-service/keys.h" +#include "meta-service/meta_service_helper.h" +#include "meta-service/meta_service_schema.h" +#include "meta-service/meta_service_tablet_stats.h" +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" +#include "rate-limiter/rate_limiter.h" + +using namespace std::chrono; + +namespace doris::cloud { + +MetaServiceImpl::MetaServiceImpl(std::shared_ptr txn_kv, + std::shared_ptr resource_mgr, + std::shared_ptr rate_limiter) { + txn_kv_ = txn_kv; + resource_mgr_ = resource_mgr; + rate_limiter_ = rate_limiter; + rate_limiter_->init(this); +} + +MetaServiceImpl::~MetaServiceImpl() = default; + +// FIXME(gavin): should it be a member function of ResourceManager? +std::string get_instance_id(const std::shared_ptr& rc_mgr, + const std::string& cloud_unique_id) { + { + [[maybe_unused]] std::string tmp_ret; + TEST_SYNC_POINT_RETURN_WITH_VALUE("get_instance_id", &tmp_ret); + } + + std::vector nodes; + std::string err = rc_mgr->get_node(cloud_unique_id, &nodes); + { TEST_SYNC_POINT_CALLBACK("get_instance_id_err", &err); } + if (!err.empty()) { + // cache can't find cloud_unique_id, so degraded by parse cloud_unique_id + // cloud_unique_id encode: ${version}:${instance_id}:${unique_id} + // check it split by ':' c + auto vec = split(cloud_unique_id, ':'); + std::stringstream ss; + for (int i = 0; i < vec.size(); ++i) { + ss << "idx " << i << "= [" << vec[i] << "] "; + } + LOG(INFO) << "degraded to get instance_id, cloud_unique_id: " << cloud_unique_id + << "after split: " << ss.str(); + if (vec.size() != 3) { + LOG(WARNING) << "cloud unique id is not degraded format, failed to check instance " + "info, cloud_unique_id=" + << cloud_unique_id << " , err=" << err; + return ""; + } + // version: vec[0], instance_id: vec[1], unique_id: vec[2] + switch (std::atoi(vec[0].c_str())) { + case 1: + // just return instance id; + return vec[1]; + default: + LOG(WARNING) << "cloud unique id degraded state, but version not eq configure, " + "cloud_unique_id=" + << cloud_unique_id << ", err=" << err; + return ""; + } + } + + std::string instance_id; + for (auto& i : nodes) { + if (!instance_id.empty() && instance_id != i.instance_id) { + LOG(WARNING) << "cloud_unique_id is one-to-many instance_id, " + << " cloud_unique_id=" << cloud_unique_id + << " current_instance_id=" << instance_id + << " later_instance_id=" << i.instance_id; + } + instance_id = i.instance_id; // The last wins + } + return instance_id; +} + +// Return `true` if tablet has been dropped; otherwise or it may not determine when meeting errors, return false +bool is_dropped_tablet(Transaction* txn, const std::string& instance_id, int64_t index_id, + int64_t partition_id) { + auto key = recycle_index_key({instance_id, index_id}); + std::string val; + TxnErrorCode err = txn->get(key, &val); + if (err == TxnErrorCode::TXN_OK) { + RecycleIndexPB pb; + if (!pb.ParseFromString(val)) [[unlikely]] { + LOG_WARNING("malformed recycle index pb").tag("key", hex(key)); + return false; + } + return pb.state() == RecycleIndexPB::DROPPED || pb.state() == RecycleIndexPB::RECYCLING; + } + if (err != TxnErrorCode::TXN_KEY_NOT_FOUND) { // Get kv failed, cannot determine, return false + return false; + } + key = recycle_partition_key({instance_id, partition_id}); + err = txn->get(key, &val); + if (err == TxnErrorCode::TXN_OK) { + RecyclePartitionPB pb; + if (!pb.ParseFromString(val)) [[unlikely]] { + LOG_WARNING("malformed recycle partition pb").tag("key", hex(key)); + return false; + } + return pb.state() == RecyclePartitionPB::DROPPED || + pb.state() == RecyclePartitionPB::RECYCLING; + } + return false; +} + +void get_tablet_idx(MetaServiceCode& code, std::string& msg, Transaction* txn, + const std::string& instance_id, int64_t tablet_id, TabletIndexPB& tablet_idx) { + std::string key, val; + meta_tablet_idx_key({instance_id, tablet_id}, &key); + TxnErrorCode err = txn->get(key, &val); + if (err != TxnErrorCode::TXN_OK) { + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = MetaServiceCode::TABLET_NOT_FOUND; + } else { + code = cast_as(err); + } + msg = fmt::format("failed to get tablet_idx, err={} tablet_id={} ", err, tablet_id); + return; + } + if (!tablet_idx.ParseFromString(val)) [[unlikely]] { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = fmt::format("malformed tablet index value, key={}", hex(key)); + return; + } + if (tablet_id != tablet_idx.tablet_id()) [[unlikely]] { + code = MetaServiceCode::UNDEFINED_ERR; + msg = "internal error"; + LOG(WARNING) << "unexpected error given_tablet_id=" << tablet_id + << " idx_pb_tablet_id=" << tablet_idx.tablet_id(); + return; + } +} + +void MetaServiceImpl::get_version(::google::protobuf::RpcController* controller, + const GetVersionRequest* request, GetVersionResponse* response, + ::google::protobuf::Closure* done) { + if (request->batch_mode()) { + batch_get_version(controller, request, response, done); + return; + } + + RPC_PREPROCESS(get_version); + std::string cloud_unique_id; + if (request->has_cloud_unique_id()) { + cloud_unique_id = request->cloud_unique_id(); + } + + int64_t db_id = request->has_db_id() ? request->db_id() : -1; + int64_t table_id = request->has_table_id() ? request->table_id() : -1; + int64_t partition_id = request->has_partition_id() ? request->partition_id() : -1; + if (db_id == -1 || table_id == -1 || partition_id == -1) { + msg = "params error, db_id=" + std::to_string(db_id) + + " table_id=" + std::to_string(table_id) + + " partition_id=" + std::to_string(partition_id); + code = MetaServiceCode::INVALID_ARGUMENT; + LOG(WARNING) << msg; + return; + } + + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); + return; + } + RPC_RATE_LIMIT(get_version) + VersionKeyInfo ver_key_info {instance_id, db_id, table_id, partition_id}; + std::string ver_key; + version_key(ver_key_info, &ver_key); + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + msg = "failed to create txn"; + code = cast_as(err); + return; + } + + std::string ver_val; + VersionPB version_pb; + // 0 for success get a key, 1 for key not found, negative for error + err = txn->get(ver_key, &ver_val); + VLOG_DEBUG << "xxx get version_key=" << hex(ver_key); + if (err == TxnErrorCode::TXN_OK) { + if (!version_pb.ParseFromString(ver_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "malformed version value"; + return; + } + response->set_version(version_pb.version()); + { TEST_SYNC_POINT_CALLBACK("get_version_code", &code); } + return; + } else if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + msg = "not found"; + code = MetaServiceCode::VERSION_NOT_FOUND; + return; + } + msg = fmt::format("failed to get txn, err={}", err); + code = cast_as(err); +} + +void MetaServiceImpl::batch_get_version(::google::protobuf::RpcController* controller, + const GetVersionRequest* request, + GetVersionResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(get_version); + + std::string cloud_unique_id; + if (request->has_cloud_unique_id()) { + cloud_unique_id = request->cloud_unique_id(); + } + + if (request->db_ids_size() == 0 || request->table_ids_size() == 0 || + request->table_ids_size() != request->partition_ids_size() || + request->db_ids_size() != request->partition_ids_size()) { + msg = "param error, num db_ids=" + std::to_string(request->db_ids_size()) + + " num table_ids=" + std::to_string(request->table_ids_size()) + + " num partition_ids=" + std::to_string(request->partition_ids_size()); + code = MetaServiceCode::INVALID_ARGUMENT; + LOG(WARNING) << msg; + return; + } + + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); + return; + } + + size_t num_acquired = request->partition_ids_size(); + response->mutable_versions()->Reserve(num_acquired); + response->mutable_db_ids()->CopyFrom(request->db_ids()); + response->mutable_table_ids()->CopyFrom(request->table_ids()); + response->mutable_partition_ids()->CopyFrom(request->partition_ids()); + + while (code == MetaServiceCode::OK && + response->versions_size() < response->partition_ids_size()) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + msg = "failed to create txn"; + code = cast_as(err); + break; + } + for (size_t i = response->versions_size(); i < num_acquired; ++i) { + int64_t db_id = request->db_ids(i); + int64_t table_id = request->table_ids(i); + int64_t partition_id = request->partition_ids(i); + std::string ver_key = version_key({instance_id, db_id, table_id, partition_id}); + + // TODO(walter) support batch get. + std::string ver_val; + err = txn->get(ver_key, &ver_val, true); + TEST_SYNC_POINT_CALLBACK("batch_get_version_err", &err); + VLOG_DEBUG << "xxx get version_key=" << hex(ver_key); + if (err == TxnErrorCode::TXN_OK) { + VersionPB version_pb; + if (!version_pb.ParseFromString(ver_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "malformed version value"; + break; + } + response->add_versions(version_pb.version()); + } else if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + // return -1 if the target version is not exists. + response->add_versions(-1); + } else if (err == TxnErrorCode::TXN_TOO_OLD) { + // txn too old, fallback to non-snapshot versions. + LOG(WARNING) << "batch_get_version execution time exceeds the txn mvcc window, " + "fallback to acquire non-snapshot versions, partition_ids_size=" + << request->partition_ids_size() << ", index=" << i; + break; + } else { + msg = fmt::format("failed to get txn, err={}", err); + code = cast_as(err); + break; + } + } + } + if (code != MetaServiceCode::OK) { + response->clear_partition_ids(); + response->clear_table_ids(); + response->clear_versions(); + } +} + +void internal_create_tablet(MetaServiceCode& code, std::string& msg, + const doris::TabletMetaCloudPB& meta, std::shared_ptr txn_kv, + const std::string& instance_id, + std::set>& saved_schema) { + doris::TabletMetaCloudPB tablet_meta(meta); + bool has_first_rowset = tablet_meta.rs_metas_size() > 0; + + // TODO: validate tablet meta, check existence + int64_t table_id = tablet_meta.table_id(); + int64_t index_id = tablet_meta.index_id(); + int64_t partition_id = tablet_meta.partition_id(); + int64_t tablet_id = tablet_meta.tablet_id(); + + if (!tablet_meta.has_schema() && !tablet_meta.has_schema_version()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "tablet_meta must have either schema or schema_version"; + return; + } + + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to init txn"; + return; + } + + std::string rs_key, rs_val; + if (has_first_rowset) { + // Put first rowset if needed + auto first_rowset = tablet_meta.mutable_rs_metas(0); + if (config::write_schema_kv) { // detach schema from rowset meta + first_rowset->set_index_id(index_id); + first_rowset->set_schema_version(tablet_meta.has_schema_version() + ? tablet_meta.schema_version() + : tablet_meta.schema().schema_version()); + first_rowset->set_allocated_tablet_schema(nullptr); + } + MetaRowsetKeyInfo rs_key_info {instance_id, tablet_id, first_rowset->end_version()}; + meta_rowset_key(rs_key_info, &rs_key); + if (!first_rowset->SerializeToString(&rs_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + msg = "failed to serialize first rowset meta"; + return; + } + txn->put(rs_key, rs_val); + tablet_meta.clear_rs_metas(); // Strip off rowset meta + } + + if (tablet_meta.has_schema()) { + // Temporary hard code to fix wrong column type string generated by FE + auto fix_column_type = [](doris::TabletSchemaCloudPB* schema) { + for (auto& column : *schema->mutable_column()) { + if (column.type() == "DECIMAL128") { + column.mutable_type()->push_back('I'); + } + } + }; + if (config::write_schema_kv) { + // detach TabletSchemaCloudPB from TabletMetaCloudPB + tablet_meta.set_schema_version(tablet_meta.schema().schema_version()); + auto [_, success] = saved_schema.emplace(index_id, tablet_meta.schema_version()); + if (success) { // schema may not be saved + fix_column_type(tablet_meta.mutable_schema()); + auto schema_key = + meta_schema_key({instance_id, index_id, tablet_meta.schema_version()}); + put_schema_kv(code, msg, txn.get(), schema_key, tablet_meta.schema()); + if (code != MetaServiceCode::OK) return; + } + tablet_meta.set_allocated_schema(nullptr); + } else { + fix_column_type(tablet_meta.mutable_schema()); + } + } + + MetaTabletKeyInfo key_info {instance_id, table_id, index_id, partition_id, tablet_id}; + std::string key; + std::string val; + meta_tablet_key(key_info, &key); + if (!tablet_meta.SerializeToString(&val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + msg = "failed to serialize tablet meta"; + return; + } + txn->put(key, val); + LOG(INFO) << "xxx put tablet_key=" << hex(key); + + // Index tablet_id -> table_id, index_id, partition_id + std::string key1; + std::string val1; + MetaTabletIdxKeyInfo key_info1 {instance_id, tablet_id}; + meta_tablet_idx_key(key_info1, &key1); + TabletIndexPB tablet_table; + // tablet_table.set_db_id(db_id); + tablet_table.set_table_id(table_id); + tablet_table.set_index_id(index_id); + tablet_table.set_partition_id(partition_id); + tablet_table.set_tablet_id(tablet_id); + if (!tablet_table.SerializeToString(&val1)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + msg = "failed to serialize tablet table value"; + return; + } + txn->put(key1, val1); + LOG(INFO) << "put tablet_idx tablet_id=" << tablet_id << " key=" << hex(key1); + + // Create stats info for the tablet + auto stats_key = stats_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + std::string stats_val; + TabletStatsPB stats_pb; + stats_pb.set_num_rowsets(1); + stats_pb.set_num_segments(0); + stats_pb.mutable_idx()->set_table_id(table_id); + stats_pb.mutable_idx()->set_index_id(index_id); + stats_pb.mutable_idx()->set_partition_id(partition_id); + stats_pb.mutable_idx()->set_tablet_id(tablet_id); + stats_pb.set_base_compaction_cnt(0); + stats_pb.set_cumulative_compaction_cnt(0); + // set cumulative point to 2 to not compact rowset [0-1] + stats_pb.set_cumulative_point(2); + stats_val = stats_pb.SerializeAsString(); + DCHECK(!stats_val.empty()); + txn->put(stats_key, stats_val); + LOG(INFO) << "put tablet stats, tablet_id=" << tablet_id << " key=" << hex(stats_key); + + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to save tablet meta, ret={}", err); + return; + } +} + +void MetaServiceImpl::create_tablets(::google::protobuf::RpcController* controller, + const CreateTabletsRequest* request, + CreateTabletsResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(create_tablets); + + if (request->tablet_metas_size() == 0) { + msg = "no tablet meta"; + code = MetaServiceCode::INVALID_ARGUMENT; + return; + } + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); + return; + } + RPC_RATE_LIMIT(create_tablets) + // [index_id, schema_version] + std::set> saved_schema; + for (auto& tablet_meta : request->tablet_metas()) { + internal_create_tablet(code, msg, tablet_meta, txn_kv_, instance_id, saved_schema); + if (code != MetaServiceCode::OK) { + return; + } + } +} + +void internal_get_tablet(MetaServiceCode& code, std::string& msg, const std::string& instance_id, + Transaction* txn, int64_t tablet_id, doris::TabletMetaCloudPB* tablet_meta, + bool skip_schema) { + // TODO: validate request + TabletIndexPB tablet_idx; + get_tablet_idx(code, msg, txn, instance_id, tablet_id, tablet_idx); + if (code != MetaServiceCode::OK) return; + + MetaTabletKeyInfo key_info1 {instance_id, tablet_idx.table_id(), tablet_idx.index_id(), + tablet_idx.partition_id(), tablet_id}; + std::string key1, val1; + meta_tablet_key(key_info1, &key1); + TxnErrorCode err = txn->get(key1, &val1); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = MetaServiceCode::TABLET_NOT_FOUND; + msg = "failed to get tablet, err=not found"; + return; + } else if (err != TxnErrorCode::TXN_OK) [[unlikely]] { + code = cast_as(err); + msg = "failed to get tablet, err=internal error"; + return; + } + + if (!tablet_meta->ParseFromString(val1)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "malformed tablet meta, unable to initialize"; + return; + } + + if (tablet_meta->has_schema()) { // tablet meta saved before detach schema kv + tablet_meta->set_schema_version(tablet_meta->schema().schema_version()); + } + if (!tablet_meta->has_schema() && !skip_schema) { + if (!tablet_meta->has_schema_version()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "tablet_meta must have either schema or schema_version"; + return; + } + auto key = meta_schema_key( + {instance_id, tablet_meta->index_id(), tablet_meta->schema_version()}); + ValueBuf val_buf; + err = cloud::get(txn, key, &val_buf); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to get schema, err={}", err == TxnErrorCode::TXN_KEY_NOT_FOUND + ? "not found" + : "internal error"); + return; + } + if (!parse_schema_value(val_buf, tablet_meta->mutable_schema())) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = fmt::format("malformed schema value, key={}", key); + return; + } + } +} + +void MetaServiceImpl::update_tablet(::google::protobuf::RpcController* controller, + const UpdateTabletRequest* request, + UpdateTabletResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(update_tablet); + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(WARNING) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); + return; + } + RPC_RATE_LIMIT(update_tablet) + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to init txn"; + return; + } + for (const TabletMetaInfoPB& tablet_meta_info : request->tablet_meta_infos()) { + doris::TabletMetaCloudPB tablet_meta; + internal_get_tablet(code, msg, instance_id, txn.get(), tablet_meta_info.tablet_id(), + &tablet_meta, true); + if (code != MetaServiceCode::OK) { + return; + } + if (tablet_meta_info.has_is_in_memory()) { // deprecate after 3.0.0 + tablet_meta.set_is_in_memory(tablet_meta_info.is_in_memory()); + } else if (tablet_meta_info.has_is_persistent()) { // deprecate after 3.0.0 + tablet_meta.set_is_persistent(tablet_meta_info.is_persistent()); + } else if (tablet_meta_info.has_ttl_seconds()) { + tablet_meta.set_ttl_seconds(tablet_meta_info.ttl_seconds()); + } + int64_t table_id = tablet_meta.table_id(); + int64_t index_id = tablet_meta.index_id(); + int64_t partition_id = tablet_meta.partition_id(); + int64_t tablet_id = tablet_meta.tablet_id(); + + MetaTabletKeyInfo key_info {instance_id, table_id, index_id, partition_id, tablet_id}; + std::string key; + std::string val; + meta_tablet_key(key_info, &key); + if (!tablet_meta.SerializeToString(&val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + msg = "failed to serialize tablet meta"; + return; + } + txn->put(key, val); + LOG(INFO) << "xxx put tablet_key=" << hex(key); + } + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to update tablet meta, err=" << err; + msg = ss.str(); + return; + } +} + +void MetaServiceImpl::update_tablet_schema(::google::protobuf::RpcController* controller, + const UpdateTabletSchemaRequest* request, + UpdateTabletSchemaResponse* response, + ::google::protobuf::Closure* done) { + DCHECK(false) << "should not call update_tablet_schema"; + RPC_PREPROCESS(update_tablet_schema); + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(WARNING) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); + return; + } + + RPC_RATE_LIMIT(update_tablet_schema) + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to init txn"; + return; + } + + doris::TabletMetaCloudPB tablet_meta; + internal_get_tablet(code, msg, instance_id, txn.get(), request->tablet_id(), &tablet_meta, + true); + if (code != MetaServiceCode::OK) { + return; + } + + std::string schema_key, schema_val; + while (request->has_tablet_schema()) { + if (!config::write_schema_kv) { + tablet_meta.mutable_schema()->CopyFrom(request->tablet_schema()); + break; + } + tablet_meta.set_schema_version(request->tablet_schema().schema_version()); + meta_schema_key({instance_id, tablet_meta.index_id(), tablet_meta.schema_version()}, + &schema_key); + if (txn->get(schema_key, &schema_val, true) == TxnErrorCode::TXN_OK) { + break; // schema has already been saved + } + if (!request->tablet_schema().SerializeToString(&schema_val)) [[unlikely]] { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + msg = "failed to serialize tablet schema value"; + return; + } + txn->put(schema_key, schema_val); + break; + } + + int64_t table_id = tablet_meta.table_id(); + int64_t index_id = tablet_meta.index_id(); + int64_t partition_id = tablet_meta.partition_id(); + int64_t tablet_id = tablet_meta.tablet_id(); + MetaTabletKeyInfo key_info {instance_id, table_id, index_id, partition_id, tablet_id}; + std::string key; + std::string val; + meta_tablet_key(key_info, &key); + if (!tablet_meta.SerializeToString(&val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + msg = "failed to serialize tablet meta"; + return; + } + txn->put(key, val); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to update tablet meta, err=" << err; + msg = ss.str(); + return; + } +} + +void MetaServiceImpl::get_tablet(::google::protobuf::RpcController* controller, + const GetTabletRequest* request, GetTabletResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(get_tablet); + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); + return; + } + RPC_RATE_LIMIT(get_tablet) + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to init txn"; + return; + } + internal_get_tablet(code, msg, instance_id, txn.get(), request->tablet_id(), + response->mutable_tablet_meta(), false); +} + +static void set_schema_in_existed_rowset(MetaServiceCode& code, std::string& msg, Transaction* txn, + const std::string& instance_id, + doris::RowsetMetaCloudPB& rowset_meta, + doris::RowsetMetaCloudPB& existed_rowset_meta) { + DCHECK(existed_rowset_meta.has_index_id()); + if (!existed_rowset_meta.has_schema_version()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "rowset_meta must have either schema or schema_version"; + return; + } + // Currently, schema version of `existed_rowset_meta` and `rowset_meta` MUST be equal + DCHECK_EQ(existed_rowset_meta.schema_version(), + rowset_meta.has_tablet_schema() ? rowset_meta.tablet_schema().schema_version() + : rowset_meta.schema_version()); + if (rowset_meta.has_tablet_schema() && + rowset_meta.tablet_schema().schema_version() == existed_rowset_meta.schema_version()) { + if (existed_rowset_meta.GetArena() && + rowset_meta.tablet_schema().GetArena() == existed_rowset_meta.GetArena()) { + existed_rowset_meta.unsafe_arena_set_allocated_tablet_schema( + rowset_meta.mutable_tablet_schema()); + } else { + existed_rowset_meta.mutable_tablet_schema()->CopyFrom(rowset_meta.tablet_schema()); + } + } else { + // get schema from txn kv + std::string schema_key = meta_schema_key({instance_id, existed_rowset_meta.index_id(), + existed_rowset_meta.schema_version()}); + ValueBuf val_buf; + TxnErrorCode err = cloud::get(txn, schema_key, &val_buf, true); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format( + "failed to get schema, schema_version={}: {}", rowset_meta.schema_version(), + err == TxnErrorCode::TXN_KEY_NOT_FOUND ? "not found" : "internal error"); + return; + } + if (!parse_schema_value(val_buf, existed_rowset_meta.mutable_tablet_schema())) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = fmt::format("malformed schema value, key={}", schema_key); + return; + } + } +} + +/** + * 0. Construct the corresponding rowset commit_key according to the info in request + * 1. Check whether this rowset has already been committed through commit_key + * a. if has been committed, abort prepare_rowset + * b. else, goto 2 + * 2. Construct recycle rowset kv which contains object path + * 3. Put recycle rowset kv + */ +void MetaServiceImpl::prepare_rowset(::google::protobuf::RpcController* controller, + const CreateRowsetRequest* request, + CreateRowsetResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(prepare_rowset); + if (!request->has_rowset_meta()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "no rowset meta"; + return; + } + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); + return; + } + doris::RowsetMetaCloudPB rowset_meta(request->rowset_meta()); + if (!rowset_meta.has_tablet_schema() && !rowset_meta.has_schema_version()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "rowset_meta must have either schema or schema_version"; + return; + } + RPC_RATE_LIMIT(prepare_rowset) + // temporary == true is for load txn, schema change, compaction + // temporary == false currently no such situation + bool temporary = request->has_temporary() ? request->temporary() : false; + int64_t tablet_id = rowset_meta.tablet_id(); + int64_t end_version = rowset_meta.end_version(); + const auto& rowset_id = rowset_meta.rowset_id_v2(); + + std::string commit_key; + std::string commit_val; + + if (temporary) { // load txn, schema change, compaction + int64_t txn_id = rowset_meta.txn_id(); + MetaRowsetTmpKeyInfo key_info {instance_id, txn_id, tablet_id}; + meta_rowset_tmp_key(key_info, &commit_key); + } else { // ? + MetaRowsetKeyInfo key_info {instance_id, tablet_id, end_version}; + meta_rowset_key(key_info, &commit_key); + } + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + return; + } + + // Check if commit key already exists. + err = txn->get(commit_key, &commit_val); + if (err == TxnErrorCode::TXN_OK) { + auto existed_rowset_meta = response->mutable_existed_rowset_meta(); + if (!existed_rowset_meta->ParseFromString(commit_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = fmt::format("malformed rowset meta value. key={}", hex(commit_key)); + return; + } + if (!existed_rowset_meta->has_index_id()) { + if (rowset_meta.has_index_id()) { + existed_rowset_meta->set_index_id(rowset_meta.index_id()); + } else { + TabletIndexPB tablet_idx; + get_tablet_idx(code, msg, txn.get(), instance_id, rowset_meta.tablet_id(), + tablet_idx); + if (code != MetaServiceCode::OK) return; + existed_rowset_meta->set_index_id(tablet_idx.index_id()); + rowset_meta.set_index_id(tablet_idx.index_id()); + } + } + if (!existed_rowset_meta->has_tablet_schema()) { + set_schema_in_existed_rowset(code, msg, txn.get(), instance_id, rowset_meta, + *existed_rowset_meta); + if (code != MetaServiceCode::OK) return; + } else { + existed_rowset_meta->set_schema_version( + existed_rowset_meta->tablet_schema().schema_version()); + } + code = MetaServiceCode::ALREADY_EXISTED; + msg = "rowset already exists"; + return; + } + if (err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = cast_as(err); + msg = "failed to check whether rowset exists"; + return; + } + + std::string prepare_key; + std::string prepare_val; + RecycleRowsetKeyInfo prepare_key_info {instance_id, tablet_id, rowset_id}; + recycle_rowset_key(prepare_key_info, &prepare_key); + RecycleRowsetPB prepare_rowset; + using namespace std::chrono; + int64_t now = duration_cast(system_clock::now().time_since_epoch()).count(); + prepare_rowset.set_creation_time(now); + prepare_rowset.set_expiration(request->rowset_meta().txn_expiration()); + // Schema is useless for PREPARE type recycle rowset, set it to null to reduce storage space + rowset_meta.set_allocated_tablet_schema(nullptr); + prepare_rowset.mutable_rowset_meta()->CopyFrom(rowset_meta); + prepare_rowset.set_type(RecycleRowsetPB::PREPARE); + prepare_rowset.SerializeToString(&prepare_val); + DCHECK_GT(prepare_rowset.expiration(), 0); + txn->put(prepare_key, prepare_val); + LOG(INFO) << "xxx put" << (temporary ? " tmp " : " ") << "prepare_rowset_key " + << hex(prepare_key) << " associated commit_rowset_key " << hex(commit_key) + << " value_size " << prepare_val.size(); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to save recycle rowset, err=" << err; + msg = ss.str(); + return; + } +} + +/** + * 0. Construct the corresponding rowset commit_key and commit_value according + * to the info in request + * 1. Check whether this rowset has already been committed through commit_key + * a. if has been committed + * 1. if committed value is same with commit_value, it may be a redundant + * retry request, return ok + * 2. else, abort commit_rowset + * b. else, goto 2 + * 2. Construct the corresponding rowset prepare_key(recycle rowset) + * 3. Remove prepare_key and put commit rowset kv + */ +void MetaServiceImpl::commit_rowset(::google::protobuf::RpcController* controller, + const CreateRowsetRequest* request, + CreateRowsetResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(commit_rowset); + if (!request->has_rowset_meta()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "no rowset meta"; + return; + } + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); + return; + } + doris::RowsetMetaCloudPB rowset_meta(request->rowset_meta()); + if (!rowset_meta.has_tablet_schema() && !rowset_meta.has_schema_version()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "rowset_meta must have either schema or schema_version"; + return; + } + RPC_RATE_LIMIT(commit_rowset) + // temporary == true is for load txn, schema change, compaction + // temporary == false currently no such situation + bool temporary = request->has_temporary() ? request->temporary() : false; + int64_t tablet_id = rowset_meta.tablet_id(); + int64_t end_version = rowset_meta.end_version(); + const auto& rowset_id = rowset_meta.rowset_id_v2(); + + std::string commit_key; + std::string commit_val; + + if (temporary) { // load txn, schema change, compaction + int64_t txn_id = rowset_meta.txn_id(); + MetaRowsetTmpKeyInfo key_info {instance_id, txn_id, tablet_id}; + meta_rowset_tmp_key(key_info, &commit_key); + } else { // ? + MetaRowsetKeyInfo key_info {instance_id, tablet_id, end_version}; + meta_rowset_key(key_info, &commit_key); + } + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + return; + } + + // Check if commit key already exists. + std::string existed_commit_val; + err = txn->get(commit_key, &existed_commit_val); + if (err == TxnErrorCode::TXN_OK) { + auto existed_rowset_meta = response->mutable_existed_rowset_meta(); + if (!existed_rowset_meta->ParseFromString(existed_commit_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = fmt::format("malformed rowset meta value. key={}", hex(commit_key)); + return; + } + if (existed_rowset_meta->rowset_id_v2() == rowset_meta.rowset_id_v2()) { + // Same request, return OK + response->set_allocated_existed_rowset_meta(nullptr); + return; + } + if (!existed_rowset_meta->has_index_id()) { + if (rowset_meta.has_index_id()) { + existed_rowset_meta->set_index_id(rowset_meta.index_id()); + } else { + TabletIndexPB tablet_idx; + get_tablet_idx(code, msg, txn.get(), instance_id, rowset_meta.tablet_id(), + tablet_idx); + if (code != MetaServiceCode::OK) return; + existed_rowset_meta->set_index_id(tablet_idx.index_id()); + } + } + if (!existed_rowset_meta->has_tablet_schema()) { + set_schema_in_existed_rowset(code, msg, txn.get(), instance_id, rowset_meta, + *existed_rowset_meta); + if (code != MetaServiceCode::OK) return; + } else { + existed_rowset_meta->set_schema_version( + existed_rowset_meta->tablet_schema().schema_version()); + } + code = MetaServiceCode::ALREADY_EXISTED; + msg = "rowset already exists"; + return; + } + if (err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = cast_as(err); + msg = "failed to check whether rowset exists"; + return; + } + // write schema kv if rowset_meta has schema + if (config::write_schema_kv && rowset_meta.has_tablet_schema()) { + if (!rowset_meta.has_index_id()) { + TabletIndexPB tablet_idx; + get_tablet_idx(code, msg, txn.get(), instance_id, rowset_meta.tablet_id(), tablet_idx); + if (code != MetaServiceCode::OK) return; + rowset_meta.set_index_id(tablet_idx.index_id()); + } + DCHECK(rowset_meta.tablet_schema().has_schema_version()); + DCHECK_GE(rowset_meta.tablet_schema().schema_version(), 0); + rowset_meta.set_schema_version(rowset_meta.tablet_schema().schema_version()); + auto schema_key = meta_schema_key( + {instance_id, rowset_meta.index_id(), rowset_meta.schema_version()}); + put_schema_kv(code, msg, txn.get(), schema_key, rowset_meta.tablet_schema()); + if (code != MetaServiceCode::OK) return; + rowset_meta.set_allocated_tablet_schema(nullptr); + } + + std::string prepare_key; + RecycleRowsetKeyInfo prepare_key_info {instance_id, tablet_id, rowset_id}; + recycle_rowset_key(prepare_key_info, &prepare_key); + DCHECK_GT(rowset_meta.txn_expiration(), 0); + if (!rowset_meta.SerializeToString(&commit_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + msg = "failed to serialize rowset meta"; + return; + } + + txn->remove(prepare_key); + txn->put(commit_key, commit_val); + LOG(INFO) << "xxx put" << (temporary ? " tmp " : " ") << "commit_rowset_key " << hex(commit_key) + << " delete prepare_rowset_key " << hex(prepare_key) << " value_size " + << commit_val.size(); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to save rowset meta, err=" << err; + msg = ss.str(); + return; + } +} + +void MetaServiceImpl::update_tmp_rowset(::google::protobuf::RpcController* controller, + const CreateRowsetRequest* request, + CreateRowsetResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(update_tmp_rowset); + if (!request->has_rowset_meta()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "no rowset meta"; + return; + } + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); + return; + } + doris::RowsetMetaCloudPB rowset_meta(request->rowset_meta()); + if (!rowset_meta.has_tablet_schema() && !rowset_meta.has_schema_version()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "rowset_meta must have either schema or schema_version"; + return; + } + RPC_RATE_LIMIT(update_tmp_rowset) + int64_t tablet_id = rowset_meta.tablet_id(); + + std::string update_key; + std::string update_val; + + int64_t txn_id = rowset_meta.txn_id(); + MetaRowsetTmpKeyInfo key_info {instance_id, txn_id, tablet_id}; + meta_rowset_tmp_key(key_info, &update_key); + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + return; + } + + // Check if commit key already exists. + std::string existed_commit_val; + err = txn->get(update_key, &existed_commit_val); + if (err == TxnErrorCode::TXN_OK) { + auto existed_rowset_meta = response->mutable_existed_rowset_meta(); + if (!existed_rowset_meta->ParseFromString(existed_commit_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = fmt::format("malformed rowset meta value. key={}", hex(update_key)); + return; + } + } else if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = MetaServiceCode::ROWSET_META_NOT_FOUND; + LOG_WARNING( + "fail to find the rowset meta with key={}, instance_id={}, txn_id={}, " + "tablet_id={}, rowset_id={}", + hex(update_key), instance_id, rowset_meta.txn_id(), tablet_id, + rowset_meta.rowset_id_v2()); + msg = "can't find the rowset"; + return; + } else { + code = cast_as(err); + LOG_WARNING( + "internal error, fail to find the rowset meta with key={}, instance_id={}, " + "txn_id={}, tablet_id={}, rowset_id={}", + hex(update_key), instance_id, rowset_meta.txn_id(), tablet_id, + rowset_meta.rowset_id_v2()); + msg = "failed to check whether rowset exists"; + return; + } + + DCHECK_GT(rowset_meta.txn_expiration(), 0); + if (!rowset_meta.SerializeToString(&update_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + msg = "failed to serialize rowset meta"; + return; + } + + txn->put(update_key, update_val); + LOG(INFO) << "xxx put " + << "update_rowset_key " << hex(update_key) << " value_size " << update_val.size(); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to update rowset meta, err=" << err; + msg = ss.str(); + return; + } +} + +void internal_get_rowset(Transaction* txn, int64_t start, int64_t end, + const std::string& instance_id, int64_t tablet_id, MetaServiceCode& code, + std::string& msg, GetRowsetResponse* response) { + LOG(INFO) << "get_rowset start=" << start << ", end=" << end; + MetaRowsetKeyInfo key_info0 {instance_id, tablet_id, start}; + MetaRowsetKeyInfo key_info1 {instance_id, tablet_id, end + 1}; + std::string key0; + std::string key1; + meta_rowset_key(key_info0, &key0); + meta_rowset_key(key_info1, &key1); + std::unique_ptr it; + + int num_rowsets = 0; + std::unique_ptr> defer_log_range( + (int*)0x01, [key0, key1, &num_rowsets](int*) { + LOG(INFO) << "get rowset meta, num_rowsets=" << num_rowsets << " range=[" + << hex(key0) << "," << hex(key1) << "]"; + }); + + std::stringstream ss; + do { + TxnErrorCode err = txn->get(key0, key1, &it); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "internal error, failed to get rowset, err=" << err; + msg = ss.str(); + LOG(WARNING) << msg; + return; + } + + while (it->has_next()) { + auto [k, v] = it->next(); + auto rs = response->add_rowset_meta(); + if (!rs->ParseFromArray(v.data(), v.size())) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "malformed rowset meta, unable to deserialize"; + LOG(WARNING) << msg << " key=" << hex(k); + return; + } + ++num_rowsets; + if (!it->has_next()) key0 = k; + } + key0.push_back('\x00'); // Update to next smallest key for iteration + } while (it->more()); +} + +std::vector> calc_sync_versions(int64_t req_bc_cnt, int64_t bc_cnt, + int64_t req_cc_cnt, int64_t cc_cnt, + int64_t req_cp, int64_t cp, + int64_t req_start, int64_t req_end) { + using Version = std::pair; + // combine `v1` `v2` to `v1`, return true if success + static auto combine_if_overlapping = [](Version& v1, Version& v2) -> bool { + if (v1.second + 1 < v2.first || v2.second + 1 < v1.first) return false; + v1.first = std::min(v1.first, v2.first); + v1.second = std::max(v1.second, v2.second); + return true; + }; + // [xxx]: compacted versions + // ^~~~~: cumulative point + // ^___^: related versions + std::vector versions; + if (req_bc_cnt < bc_cnt) { + // * for any BC happended + // BE [=][=][=][=][=====][=][=] + // ^~~~~ req_cp + // MS [xxxxxxxxxx][xxxxxxxxxxxxxx][=======][=][=] + // ^~~~~~~ ms_cp + // ^_________________________^ versions_return: [0, ms_cp - 1] + versions.emplace_back(0, cp - 1); + } + + if (req_cc_cnt < cc_cnt) { + Version cc_version; + if (req_cp < cp && req_cc_cnt + 1 == cc_cnt) { + // * only one CC happened and CP changed + // BE [=][=][=][=][=====][=][=] + // ^~~~~ req_cp + // MS [=][=][=][=][xxxxxxxxxxxxxx][=======][=][=] + // ^~~~~~~ ms_cp + // ^____________^ related_versions: [req_cp, ms_cp - 1] + // + cc_version = {req_cp, cp - 1}; + } else { + // * more than one CC happened and CP changed + // BE [=][=][=][=][=====][=][=] + // ^~~~~ req_cp + // MS [=][=][=][=][xxxxxxxxxxxxxx][xxxxxxx][=][=] + // ^~~~~~~ ms_cp + // ^_____________________^ related_versions: [req_cp, max] + // + // * more than one CC happened and CP remain unchanged + // BE [=][=][=][=][=====][=][=] + // ^~~~~ req_cp + // MS [=][=][=][=][xxxxxxxxxxxxxx][xxxxxxx][=][=] + // ^~~~~~~~~~~~~~ ms_cp + // ^_____________________^ related_versions: [req_cp, max] + // there may be holes if we don't return all version + // after ms_cp, however it can be optimized. + cc_version = {req_cp, std::numeric_limits::max() - 1}; + } + if (versions.empty() || !combine_if_overlapping(versions.front(), cc_version)) { + versions.push_back(cc_version); + } + } + + Version query_version {req_start, req_end}; + bool combined = false; + for (auto& v : versions) { + if ((combined = combine_if_overlapping(v, query_version))) break; + } + if (!combined) { + versions.push_back(query_version); + } + std::sort(versions.begin(), versions.end(), + [](const Version& v1, const Version& v2) { return v1.first < v2.first; }); + return versions; +} + +void MetaServiceImpl::get_rowset(::google::protobuf::RpcController* controller, + const GetRowsetRequest* request, GetRowsetResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(get_rowset); + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); + return; + } + RPC_RATE_LIMIT(get_rowset) + int64_t tablet_id = request->idx().has_tablet_id() ? request->idx().tablet_id() : -1; + if (tablet_id <= 0) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "no valid tablet_id given"; + return; + } + + if (!request->has_base_compaction_cnt() || !request->has_cumulative_compaction_cnt() || + !request->has_cumulative_point()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "no valid compaction_cnt or cumulative_point given"; + return; + } + int64_t req_bc_cnt = request->base_compaction_cnt(); + int64_t req_cc_cnt = request->cumulative_compaction_cnt(); + int64_t req_cp = request->cumulative_point(); + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + return; + } + + TabletIndexPB idx(request->idx()); + // Get tablet id index from kv + if (!idx.has_table_id() || !idx.has_index_id() || !idx.has_partition_id()) { + get_tablet_idx(code, msg, txn.get(), instance_id, tablet_id, idx); + if (code != MetaServiceCode::OK) return; + } + // TODO(plat1ko): Judge if tablet has been dropped (in dropped index/partition) + + TabletStatsPB tablet_stat; + internal_get_tablet_stats(code, msg, txn.get(), instance_id, idx, tablet_stat, true); + if (code != MetaServiceCode::OK) return; + VLOG_DEBUG << "tablet_id=" << tablet_id << " stats=" << proto_to_json(tablet_stat); + + int64_t bc_cnt = tablet_stat.base_compaction_cnt(); + int64_t cc_cnt = tablet_stat.cumulative_compaction_cnt(); + int64_t cp = tablet_stat.cumulative_point(); + + response->mutable_stats()->CopyFrom(tablet_stat); + + int64_t req_start = request->start_version(); + int64_t req_end = request->end_version(); + req_end = req_end < 0 ? std::numeric_limits::max() - 1 : req_end; + + LOG(INFO) << "req_bc_cnt=" << req_bc_cnt << ", bc_cnt=" << bc_cnt + << ", req_cc_cnt=" << req_cc_cnt << ", cc_cnt=" << cc_cnt << ", req_cp=" << req_cp + << ", cp=" << cp; + //========================================================================== + // Find version ranges to be synchronized due to compaction + //========================================================================== + if (req_bc_cnt > bc_cnt || req_cc_cnt > cc_cnt || req_cp > cp) { + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "no valid compaction_cnt or cumulative_point given. req_bc_cnt=" << req_bc_cnt + << ", bc_cnt=" << bc_cnt << ", req_cc_cnt=" << req_cc_cnt << ", cc_cnt=" << cc_cnt + << ", req_cp=" << req_cp << ", cp=" << cp; + msg = ss.str(); + return; + } + auto versions = calc_sync_versions(req_bc_cnt, bc_cnt, req_cc_cnt, cc_cnt, req_cp, cp, + req_start, req_end); + for (auto [start, end] : versions) { + internal_get_rowset(txn.get(), start, end, instance_id, tablet_id, code, msg, response); + if (code != MetaServiceCode::OK) { + return; + } + } + + // get referenced schema + std::unordered_map version_to_schema; + for (auto& rowset_meta : *response->mutable_rowset_meta()) { + if (rowset_meta.has_tablet_schema()) { + version_to_schema.emplace(rowset_meta.tablet_schema().schema_version(), + rowset_meta.mutable_tablet_schema()); + rowset_meta.set_schema_version(rowset_meta.tablet_schema().schema_version()); + } + rowset_meta.set_index_id(idx.index_id()); + } + auto arena = response->GetArena(); + for (auto& rowset_meta : *response->mutable_rowset_meta()) { + if (rowset_meta.has_tablet_schema()) continue; + if (!rowset_meta.has_schema_version()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = fmt::format( + "rowset_meta must have either schema or schema_version, " + "rowset_version=[{}-{}]", + rowset_meta.start_version(), rowset_meta.end_version()); + return; + } + if (auto it = version_to_schema.find(rowset_meta.schema_version()); + it != version_to_schema.end()) { + if (arena != nullptr) { + rowset_meta.set_allocated_tablet_schema(it->second); + } else { + rowset_meta.mutable_tablet_schema()->CopyFrom(*it->second); + } + } else { + auto key = meta_schema_key({instance_id, idx.index_id(), rowset_meta.schema_version()}); + ValueBuf val_buf; + TxnErrorCode err = cloud::get(txn.get(), key, &val_buf); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format( + "failed to get schema, schema_version={}, rowset_version=[{}-{}]: {}", + rowset_meta.schema_version(), rowset_meta.start_version(), + rowset_meta.end_version(), + err == TxnErrorCode::TXN_KEY_NOT_FOUND ? "not found" : "internal error"); + return; + } + auto schema = rowset_meta.mutable_tablet_schema(); + if (!parse_schema_value(val_buf, schema)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = fmt::format("malformed schema value, key={}", key); + return; + } + version_to_schema.emplace(rowset_meta.schema_version(), schema); + } + } +} + +void MetaServiceImpl::get_tablet_stats(::google::protobuf::RpcController* controller, + const GetTabletStatsRequest* request, + GetTabletStatsResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(get_tablet_stats); + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); + return; + } + RPC_RATE_LIMIT(get_tablet_stats) + + std::unique_ptr txn; + for (auto& i : request->tablet_idx()) { + TabletIndexPB idx(i); + // FIXME(plat1ko): Get all tablet stats in one txn + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to create txn, tablet_id={}", idx.tablet_id()); + return; + } + if (!(/* idx.has_db_id() && */ idx.has_table_id() && idx.has_index_id() && + idx.has_partition_id() && i.has_tablet_id())) { + get_tablet_idx(code, msg, txn.get(), instance_id, idx.tablet_id(), idx); + if (code != MetaServiceCode::OK) return; + } + auto tablet_stats = response->add_tablet_stats(); + internal_get_tablet_stats(code, msg, txn.get(), instance_id, idx, *tablet_stats, true); + if (code != MetaServiceCode::OK) { + response->clear_tablet_stats(); + break; + } +#ifdef NDEBUG + // Force data size >= 0 to reduce the losses caused by bugs + if (tablet_stats->data_size() < 0) tablet_stats->set_data_size(0); +#endif + } +} + +static bool check_delete_bitmap_lock(MetaServiceCode& code, std::string& msg, std::stringstream& ss, + std::unique_ptr& txn, std::string& instance_id, + int64_t table_id, int64_t lock_id, int64_t lock_initiator) { + std::string lock_key = meta_delete_bitmap_update_lock_key({instance_id, table_id, -1}); + std::string lock_val; + DeleteBitmapUpdateLockPB lock_info; + auto err = txn->get(lock_key, &lock_val); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + msg = "lock id key not found"; + code = MetaServiceCode::LOCK_EXPIRED; + return false; + } + if (err != TxnErrorCode::TXN_OK) { + ss << "failed to get delete bitmap lock info, err=" << err; + msg = ss.str(); + code = cast_as(err); + return false; + } + if (!lock_info.ParseFromString(lock_val)) [[unlikely]] { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse DeleteBitmapUpdateLockPB"; + return false; + } + if (lock_info.lock_id() != lock_id) { + ss << "lock id not match, locked by lock_id=" << lock_info.lock_id(); + msg = ss.str(); + code = MetaServiceCode::LOCK_EXPIRED; + return false; + } + bool found = false; + for (auto initiator : lock_info.initiators()) { + if (lock_initiator == initiator) { + found = true; + break; + } + } + if (!found) { + msg = "lock initiator not exist"; + code = MetaServiceCode::LOCK_EXPIRED; + return false; + } + return true; +} + +static bool process_pending_delete_bitmap(MetaServiceCode& code, std::string& msg, + std::stringstream& ss, std::unique_ptr& txn, + std::string& instance_id, int64_t tablet_id) { + std::string pending_key = meta_pending_delete_bitmap_key({instance_id, tablet_id}); + std::string pending_val; + auto err = txn->get(pending_key, &pending_val); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + ss << "failed to get delete bitmap pending info, instance_id=" << instance_id + << " tablet_id=" << tablet_id << " key=" << hex(pending_key) << " err=" << err; + msg = ss.str(); + code = cast_as(err); + return false; + } + + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + return true; + } + + // delete delete bitmap of expired txn + PendingDeleteBitmapPB pending_info; + if (!pending_info.ParseFromString(pending_val)) [[unlikely]] { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse PendingDeleteBitmapPB"; + return false; + } + for (auto& delete_bitmap_key : pending_info.delete_bitmap_keys()) { + // FIXME: Don't expose the implementation details of splitting large value + // remove large value (>90*1000) + std::string end_key = delete_bitmap_key; + encode_int64(INT64_MAX, &end_key); + txn->remove(delete_bitmap_key, end_key); + LOG(INFO) << "xxx remove pending delete bitmap, delete_bitmap_key=" + << hex(delete_bitmap_key); + } + return true; +} + +void MetaServiceImpl::update_delete_bitmap(google::protobuf::RpcController* controller, + const UpdateDeleteBitmapRequest* request, + UpdateDeleteBitmapResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(update_delete_bitmap); + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + if (cloud_unique_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "cloud unique id not set"; + return; + } + + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(WARNING) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); + return; + } + RPC_RATE_LIMIT(update_delete_bitmap) + + uint32_t fdb_txn_size = 0; + auto table_id = request->table_id(); + auto tablet_id = request->tablet_id(); + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to init txn"; + return; + } + + // 1. Check whether the lock expires + if (!check_delete_bitmap_lock(code, msg, ss, txn, instance_id, table_id, request->lock_id(), + request->initiator())) { + LOG(WARNING) << "failed to check delete bitmap lock, table_id=" << table_id + << " request lock_id=" << request->lock_id() + << " request initiator=" << request->initiator() << " msg" << msg; + return; + } + + // 2. Process pending delete bitmap + if (!process_pending_delete_bitmap(code, msg, ss, txn, instance_id, tablet_id)) { + return; + } + + // 3. store all pending delete bitmap for this txn + PendingDeleteBitmapPB delete_bitmap_keys; + for (size_t i = 0; i < request->rowset_ids_size(); ++i) { + MetaDeleteBitmapInfo key_info {instance_id, tablet_id, request->rowset_ids(i), + request->versions(i), request->segment_ids(i)}; + std::string key; + meta_delete_bitmap_key(key_info, &key); + delete_bitmap_keys.add_delete_bitmap_keys(key); + } + // no need to record pending key for compaction or schema change, + // because delete bitmap will attach to new rowset, just delete new rowset if failed + // lock_id > 0 : load + // lock_id = -1 : compaction + // lock_id = -2 : schema change + if (request->lock_id() > 0) { + std::string pending_val; + if (!delete_bitmap_keys.SerializeToString(&pending_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + msg = "failed to serialize pending delete bitmap"; + return; + } + std::string pending_key = meta_pending_delete_bitmap_key({instance_id, tablet_id}); + txn->put(pending_key, pending_val); + fdb_txn_size = fdb_txn_size + pending_key.size() + pending_val.size(); + LOG(INFO) << "xxx update delete bitmap put pending_key=" << hex(pending_key) + << " lock_id=" << request->lock_id() << " value_size: " << pending_val.size(); + } + + // 4. Update delete bitmap for curent txn + for (size_t i = 0; i < request->rowset_ids_size(); ++i) { + auto& key = delete_bitmap_keys.delete_bitmap_keys(i); + auto& val = request->segment_delete_bitmaps(i); + + // Split into multiple fdb transactions, because the size of one fdb + // transaction can't exceed 10MB. + if (fdb_txn_size + key.size() + val.size() > 9 * 1024 * 1024) { + LOG(INFO) << "fdb txn size more than 9MB, current size: " << fdb_txn_size + << " lock_id=" << request->lock_id(); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to update delete bitmap, err=" << err; + msg = ss.str(); + return; + } + fdb_txn_size = 0; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to init txn"; + return; + } + if (!check_delete_bitmap_lock(code, msg, ss, txn, instance_id, table_id, + request->lock_id(), request->initiator())) { + LOG(WARNING) << "failed to check delete bitmap lock, table_id=" << table_id + << " request lock_id=" << request->lock_id() + << " request initiator=" << request->initiator() << " msg" << msg; + return; + } + } + // splitting large values (>90*1000) into multiple KVs + cloud::put(txn.get(), key, val, 0); + fdb_txn_size = fdb_txn_size + key.size() + val.size(); + LOG(INFO) << "xxx update delete bitmap put delete_bitmap_key=" << hex(key) + << " lock_id=" << request->lock_id() << " value_size: " << val.size(); + } + + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to update delete bitmap, err=" << err; + msg = ss.str(); + return; + } +} + +void MetaServiceImpl::get_delete_bitmap(google::protobuf::RpcController* controller, + const GetDeleteBitmapRequest* request, + GetDeleteBitmapResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(get_delete_bitmap); + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + if (cloud_unique_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "cloud unique id not set"; + return; + } + + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(WARNING) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); + return; + } + RPC_RATE_LIMIT(get_delete_bitmap) + + auto tablet_id = request->tablet_id(); + auto& rowset_ids = request->rowset_ids(); + auto& begin_versions = request->begin_versions(); + auto& end_versions = request->end_versions(); + if (rowset_ids.size() != begin_versions.size() || rowset_ids.size() != end_versions.size()) { + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "rowset and version size not match. " + << " rowset_size=" << rowset_ids.size() + << " begin_version_size=" << begin_versions.size() + << " end_version_size=" << end_versions.size(); + msg = ss.str(); + return; + } + + for (size_t i = 0; i < rowset_ids.size(); i++) { + // create a new transaction every time, avoid using one transaction that takes too long + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to init txn"; + return; + } + MetaDeleteBitmapInfo start_key_info {instance_id, tablet_id, rowset_ids[i], + begin_versions[i], 0}; + MetaDeleteBitmapInfo end_key_info {instance_id, tablet_id, rowset_ids[i], end_versions[i], + INT64_MAX}; + std::string start_key; + std::string end_key; + meta_delete_bitmap_key(start_key_info, &start_key); + meta_delete_bitmap_key(end_key_info, &end_key); + + // in order to get splitted large value + encode_int64(INT64_MAX, &end_key); + + std::unique_ptr it; + int64_t last_ver = -1; + int64_t last_seg_id = -1; + do { + err = txn->get(start_key, end_key, &it); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "internal error, failed to get delete bitmap, ret=" << err; + msg = ss.str(); + return; + } + + while (it->has_next()) { + auto [k, v] = it->next(); + auto k1 = k; + k1.remove_prefix(1); + std::vector, int, int>> out; + decode_key(&k1, &out); + // 0x01 "meta" ${instance_id} "delete_bitmap" ${tablet_id} + // ${rowset_id0} ${version1} ${segment_id0} -> DeleteBitmapPB + auto ver = std::get(std::get<0>(out[5])); + auto seg_id = std::get(std::get<0>(out[6])); + + // FIXME: Don't expose the implementation details of splitting large value. + // merge splitted large values (>90*1000) + if (ver != last_ver || seg_id != last_seg_id) { + response->add_rowset_ids(rowset_ids[i]); + response->add_segment_ids(seg_id); + response->add_versions(ver); + response->add_segment_delete_bitmaps(std::string(v)); + last_ver = ver; + last_seg_id = seg_id; + } else { + response->mutable_segment_delete_bitmaps()->rbegin()->append(v); + } + } + start_key = it->next_begin_key(); // Update to next smallest key for iteration + } while (it->more()); + } + + if (request->has_idx()) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to init txn"; + return; + } + TabletIndexPB idx(request->idx()); + TabletStatsPB tablet_stat; + internal_get_tablet_stats(code, msg, txn.get(), instance_id, idx, tablet_stat, + true /*snapshot_read*/); + if (code != MetaServiceCode::OK) { + return; + } + // The requested compaction state and the actual compaction state are different, which indicates that + // the requested rowsets are expired and their delete bitmap may have been deleted. + if (request->base_compaction_cnt() != tablet_stat.base_compaction_cnt() || + request->cumulative_compaction_cnt() != tablet_stat.cumulative_compaction_cnt() || + request->cumulative_point() != tablet_stat.cumulative_point()) { + code = MetaServiceCode::ROWSETS_EXPIRED; + msg = "rowsets are expired"; + return; + } + } +} + +void MetaServiceImpl::get_delete_bitmap_update_lock(google::protobuf::RpcController* controller, + const GetDeleteBitmapUpdateLockRequest* request, + GetDeleteBitmapUpdateLockResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(get_delete_bitmap_update_lock); + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + if (cloud_unique_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "cloud unique id not set"; + return; + } + + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << cloud_unique_id; + return; + } + + RPC_RATE_LIMIT(get_delete_bitmap_update_lock) + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to init txn"; + return; + } + auto table_id = request->table_id(); + std::string lock_key = meta_delete_bitmap_update_lock_key({instance_id, table_id, -1}); + std::string lock_val; + DeleteBitmapUpdateLockPB lock_info; + err = txn->get(lock_key, &lock_val); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + ss << "failed to get delete bitmap update lock, instance_id=" << instance_id + << " table_id=" << table_id << " key=" << hex(lock_key) << " err=" << err; + msg = ss.str(); + code = MetaServiceCode::KV_TXN_GET_ERR; + return; + } + using namespace std::chrono; + int64_t now = duration_cast(system_clock::now().time_since_epoch()).count(); + if (err == TxnErrorCode::TXN_OK) { + if (!lock_info.ParseFromString(lock_val)) [[unlikely]] { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse DeleteBitmapUpdateLockPB"; + return; + } + if (lock_info.expiration() > 0 && lock_info.expiration() < now) { + LOG(INFO) << "delete bitmap lock expired, continue to process. lock_id=" + << lock_info.lock_id() << " table_id=" << table_id << " now=" << now; + lock_info.clear_initiators(); + } else if (lock_info.lock_id() != request->lock_id()) { + ss << "already be locked. request lock_id=" << request->lock_id() + << " locked by lock_id=" << lock_info.lock_id() << " table_id=" << table_id + << " now=" << now << " expiration=" << lock_info.expiration(); + msg = ss.str(); + code = MetaServiceCode::LOCK_CONFLICT; + return; + } + } + + lock_info.set_lock_id(request->lock_id()); + lock_info.set_expiration(now + request->expiration()); + bool found = false; + for (auto initiator : lock_info.initiators()) { + if (request->initiator() == initiator) { + found = true; + break; + } + } + if (!found) { + lock_info.add_initiators(request->initiator()); + } + lock_info.SerializeToString(&lock_val); + if (lock_val.empty()) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + msg = "pb serialization error"; + return; + } + txn->put(lock_key, lock_val); + LOG(INFO) << "xxx put lock_key=" << hex(lock_key) << " lock_id=" << request->lock_id() + << " initiators_size: " << lock_info.initiators_size(); + + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get_delete_bitmap_update_lock, err=" << err; + msg = ss.str(); + return; + } +} + +std::pair MetaServiceImpl::get_instance_info( + const std::string& instance_id, const std::string& cloud_unique_id, + InstanceInfoPB* instance) { + std::string cloned_instance_id = instance_id; + if (instance_id.empty()) { + if (cloud_unique_id.empty()) { + return {MetaServiceCode::INVALID_ARGUMENT, "empty instance_id and cloud_unique_id"}; + } + // get instance_id by cloud_unique_id + cloned_instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (cloned_instance_id.empty()) { + std::string msg = + fmt::format("cannot find instance_id with cloud_unique_id={}", cloud_unique_id); + return {MetaServiceCode::INVALID_ARGUMENT, std::move(msg)}; + } + } + + std::unique_ptr txn0; + TxnErrorCode err = txn_kv_->create_txn(&txn0); + if (err != TxnErrorCode::TXN_OK) { + return {cast_as(err), "failed to create txn"}; + } + + std::shared_ptr txn(txn0.release()); + auto [c0, m0] = resource_mgr_->get_instance(txn, cloned_instance_id, instance); + if (c0 != TxnErrorCode::TXN_OK) { + return {cast_as(err), "failed to get instance, info=" + m0}; + } + + // maybe do not decrypt ak/sk? + MetaServiceCode code = MetaServiceCode::OK; + std::string msg; + decrypt_instance_info(*instance, cloned_instance_id, code, msg, txn); + return {code, std::move(msg)}; +} + +} // namespace doris::cloud diff --git a/cloud/src/meta-service/meta_service.h b/cloud/src/meta-service/meta_service.h new file mode 100644 index 00000000000000..4aac7cbcff3453 --- /dev/null +++ b/cloud/src/meta-service/meta_service.h @@ -0,0 +1,642 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +#include "common/config.h" +#include "common/logging.h" +#include "common/sync_point.h" +#include "meta-service/txn_kv.h" +#include "rate-limiter/rate_limiter.h" +#include "resource-manager/resource_manager.h" + +namespace doris::cloud { + +class Transaction; + +class MetaServiceImpl : public cloud::MetaService { +public: + MetaServiceImpl(std::shared_ptr txn_kv, std::shared_ptr resource_mgr, + std::shared_ptr rate_controller); + ~MetaServiceImpl() override; + + [[nodiscard]] const std::shared_ptr& txn_kv() const { return txn_kv_; } + [[nodiscard]] const std::shared_ptr& rate_limiter() const { return rate_limiter_; } + [[nodiscard]] const std::shared_ptr& resource_mgr() const { + return resource_mgr_; + } + + void begin_txn(::google::protobuf::RpcController* controller, const BeginTxnRequest* request, + BeginTxnResponse* response, ::google::protobuf::Closure* done) override; + + void precommit_txn(::google::protobuf::RpcController* controller, + const PrecommitTxnRequest* request, PrecommitTxnResponse* response, + ::google::protobuf::Closure* done) override; + + void commit_txn(::google::protobuf::RpcController* controller, const CommitTxnRequest* request, + CommitTxnResponse* response, ::google::protobuf::Closure* done) override; + + void abort_txn(::google::protobuf::RpcController* controller, const AbortTxnRequest* request, + AbortTxnResponse* response, ::google::protobuf::Closure* done) override; + + // clang-format off + void get_txn(::google::protobuf::RpcController* controller, + const GetTxnRequest* request, + GetTxnResponse* response, + ::google::protobuf::Closure* done) override; + // clang-format on + + void get_current_max_txn_id(::google::protobuf::RpcController* controller, + const GetCurrentMaxTxnRequest* request, + GetCurrentMaxTxnResponse* response, + ::google::protobuf::Closure* done) override; + + void check_txn_conflict(::google::protobuf::RpcController* controller, + const CheckTxnConflictRequest* request, + CheckTxnConflictResponse* response, + ::google::protobuf::Closure* done) override; + + void clean_txn_label(::google::protobuf::RpcController* controller, + const CleanTxnLabelRequest* request, CleanTxnLabelResponse* response, + ::google::protobuf::Closure* done) override; + + void get_version(::google::protobuf::RpcController* controller, + const GetVersionRequest* request, GetVersionResponse* response, + ::google::protobuf::Closure* done) override; + + void batch_get_version(::google::protobuf::RpcController* controller, + const GetVersionRequest* request, GetVersionResponse* response, + ::google::protobuf::Closure* done); + + void create_tablets(::google::protobuf::RpcController* controller, + const CreateTabletsRequest* request, CreateTabletsResponse* response, + ::google::protobuf::Closure* done) override; + + void update_tablet(::google::protobuf::RpcController* controller, + const UpdateTabletRequest* request, UpdateTabletResponse* response, + ::google::protobuf::Closure* done) override; + + void update_tablet_schema(::google::protobuf::RpcController* controller, + const UpdateTabletSchemaRequest* request, + UpdateTabletSchemaResponse* response, + ::google::protobuf::Closure* done) override; + + void get_tablet(::google::protobuf::RpcController* controller, const GetTabletRequest* request, + GetTabletResponse* response, ::google::protobuf::Closure* done) override; + + void prepare_rowset(::google::protobuf::RpcController* controller, + const CreateRowsetRequest* request, CreateRowsetResponse* response, + ::google::protobuf::Closure* done) override; + + void commit_rowset(::google::protobuf::RpcController* controller, + const CreateRowsetRequest* request, CreateRowsetResponse* response, + ::google::protobuf::Closure* done) override; + + void update_tmp_rowset(::google::protobuf::RpcController* controller, + const CreateRowsetRequest* request, CreateRowsetResponse* response, + ::google::protobuf::Closure* done) override; + + void get_rowset(::google::protobuf::RpcController* controller, const GetRowsetRequest* request, + GetRowsetResponse* response, ::google::protobuf::Closure* done) override; + + void prepare_index(::google::protobuf::RpcController* controller, const IndexRequest* request, + IndexResponse* response, ::google::protobuf::Closure* done) override; + + void commit_index(::google::protobuf::RpcController* controller, const IndexRequest* request, + IndexResponse* response, ::google::protobuf::Closure* done) override; + + void drop_index(::google::protobuf::RpcController* controller, const IndexRequest* request, + IndexResponse* response, ::google::protobuf::Closure* done) override; + + void prepare_partition(::google::protobuf::RpcController* controller, + const PartitionRequest* request, PartitionResponse* response, + ::google::protobuf::Closure* done) override; + + void commit_partition(::google::protobuf::RpcController* controller, + const PartitionRequest* request, PartitionResponse* response, + ::google::protobuf::Closure* done) override; + + void drop_partition(::google::protobuf::RpcController* controller, + const PartitionRequest* request, PartitionResponse* response, + ::google::protobuf::Closure* done) override; + + void get_tablet_stats(::google::protobuf::RpcController* controller, + const GetTabletStatsRequest* request, GetTabletStatsResponse* response, + ::google::protobuf::Closure* done) override; + + void start_tablet_job(::google::protobuf::RpcController* controller, + const StartTabletJobRequest* request, StartTabletJobResponse* response, + ::google::protobuf::Closure* done) override; + + void finish_tablet_job(::google::protobuf::RpcController* controller, + const FinishTabletJobRequest* request, FinishTabletJobResponse* response, + ::google::protobuf::Closure* done) override; + + void http(::google::protobuf::RpcController* controller, const MetaServiceHttpRequest* request, + MetaServiceHttpResponse* response, ::google::protobuf::Closure* done) override; + + void get_obj_store_info(google::protobuf::RpcController* controller, + const GetObjStoreInfoRequest* request, + GetObjStoreInfoResponse* response, + ::google::protobuf::Closure* done) override; + + void alter_obj_store_info(google::protobuf::RpcController* controller, + const AlterObjStoreInfoRequest* request, + AlterObjStoreInfoResponse* response, + ::google::protobuf::Closure* done) override; + + void update_ak_sk(google::protobuf::RpcController* controller, const UpdateAkSkRequest* request, + UpdateAkSkResponse* response, ::google::protobuf::Closure* done) override; + + void create_instance(google::protobuf::RpcController* controller, + const CreateInstanceRequest* request, CreateInstanceResponse* response, + ::google::protobuf::Closure* done) override; + + void alter_instance(google::protobuf::RpcController* controller, + const AlterInstanceRequest* request, AlterInstanceResponse* response, + ::google::protobuf::Closure* done) override; + + void get_instance(google::protobuf::RpcController* controller, + const GetInstanceRequest* request, GetInstanceResponse* response, + ::google::protobuf::Closure* done) override; + + void alter_cluster(google::protobuf::RpcController* controller, + const AlterClusterRequest* request, AlterClusterResponse* response, + ::google::protobuf::Closure* done) override; + + void get_cluster(google::protobuf::RpcController* controller, const GetClusterRequest* request, + GetClusterResponse* response, ::google::protobuf::Closure* done) override; + + void create_stage(google::protobuf::RpcController* controller, + const CreateStageRequest* request, CreateStageResponse* response, + ::google::protobuf::Closure* done) override; + + void get_stage(google::protobuf::RpcController* controller, const GetStageRequest* request, + GetStageResponse* response, ::google::protobuf::Closure* done) override; + + void drop_stage(google::protobuf::RpcController* controller, const DropStageRequest* request, + DropStageResponse* response, ::google::protobuf::Closure* done) override; + + void get_iam(google::protobuf::RpcController* controller, const GetIamRequest* request, + GetIamResponse* response, ::google::protobuf::Closure* done) override; + + void alter_iam(google::protobuf::RpcController* controller, const AlterIamRequest* request, + AlterIamResponse* response, ::google::protobuf::Closure* done) override; + + void alter_ram_user(google::protobuf::RpcController* controller, + const AlterRamUserRequest* request, AlterRamUserResponse* response, + ::google::protobuf::Closure* done) override; + + void begin_copy(google::protobuf::RpcController* controller, const BeginCopyRequest* request, + BeginCopyResponse* response, ::google::protobuf::Closure* done) override; + + void finish_copy(google::protobuf::RpcController* controller, const FinishCopyRequest* request, + FinishCopyResponse* response, ::google::protobuf::Closure* done) override; + + void get_copy_job(google::protobuf::RpcController* controller, const GetCopyJobRequest* request, + GetCopyJobResponse* response, ::google::protobuf::Closure* done) override; + + void get_copy_files(google::protobuf::RpcController* controller, + const GetCopyFilesRequest* request, GetCopyFilesResponse* response, + ::google::protobuf::Closure* done) override; + + // filter files that are loading or loaded in the input files, return files that are not loaded + void filter_copy_files(google::protobuf::RpcController* controller, + const FilterCopyFilesRequest* request, FilterCopyFilesResponse* response, + ::google::protobuf::Closure* done) override; + + void update_delete_bitmap(google::protobuf::RpcController* controller, + const UpdateDeleteBitmapRequest* request, + UpdateDeleteBitmapResponse* response, + ::google::protobuf::Closure* done) override; + void get_delete_bitmap(google::protobuf::RpcController* controller, + const GetDeleteBitmapRequest* request, GetDeleteBitmapResponse* response, + ::google::protobuf::Closure* done) override; + + void get_delete_bitmap_update_lock(google::protobuf::RpcController* controller, + const GetDeleteBitmapUpdateLockRequest* request, + GetDeleteBitmapUpdateLockResponse* response, + ::google::protobuf::Closure* done) override; + + // cloud control get cluster's status by this api + void get_cluster_status(google::protobuf::RpcController* controller, + const GetClusterStatusRequest* request, + GetClusterStatusResponse* response, + ::google::protobuf::Closure* done) override; + + // ATTN: If you add a new method, please also add the corresponding implementation in `MetaServiceProxy`. + + std::pair get_instance_info(const std::string& instance_id, + const std::string& cloud_unique_id, + InstanceInfoPB* instance); + +private: + std::pair alter_instance( + const AlterInstanceRequest* request, + std::function(InstanceInfoPB*)> action); + + std::shared_ptr txn_kv_; + std::shared_ptr resource_mgr_; + std::shared_ptr rate_limiter_; +}; + +class MetaServiceProxy final : public MetaService { +public: + MetaServiceProxy(std::unique_ptr service) : impl_(std::move(service)) {} + ~MetaServiceProxy() override = default; + MetaServiceProxy(const MetaServiceProxy&) = delete; + MetaServiceProxy& operator=(const MetaServiceProxy&) = delete; + + [[nodiscard]] const std::shared_ptr& txn_kv() const { return impl_->txn_kv(); } + [[nodiscard]] const std::shared_ptr& rate_limiter() const { + return impl_->rate_limiter(); + } + [[nodiscard]] const std::shared_ptr& resource_mgr() const { + return impl_->resource_mgr(); + } + + void begin_txn(::google::protobuf::RpcController* controller, const BeginTxnRequest* request, + BeginTxnResponse* response, ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::begin_txn, controller, request, response, done); + } + + void precommit_txn(::google::protobuf::RpcController* controller, + const PrecommitTxnRequest* request, PrecommitTxnResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::precommit_txn, controller, request, response, done); + } + + void commit_txn(::google::protobuf::RpcController* controller, const CommitTxnRequest* request, + CommitTxnResponse* response, ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::commit_txn, controller, request, response, done); + } + + void abort_txn(::google::protobuf::RpcController* controller, const AbortTxnRequest* request, + AbortTxnResponse* response, ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::abort_txn, controller, request, response, done); + } + + void get_txn(::google::protobuf::RpcController* controller, const GetTxnRequest* request, + GetTxnResponse* response, ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::get_txn, controller, request, response, done); + } + + void get_current_max_txn_id(::google::protobuf::RpcController* controller, + const GetCurrentMaxTxnRequest* request, + GetCurrentMaxTxnResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::get_current_max_txn_id, controller, request, response, done); + } + + void check_txn_conflict(::google::protobuf::RpcController* controller, + const CheckTxnConflictRequest* request, + CheckTxnConflictResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::check_txn_conflict, controller, request, response, done); + } + + void clean_txn_label(::google::protobuf::RpcController* controller, + const CleanTxnLabelRequest* request, CleanTxnLabelResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::clean_txn_label, controller, request, response, done); + } + + void get_version(::google::protobuf::RpcController* controller, + const GetVersionRequest* request, GetVersionResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::get_version, controller, request, response, done); + } + + void create_tablets(::google::protobuf::RpcController* controller, + const CreateTabletsRequest* request, CreateTabletsResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::create_tablets, controller, request, response, done); + } + + void update_tablet(::google::protobuf::RpcController* controller, + const UpdateTabletRequest* request, UpdateTabletResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::update_tablet, controller, request, response, done); + } + + void update_tablet_schema(::google::protobuf::RpcController* controller, + const UpdateTabletSchemaRequest* request, + UpdateTabletSchemaResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::update_tablet_schema, controller, request, response, done); + } + + void get_tablet(::google::protobuf::RpcController* controller, const GetTabletRequest* request, + GetTabletResponse* response, ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::get_tablet, controller, request, response, done); + } + + void prepare_rowset(::google::protobuf::RpcController* controller, + const CreateRowsetRequest* request, CreateRowsetResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::prepare_rowset, controller, request, response, done); + } + + void commit_rowset(::google::protobuf::RpcController* controller, + const CreateRowsetRequest* request, CreateRowsetResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::commit_rowset, controller, request, response, done); + } + + void update_tmp_rowset(::google::protobuf::RpcController* controller, + const CreateRowsetRequest* request, CreateRowsetResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::update_tmp_rowset, controller, request, response, done); + } + + void get_rowset(::google::protobuf::RpcController* controller, const GetRowsetRequest* request, + GetRowsetResponse* response, ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::get_rowset, controller, request, response, done); + } + + void prepare_index(::google::protobuf::RpcController* controller, const IndexRequest* request, + IndexResponse* response, ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::prepare_index, controller, request, response, done); + } + + void commit_index(::google::protobuf::RpcController* controller, const IndexRequest* request, + IndexResponse* response, ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::commit_index, controller, request, response, done); + } + + void drop_index(::google::protobuf::RpcController* controller, const IndexRequest* request, + IndexResponse* response, ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::drop_index, controller, request, response, done); + } + + void prepare_partition(::google::protobuf::RpcController* controller, + const PartitionRequest* request, PartitionResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::prepare_partition, controller, request, response, done); + } + + void commit_partition(::google::protobuf::RpcController* controller, + const PartitionRequest* request, PartitionResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::commit_partition, controller, request, response, done); + } + + void drop_partition(::google::protobuf::RpcController* controller, + const PartitionRequest* request, PartitionResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::drop_partition, controller, request, response, done); + } + + void get_tablet_stats(::google::protobuf::RpcController* controller, + const GetTabletStatsRequest* request, GetTabletStatsResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::get_tablet_stats, controller, request, response, done); + } + + void start_tablet_job(::google::protobuf::RpcController* controller, + const StartTabletJobRequest* request, StartTabletJobResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::start_tablet_job, controller, request, response, done); + } + + void finish_tablet_job(::google::protobuf::RpcController* controller, + const FinishTabletJobRequest* request, FinishTabletJobResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::finish_tablet_job, controller, request, response, done); + } + + void http(::google::protobuf::RpcController* controller, const MetaServiceHttpRequest* request, + MetaServiceHttpResponse* response, ::google::protobuf::Closure* done) override { + impl_->http(controller, request, response, done); + } + + void get_obj_store_info(google::protobuf::RpcController* controller, + const GetObjStoreInfoRequest* request, + GetObjStoreInfoResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::get_obj_store_info, controller, request, response, done); + } + + void alter_obj_store_info(google::protobuf::RpcController* controller, + const AlterObjStoreInfoRequest* request, + AlterObjStoreInfoResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::alter_obj_store_info, controller, request, response, done); + } + + void update_ak_sk(google::protobuf::RpcController* controller, const UpdateAkSkRequest* request, + UpdateAkSkResponse* response, ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::update_ak_sk, controller, request, response, done); + } + + void create_instance(google::protobuf::RpcController* controller, + const CreateInstanceRequest* request, CreateInstanceResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::create_instance, controller, request, response, done); + } + + void get_instance(google::protobuf::RpcController* controller, + const GetInstanceRequest* request, GetInstanceResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::get_instance, controller, request, response, done); + } + + void alter_instance(google::protobuf::RpcController* controller, + const AlterInstanceRequest* request, AlterInstanceResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::alter_instance, controller, request, response, done); + } + + void alter_cluster(google::protobuf::RpcController* controller, + const AlterClusterRequest* request, AlterClusterResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::alter_cluster, controller, request, response, done); + } + + void get_cluster(google::protobuf::RpcController* controller, const GetClusterRequest* request, + GetClusterResponse* response, ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::get_cluster, controller, request, response, done); + } + + void create_stage(google::protobuf::RpcController* controller, + const CreateStageRequest* request, CreateStageResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::create_stage, controller, request, response, done); + } + + void get_stage(google::protobuf::RpcController* controller, const GetStageRequest* request, + GetStageResponse* response, ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::get_stage, controller, request, response, done); + } + + void drop_stage(google::protobuf::RpcController* controller, const DropStageRequest* request, + DropStageResponse* response, ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::drop_stage, controller, request, response, done); + } + + void get_iam(google::protobuf::RpcController* controller, const GetIamRequest* request, + GetIamResponse* response, ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::get_iam, controller, request, response, done); + } + + void alter_iam(google::protobuf::RpcController* controller, const AlterIamRequest* request, + AlterIamResponse* response, ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::alter_iam, controller, request, response, done); + } + + void alter_ram_user(google::protobuf::RpcController* controller, + const AlterRamUserRequest* request, AlterRamUserResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::alter_ram_user, controller, request, response, done); + } + + void begin_copy(google::protobuf::RpcController* controller, const BeginCopyRequest* request, + BeginCopyResponse* response, ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::begin_copy, controller, request, response, done); + } + + void finish_copy(google::protobuf::RpcController* controller, const FinishCopyRequest* request, + FinishCopyResponse* response, ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::finish_copy, controller, request, response, done); + } + + void get_copy_job(google::protobuf::RpcController* controller, const GetCopyJobRequest* request, + GetCopyJobResponse* response, ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::get_copy_job, controller, request, response, done); + } + + void get_copy_files(google::protobuf::RpcController* controller, + const GetCopyFilesRequest* request, GetCopyFilesResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::get_copy_files, controller, request, response, done); + } + + // filter files that are loading or loaded in the input files, return files that are not loaded + void filter_copy_files(google::protobuf::RpcController* controller, + const FilterCopyFilesRequest* request, FilterCopyFilesResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::filter_copy_files, controller, request, response, done); + } + + void update_delete_bitmap(google::protobuf::RpcController* controller, + const UpdateDeleteBitmapRequest* request, + UpdateDeleteBitmapResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::update_delete_bitmap, controller, request, response, done); + } + + void get_delete_bitmap(google::protobuf::RpcController* controller, + const GetDeleteBitmapRequest* request, GetDeleteBitmapResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::get_delete_bitmap, controller, request, response, done); + } + + void get_delete_bitmap_update_lock(google::protobuf::RpcController* controller, + const GetDeleteBitmapUpdateLockRequest* request, + GetDeleteBitmapUpdateLockResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::get_delete_bitmap_update_lock, controller, request, response, + done); + } + + // cloud control get cluster's status by this api + void get_cluster_status(google::protobuf::RpcController* controller, + const GetClusterStatusRequest* request, + GetClusterStatusResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::get_cluster_status, controller, request, response, done); + } + +private: + template + using MetaServiceMethod = void (cloud::MetaService::*)(::google::protobuf::RpcController*, + const Request*, Response*, + ::google::protobuf::Closure*); + + template + void call_impl(MetaServiceMethod method, + ::google::protobuf::RpcController* ctrl, const Request* req, Response* resp, + ::google::protobuf::Closure* done) { + static_assert(std::is_base_of_v<::google::protobuf::Message, Request>); + static_assert(std::is_base_of_v<::google::protobuf::Message, Response>); + + brpc::ClosureGuard done_guard(done); + if (!config::enable_txn_store_retry) { + (impl_.get()->*method)(ctrl, req, resp, brpc::DoNothing()); + if (DCHECK_IS_ON()) { + MetaServiceCode code = resp->status().code(); + DCHECK_NE(code, MetaServiceCode::KV_TXN_STORE_GET_RETRYABLE) + << "KV_TXN_STORE_GET_RETRYABLE should not be sent back to client"; + DCHECK_NE(code, MetaServiceCode::KV_TXN_STORE_COMMIT_RETRYABLE) + << "KV_TXN_STORE_COMMIT_RETRYABLE should not be sent back to client"; + DCHECK_NE(code, MetaServiceCode::KV_TXN_STORE_CREATE_RETRYABLE) + << "KV_TXN_STORE_CREATE_RETRYABLE should not be sent back to client"; + } + return; + } + + TEST_SYNC_POINT("MetaServiceProxy::call_impl:1"); + + int32_t retry_times = config::txn_store_retry_times; + uint64_t duration_ms = 0; + std::uniform_int_distribution u(20, 200); + std::uniform_int_distribution u2(500, 1000); + auto rng = std::default_random_engine { + static_cast(std::chrono::steady_clock::now().time_since_epoch().count())}; + while (true) { + (impl_.get()->*method)(ctrl, req, resp, brpc::DoNothing()); + MetaServiceCode code = resp->status().code(); + if (code != MetaServiceCode::KV_TXN_STORE_GET_RETRYABLE && + code != MetaServiceCode::KV_TXN_STORE_COMMIT_RETRYABLE && + code != MetaServiceCode::KV_TXN_STORE_CREATE_RETRYABLE) { + return; + } + + TEST_SYNC_POINT("MetaServiceProxy::call_impl:2"); + if (--retry_times < 0) { + resp->mutable_status()->set_code( + code == MetaServiceCode::KV_TXN_STORE_COMMIT_RETRYABLE ? KV_TXN_COMMIT_ERR + : code == MetaServiceCode::KV_TXN_STORE_GET_RETRYABLE ? KV_TXN_GET_ERR + : KV_TXN_CREATE_ERR); + return; + } + + duration_ms = retry_times > 10 ? u(rng) : u2(rng); + TEST_SYNC_POINT_CALLBACK("MetaServiceProxy::call_impl_duration_ms", &duration_ms); + LOG(WARNING) << __PRETTY_FUNCTION__ << " sleep " << duration_ms + << " ms before next round, retry times left: " << retry_times + << ", code: " << MetaServiceCode_Name(code) + << ", msg: " << resp->status().msg(); + bthread_usleep(duration_ms * 1000); + } + } + + std::unique_ptr impl_; +}; + +} // namespace doris::cloud \ No newline at end of file diff --git a/cloud/src/meta-service/meta_service_helper.h b/cloud/src/meta-service/meta_service_helper.h new file mode 100644 index 00000000000000..86d500b2cd0344 --- /dev/null +++ b/cloud/src/meta-service/meta_service_helper.h @@ -0,0 +1,195 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include + +#include +#include +#include + +#include "common/bvars.h" +#include "common/config.h" +#include "common/logging.h" +#include "common/stopwatch.h" +#include "common/util.h" +#include "meta-service/keys.h" +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" +#include "resource-manager/resource_manager.h" + +namespace doris::cloud { + +template +void begin_rpc(std::string_view func_name, brpc::Controller* ctrl, const Request* req) { + if constexpr (std::is_same_v) { + LOG(INFO) << "begin " << func_name << " from " << ctrl->remote_side(); + } else if constexpr (std::is_same_v) { + LOG(INFO) << "begin " << func_name << " from " << ctrl->remote_side(); + } else if constexpr (std::is_same_v) { + LOG(INFO) << "begin " << func_name << " from " << ctrl->remote_side() + << " tablet_id=" << req->tablet_id() << " lock_id=" << req->lock_id() + << " initiator=" << req->initiator() + << " delete_bitmap_size=" << req->segment_delete_bitmaps_size(); + } else if constexpr (std::is_same_v) { + LOG(INFO) << "begin " << func_name << " from " << ctrl->remote_side() + << " tablet_id=" << req->tablet_id() << " rowset_size=" << req->rowset_ids_size(); + } else if constexpr (std::is_same_v) { + VLOG_DEBUG << "begin " << func_name << " from " << ctrl->remote_side() + << " tablet size: " << req->tablet_idx().size(); + } else if constexpr (std::is_same_v || + std::is_same_v || + std::is_same_v) { + VLOG_DEBUG << "begin " << func_name << " from " << ctrl->remote_side() + << " request=" << req->ShortDebugString(); + } else { + LOG(INFO) << "begin " << func_name << " from " << ctrl->remote_side() + << " request=" << req->ShortDebugString(); + } +} + +template +void finish_rpc(std::string_view func_name, brpc::Controller* ctrl, Response* res) { + if constexpr (std::is_same_v) { + if (res->status().code() != MetaServiceCode::OK) { + res->clear_table_ids(); + res->clear_partition_ids(); + res->clear_versions(); + } + LOG(INFO) << "finish " << func_name << " from " << ctrl->remote_side() + << " response=" << res->ShortDebugString(); + } else if constexpr (std::is_same_v) { + if (res->status().code() != MetaServiceCode::OK) { + res->clear_rowset_meta(); + } + VLOG_DEBUG << "finish " << func_name << " from " << ctrl->remote_side() + << " status=" << res->status().ShortDebugString(); + } else if constexpr (std::is_same_v) { + VLOG_DEBUG << "finish " << func_name << " from " << ctrl->remote_side() + << " status=" << res->status().ShortDebugString() + << " tablet size: " << res->tablet_stats().size(); + } else if constexpr (std::is_same_v || + std::is_same_v) { + VLOG_DEBUG << "finish " << func_name << " from " << ctrl->remote_side() + << " response=" << res->ShortDebugString(); + } else if constexpr (std::is_same_v) { + LOG(INFO) << "finish " << func_name << " from " << ctrl->remote_side() + << " status=" << res->status().ShortDebugString() + << " delete_bitmap_size=" << res->segment_delete_bitmaps_size(); + + } else { + LOG(INFO) << "finish " << func_name << " from " << ctrl->remote_side() + << " response=" << res->ShortDebugString(); + } +} + +enum ErrCategory { CREATE, READ, COMMIT }; + +template +inline MetaServiceCode cast_as(TxnErrorCode code) { + switch (code) { + case TxnErrorCode::TXN_OK: + return MetaServiceCode::OK; + case TxnErrorCode::TXN_CONFLICT: + return MetaServiceCode::KV_TXN_CONFLICT; + case TxnErrorCode::TXN_TOO_OLD: + case TxnErrorCode::TXN_RETRYABLE_NOT_COMMITTED: + if (config::enable_txn_store_retry) { + if constexpr (category == ErrCategory::READ) { + return MetaServiceCode::KV_TXN_STORE_GET_RETRYABLE; + } else { + return MetaServiceCode::KV_TXN_STORE_COMMIT_RETRYABLE; + } + } + [[fallthrough]]; + case TxnErrorCode::TXN_KEY_NOT_FOUND: + case TxnErrorCode::TXN_MAYBE_COMMITTED: + case TxnErrorCode::TXN_TIMEOUT: + case TxnErrorCode::TXN_INVALID_ARGUMENT: + case TxnErrorCode::TXN_UNIDENTIFIED_ERROR: + case TxnErrorCode::TXN_KEY_TOO_LARGE: + case TxnErrorCode::TXN_VALUE_TOO_LARGE: + case TxnErrorCode::TXN_BYTES_TOO_LARGE: + if constexpr (category == ErrCategory::READ) { + return MetaServiceCode::KV_TXN_GET_ERR; + } else if constexpr (category == ErrCategory::CREATE) { + return MetaServiceCode::KV_TXN_CREATE_ERR; + } else { + return MetaServiceCode::KV_TXN_COMMIT_ERR; + } + default: + return MetaServiceCode::UNDEFINED_ERR; + } +} + +#define RPC_PREPROCESS(func_name) \ + StopWatch sw; \ + auto ctrl = static_cast(controller); \ + begin_rpc(#func_name, ctrl, request); \ + brpc::ClosureGuard closure_guard(done); \ + [[maybe_unused]] std::stringstream ss; \ + [[maybe_unused]] MetaServiceCode code = MetaServiceCode::OK; \ + [[maybe_unused]] std::string msg; \ + [[maybe_unused]] std::string instance_id; \ + [[maybe_unused]] bool drop_request = false; \ + std::unique_ptr> defer_status((int*)0x01, [&](int*) { \ + response->mutable_status()->set_code(code); \ + response->mutable_status()->set_msg(msg); \ + finish_rpc(#func_name, ctrl, response); \ + closure_guard.reset(nullptr); \ + if (config::use_detailed_metrics && !instance_id.empty() && !drop_request) { \ + g_bvar_ms_##func_name.put(instance_id, sw.elapsed_us()); \ + } \ + }); + +#define RPC_RATE_LIMIT(func_name) \ + if (config::enable_rate_limit && config::use_detailed_metrics && !instance_id.empty()) { \ + auto rate_limiter = rate_limiter_->get_rpc_rate_limiter(#func_name); \ + assert(rate_limiter != nullptr); \ + std::function get_bvar_qps = [&] { \ + return g_bvar_ms_##func_name.get(instance_id)->qps(); \ + }; \ + if (!rate_limiter->get_qps_token(instance_id, get_bvar_qps)) { \ + drop_request = true; \ + code = MetaServiceCode::MAX_QPS_LIMIT; \ + msg = "reach max qps limit"; \ + return; \ + } \ + } + +// FIXME(gavin): should it be a member function of ResourceManager? +std::string get_instance_id(const std::shared_ptr& rc_mgr, + const std::string& cloud_unique_id); + +int decrypt_instance_info(InstanceInfoPB& instance, const std::string& instance_id, + MetaServiceCode& code, std::string& msg, + std::shared_ptr& txn); + +/** + * Notifies other metaservice to refresh instance + */ +void notify_refresh_instance(std::shared_ptr txn_kv, const std::string& instance_id); + +void get_tablet_idx(MetaServiceCode& code, std::string& msg, Transaction* txn, + const std::string& instance_id, int64_t tablet_id, TabletIndexPB& tablet_idx); + +bool is_dropped_tablet(Transaction* txn, const std::string& instance_id, int64_t index_id, + int64_t partition_id); + +} // namespace doris::cloud diff --git a/cloud/src/meta-service/meta_service_http.cpp b/cloud/src/meta-service/meta_service_http.cpp new file mode 100644 index 00000000000000..6116664f4e630b --- /dev/null +++ b/cloud/src/meta-service/meta_service_http.cpp @@ -0,0 +1,517 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "meta_service_http.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +#include "common/config.h" +#include "common/logging.h" +#include "meta_service.h" + +namespace doris::cloud { + +#define PARSE_MESSAGE_OR_RETURN(ctrl, req) \ + do { \ + std::string body = ctrl->request_attachment().to_string(); \ + auto& unresolved_path = ctrl->http_request().unresolved_path(); \ + auto st = parse_json_message(unresolved_path, body, &req); \ + if (!st.ok()) { \ + std::string msg = "parse http request '" + unresolved_path + "': " + st.ToString(); \ + LOG_WARNING(msg).tag("body", body); \ + return http_json_reply(MetaServiceCode::PROTOBUF_PARSE_ERR, msg); \ + } \ + } while (0) + +extern std::string get_instance_id(const std::shared_ptr& rc_mgr, + const std::string& cloud_unique_id); + +extern int decrypt_instance_info(InstanceInfoPB& instance, const std::string& instance_id, + MetaServiceCode& code, std::string& msg, + std::shared_ptr& txn); + +template +static google::protobuf::util::Status parse_json_message(const std::string& unresolved_path, + const std::string& body, Message* req) { + static_assert(std::is_base_of_v); + auto st = google::protobuf::util::JsonStringToMessage(body, req); + if (!st.ok()) { + std::string msg = "failed to strictly parse http request for '" + unresolved_path + + "' error: " + st.ToString(); + LOG_WARNING(msg).tag("body", body); + + // ignore unknown fields + google::protobuf::util::JsonParseOptions json_parse_options; + json_parse_options.ignore_unknown_fields = true; + return google::protobuf::util::JsonStringToMessage(body, req, json_parse_options); + } + return {}; +} + +std::tuple convert_ms_code_to_http_code(MetaServiceCode ret) { + switch (ret) { + case OK: + return {200, "OK"}; + case INVALID_ARGUMENT: + case PROTOBUF_PARSE_ERR: + return {400, "INVALID_ARGUMENT"}; + case CLUSTER_NOT_FOUND: + return {404, "NOT_FOUND"}; + case ALREADY_EXISTED: + return {409, "ALREADY_EXISTED"}; + case KV_TXN_CREATE_ERR: + case KV_TXN_GET_ERR: + case KV_TXN_COMMIT_ERR: + case PROTOBUF_SERIALIZE_ERR: + case TXN_GEN_ID_ERR: + case TXN_DUPLICATED_REQ: + case TXN_LABEL_ALREADY_USED: + case TXN_INVALID_STATUS: + case TXN_LABEL_NOT_FOUND: + case TXN_ID_NOT_FOUND: + case TXN_ALREADY_ABORTED: + case TXN_ALREADY_VISIBLE: + case TXN_ALREADY_PRECOMMITED: + case VERSION_NOT_FOUND: + case UNDEFINED_ERR: + default: + return {500, "INTERNAL_ERROR"}; + } +} + +HttpResponse http_json_reply(MetaServiceCode code, const std::string& msg, + std::optional body) { + auto [status_code, status_msg] = convert_ms_code_to_http_code(code); + rapidjson::Document d; + d.SetObject(); + if (code == MetaServiceCode::OK) { + d.AddMember("code", "OK", d.GetAllocator()); + d.AddMember("msg", "", d.GetAllocator()); + } else { + d.AddMember("code", rapidjson::StringRef(status_msg.data(), status_msg.size()), + d.GetAllocator()); + d.AddMember("msg", rapidjson::StringRef(msg.data(), msg.size()), d.GetAllocator()); + } + + rapidjson::Document result; + if (body.has_value()) { + rapidjson::ParseResult ok = result.Parse(body->c_str()); + if (!ok) { + LOG_WARNING("JSON parse error") + .tag("code", rapidjson::GetParseError_En(ok.Code())) + .tag("offset", ok.Offset()); + d.AddMember("code", "INTERNAL_ERROR", d.GetAllocator()); + d.AddMember("msg", "JSON parse error", d.GetAllocator()); + } else { + d.AddMember("result", result, d.GetAllocator()); + } + } + + rapidjson::StringBuffer sb; + rapidjson::PrettyWriter writer(sb); + d.Accept(writer); + return {status_code, msg, sb.GetString()}; +} + +static std::string format_http_request(const brpc::HttpHeader& request) { + auto& unresolved_path = request.unresolved_path(); + auto& uri = request.uri(); + std::stringstream ss; + ss << "\nuri_path=" << uri.path(); + ss << "\nunresolved_path=" << unresolved_path; + ss << "\nmethod=" << brpc::HttpMethod2Str(request.method()); + ss << "\nquery strings:"; + for (auto it = uri.QueryBegin(); it != uri.QueryEnd(); ++it) { + ss << "\n" << it->first << "=" << it->second; + } + ss << "\nheaders:"; + for (auto it = request.HeaderBegin(); it != request.HeaderEnd(); ++it) { + ss << "\n" << it->first << ":" << it->second; + } + return ss.str(); +} + +static std::string_view remove_version_prefix(std::string_view path) { + if (path.size() > 3 && path.substr(0, 3) == "v1/") path.remove_prefix(3); + return path; +} + +static HttpResponse process_alter_cluster(MetaServiceImpl* service, brpc::Controller* ctrl) { + static std::unordered_map operations { + {"add_cluster", AlterClusterRequest::ADD_CLUSTER}, + {"drop_cluster", AlterClusterRequest::DROP_CLUSTER}, + {"rename_cluster", AlterClusterRequest::RENAME_CLUSTER}, + {"update_cluster_endpoint", AlterClusterRequest::UPDATE_CLUSTER_ENDPOINT}, + {"update_cluster_mysql_user_name", AlterClusterRequest::UPDATE_CLUSTER_MYSQL_USER_NAME}, + {"add_node", AlterClusterRequest::ADD_NODE}, + {"drop_node", AlterClusterRequest::DROP_NODE}, + {"decommission_node", AlterClusterRequest::DECOMMISSION_NODE}, + {"set_cluster_status", AlterClusterRequest::SET_CLUSTER_STATUS}, + {"notify_decommissioned", AlterClusterRequest::NOTIFY_DECOMMISSIONED}, + }; + + auto& path = ctrl->http_request().unresolved_path(); + auto body = ctrl->request_attachment().to_string(); + auto it = operations.find(remove_version_prefix(path)); + if (it == operations.end()) { + std::string msg = "not supportted alter cluster operation: " + path; + return http_json_reply(MetaServiceCode::INVALID_ARGUMENT, msg); + } + + AlterClusterRequest req; + PARSE_MESSAGE_OR_RETURN(ctrl, req); + + req.set_op(it->second); + AlterClusterResponse resp; + service->alter_cluster(ctrl, &req, &resp, nullptr); + return http_json_reply(resp.status()); +} + +static HttpResponse process_get_obj_store_info(MetaServiceImpl* service, brpc::Controller* ctrl) { + GetObjStoreInfoRequest req; + PARSE_MESSAGE_OR_RETURN(ctrl, req); + + GetObjStoreInfoResponse resp; + service->get_obj_store_info(ctrl, &req, &resp, nullptr); + return http_json_reply_message(resp.status(), resp); +} + +static HttpResponse process_alter_obj_store_info(MetaServiceImpl* service, brpc::Controller* ctrl) { + static std::unordered_map operations { + {"add_obj_info", AlterObjStoreInfoRequest::ADD_OBJ_INFO}, + {"legacy_update_ak_sk", AlterObjStoreInfoRequest::LEGACY_UPDATE_AK_SK}, + }; + + auto& path = ctrl->http_request().unresolved_path(); + auto it = operations.find(remove_version_prefix(path)); + if (it == operations.end()) { + std::string msg = "not supportted alter obj store info operation: " + path; + return http_json_reply(MetaServiceCode::INVALID_ARGUMENT, msg); + } + + AlterObjStoreInfoRequest req; + PARSE_MESSAGE_OR_RETURN(ctrl, req); + req.set_op(it->second); + + AlterObjStoreInfoResponse resp; + service->alter_obj_store_info(ctrl, &req, &resp, nullptr); + return http_json_reply(resp.status()); +} + +static HttpResponse process_update_ak_sk(MetaServiceImpl* service, brpc::Controller* ctrl) { + UpdateAkSkRequest req; + PARSE_MESSAGE_OR_RETURN(ctrl, req); + UpdateAkSkResponse resp; + service->update_ak_sk(ctrl, &req, &resp, nullptr); + return http_json_reply(resp.status()); +} + +static HttpResponse process_create_instance(MetaServiceImpl* service, brpc::Controller* ctrl) { + CreateInstanceRequest req; + PARSE_MESSAGE_OR_RETURN(ctrl, req); + CreateInstanceResponse resp; + service->create_instance(ctrl, &req, &resp, nullptr); + return http_json_reply(resp.status()); +} + +static HttpResponse process_alter_instance(MetaServiceImpl* service, brpc::Controller* ctrl) { + static std::unordered_map> + operations {{"rename_instance", {AlterInstanceRequest::RENAME}}, + {"enable_instance_sse", {AlterInstanceRequest::ENABLE_SSE}}, + {"disable_instance_sse", {AlterInstanceRequest::DISABLE_SSE}}, + {"drop_instance", {AlterInstanceRequest::DROP}}, + {"set_instance_status", + {AlterInstanceRequest::SET_NORMAL, AlterInstanceRequest::SET_OVERDUE}}}; + + auto& path = ctrl->http_request().unresolved_path(); + auto it = operations.find(remove_version_prefix(path)); + if (it == operations.end()) { + std::string msg = "not supportted alter instance operation: '" + path + + "', remove version prefix=" + std::string(remove_version_prefix(path)); + return http_json_reply(MetaServiceCode::INVALID_ARGUMENT, msg); + } + + AlterInstanceRequest req; + PARSE_MESSAGE_OR_RETURN(ctrl, req); + // for unresolved path whose corresponding operation is signal, we need set operation by ourselves. + if ((it->second).size() == 1) { + req.set_op((it->second)[0]); + } + AlterInstanceResponse resp; + service->alter_instance(ctrl, &req, &resp, nullptr); + return http_json_reply(resp.status()); +} + +static HttpResponse process_abort_txn(MetaServiceImpl* service, brpc::Controller* ctrl) { + AbortTxnRequest req; + PARSE_MESSAGE_OR_RETURN(ctrl, req); + AbortTxnResponse resp; + service->abort_txn(ctrl, &req, &resp, nullptr); + return http_json_reply(resp.status()); +} + +static HttpResponse process_abort_tablet_job(MetaServiceImpl* service, brpc::Controller* ctrl) { + FinishTabletJobRequest req; + PARSE_MESSAGE_OR_RETURN(ctrl, req); + req.set_action(FinishTabletJobRequest::ABORT); + FinishTabletJobResponse resp; + service->finish_tablet_job(ctrl, &req, &resp, nullptr); + return http_json_reply(resp.status()); +} + +static HttpResponse process_alter_ram_user(MetaServiceImpl* service, brpc::Controller* ctrl) { + AlterRamUserRequest req; + PARSE_MESSAGE_OR_RETURN(ctrl, req); + AlterRamUserResponse resp; + service->alter_ram_user(ctrl, &req, &resp, nullptr); + return http_json_reply(resp.status()); +} + +static HttpResponse process_alter_iam(MetaServiceImpl* service, brpc::Controller* ctrl) { + AlterIamRequest req; + PARSE_MESSAGE_OR_RETURN(ctrl, req); + AlterIamResponse resp; + service->alter_iam(ctrl, &req, &resp, nullptr); + return http_json_reply(resp.status()); +} + +static HttpResponse process_decode_key(MetaServiceImpl*, brpc::Controller* ctrl) { + auto& uri = ctrl->http_request().uri(); + std::string_view key = http_query(uri, "key"); + if (key.empty()) { + return http_json_reply(MetaServiceCode::INVALID_ARGUMENT, "no key to decode"); + } + + bool unicode = http_query(uri, "unicode") != "false"; + std::string body = prettify_key(key, unicode); + if (body.empty()) { + std::string msg = "failed to decode key, key=" + std::string(key); + return http_json_reply(MetaServiceCode::INVALID_ARGUMENT, msg); + } + return http_text_reply(MetaServiceCode::OK, "", body); +} + +static HttpResponse process_encode_key(MetaServiceImpl*, brpc::Controller* ctrl) { + return process_http_encode_key(ctrl->http_request().uri()); +} + +static HttpResponse process_get_value(MetaServiceImpl* service, brpc::Controller* ctrl) { + return process_http_get_value(service->txn_kv().get(), ctrl->http_request().uri()); +} + +static HttpResponse process_get_instance_info(MetaServiceImpl* service, brpc::Controller* ctrl) { + auto& uri = ctrl->http_request().uri(); + std::string_view instance_id = http_query(uri, "instance_id"); + std::string_view cloud_unique_id = http_query(uri, "cloud_unique_id"); + + InstanceInfoPB instance; + auto [code, msg] = service->get_instance_info(std::string(instance_id), + std::string(cloud_unique_id), &instance); + return http_json_reply_message(code, msg, instance); +} + +static HttpResponse process_get_cluster(MetaServiceImpl* service, brpc::Controller* ctrl) { + GetClusterRequest req; + PARSE_MESSAGE_OR_RETURN(ctrl, req); + + bool get_all_cluster_info = false; + // if cluster_id、cluster_name、mysql_user_name all empty, get this instance's all cluster info. + if (req.cluster_id().empty() && req.cluster_name().empty() && req.mysql_user_name().empty()) { + get_all_cluster_info = true; + } + + GetClusterResponse resp; + service->get_cluster(ctrl, &req, &resp, nullptr); + + if (resp.status().code() == MetaServiceCode::OK) { + if (get_all_cluster_info) { + return http_json_reply_message(resp.status(), resp); + } else { + // ATTN: only returns the first cluster pb. + return http_json_reply_message(resp.status(), resp.cluster(0)); + } + } else { + return http_json_reply(resp.status()); + } +} + +static HttpResponse process_get_tablet_stats(MetaServiceImpl* service, brpc::Controller* ctrl) { + GetTabletStatsRequest req; + PARSE_MESSAGE_OR_RETURN(ctrl, req); + GetTabletStatsResponse resp; + service->get_tablet_stats(ctrl, &req, &resp, nullptr); + + std::string body; + if (resp.status().code() == MetaServiceCode::OK) { + body = resp.DebugString(); + } + return http_text_reply(resp.status(), body); +} + +static HttpResponse process_get_stage(MetaServiceImpl* service, brpc::Controller* ctrl) { + GetStageRequest req; + PARSE_MESSAGE_OR_RETURN(ctrl, req); + GetStageResponse resp; + service->get_stage(ctrl, &req, &resp, nullptr); + return http_json_reply_message(resp.status(), resp); +} + +static HttpResponse process_get_cluster_status(MetaServiceImpl* service, brpc::Controller* ctrl) { + GetClusterStatusRequest req; + PARSE_MESSAGE_OR_RETURN(ctrl, req); + GetClusterStatusResponse resp; + service->get_cluster_status(ctrl, &req, &resp, nullptr); + return http_json_reply_message(resp.status(), resp); +} + +static HttpResponse process_unknown(MetaServiceImpl*, brpc::Controller*) { + // ATTN: To be compatible with cloud manager versions higher than this MS + return http_json_reply(MetaServiceCode::OK, ""); +} + +void MetaServiceImpl::http(::google::protobuf::RpcController* controller, + const MetaServiceHttpRequest*, MetaServiceHttpResponse*, + ::google::protobuf::Closure* done) { + using HttpHandler = HttpResponse (*)(MetaServiceImpl*, brpc::Controller*); + static std::unordered_map http_handlers { + // for alter cluster. + {"add_cluster", process_alter_cluster}, + {"drop_cluster", process_alter_cluster}, + {"rename_cluster", process_alter_cluster}, + {"update_cluster_endpoint", process_alter_cluster}, + {"update_cluster_mysql_user_name", process_alter_cluster}, + {"add_node", process_alter_cluster}, + {"drop_node", process_alter_cluster}, + {"decommission_node", process_alter_cluster}, + {"set_cluster_status", process_alter_cluster}, + {"notify_decommissioned", process_alter_cluster}, + {"v1/add_cluster", process_alter_cluster}, + {"v1/drop_cluster", process_alter_cluster}, + {"v1/rename_cluster", process_alter_cluster}, + {"v1/update_cluster_endpoint", process_alter_cluster}, + {"v1/update_cluster_mysql_user_name", process_alter_cluster}, + {"v1/add_node", process_alter_cluster}, + {"v1/drop_node", process_alter_cluster}, + {"v1/decommission_node", process_alter_cluster}, + {"v1/set_cluster_status", process_alter_cluster}, + // for alter instance + {"create_instance", process_create_instance}, + {"drop_instance", process_alter_instance}, + {"rename_instance", process_alter_instance}, + {"enable_instance_sse", process_alter_instance}, + {"disable_instance_sse", process_alter_instance}, + {"set_instance_status", process_alter_instance}, + {"v1/create_instance", process_create_instance}, + {"v1/drop_instance", process_alter_instance}, + {"v1/rename_instance", process_alter_instance}, + {"v1/enable_instance_sse", process_alter_instance}, + {"v1/disable_instance_sse", process_alter_instance}, + {"v1/set_instance_status", process_alter_instance}, + // for alter obj store info + {"add_obj_info", process_alter_obj_store_info}, + {"legacy_update_ak_sk", process_alter_obj_store_info}, + {"update_ak_sk", process_update_ak_sk}, + {"v1/add_obj_info", process_alter_obj_store_info}, + {"v1/legacy_update_ak_sk", process_alter_obj_store_info}, + {"v1/update_ak_sk", process_update_ak_sk}, + // for tools + {"decode_key", process_decode_key}, + {"encode_key", process_encode_key}, + {"get_value", process_get_value}, + {"v1/decode_key", process_decode_key}, + {"v1/encode_key", process_encode_key}, + {"v1/get_value", process_get_value}, + // for get + {"get_instance", process_get_instance_info}, + {"get_obj_store_info", process_get_obj_store_info}, + {"get_cluster", process_get_cluster}, + {"get_tablet_stats", process_get_tablet_stats}, + {"get_stage", process_get_stage}, + {"get_cluster_status", process_get_cluster_status}, + {"v1/get_instance", process_get_instance_info}, + {"v1/get_obj_store_info", process_get_obj_store_info}, + {"v1/get_cluster", process_get_cluster}, + {"v1/get_tablet_stats", process_get_tablet_stats}, + {"v1/get_stage", process_get_stage}, + {"v1/get_cluster_status", process_get_cluster_status}, + // misc + {"abort_txn", process_abort_txn}, + {"abort_tablet_job", process_abort_tablet_job}, + {"alter_ram_user", process_alter_ram_user}, + {"alter_iam", process_alter_iam}, + {"v1/abort_txn", process_abort_txn}, + {"v1/abort_tablet_job", process_abort_tablet_job}, + {"v1/alter_ram_user", process_alter_ram_user}, + {"v1/alter_iam", process_alter_iam}, + }; + + auto cntl = static_cast(controller); + brpc::ClosureGuard closure_guard(done); + + // Prepare input request info + LOG(INFO) << "rpc from " << cntl->remote_side() + << " request: " << cntl->http_request().uri().path(); + std::string http_request = format_http_request(cntl->http_request()); + + // Auth + auto token = http_query(cntl->http_request().uri(), "token"); + if (token != config::http_token) { + std::string body = fmt::format("incorrect token, token={}", + (token.empty() ? std::string_view("(not given)") : token)); + cntl->http_response().set_status_code(403); + cntl->response_attachment().append(body); + cntl->response_attachment().append("\n"); + LOG(WARNING) << "failed to handle http from " << cntl->remote_side() + << " request: " << http_request << " msg: " << body; + return; + } + + // Process http request + auto& unresolved_path = cntl->http_request().unresolved_path(); + HttpHandler handler = process_unknown; + auto it = http_handlers.find(unresolved_path); + if (it != http_handlers.end()) { + handler = it->second; + } + + auto [status_code, msg, body] = handler(this, cntl); + cntl->http_response().set_status_code(status_code); + cntl->response_attachment().append(body); + cntl->response_attachment().append("\n"); + + int ret = cntl->http_response().status_code(); + LOG(INFO) << (ret == 200 ? "succ to " : "failed to ") << __PRETTY_FUNCTION__ << " " + << cntl->remote_side() << " request=\n" + << http_request << "\n ret=" << ret << " msg=" << msg; +} + +} // namespace doris::cloud diff --git a/cloud/src/meta-service/meta_service_http.h b/cloud/src/meta-service/meta_service_http.h new file mode 100644 index 00000000000000..ead53f0630fe15 --- /dev/null +++ b/cloud/src/meta-service/meta_service_http.h @@ -0,0 +1,77 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include + +#include +#include +#include + +#include "common/util.h" + +namespace doris::cloud { + +struct HttpResponse { + int status_code; + std::string msg; + std::string body; +}; + +std::tuple convert_ms_code_to_http_code(MetaServiceCode ret); + +HttpResponse http_json_reply(MetaServiceCode code, const std::string& msg, + std::optional body = {}); + +HttpResponse process_http_get_value(TxnKv* txn_kv, const brpc::URI& uri); + +HttpResponse process_http_encode_key(const brpc::URI& uri); + +/// Return the query value or an empty string if not exists. +inline static std::string_view http_query(const brpc::URI& uri, const char* name) { + return uri.GetQuery(name) ? *uri.GetQuery(name) : std::string_view(); +} + +inline static HttpResponse http_json_reply(const MetaServiceResponseStatus& status, + std::optional body = {}) { + return http_json_reply(status.code(), status.msg(), body); +} + +inline static HttpResponse http_json_reply_message(MetaServiceCode code, const std::string& msg, + const google::protobuf::Message& body) { + return http_json_reply(code, msg, proto_to_json(body)); +} + +inline static HttpResponse http_json_reply_message(const MetaServiceResponseStatus& status, + const google::protobuf::Message& msg) { + return http_json_reply(status, proto_to_json(msg)); +} + +inline static HttpResponse http_text_reply(MetaServiceCode code, const std::string& msg, + const std::string& body) { + auto [status_code, _] = convert_ms_code_to_http_code(code); + return {status_code, msg, body}; +} + +inline static HttpResponse http_text_reply(const MetaServiceResponseStatus& status, + const std::string& body) { + return http_text_reply(status.code(), status.msg(), body); +} + +} // namespace doris::cloud diff --git a/cloud/src/meta-service/meta_service_job.cpp b/cloud/src/meta-service/meta_service_job.cpp new file mode 100644 index 00000000000000..b57fadf999f886 --- /dev/null +++ b/cloud/src/meta-service/meta_service_job.cpp @@ -0,0 +1,1229 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include +#include +#include +#include + +#include +#include + +#include "common/bvars.h" +#include "common/config.h" +#include "common/logging.h" +#include "common/stopwatch.h" +#include "common/sync_point.h" +#include "common/util.h" +#include "meta-service/keys.h" +#include "meta-service/meta_service_helper.h" +#include "meta-service/meta_service_tablet_stats.h" +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" +#include "meta_service.h" + +// Empty string not is not processed +template +static inline constexpr size_t get_file_name_offset(const T (&s)[S], size_t i = S - 1) { + return (s[i] == '/' || s[i] == '\\') ? i + 1 : (i > 0 ? get_file_name_offset(s, i - 1) : 0); +} +#define SS (ss << &__FILE__[get_file_name_offset(__FILE__)] << ":" << __LINE__ << " ") +#define INSTANCE_LOG(severity) (LOG(severity) << '(' << instance_id << ')') + +namespace doris::cloud { + +static constexpr int COMPACTION_DELETE_BITMAP_LOCK_ID = -1; +static constexpr int SCHEMA_CHANGE_DELETE_BITMAP_LOCK_ID = -2; + +void start_compaction_job(MetaServiceCode& code, std::string& msg, std::stringstream& ss, + std::unique_ptr& txn, const StartTabletJobRequest* request, + StartTabletJobResponse* response, std::string& instance_id, + bool& need_commit) { + auto& compaction = request->job().compaction(0); + if (!compaction.has_id() || compaction.id().empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "no job id specified"; + return; + } + + // check compaction_cnt to avoid compact on expired tablet cache + if (!compaction.has_base_compaction_cnt() || !compaction.has_cumulative_compaction_cnt()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "no valid compaction_cnt given"; + return; + } + + if (compaction.expiration() <= 0 && + compaction.type() != TabletCompactionJobPB::EMPTY_CUMULATIVE) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "no valid expiration given"; + return; + } + + if (compaction.lease() <= 0) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "no valid lease given"; + return; + } + + int64_t table_id = request->job().idx().table_id(); + int64_t index_id = request->job().idx().index_id(); + int64_t partition_id = request->job().idx().partition_id(); + int64_t tablet_id = request->job().idx().tablet_id(); + std::string stats_key = + stats_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + std::string stats_val; + TxnErrorCode err = txn->get(stats_key, &stats_val); + if (err != TxnErrorCode::TXN_OK) { + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::TABLET_NOT_FOUND + : cast_as(err); + SS << (err == TxnErrorCode::TXN_KEY_NOT_FOUND ? "not found" : "get kv error") + << " when get tablet stats, tablet_id=" << tablet_id << " key=" << hex(stats_key) + << " err=" << err; + msg = ss.str(); + return; + } + TabletStatsPB stats; + CHECK(stats.ParseFromString(stats_val)); + if (compaction.base_compaction_cnt() < stats.base_compaction_cnt() || + compaction.cumulative_compaction_cnt() < stats.cumulative_compaction_cnt()) { + code = MetaServiceCode::STALE_TABLET_CACHE; + SS << "could not perform compaction on expired tablet cache." + << " req_base_compaction_cnt=" << compaction.base_compaction_cnt() + << ", base_compaction_cnt=" << stats.base_compaction_cnt() + << ", req_cumulative_compaction_cnt=" << compaction.cumulative_compaction_cnt() + << ", cumulative_compaction_cnt=" << stats.cumulative_compaction_cnt(); + msg = ss.str(); + return; + } + + auto job_key = job_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + std::string job_val; + TabletJobInfoPB job_pb; + err = txn->get(job_key, &job_val); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + SS << "failed to get tablet job, instance_id=" << instance_id << " tablet_id=" << tablet_id + << " key=" << hex(job_key) << " err=" << err; + msg = ss.str(); + code = cast_as(err); + return; + } + while (err == TxnErrorCode::TXN_OK) { + job_pb.ParseFromString(job_val); + if (job_pb.compaction().empty()) { + break; + } + auto& compactions = *job_pb.mutable_compaction(); + // Remove expired compaction jobs + // clang-format off + int64_t now = time(nullptr); + compactions.erase(std::remove_if(compactions.begin(), compactions.end(), [&](auto& c) { + DCHECK(c.expiration() > 0 || c.type() == TabletCompactionJobPB::EMPTY_CUMULATIVE) << proto_to_json(c); + DCHECK(c.lease() > 0) << proto_to_json(c); + if (c.expiration() > 0 && c.expiration() < now) { + INSTANCE_LOG(INFO) + << "got an expired job. job=" << proto_to_json(c) << " now=" << now; + return true; + } + if (c.lease() > 0 && c.lease() < now) { + INSTANCE_LOG(INFO) + << "got a job exceeding lease. job=" << proto_to_json(c) << " now=" << now; + return true; + } + return false; + }), compactions.end()); + // clang-format on + // Check conflict job + if (compaction.type() == TabletCompactionJobPB::FULL) { + // Full compaction is generally used for data correctness repair + // for MOW table, so priority should be given to performing full + // compaction operations and canceling other types of compaction. + compactions.Clear(); + } else if (compaction.input_versions().empty()) { + // Unknown input version range, doesn't support parallel compaction of same type + for (auto& c : compactions) { + if (c.type() != compaction.type() && c.type() != TabletCompactionJobPB::FULL) + continue; + if (c.id() == compaction.id()) return; // Same job, return OK to keep idempotency + msg = fmt::format("compaction has already started, tablet_id={} job={}", tablet_id, + proto_to_json(c)); + code = MetaServiceCode::JOB_TABLET_BUSY; + return; + } + } else { + DCHECK_EQ(compaction.input_versions_size(), 2) << proto_to_json(compaction); + DCHECK_LE(compaction.input_versions(0), compaction.input_versions(1)) + << proto_to_json(compaction); + auto version_not_conflict = [](const TabletCompactionJobPB& a, + const TabletCompactionJobPB& b) { + return a.input_versions(0) > b.input_versions(1) || + a.input_versions(1) < b.input_versions(0); + }; + for (auto& c : compactions) { + if (c.type() != compaction.type() && c.type() != TabletCompactionJobPB::FULL) + continue; + if (c.input_versions_size() > 0 && version_not_conflict(c, compaction)) continue; + if (c.id() == compaction.id()) return; // Same job, return OK to keep idempotency + msg = fmt::format("compaction has already started, tablet_id={} job={}", tablet_id, + proto_to_json(c)); + code = MetaServiceCode::JOB_TABLET_BUSY; + // Unknown version range of started compaction, BE should not retry other version range + if (c.input_versions_size() == 0) return; + // Notify version ranges in started compaction to BE, so BE can retry other version range + for (auto& c : compactions) { + if (c.type() == compaction.type() || c.type() == TabletCompactionJobPB::FULL) { + // If there are multiple started compaction of same type, they all must has input version range + DCHECK_EQ(c.input_versions_size(), 2) << proto_to_json(c); + response->add_version_in_compaction(c.input_versions(0)); + response->add_version_in_compaction(c.input_versions(1)); + } + } + return; + } + } + break; + } + if (!job_pb.has_idx()) { + job_pb.mutable_idx()->CopyFrom(request->job().idx()); + } + job_pb.add_compaction()->CopyFrom(compaction); + job_pb.SerializeToString(&job_val); + if (job_val.empty()) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + msg = "pb serialization error"; + return; + } + INSTANCE_LOG(INFO) << "compaction job to save job=" << proto_to_json(compaction); + txn->put(job_key, job_val); + need_commit = true; +} + +void start_schema_change_job(MetaServiceCode& code, std::string& msg, std::stringstream& ss, + std::unique_ptr& txn, + const StartTabletJobRequest* request, std::string& instance_id, + bool& need_commit) { + auto& schema_change = request->job().schema_change(); + if (!schema_change.has_id() || schema_change.id().empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "no job id specified"; + return; + } + if (!schema_change.has_initiator()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "no initiator specified"; + return; + } + + // check new_tablet state + int64_t new_tablet_id = schema_change.new_tablet_idx().tablet_id(); + if (new_tablet_id <= 0) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "no valid new_tablet_id given"; + return; + } + int64_t table_id = request->job().idx().table_id(); + int64_t index_id = request->job().idx().index_id(); + int64_t partition_id = request->job().idx().partition_id(); + int64_t tablet_id = request->job().idx().tablet_id(); + if (new_tablet_id == tablet_id) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "not allow new_tablet_id same with base_tablet_id"; + return; + } + auto& new_tablet_idx = const_cast(schema_change.new_tablet_idx()); + if (!new_tablet_idx.has_table_id() || !new_tablet_idx.has_index_id() || + !new_tablet_idx.has_partition_id()) { + get_tablet_idx(code, msg, txn.get(), instance_id, new_tablet_id, new_tablet_idx); + if (code != MetaServiceCode::OK) return; + } + MetaTabletKeyInfo new_tablet_key_info {instance_id, new_tablet_idx.table_id(), + new_tablet_idx.index_id(), new_tablet_idx.partition_id(), + new_tablet_id}; + std::string new_tablet_key; + std::string new_tablet_val; + doris::TabletMetaCloudPB new_tablet_meta; + meta_tablet_key(new_tablet_key_info, &new_tablet_key); + TxnErrorCode err = txn->get(new_tablet_key, &new_tablet_val); + if (err != TxnErrorCode::TXN_OK) { + SS << "failed to get new tablet meta" + << (err == TxnErrorCode::TXN_KEY_NOT_FOUND ? " (not found)" : "") + << " instance_id=" << instance_id << " tablet_id=" << new_tablet_id + << " key=" << hex(new_tablet_key) << " err=" << err; + msg = ss.str(); + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::TABLET_NOT_FOUND + : cast_as(err); + return; + } + if (!new_tablet_meta.ParseFromString(new_tablet_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "malformed tablet meta"; + return; + } + + if (new_tablet_meta.tablet_state() == doris::TabletStatePB::PB_RUNNING) { + code = MetaServiceCode::JOB_ALREADY_SUCCESS; + msg = "schema_change job already success"; + return; + } + if (!new_tablet_meta.has_tablet_state() || + new_tablet_meta.tablet_state() != doris::TabletStatePB::PB_NOTREADY) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "invalid new tablet state"; + return; + } + + auto job_key = job_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + std::string job_val; + TabletJobInfoPB job_pb; + err = txn->get(job_key, &job_val); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + SS << "failed to get tablet job, instance_id=" << instance_id << " tablet_id=" << tablet_id + << " key=" << hex(job_key) << " err=" << err; + msg = ss.str(); + code = cast_as(err); + return; + } + job_pb.mutable_idx()->CopyFrom(request->job().idx()); + // FE can ensure that a tablet does not have more than one schema_change job at the same time, + // so we can directly preempt previous schema_change job. + job_pb.mutable_schema_change()->CopyFrom(schema_change); + job_pb.SerializeToString(&job_val); + if (job_val.empty()) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + msg = "pb serialization error"; + return; + } + INSTANCE_LOG(INFO) << "schema_change job to save job=" << proto_to_json(schema_change); + txn->put(job_key, job_val); + need_commit = true; +} + +void MetaServiceImpl::start_tablet_job(::google::protobuf::RpcController* controller, + const StartTabletJobRequest* request, + StartTabletJobResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(start_tablet_job); + std::string cloud_unique_id = request->cloud_unique_id(); + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + SS << "cannot find instance_id with cloud_unique_id=" + << (cloud_unique_id.empty() ? "(empty)" : cloud_unique_id); + msg = ss.str(); + return; + } + RPC_RATE_LIMIT(start_tablet_job) + if (!request->has_job() || + (request->job().compaction().empty() && !request->job().has_schema_change())) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "no valid job specified"; + return; + } + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + return; + } + + int64_t tablet_id = request->job().idx().tablet_id(); + if (tablet_id <= 0) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "no valid tablet_id given"; + return; + } + auto& tablet_idx = const_cast(request->job().idx()); + if (!tablet_idx.has_table_id() || !tablet_idx.has_index_id() || + !tablet_idx.has_partition_id()) { + get_tablet_idx(code, msg, txn.get(), instance_id, tablet_id, tablet_idx); + if (code != MetaServiceCode::OK) return; + } + // Check if tablet has been dropped + if (is_dropped_tablet(txn.get(), instance_id, tablet_idx.index_id(), + tablet_idx.partition_id())) { + code = MetaServiceCode::TABLET_NOT_FOUND; + msg = fmt::format("tablet {} has been dropped", tablet_id); + return; + } + + bool need_commit = false; + std::unique_ptr> defer_commit( + (int*)0x01, [&ss, &txn, &code, &msg, &need_commit](int*) { + if (!need_commit) return; + TxnErrorCode err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to commit job kv, err=" << err; + msg = ss.str(); + return; + } + }); + + if (!request->job().compaction().empty()) { + start_compaction_job(code, msg, ss, txn, request, response, instance_id, need_commit); + return; + } + + if (request->job().has_schema_change()) { + start_schema_change_job(code, msg, ss, txn, request, instance_id, need_commit); + return; + } +} + +static bool check_and_remove_delete_bitmap_update_lock(MetaServiceCode& code, std::string& msg, + std::stringstream& ss, + std::unique_ptr& txn, + std::string& instance_id, int64_t table_id, + int64_t lock_id, int64_t lock_initiator) { + std::string lock_key = meta_delete_bitmap_update_lock_key({instance_id, table_id, -1}); + std::string lock_val; + TxnErrorCode err = txn->get(lock_key, &lock_val); + LOG(INFO) << "get delete bitmap update lock info, table_id=" << table_id + << " key=" << hex(lock_key) << " err=" << err; + if (err != TxnErrorCode::TXN_OK) { + ss << "failed to get delete bitmap update lock key, instance_id=" << instance_id + << " table_id=" << table_id << " key=" << hex(lock_key) << " err=" << err; + msg = ss.str(); + code = cast_as(err); + return false; + } + DeleteBitmapUpdateLockPB lock_info; + if (!lock_info.ParseFromString(lock_val)) [[unlikely]] { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse DeleteBitmapUpdateLockPB"; + return false; + } + if (lock_info.lock_id() != lock_id) { + msg = "lock id not match"; + code = MetaServiceCode::LOCK_EXPIRED; + return false; + } + bool found = false; + auto initiators = lock_info.mutable_initiators(); + for (auto iter = initiators->begin(); iter != initiators->end(); iter++) { + if (*iter == lock_initiator) { + initiators->erase(iter); + found = true; + break; + } + } + if (!found) { + msg = "lock initiator not exist"; + code = MetaServiceCode::LOCK_EXPIRED; + return false; + } + if (initiators->empty()) { + INSTANCE_LOG(INFO) << "remove delete bitmap lock, table_id=" << table_id + << " key=" << hex(lock_key); + txn->remove(lock_key); + return true; + } + lock_info.SerializeToString(&lock_val); + if (lock_val.empty()) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + msg = "pb serialization error"; + return false; + } + INSTANCE_LOG(INFO) << "remove delete bitmap lock initiator, table_id=" << table_id + << ", key=" << hex(lock_key) << ", initiator=" << lock_initiator + << " initiators_size=" << lock_info.initiators_size(); + txn->put(lock_key, lock_val); + return true; +} + +static void remove_delete_bitmap_update_lock(std::unique_ptr& txn, + const std::string& instance_id, int64_t table_id, + int64_t lock_id, int64_t lock_initiator) { + std::string lock_key = meta_delete_bitmap_update_lock_key({instance_id, table_id, -1}); + std::string lock_val; + TxnErrorCode err = txn->get(lock_key, &lock_val); + LOG(INFO) << "get delete bitmap update lock info, table_id=" << table_id + << " key=" << hex(lock_key) << " err=" << err; + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to get delete bitmap update lock key, instance_id=" << instance_id + << " table_id=" << table_id << " key=" << hex(lock_key) << " err=" << err; + return; + } + DeleteBitmapUpdateLockPB lock_info; + if (!lock_info.ParseFromString(lock_val)) [[unlikely]] { + LOG(WARNING) << "failed to parse DeleteBitmapUpdateLockPB, instance_id=" << instance_id + << " table_id=" << table_id << " key=" << hex(lock_key); + return; + } + if (lock_info.lock_id() != lock_id) { + return; + } + bool found = false; + auto initiators = lock_info.mutable_initiators(); + for (auto iter = initiators->begin(); iter != initiators->end(); iter++) { + if (*iter == lock_initiator) { + initiators->erase(iter); + found = true; + break; + } + } + if (!found) { + return; + } + if (initiators->empty()) { + INSTANCE_LOG(INFO) << "remove delete bitmap lock, table_id=" << table_id + << " key=" << hex(lock_key); + txn->remove(lock_key); + return; + } + lock_info.SerializeToString(&lock_val); + if (lock_val.empty()) { + INSTANCE_LOG(WARNING) << "failed to seiralize lock_info, table_id=" << table_id + << " key=" << hex(lock_key); + return; + } + INSTANCE_LOG(INFO) << "remove delete bitmap lock initiator, table_id=" << table_id + << ", key=" << hex(lock_key) << ", initiator=" << lock_initiator + << " initiators_size=" << lock_info.initiators_size(); + txn->put(lock_key, lock_val); +} + +void process_compaction_job(MetaServiceCode& code, std::string& msg, std::stringstream& ss, + std::unique_ptr& txn, + const FinishTabletJobRequest* request, + FinishTabletJobResponse* response, TabletJobInfoPB& recorded_job, + std::string& instance_id, std::string& job_key, bool& need_commit) { + //========================================================================== + // check + //========================================================================== + int64_t table_id = request->job().idx().table_id(); + int64_t index_id = request->job().idx().index_id(); + int64_t partition_id = request->job().idx().partition_id(); + int64_t tablet_id = request->job().idx().tablet_id(); + if (recorded_job.compaction().empty()) { + SS << "there is no running compaction, tablet_id=" << tablet_id; + msg = ss.str(); + code = MetaServiceCode::INVALID_ARGUMENT; + return; + } + + auto& compaction = request->job().compaction(0); + + auto recorded_compaction = recorded_job.mutable_compaction()->begin(); + for (; recorded_compaction != recorded_job.mutable_compaction()->end(); ++recorded_compaction) { + if (recorded_compaction->id() == compaction.id()) break; + } + if (recorded_compaction == recorded_job.mutable_compaction()->end()) { + SS << "unmatched job id, recorded_job=" << proto_to_json(recorded_job) + << " given_job=" << proto_to_json(compaction); + code = MetaServiceCode::INVALID_ARGUMENT; + msg = ss.str(); + return; + } + + using namespace std::chrono; + int64_t now = duration_cast(system_clock::now().time_since_epoch()).count(); + if (recorded_compaction->expiration() > 0 && recorded_compaction->expiration() < now) { + code = MetaServiceCode::JOB_EXPIRED; + SS << "expired compaction job, tablet_id=" << tablet_id + << " job=" << proto_to_json(*recorded_compaction); + msg = ss.str(); + // FIXME: Just remove or notify to abort? + // LOG(INFO) << "remove expired job, tablet_id=" << tablet_id << " key=" << hex(job_key); + return; + } + + if (request->action() != FinishTabletJobRequest::COMMIT && + request->action() != FinishTabletJobRequest::ABORT && + request->action() != FinishTabletJobRequest::LEASE) { + SS << "unsupported action, tablet_id=" << tablet_id << " action=" << request->action(); + msg = ss.str(); + code = MetaServiceCode::INVALID_ARGUMENT; + return; + } + + //========================================================================== + // Lease + //========================================================================== + if (request->action() == FinishTabletJobRequest::LEASE) { + if (compaction.lease() <= 0 || recorded_compaction->lease() > compaction.lease()) { + ss << "invalid lease. recoreded_lease=" << recorded_compaction->lease() + << " req_lease=" << compaction.lease(); + msg = ss.str(); + code = MetaServiceCode::INVALID_ARGUMENT; + return; + } + recorded_compaction->set_lease(compaction.lease()); + auto job_val = recorded_job.SerializeAsString(); + txn->put(job_key, job_val); + INSTANCE_LOG(INFO) << "lease tablet compaction job, tablet_id=" << tablet_id + << " key=" << hex(job_key); + need_commit = true; + return; + } + + //========================================================================== + // Abort + //========================================================================== + if (request->action() == FinishTabletJobRequest::ABORT) { + // TODO(gavin): mv tmp rowsets to recycle or remove them directly + recorded_job.mutable_compaction()->erase(recorded_compaction); + auto job_val = recorded_job.SerializeAsString(); + txn->put(job_key, job_val); + INSTANCE_LOG(INFO) << "abort tablet compaction job, tablet_id=" << tablet_id + << " key=" << hex(job_key); + if (compaction.has_delete_bitmap_lock_initiator()) { + remove_delete_bitmap_update_lock(txn, instance_id, table_id, + COMPACTION_DELETE_BITMAP_LOCK_ID, + compaction.delete_bitmap_lock_initiator()); + } + need_commit = true; + return; + } + + //========================================================================== + // Commit + //========================================================================== + // + // 1. update tablet stats + // 2. move compaction input rowsets to recycle + // 3. change tmp rowset to formal rowset + // 4. remove compaction job + // + //========================================================================== + // Update tablet stats + //========================================================================== + auto stats = response->mutable_stats(); + TabletStats detached_stats; + // ATTN: The condition that snapshot read can be used to get tablet stats is: all other transactions that put tablet stats + // can make read write conflicts with this transaction on other keys. Currently, if all meta-service nodes are running + // with `config::split_tablet_stats = true` can meet the condition. + internal_get_tablet_stats(code, msg, txn.get(), instance_id, request->job().idx(), *stats, + detached_stats, config::snapshot_get_tablet_stats); + if (compaction.type() == TabletCompactionJobPB::EMPTY_CUMULATIVE) { + stats->set_cumulative_compaction_cnt(stats->cumulative_compaction_cnt() + 1); + stats->set_cumulative_point(compaction.output_cumulative_point()); + stats->set_last_cumu_compaction_time_ms(now * 1000); + } else if (compaction.type() == TabletCompactionJobPB::CUMULATIVE) { + // clang-format off + stats->set_cumulative_compaction_cnt(stats->cumulative_compaction_cnt() + 1); + if (compaction.output_cumulative_point() > stats->cumulative_point()) { + // After supporting parallel cumu compaction, compaction with older cumu point may be committed after + // new cumu point has been set, MUST NOT set cumu point back to old value + stats->set_cumulative_point(compaction.output_cumulative_point()); + } + stats->set_num_rows(stats->num_rows() + (compaction.num_output_rows() - compaction.num_input_rows())); + stats->set_data_size(stats->data_size() + (compaction.size_output_rowsets() - compaction.size_input_rowsets())); + stats->set_num_rowsets(stats->num_rowsets() + (compaction.num_output_rowsets() - compaction.num_input_rowsets())); + stats->set_num_segments(stats->num_segments() + (compaction.num_output_segments() - compaction.num_input_segments())); + stats->set_last_cumu_compaction_time_ms(now * 1000); + // clang-format on + } else if (compaction.type() == TabletCompactionJobPB::BASE) { + // clang-format off + stats->set_base_compaction_cnt(stats->base_compaction_cnt() + 1); + stats->set_num_rows(stats->num_rows() + (compaction.num_output_rows() - compaction.num_input_rows())); + stats->set_data_size(stats->data_size() + (compaction.size_output_rowsets() - compaction.size_input_rowsets())); + stats->set_num_rowsets(stats->num_rowsets() + (compaction.num_output_rowsets() - compaction.num_input_rowsets())); + stats->set_num_segments(stats->num_segments() + (compaction.num_output_segments() - compaction.num_input_segments())); + stats->set_last_base_compaction_time_ms(now * 1000); + // clang-format on + } else if (compaction.type() == TabletCompactionJobPB::FULL) { + // clang-format off + stats->set_base_compaction_cnt(stats->base_compaction_cnt() + 1); + if (compaction.output_cumulative_point() > stats->cumulative_point()) { + // After supporting parallel cumu compaction, compaction with older cumu point may be committed after + // new cumu point has been set, MUST NOT set cumu point back to old value + stats->set_cumulative_point(compaction.output_cumulative_point()); + } + stats->set_num_rows(stats->num_rows() + (compaction.num_output_rows() - compaction.num_input_rows())); + stats->set_data_size(stats->data_size() + (compaction.size_output_rowsets() - compaction.size_input_rowsets())); + stats->set_num_rowsets(stats->num_rowsets() + (compaction.num_output_rowsets() - compaction.num_input_rowsets())); + stats->set_num_segments(stats->num_segments() + (compaction.num_output_segments() - compaction.num_input_segments())); + stats->set_last_full_compaction_time_ms(now * 1000); + // clang-format on + } else { + msg = "invalid compaction type"; + code = MetaServiceCode::INVALID_ARGUMENT; + return; + } + auto stats_key = stats_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + auto stats_val = stats->SerializeAsString(); + txn->put(stats_key, stats_val); + merge_tablet_stats(*stats, detached_stats); + if (stats->data_size() < 0 || stats->num_rowsets() < 1) [[unlikely]] { + INSTANCE_LOG(ERROR) << "buggy data size, tablet_id=" << tablet_id + << " stats.data_size=" << stats->data_size() + << " compaction.size_output_rowsets=" + << compaction.size_output_rowsets() + << " compaction.size_input_rowsets= " + << compaction.size_input_rowsets(); + DCHECK(false) << "buggy data size"; + } + + VLOG_DEBUG << "update tablet stats tablet_id=" << tablet_id << " key=" << hex(stats_key) + << " stats=" << proto_to_json(*stats); + if (compaction.type() == TabletCompactionJobPB::EMPTY_CUMULATIVE) { + recorded_job.mutable_compaction()->erase(recorded_compaction); + auto job_val = recorded_job.SerializeAsString(); + txn->put(job_key, job_val); + INSTANCE_LOG(INFO) << "remove compaction job, tablet_id=" << tablet_id + << " key=" << hex(job_key); + need_commit = true; + return; + } + + // remove delete bitmap update lock for MoW table + if (compaction.has_delete_bitmap_lock_initiator()) { + bool success = check_and_remove_delete_bitmap_update_lock( + code, msg, ss, txn, instance_id, table_id, COMPACTION_DELETE_BITMAP_LOCK_ID, + compaction.delete_bitmap_lock_initiator()); + if (!success) { + return; + } + } + + //========================================================================== + // Move input rowsets to recycle + //========================================================================== + if (compaction.input_versions_size() != 2 || compaction.output_versions_size() != 1 || + compaction.output_rowset_ids_size() != 1) { + code = MetaServiceCode::INVALID_ARGUMENT; + SS << "invalid input or output versions, input_versions_size=" + << compaction.input_versions_size() + << " output_versions_size=" << compaction.output_versions_size() + << " output_rowset_ids_size=" << compaction.output_rowset_ids_size(); + msg = ss.str(); + return; + } + + auto start = compaction.input_versions(0); + auto end = compaction.input_versions(1); + auto rs_start = meta_rowset_key({instance_id, tablet_id, start}); + auto rs_end = meta_rowset_key({instance_id, tablet_id, end + 1}); + + std::unique_ptr it; + int num_rowsets = 0; + std::unique_ptr> defer_log_range( + (int*)0x01, [&rs_start, &rs_end, &num_rowsets, &instance_id](int*) { + INSTANCE_LOG(INFO) << "get rowset meta, num_rowsets=" << num_rowsets << " range=[" + << hex(rs_start) << "," << hex(rs_end) << "]"; + }); + + auto rs_start1 = rs_start; + do { + TxnErrorCode err = txn->get(rs_start1, rs_end, &it); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + SS << "internal error, failed to get rowset range, err=" << err + << " tablet_id=" << tablet_id << " range=[" << hex(rs_start1) << ", << " + << hex(rs_end) << ")"; + msg = ss.str(); + return; + } + + while (it->has_next()) { + auto [k, v] = it->next(); + + doris::RowsetMetaCloudPB rs; + if (!rs.ParseFromArray(v.data(), v.size())) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + SS << "malformed rowset meta, unable to deserialize, tablet_id=" << tablet_id + << " key=" << hex(k); + msg = ss.str(); + return; + } + + // remove delete bitmap of input rowset for MoW table + if (compaction.has_delete_bitmap_lock_initiator()) { + auto delete_bitmap_start = + meta_delete_bitmap_key({instance_id, tablet_id, rs.rowset_id_v2(), 0, 0}); + auto delete_bitmap_end = meta_delete_bitmap_key( + {instance_id, tablet_id, rs.rowset_id_v2(), INT64_MAX, INT64_MAX}); + txn->remove(delete_bitmap_start, delete_bitmap_end); + } + + auto recycle_key = recycle_rowset_key({instance_id, tablet_id, rs.rowset_id_v2()}); + RecycleRowsetPB recycle_rowset; + recycle_rowset.set_creation_time(now); + recycle_rowset.mutable_rowset_meta()->CopyFrom(rs); + recycle_rowset.set_type(RecycleRowsetPB::COMPACT); + auto recycle_val = recycle_rowset.SerializeAsString(); + txn->put(recycle_key, recycle_val); + INSTANCE_LOG(INFO) << "put recycle rowset, tablet_id=" << tablet_id + << " key=" << hex(recycle_key); + + ++num_rowsets; + if (!it->has_next()) rs_start1 = k; + } + rs_start1.push_back('\x00'); // Update to next smallest key for iteration + } while (it->more()); + + txn->remove(rs_start, rs_end); + + TEST_SYNC_POINT_CALLBACK("process_compaction_job::loop_input_done", &num_rowsets); + + if (num_rowsets < 1) { + SS << "too few input rowsets, tablet_id=" << tablet_id << " num_rowsets=" << num_rowsets; + code = MetaServiceCode::UNDEFINED_ERR; + msg = ss.str(); + recorded_job.mutable_compaction()->erase(recorded_compaction); + auto job_val = recorded_job.SerializeAsString(); + txn->put(job_key, job_val); + INSTANCE_LOG(INFO) << "remove compaction job, tablet_id=" << tablet_id + << " key=" << hex(job_key); + need_commit = true; + TEST_SYNC_POINT_CALLBACK("process_compaction_job::too_few_rowsets", &need_commit); + return; + } + + //========================================================================== + // Change tmp rowset to formal rowset + //========================================================================== + if (compaction.txn_id_size() != 1) { + SS << "invalid txn_id, txn_id_size=" << compaction.txn_id_size(); + msg = ss.str(); + code = MetaServiceCode::INVALID_ARGUMENT; + return; + } + int64_t txn_id = compaction.txn_id(0); + auto& rowset_id = compaction.output_rowset_ids(0); + if (txn_id <= 0 || rowset_id.empty()) { + SS << "invalid txn_id or rowset_id, tablet_id=" << tablet_id << " txn_id=" << txn_id + << " rowset_id=" << rowset_id; + msg = ss.str(); + code = MetaServiceCode::INVALID_ARGUMENT; + return; + } + auto tmp_rowset_key = meta_rowset_tmp_key({instance_id, txn_id, tablet_id}); + std::string tmp_rowset_val; + TxnErrorCode err = txn->get(tmp_rowset_key, &tmp_rowset_val); + if (err != TxnErrorCode::TXN_OK) { + SS << "failed to get tmp rowset key" + << (err == TxnErrorCode::TXN_KEY_NOT_FOUND ? " (not found)" : "") + << ", tablet_id=" << tablet_id << " tmp_rowset_key=" << hex(tmp_rowset_key); + msg = ss.str(); + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::UNDEFINED_ERR + : cast_as(err); + return; + } + + // We don't actually need to parse the rowset meta + doris::RowsetMetaCloudPB rs_meta; + rs_meta.ParseFromString(tmp_rowset_val); + if (rs_meta.txn_id() <= 0) { + SS << "invalid txn_id in output tmp rowset meta, tablet_id=" << tablet_id + << " txn_id=" << rs_meta.txn_id(); + msg = ss.str(); + code = MetaServiceCode::INVALID_ARGUMENT; + return; + } + + txn->remove(tmp_rowset_key); + INSTANCE_LOG(INFO) << "remove tmp rowset meta, tablet_id=" << tablet_id + << " tmp_rowset_key=" << hex(tmp_rowset_key); + + int64_t version = compaction.output_versions(0); + auto rowset_key = meta_rowset_key({instance_id, tablet_id, version}); + txn->put(rowset_key, tmp_rowset_val); + INSTANCE_LOG(INFO) << "put rowset meta, tablet_id=" << tablet_id + << " rowset_key=" << hex(rowset_key); + + //========================================================================== + // Remove compaction job + //========================================================================== + // TODO(gavin): move deleted job info into recycle or history + recorded_job.mutable_compaction()->erase(recorded_compaction); + auto job_val = recorded_job.SerializeAsString(); + txn->put(job_key, job_val); + INSTANCE_LOG(INFO) << "remove compaction job tabelt_id=" << tablet_id + << " key=" << hex(job_key); + + need_commit = true; +} + +void process_schema_change_job(MetaServiceCode& code, std::string& msg, std::stringstream& ss, + std::unique_ptr& txn, + const FinishTabletJobRequest* request, + FinishTabletJobResponse* response, TabletJobInfoPB& recorded_job, + std::string& instance_id, std::string& job_key, bool& need_commit) { + //========================================================================== + // check + //========================================================================== + int64_t tablet_id = request->job().idx().tablet_id(); + auto& schema_change = request->job().schema_change(); + int64_t new_tablet_id = schema_change.new_tablet_idx().tablet_id(); + if (new_tablet_id <= 0) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "no valid new_tablet_id given"; + return; + } + if (new_tablet_id == tablet_id) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "not allow new_tablet_id same with base_tablet_id"; + return; + } + auto& new_tablet_idx = const_cast(schema_change.new_tablet_idx()); + if (!new_tablet_idx.has_table_id() || !new_tablet_idx.has_index_id() || + !new_tablet_idx.has_partition_id()) { + get_tablet_idx(code, msg, txn.get(), instance_id, new_tablet_id, new_tablet_idx); + if (code != MetaServiceCode::OK) return; + } + int64_t new_table_id = new_tablet_idx.table_id(); + int64_t new_index_id = new_tablet_idx.index_id(); + int64_t new_partition_id = new_tablet_idx.partition_id(); + + auto new_tablet_key = meta_tablet_key( + {instance_id, new_table_id, new_index_id, new_partition_id, new_tablet_id}); + std::string new_tablet_val; + doris::TabletMetaCloudPB new_tablet_meta; + TxnErrorCode err = txn->get(new_tablet_key, &new_tablet_val); + if (err != TxnErrorCode::TXN_OK) { + SS << "failed to get new tablet meta" + << (err == TxnErrorCode::TXN_KEY_NOT_FOUND ? " (not found)" : "") + << " instance_id=" << instance_id << " tablet_id=" << new_tablet_id + << " key=" << hex(new_tablet_key) << " err=" << err; + msg = ss.str(); + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::TABLET_NOT_FOUND + : cast_as(err); + return; + } + if (!new_tablet_meta.ParseFromString(new_tablet_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "malformed tablet meta"; + return; + } + + if (new_tablet_meta.tablet_state() == doris::TabletStatePB::PB_RUNNING) { + code = MetaServiceCode::JOB_ALREADY_SUCCESS; + msg = "schema_change job already success"; + return; + } + if (!new_tablet_meta.has_tablet_state() || + new_tablet_meta.tablet_state() != doris::TabletStatePB::PB_NOTREADY) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "invalid new tablet state"; + return; + } + + if (!recorded_job.has_schema_change()) { + SS << "there is no running schema_change, tablet_id=" << tablet_id; + msg = ss.str(); + code = MetaServiceCode::INVALID_ARGUMENT; + return; + } + auto& recorded_schema_change = recorded_job.schema_change(); + using namespace std::chrono; + int64_t now = duration_cast(system_clock::now().time_since_epoch()).count(); + if (recorded_schema_change.expiration() > 0 && recorded_schema_change.expiration() < now) { + code = MetaServiceCode::JOB_EXPIRED; + SS << "expired schema_change job, tablet_id=" << tablet_id + << " job=" << proto_to_json(recorded_schema_change); + msg = ss.str(); + // FIXME: Just remove or notify to abort? + // LOG(INFO) << "remove expired job, tablet_id=" << tablet_id << " key=" << hex(job_key); + return; + } + + // MUST check initiator to let the retried BE commit this schema_change job. + if (schema_change.id() != recorded_schema_change.id() || + schema_change.initiator() != recorded_schema_change.initiator()) { + SS << "unmatched job id or initiator, recorded_id=" << recorded_schema_change.id() + << " given_id=" << schema_change.id() + << " recorded_job=" << proto_to_json(recorded_schema_change) + << " given_job=" << proto_to_json(schema_change); + code = MetaServiceCode::INVALID_ARGUMENT; + msg = ss.str(); + return; + } + + if (request->action() != FinishTabletJobRequest::COMMIT && + request->action() != FinishTabletJobRequest::ABORT) { + SS << "unsupported action, tablet_id=" << tablet_id << " action=" << request->action(); + msg = ss.str(); + code = MetaServiceCode::INVALID_ARGUMENT; + return; + } + + //========================================================================== + // Abort + //========================================================================== + if (request->action() == FinishTabletJobRequest::ABORT) { + // TODO(cyx) + return; + } + + //========================================================================== + // Commit + //========================================================================== + // + // 1. update new_tablet meta + // 2. move rowsets [2-alter_version] in new_tablet to recycle + // 3. update new_tablet stats + // 4. change tmp rowset to formal rowset + // 5. remove schema_change job (unnecessary) + // + //========================================================================== + // update tablet meta + //========================================================================== + new_tablet_meta.set_tablet_state(doris::TabletStatePB::PB_RUNNING); + new_tablet_meta.set_cumulative_layer_point(schema_change.output_cumulative_point()); + new_tablet_meta.SerializeToString(&new_tablet_val); + txn->put(new_tablet_key, new_tablet_val); + + //========================================================================== + // move rowsets [2-alter_version] to recycle + //========================================================================== + if (!schema_change.has_alter_version()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "invalid alter_version"; + return; + } + if (schema_change.alter_version() < 2) { // no need to update stats + // TODO(cyx): clear schema_change job? + need_commit = true; + return; + } + + int64_t num_remove_rows = 0; + int64_t size_remove_rowsets = 0; + int64_t num_remove_rowsets = 0; + int64_t num_remove_segments = 0; + + auto rs_start = meta_rowset_key({instance_id, new_tablet_id, 2}); + auto rs_end = meta_rowset_key({instance_id, new_tablet_id, schema_change.alter_version() + 1}); + std::unique_ptr it; + auto rs_start1 = rs_start; + do { + TxnErrorCode err = txn->get(rs_start1, rs_end, &it); + if (err != TxnErrorCode::TXN_OK) { + code = MetaServiceCode::KV_TXN_GET_ERR; + SS << "internal error, failed to get rowset range, err=" << err + << " tablet_id=" << new_tablet_id << " range=[" << hex(rs_start1) << ", << " + << hex(rs_end) << ")"; + msg = ss.str(); + return; + } + + while (it->has_next()) { + auto [k, v] = it->next(); + + doris::RowsetMetaCloudPB rs; + if (!rs.ParseFromArray(v.data(), v.size())) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + SS << "malformed rowset meta, unable to deserialize, tablet_id=" << new_tablet_id + << " key=" << hex(k); + msg = ss.str(); + return; + } + + num_remove_rows += rs.num_rows(); + size_remove_rowsets += rs.data_disk_size(); + ++num_remove_rowsets; + num_remove_segments += rs.num_segments(); + + auto recycle_key = recycle_rowset_key({instance_id, new_tablet_id, rs.rowset_id_v2()}); + RecycleRowsetPB recycle_rowset; + recycle_rowset.set_creation_time(now); + recycle_rowset.mutable_rowset_meta()->CopyFrom(rs); + recycle_rowset.set_type(RecycleRowsetPB::DROP); + auto recycle_val = recycle_rowset.SerializeAsString(); + txn->put(recycle_key, recycle_val); + INSTANCE_LOG(INFO) << "put recycle rowset, tablet_id=" << new_tablet_id + << " key=" << hex(recycle_key); + + if (!it->has_next()) rs_start1 = k; + } + rs_start1.push_back('\x00'); // Update to next smallest key for iteration + } while (it->more()); + + txn->remove(rs_start, rs_end); + + //========================================================================== + // update new_tablet stats + //========================================================================== + auto stats = response->mutable_stats(); + TabletStats detached_stats; + // ATTN: The condition that snapshot read can be used to get tablet stats is: all other transactions that put tablet stats + // can make read write conflicts with this transaction on other keys. Currently, if all meta-service nodes are running + // with `config::split_tablet_stats = true` can meet the condition. + internal_get_tablet_stats(code, msg, txn.get(), instance_id, new_tablet_idx, *stats, + detached_stats, config::snapshot_get_tablet_stats); + // clang-format off + stats->set_cumulative_point(schema_change.output_cumulative_point()); + stats->set_num_rows(stats->num_rows() + (schema_change.num_output_rows() - num_remove_rows)); + stats->set_data_size(stats->data_size() + (schema_change.size_output_rowsets() - size_remove_rowsets)); + stats->set_num_rowsets(stats->num_rowsets() + (schema_change.num_output_rowsets() - num_remove_rowsets)); + stats->set_num_segments(stats->num_segments() + (schema_change.num_output_segments() - num_remove_segments)); + // clang-format on + auto stats_key = stats_tablet_key( + {instance_id, new_table_id, new_index_id, new_partition_id, new_tablet_id}); + auto stats_val = stats->SerializeAsString(); + txn->put(stats_key, stats_val); + merge_tablet_stats(*stats, detached_stats); + VLOG_DEBUG << "update tablet stats tablet_id=" << tablet_id << " key=" << hex(stats_key) + << " stats=" << proto_to_json(*stats); + //========================================================================== + // change tmp rowset to formal rowset + //========================================================================== + if (schema_change.txn_ids().empty() || schema_change.output_versions().empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty txn_ids or output_versions"; + return; + } + + // process mow table, check lock + if (new_tablet_meta.enable_unique_key_merge_on_write()) { + bool success = check_and_remove_delete_bitmap_update_lock( + code, msg, ss, txn, instance_id, new_table_id, SCHEMA_CHANGE_DELETE_BITMAP_LOCK_ID, + schema_change.delete_bitmap_lock_initiator()); + if (!success) { + return; + } + } + + for (size_t i = 0; i < schema_change.txn_ids().size(); ++i) { + auto tmp_rowset_key = + meta_rowset_tmp_key({instance_id, schema_change.txn_ids().at(i), new_tablet_id}); + std::string tmp_rowset_val; + // FIXME: async get + TxnErrorCode err = txn->get(tmp_rowset_key, &tmp_rowset_val); + if (err != TxnErrorCode::TXN_OK) { + SS << "failed to get tmp rowset key" + << (err == TxnErrorCode::TXN_KEY_NOT_FOUND ? " (not found)" : "") + << ", tablet_id=" << new_tablet_id << " tmp_rowset_key=" << hex(tmp_rowset_key); + msg = ss.str(); + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::UNDEFINED_ERR + : cast_as(err); + return; + } + auto rowset_key = meta_rowset_key( + {instance_id, new_tablet_id, schema_change.output_versions().at(i)}); + txn->put(rowset_key, tmp_rowset_val); + txn->remove(tmp_rowset_key); + } + + //========================================================================== + // remove schema_change job + //========================================================================== + recorded_job.clear_schema_change(); + auto job_val = recorded_job.SerializeAsString(); + txn->put(job_key, job_val); + INSTANCE_LOG(INFO) << "remove schema_change job tablet_id=" << tablet_id + << " key=" << hex(job_key); + + need_commit = true; +} + +void MetaServiceImpl::finish_tablet_job(::google::protobuf::RpcController* controller, + const FinishTabletJobRequest* request, + FinishTabletJobResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(finish_tablet_job); + std::string cloud_unique_id = request->cloud_unique_id(); + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + SS << "cannot find instance_id with cloud_unique_id=" + << (cloud_unique_id.empty() ? "(empty)" : cloud_unique_id); + msg = ss.str(); + LOG(INFO) << msg; + return; + } + RPC_RATE_LIMIT(finish_tablet_job) + if (!request->has_job() || + (request->job().compaction().empty() && !request->job().has_schema_change())) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "no valid job specified"; + return; + } + + bool need_commit = false; + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + return; + } + + int64_t tablet_id = request->job().idx().tablet_id(); + if (tablet_id <= 0) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "no valid tablet_id given"; + return; + } + auto& tablet_idx = const_cast(request->job().idx()); + if (!tablet_idx.has_table_id() || !tablet_idx.has_index_id() || + !tablet_idx.has_partition_id()) { + get_tablet_idx(code, msg, txn.get(), instance_id, tablet_id, tablet_idx); + if (code != MetaServiceCode::OK) return; + } + // Check if tablet has been dropped + if (is_dropped_tablet(txn.get(), instance_id, tablet_idx.index_id(), + tablet_idx.partition_id())) { + code = MetaServiceCode::TABLET_NOT_FOUND; + msg = fmt::format("tablet {} has been dropped", tablet_id); + return; + } + + // TODO(gavin): remove duplicated code with start_tablet_job() + // Begin to process finish tablet job + std::string job_key = job_tablet_key({instance_id, tablet_idx.table_id(), tablet_idx.index_id(), + tablet_idx.partition_id(), tablet_id}); + std::string job_val; + err = txn->get(job_key, &job_val); + if (err != TxnErrorCode::TXN_OK) { + SS << (err == TxnErrorCode::TXN_KEY_NOT_FOUND ? "job not found," : "internal error,") + << " instance_id=" << instance_id << " tablet_id=" << tablet_id + << " job=" << proto_to_json(request->job()); + msg = ss.str(); + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::INVALID_ARGUMENT + : cast_as(err); + return; + } + TabletJobInfoPB recorded_job; + recorded_job.ParseFromString(job_val); + VLOG_DEBUG << "get tablet job, tablet_id=" << tablet_id + << " job=" << proto_to_json(recorded_job); + + std::unique_ptr> defer_commit( + (int*)0x01, [&ss, &txn, &code, &msg, &need_commit](int*) { + if (!need_commit) return; + TxnErrorCode err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to commit job kv, err=" << err; + msg = ss.str(); + return; + } + }); + + // Process compaction commit + if (!request->job().compaction().empty()) { + process_compaction_job(code, msg, ss, txn, request, response, recorded_job, instance_id, + job_key, need_commit); + return; + } + + // Process schema change commit + if (request->job().has_schema_change()) { + process_schema_change_job(code, msg, ss, txn, request, response, recorded_job, instance_id, + job_key, need_commit); + return; + } +} + +#undef SS +#undef INSTANCE_LOG +} // namespace doris::cloud diff --git a/cloud/src/meta-service/meta_service_partition.cpp b/cloud/src/meta-service/meta_service_partition.cpp new file mode 100644 index 00000000000000..17c67fb22ffd0c --- /dev/null +++ b/cloud/src/meta-service/meta_service_partition.cpp @@ -0,0 +1,590 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include +#include + +#include "common/logging.h" +#include "meta-service/keys.h" +#include "meta-service/meta_service_helper.h" +#include "meta-service/txn_kv_error.h" +#include "meta_service.h" + +namespace doris::cloud { + +// ATTN: xxx_id MUST NOT be reused +// +// UNKNOWN +// | +// +----------+---------+ +// | | +// (prepare_xxx) (drop_xxx) +// | | +// v v +// PREPARED--(drop_xxx)-->DROPPED +// | | +// |----------+---------+ +// | | +// | (begin_recycle_xxx) +// | | +// (commit_xxx) v +// | RECYCLING RECYCLING --(drop_xxx)-> RECYCLING +// | | +// | (finish_recycle_xxx) UNKNOWN --(commit_xxx)-> UNKNOWN +// | | if xxx exists +// +----------+ +// | +// v +// UNKNOWN + +// Return TXN_OK if exists, TXN_KEY_NOT_FOUND if not exists, otherwise error +static TxnErrorCode index_exists(Transaction* txn, const std::string& instance_id, + const IndexRequest* req) { + auto tablet_key = meta_tablet_key({instance_id, req->table_id(), req->index_ids(0), 0, 0}); + auto tablet_key_end = + meta_tablet_key({instance_id, req->table_id(), req->index_ids(0), INT64_MAX, 0}); + std::unique_ptr it; + + TxnErrorCode err = txn->get(tablet_key, tablet_key_end, &it, false, 1); + if (err != TxnErrorCode::TXN_OK) { + LOG_WARNING("failed to get kv").tag("err", err); + return err; + } + return it->has_next() ? TxnErrorCode::TXN_OK : TxnErrorCode::TXN_KEY_NOT_FOUND; +} + +void MetaServiceImpl::prepare_index(::google::protobuf::RpcController* controller, + const IndexRequest* request, IndexResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(prepare_index); + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + return; + } + AnnotateTag tag_instance_id("instance_id", instance_id); + + RPC_RATE_LIMIT(prepare_index) + + if (request->index_ids().empty() || !request->has_table_id()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty index_ids or table_id"; + return; + } + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + return; + } + err = index_exists(txn.get(), instance_id, request); + // If index has existed, this might be a stale request + if (err == TxnErrorCode::TXN_OK) { + code = MetaServiceCode::ALREADY_EXISTED; + msg = "index already existed"; + return; + } + if (err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = cast_as(err); + msg = fmt::format("failed to check index existence, err={}", err); + return; + } + + std::string to_save_val; + { + RecycleIndexPB pb; + pb.set_table_id(request->table_id()); + pb.set_creation_time(::time(nullptr)); + pb.set_expiration(request->expiration()); + pb.set_state(RecycleIndexPB::PREPARED); + pb.SerializeToString(&to_save_val); + } + for (auto index_id : request->index_ids()) { + auto key = recycle_index_key({instance_id, index_id}); + std::string val; + err = txn->get(key, &val); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { // UNKNOWN + LOG_INFO("put recycle index").tag("key", hex(key)); + txn->put(key, to_save_val); + continue; + } + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to get kv, err={}", err); + LOG_WARNING(msg); + return; + } + RecycleIndexPB pb; + if (!pb.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "malformed recycle index value"; + LOG_WARNING(msg).tag("index_id", index_id); + return; + } + if (pb.state() != RecycleIndexPB::PREPARED) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = fmt::format("invalid recycle index state: {}", + RecycleIndexPB::State_Name(pb.state())); + return; + } + // else, duplicate request, OK + } + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to commit txn: {}", err); + return; + } +} + +void MetaServiceImpl::commit_index(::google::protobuf::RpcController* controller, + const IndexRequest* request, IndexResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(commit_index); + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + return; + } + RPC_RATE_LIMIT(commit_index) + + if (request->index_ids().empty() || !request->has_table_id()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty index_ids or table_id"; + return; + } + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + return; + } + + for (auto index_id : request->index_ids()) { + auto key = recycle_index_key({instance_id, index_id}); + std::string val; + err = txn->get(key, &val); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { // UNKNOWN + err = index_exists(txn.get(), instance_id, request); + // If index has existed, this might be a duplicate request + if (err == TxnErrorCode::TXN_OK) { + return; // Index committed, OK + } + if (err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = cast_as(err); + msg = "failed to check index existence"; + return; + } + // Index recycled + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "index has been recycled"; + return; + } + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to get kv, err={}", err); + LOG_WARNING(msg); + return; + } + RecycleIndexPB pb; + if (!pb.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "malformed recycle index value"; + LOG_WARNING(msg).tag("index_id", index_id); + return; + } + if (pb.state() != RecycleIndexPB::PREPARED) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = fmt::format("invalid recycle index state: {}", + RecycleIndexPB::State_Name(pb.state())); + return; + } + LOG_INFO("remove recycle index").tag("key", hex(key)); + txn->remove(key); + } + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to commit txn: {}", err); + return; + } +} + +void MetaServiceImpl::drop_index(::google::protobuf::RpcController* controller, + const IndexRequest* request, IndexResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(drop_index); + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + return; + } + RPC_RATE_LIMIT(drop_index) + + if (request->index_ids().empty() || !request->has_table_id()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty index_ids or table_id"; + return; + } + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + return; + } + + std::string to_save_val; + { + RecycleIndexPB pb; + pb.set_table_id(request->table_id()); + pb.set_creation_time(::time(nullptr)); + pb.set_expiration(request->expiration()); + pb.set_state(RecycleIndexPB::DROPPED); + pb.SerializeToString(&to_save_val); + } + bool need_commit = false; + for (auto index_id : request->index_ids()) { + auto key = recycle_index_key({instance_id, index_id}); + std::string val; + err = txn->get(key, &val); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { // UNKNOWN + LOG_INFO("put recycle index").tag("key", hex(key)); + txn->put(key, to_save_val); + need_commit = true; + continue; + } + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to get kv, err={}", err); + LOG_WARNING(msg); + return; + } + RecycleIndexPB pb; + if (!pb.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "malformed recycle index value"; + LOG_WARNING(msg).tag("index_id", index_id); + return; + } + switch (pb.state()) { + case RecycleIndexPB::PREPARED: + LOG_INFO("put recycle index").tag("key", hex(key)); + txn->put(key, to_save_val); + need_commit = true; + break; + case RecycleIndexPB::DROPPED: + case RecycleIndexPB::RECYCLING: + break; + default: + code = MetaServiceCode::INVALID_ARGUMENT; + msg = fmt::format("invalid recycle index state: {}", + RecycleIndexPB::State_Name(pb.state())); + return; + } + } + if (!need_commit) return; + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to commit txn: {}", err); + return; + } +} + +// Return TXN_OK if exists, TXN_KEY_NOT_FOUND if not exists, otherwise error +static TxnErrorCode partition_exists(Transaction* txn, const std::string& instance_id, + const PartitionRequest* req) { + auto tablet_key = meta_tablet_key( + {instance_id, req->table_id(), req->index_ids(0), req->partition_ids(0), 0}); + auto tablet_key_end = meta_tablet_key( + {instance_id, req->table_id(), req->index_ids(0), req->partition_ids(0), INT64_MAX}); + std::unique_ptr it; + + TxnErrorCode err = txn->get(tablet_key, tablet_key_end, &it, false, 1); + if (err != TxnErrorCode::TXN_OK) { + LOG_WARNING("failed to get kv").tag("err", err); + return err; + } + return it->has_next() ? TxnErrorCode::TXN_OK : TxnErrorCode::TXN_KEY_NOT_FOUND; +} + +void MetaServiceImpl::prepare_partition(::google::protobuf::RpcController* controller, + const PartitionRequest* request, + PartitionResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(prepare_partition); + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + return; + } + AnnotateTag tag_instance_id("instance_id", instance_id); + + RPC_RATE_LIMIT(prepare_partition) + + if (request->partition_ids().empty() || request->index_ids().empty() || + !request->has_table_id()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty partition_ids or index_ids or table_id"; + return; + } + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + return; + } + err = partition_exists(txn.get(), instance_id, request); + // If index has existed, this might be a stale request + if (err == TxnErrorCode::TXN_OK) { + code = MetaServiceCode::ALREADY_EXISTED; + msg = "index already existed"; + return; + } + if (err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = cast_as(err); + msg = "failed to check index existence"; + return; + } + + std::string to_save_val; + { + RecyclePartitionPB pb; + if (request->db_id() > 0) pb.set_db_id(request->db_id()); + pb.set_table_id(request->table_id()); + *pb.mutable_index_id() = request->index_ids(); + pb.set_creation_time(::time(nullptr)); + pb.set_expiration(request->expiration()); + pb.set_state(RecyclePartitionPB::PREPARED); + pb.SerializeToString(&to_save_val); + } + for (auto part_id : request->partition_ids()) { + auto key = recycle_partition_key({instance_id, part_id}); + std::string val; + err = txn->get(key, &val); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { // UNKNOWN + LOG_INFO("put recycle partition").tag("key", hex(key)); + txn->put(key, to_save_val); + continue; + } + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to get kv, err={}", err); + LOG_WARNING(msg); + return; + } + RecyclePartitionPB pb; + if (!pb.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "malformed recycle partition value"; + LOG_WARNING(msg).tag("partition_id", part_id); + return; + } + if (pb.state() != RecyclePartitionPB::PREPARED) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = fmt::format("invalid recycle index state: {}", + RecyclePartitionPB::State_Name(pb.state())); + return; + } + // else, duplicate request, OK + } + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to commit txn: {}", err); + return; + } +} + +void MetaServiceImpl::commit_partition(::google::protobuf::RpcController* controller, + const PartitionRequest* request, PartitionResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(commit_partition); + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + return; + } + RPC_RATE_LIMIT(commit_partition) + + if (request->partition_ids().empty() || !request->has_table_id()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty partition_ids or index_ids or table_id"; + return; + } + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + return; + } + + for (auto part_id : request->partition_ids()) { + auto key = recycle_partition_key({instance_id, part_id}); + std::string val; + err = txn->get(key, &val); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { // UNKNOWN + // Compatible with requests without `index_ids` + if (!request->index_ids().empty()) { + err = partition_exists(txn.get(), instance_id, request); + // If partition has existed, this might be a duplicate request + if (err == TxnErrorCode::TXN_OK) { + return; // Partition committed, OK + } + if (err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = cast_as(err); + msg = "failed to check partition existence"; + return; + } + } + // Index recycled + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "partition has been recycled"; + return; + } + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to get kv, err={}", err); + LOG_WARNING(msg); + return; + } + RecyclePartitionPB pb; + if (!pb.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "malformed recycle partition value"; + LOG_WARNING(msg).tag("partition_id", part_id); + return; + } + if (pb.state() != RecyclePartitionPB::PREPARED) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = fmt::format("invalid recycle partition state: {}", + RecyclePartitionPB::State_Name(pb.state())); + return; + } + LOG_INFO("remove recycle partition").tag("key", hex(key)); + txn->remove(key); + } + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to commit txn: {}", err); + return; + } +} + +void MetaServiceImpl::drop_partition(::google::protobuf::RpcController* controller, + const PartitionRequest* request, PartitionResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(drop_partition); + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + return; + } + RPC_RATE_LIMIT(drop_partition) + + if (request->partition_ids().empty() || request->index_ids().empty() || + !request->has_table_id()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty partition_ids or index_ids or table_id"; + return; + } + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + return; + } + std::string to_save_val; + { + RecyclePartitionPB pb; + if (request->db_id() > 0) pb.set_db_id(request->db_id()); + pb.set_table_id(request->table_id()); + *pb.mutable_index_id() = request->index_ids(); + pb.set_creation_time(::time(nullptr)); + pb.set_expiration(request->expiration()); + pb.set_state(RecyclePartitionPB::DROPPED); + pb.SerializeToString(&to_save_val); + } + bool need_commit = false; + for (auto part_id : request->partition_ids()) { + auto key = recycle_partition_key({instance_id, part_id}); + std::string val; + err = txn->get(key, &val); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { // UNKNOWN + LOG_INFO("put recycle partition").tag("key", hex(key)); + txn->put(key, to_save_val); + need_commit = true; + continue; + } + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to get kv, err={}", err); + LOG_WARNING(msg); + return; + } + RecyclePartitionPB pb; + if (!pb.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "malformed recycle partition value"; + LOG_WARNING(msg).tag("partition_id", part_id); + return; + } + switch (pb.state()) { + case RecyclePartitionPB::PREPARED: + LOG_INFO("put recycle partition").tag("key", hex(key)); + txn->put(key, to_save_val); + need_commit = true; + break; + case RecyclePartitionPB::DROPPED: + case RecyclePartitionPB::RECYCLING: + break; + default: + code = MetaServiceCode::INVALID_ARGUMENT; + msg = fmt::format("invalid recycle partition state: {}", + RecyclePartitionPB::State_Name(pb.state())); + return; + } + } + if (!need_commit) return; + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to commit txn: {}", err); + return; + } +} + +} // namespace doris::cloud diff --git a/cloud/src/meta-service/meta_service_resource.cpp b/cloud/src/meta-service/meta_service_resource.cpp new file mode 100644 index 00000000000000..4ce1cee596a132 --- /dev/null +++ b/cloud/src/meta-service/meta_service_resource.cpp @@ -0,0 +1,2980 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include +#include +#include + +#include + +#include "common/encryption_util.h" +#include "common/logging.h" +#include "common/string_util.h" +#include "common/sync_point.h" +#include "meta-service/meta_service.h" +#include "meta-service/meta_service_helper.h" +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" + +using namespace std::chrono; + +namespace doris::cloud { + +static void* run_bthread_work(void* arg) { + auto f = reinterpret_cast*>(arg); + (*f)(); + delete f; + return nullptr; +} + +static std::string_view print_cluster_status(const ClusterStatus& status) { + switch (status) { + case ClusterStatus::UNKNOWN: + return "UNKNOWN"; + case ClusterStatus::NORMAL: + return "NORMAL"; + case ClusterStatus::SUSPENDED: + return "SUSPENDED"; + case ClusterStatus::TO_RESUME: + return "TO_RESUME"; + default: + return "UNKNOWN"; + } +} + +static int encrypt_ak_sk_helper(const std::string plain_ak, const std::string plain_sk, + EncryptionInfoPB* encryption_info, AkSkPair* cipher_ak_sk_pair, + MetaServiceCode& code, std::string& msg) { + std::string key; + int64_t key_id; + int ret = get_newest_encryption_key_for_ak_sk(&key_id, &key); + { + TEST_SYNC_POINT_CALLBACK("encrypt_ak_sk:get_encryption_key_ret", &ret); + TEST_SYNC_POINT_CALLBACK("encrypt_ak_sk:get_encryption_key", &key); + TEST_SYNC_POINT_CALLBACK("encrypt_ak_sk:get_encryption_key_id", &key_id); + } + if (ret != 0) { + msg = "failed to get encryption key"; + code = MetaServiceCode::ERR_ENCRYPT; + LOG(WARNING) << msg; + return -1; + } + auto& encryption_method = get_encryption_method_for_ak_sk(); + AkSkPair plain_ak_sk_pair {plain_ak, plain_sk}; + ret = encrypt_ak_sk(plain_ak_sk_pair, encryption_method, key, cipher_ak_sk_pair); + if (ret != 0) { + msg = "failed to encrypt"; + code = MetaServiceCode::ERR_ENCRYPT; + LOG(WARNING) << msg; + return -1; + } + encryption_info->set_key_id(key_id); + encryption_info->set_encryption_method(encryption_method); + return 0; +} + +static int decrypt_ak_sk_helper(std::string_view cipher_ak, std::string_view cipher_sk, + const EncryptionInfoPB& encryption_info, AkSkPair* plain_ak_sk_pair, + MetaServiceCode& code, std::string& msg) { + int ret = decrypt_ak_sk_helper(cipher_ak, cipher_sk, encryption_info, plain_ak_sk_pair); + if (ret != 0) { + msg = "failed to decrypt"; + code = MetaServiceCode::ERR_DECPYPT; + } + return ret; +} + +int decrypt_instance_info(InstanceInfoPB& instance, const std::string& instance_id, + MetaServiceCode& code, std::string& msg, + std::shared_ptr& txn) { + for (auto& obj_info : *instance.mutable_obj_info()) { + if (obj_info.has_encryption_info()) { + AkSkPair plain_ak_sk_pair; + int ret = decrypt_ak_sk_helper(obj_info.ak(), obj_info.sk(), obj_info.encryption_info(), + &plain_ak_sk_pair, code, msg); + if (ret != 0) return -1; + obj_info.set_ak(std::move(plain_ak_sk_pair.first)); + obj_info.set_sk(std::move(plain_ak_sk_pair.second)); + } + } + if (instance.has_ram_user() && instance.ram_user().has_encryption_info()) { + auto& ram_user = *instance.mutable_ram_user(); + AkSkPair plain_ak_sk_pair; + int ret = decrypt_ak_sk_helper(ram_user.ak(), ram_user.sk(), ram_user.encryption_info(), + &plain_ak_sk_pair, code, msg); + if (ret != 0) return -1; + ram_user.set_ak(std::move(plain_ak_sk_pair.first)); + ram_user.set_sk(std::move(plain_ak_sk_pair.second)); + } + + std::string val; + TxnErrorCode err = txn->get(system_meta_service_arn_info_key(), &val); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + // For compatibility, use arn_info of config + RamUserPB iam_user; + iam_user.set_user_id(config::arn_id); + iam_user.set_external_id(instance_id); + iam_user.set_ak(config::arn_ak); + iam_user.set_sk(config::arn_sk); + instance.mutable_iam_user()->CopyFrom(iam_user); + } else if (err == TxnErrorCode::TXN_OK) { + RamUserPB iam_user; + if (!iam_user.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse RamUserPB"; + LOG(WARNING) << msg; + return -1; + } + AkSkPair plain_ak_sk_pair; + int ret = decrypt_ak_sk_helper(iam_user.ak(), iam_user.sk(), iam_user.encryption_info(), + &plain_ak_sk_pair, code, msg); + if (ret != 0) return -1; + iam_user.set_ak(std::move(plain_ak_sk_pair.first)); + iam_user.set_sk(std::move(plain_ak_sk_pair.second)); + instance.mutable_iam_user()->CopyFrom(iam_user); + } else { + code = cast_as(err); + msg = fmt::format("failed to get arn_info_key, err={}", err); + LOG(WARNING) << msg; + return -1; + } + + for (auto& stage : *instance.mutable_stages()) { + if (stage.has_obj_info() && stage.obj_info().has_encryption_info()) { + auto& obj_info = *stage.mutable_obj_info(); + AkSkPair plain_ak_sk_pair; + int ret = decrypt_ak_sk_helper(obj_info.ak(), obj_info.sk(), obj_info.encryption_info(), + &plain_ak_sk_pair, code, msg); + if (ret != 0) return -1; + obj_info.set_ak(std::move(plain_ak_sk_pair.first)); + obj_info.set_sk(std::move(plain_ak_sk_pair.second)); + } + } + return 0; +} + +void MetaServiceImpl::get_obj_store_info(google::protobuf::RpcController* controller, + const GetObjStoreInfoRequest* request, + GetObjStoreInfoResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(get_obj_store_info); + // Prepare data + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + if (cloud_unique_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "cloud unique id not set"; + return; + } + + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << cloud_unique_id; + return; + } + RPC_RATE_LIMIT(get_obj_store_info) + InstanceKeyInfo key_info {instance_id}; + std::string key; + std::string val; + instance_key(key_info, &key); + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return; + } + err = txn->get(key, &val); + LOG(INFO) << "get instance_key=" << hex(key); + + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get instance, instance_id=" << instance_id << " err=" << err; + msg = ss.str(); + return; + } + + InstanceInfoPB instance; + if (!instance.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse InstanceInfoPB"; + return; + } + for (auto& obj_info : *instance.mutable_obj_info()) { + if (obj_info.has_encryption_info()) { + AkSkPair plain_ak_sk_pair; + int ret = decrypt_ak_sk_helper(obj_info.ak(), obj_info.sk(), obj_info.encryption_info(), + &plain_ak_sk_pair, code, msg); + if (ret != 0) return; + obj_info.set_ak(std::move(plain_ak_sk_pair.first)); + obj_info.set_sk(std::move(plain_ak_sk_pair.second)); + } + } + response->mutable_obj_info()->CopyFrom(instance.obj_info()); +} + +void MetaServiceImpl::alter_obj_store_info(google::protobuf::RpcController* controller, + const AlterObjStoreInfoRequest* request, + AlterObjStoreInfoResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(alter_obj_store_info); + // Prepare data + if (!request->has_obj() || !request->obj().has_ak() || !request->obj().has_sk()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "s3 obj info err " + proto_to_json(*request); + return; + } + + auto& obj = request->obj(); + std::string plain_ak = obj.has_ak() ? obj.ak() : ""; + std::string plain_sk = obj.has_sk() ? obj.sk() : ""; + + EncryptionInfoPB encryption_info; + AkSkPair cipher_ak_sk_pair; + if (encrypt_ak_sk_helper(plain_ak, plain_sk, &encryption_info, &cipher_ak_sk_pair, code, msg) != + 0) { + return; + } + const auto& [ak, sk] = cipher_ak_sk_pair; + std::string bucket = obj.has_bucket() ? obj.bucket() : ""; + std::string prefix = obj.has_prefix() ? obj.prefix() : ""; + std::string endpoint = obj.has_endpoint() ? obj.endpoint() : ""; + std::string external_endpoint = obj.has_external_endpoint() ? obj.external_endpoint() : ""; + std::string region = obj.has_region() ? obj.region() : ""; + + // obj size > 1k, refuse + if (obj.ByteSizeLong() > 1024) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "s3 obj info greater than 1k " + proto_to_json(*request); + return; + } + + // TODO(dx): check s3 info right + + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + if (cloud_unique_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "cloud unique id not set"; + return; + } + + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << cloud_unique_id; + return; + } + RPC_RATE_LIMIT(alter_obj_store_info) + InstanceKeyInfo key_info {instance_id}; + std::string key; + std::string val; + instance_key(key_info, &key); + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return; + } + err = txn->get(key, &val); + LOG(INFO) << "get instance_key=" << hex(key); + + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get instance, instance_id=" << instance_id << " err=" << err; + msg = ss.str(); + return; + } + + InstanceInfoPB instance; + if (!instance.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse InstanceInfoPB"; + return; + } + + if (instance.status() == InstanceInfoPB::DELETED) { + code = MetaServiceCode::CLUSTER_NOT_FOUND; + msg = "instance status has been set delete, plz check it"; + return; + } + + auto now_time = std::chrono::system_clock::now(); + uint64_t time = + std::chrono::duration_cast(now_time.time_since_epoch()).count(); + + switch (request->op()) { + case AlterObjStoreInfoRequest::LEGACY_UPDATE_AK_SK: { + // get id + std::string id = request->obj().has_id() ? request->obj().id() : "0"; + int idx = std::stoi(id); + if (idx < 1 || idx > instance.obj_info().size()) { + // err + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "id invalid, please check it"; + return; + } + auto& obj_info = + const_cast&>(instance.obj_info()); + for (auto& it : obj_info) { + if (std::stoi(it.id()) == idx) { + if (it.ak() == ak && it.sk() == sk) { + // not change, just return ok + code = MetaServiceCode::OK; + msg = ""; + return; + } + it.set_mtime(time); + it.set_ak(ak); + it.set_sk(sk); + it.mutable_encryption_info()->CopyFrom(encryption_info); + } + } + } break; + case AlterObjStoreInfoRequest::ADD_OBJ_INFO: { + if (!obj.has_provider()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "s3 conf lease provider info"; + return; + } + if (instance.obj_info().size() >= 10) { + code = MetaServiceCode::UNDEFINED_ERR; + msg = "this instance history has greater than 10 objs, please new another instance"; + return; + } + // ATTN: prefix may be empty + if (ak.empty() || sk.empty() || bucket.empty() || endpoint.empty() || region.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "s3 conf info err, please check it"; + return; + } + + auto& objs = instance.obj_info(); + for (auto& it : objs) { + if (bucket == it.bucket() && prefix == it.prefix() && endpoint == it.endpoint() && + region == it.region() && ak == it.ak() && sk == it.sk() && + obj.provider() == it.provider() && external_endpoint == it.external_endpoint()) { + // err, anything not changed + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "original obj infos has a same conf, please check it"; + return; + } + } + // calc id + cloud::ObjectStoreInfoPB last_item; + last_item.set_ctime(time); + last_item.set_mtime(time); + last_item.set_id(std::to_string(instance.obj_info().size() + 1)); + if (obj.has_user_id()) { + last_item.set_user_id(obj.user_id()); + } + last_item.set_ak(std::move(cipher_ak_sk_pair.first)); + last_item.set_sk(std::move(cipher_ak_sk_pair.second)); + last_item.mutable_encryption_info()->CopyFrom(encryption_info); + last_item.set_bucket(bucket); + // format prefix, such as `/aa/bb/`, `aa/bb//`, `//aa/bb`, ` /aa/bb` -> `aa/bb` + prefix = trim(prefix); + last_item.set_prefix(prefix); + last_item.set_endpoint(endpoint); + last_item.set_external_endpoint(external_endpoint); + last_item.set_region(region); + last_item.set_provider(obj.provider()); + last_item.set_sse_enabled(instance.sse_enabled()); + instance.add_obj_info()->CopyFrom(last_item); + } break; + default: { + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "invalid request op, op=" << request->op(); + msg = ss.str(); + return; + } + } + + LOG(INFO) << "instance " << instance_id << " has " << instance.obj_info().size() + << " s3 history info, and instance = " << proto_to_json(instance); + + val = instance.SerializeAsString(); + if (val.empty()) { + msg = "failed to serialize"; + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + return; + } + + txn->put(key, val); + LOG(INFO) << "put instance_id=" << instance_id << " instance_key=" << hex(key); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to commit kv txn, err={}", err); + LOG(WARNING) << msg; + } +} + +void MetaServiceImpl::update_ak_sk(google::protobuf::RpcController* controller, + const UpdateAkSkRequest* request, UpdateAkSkResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(update_ak_sk); + instance_id = request->has_instance_id() ? request->instance_id() : ""; + if (instance_id.empty()) { + msg = "instance id not set"; + code = MetaServiceCode::INVALID_ARGUMENT; + return; + } + if (!request->has_ram_user() && request->internal_bucket_user().empty()) { + msg = "nothing to update"; + code = MetaServiceCode::INVALID_ARGUMENT; + return; + } + RPC_RATE_LIMIT(update_ak_sk) + + InstanceKeyInfo key_info {instance_id}; + std::string key; + std::string val; + instance_key(key_info, &key); + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return; + } + err = txn->get(key, &val); + LOG(INFO) << "get instance_key=" << hex(key); + + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get instance, instance_id=" << instance_id << " err=" << err; + msg = ss.str(); + return; + } + + InstanceInfoPB instance; + if (!instance.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse InstanceInfoPB"; + return; + } + + if (instance.status() == InstanceInfoPB::DELETED) { + code = MetaServiceCode::CLUSTER_NOT_FOUND; + msg = "instance status has been set delete, plz check it"; + return; + } + + auto now_time = std::chrono::system_clock::now(); + uint64_t time = + std::chrono::duration_cast(now_time.time_since_epoch()).count(); + + std::stringstream update_record; + + // if has ram_user, encrypt and save it + if (request->has_ram_user()) { + if (request->ram_user().user_id().empty() || request->ram_user().ak().empty() || + request->ram_user().sk().empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "ram user info err " + proto_to_json(*request); + return; + } + if (!instance.has_ram_user()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "instance doesn't have ram user info"; + return; + } + auto& ram_user = request->ram_user(); + EncryptionInfoPB encryption_info; + AkSkPair cipher_ak_sk_pair; + if (encrypt_ak_sk_helper(ram_user.ak(), ram_user.sk(), &encryption_info, &cipher_ak_sk_pair, + code, msg) != 0) { + return; + } + const auto& [ak, sk] = cipher_ak_sk_pair; + auto& instance_ram_user = *instance.mutable_ram_user(); + if (ram_user.user_id() != instance_ram_user.user_id()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "ram user_id err"; + return; + } + std::string old_ak = instance_ram_user.ak(); + std::string old_sk = instance_ram_user.sk(); + if (old_ak == ak && old_sk == sk) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "ak sk eq original, please check it"; + return; + } + instance_ram_user.set_ak(std::move(cipher_ak_sk_pair.first)); + instance_ram_user.set_sk(std::move(cipher_ak_sk_pair.second)); + instance_ram_user.mutable_encryption_info()->CopyFrom(encryption_info); + update_record << "update ram_user's ak sk, instance_id: " << instance_id + << " user_id: " << ram_user.user_id() << " old: cipher ak: " << old_ak + << " cipher sk: " << old_sk << " new: cipher ak: " << ak + << " cipher sk: " << sk; + } + + bool has_found_alter_obj_info = false; + for (auto& alter_bucket_user : request->internal_bucket_user()) { + if (!alter_bucket_user.has_ak() || !alter_bucket_user.has_sk() || + !alter_bucket_user.has_user_id()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "s3 bucket info err " + proto_to_json(*request); + return; + } + std::string user_id = alter_bucket_user.user_id(); + EncryptionInfoPB encryption_info; + AkSkPair cipher_ak_sk_pair; + if (encrypt_ak_sk_helper(alter_bucket_user.ak(), alter_bucket_user.sk(), &encryption_info, + &cipher_ak_sk_pair, code, msg) != 0) { + return; + } + const auto& [ak, sk] = cipher_ak_sk_pair; + auto& obj_info = + const_cast&>(instance.obj_info()); + for (auto& it : obj_info) { + std::string old_ak = it.ak(); + std::string old_sk = it.sk(); + if (!it.has_user_id()) { + has_found_alter_obj_info = true; + // For compatibility, obj_info without a user_id only allow + // single internal_bucket_user to modify it. + if (request->internal_bucket_user_size() != 1) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "fail to update old instance's obj_info, s3 obj info err " + + proto_to_json(*request); + return; + } + if (it.ak() == ak && it.sk() == sk) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "ak sk eq original, please check it"; + return; + } + it.set_mtime(time); + it.set_user_id(user_id); + it.set_ak(ak); + it.set_sk(sk); + it.mutable_encryption_info()->CopyFrom(encryption_info); + update_record << "update obj_info's ak sk without user_id, instance_id: " + << instance_id << " obj_info_id: " << it.id() + << " new user_id: " << user_id << " old: cipher ak: " << old_ak + << " cipher sk: " << old_sk << " new: cipher ak: " << ak + << " cipher sk: " << sk; + continue; + } + if (it.user_id() == user_id) { + has_found_alter_obj_info = true; + if (it.ak() == ak && it.sk() == sk) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "ak sk eq original, please check it"; + return; + } + it.set_mtime(time); + it.set_ak(ak); + it.set_sk(sk); + it.mutable_encryption_info()->CopyFrom(encryption_info); + update_record << "update obj_info's ak sk, instance_id: " << instance_id + << " obj_info_id: " << it.id() << " user_id: " << user_id + << " old: cipher ak: " << old_ak << " cipher sk: " << old_sk + << " new: cipher ak: " << ak << " cipher sk: " << sk; + } + } + } + + if (!request->internal_bucket_user().empty() && !has_found_alter_obj_info) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "fail to find the alter obj info"; + return; + } + + LOG(INFO) << "instance " << instance_id << " has " << instance.obj_info().size() + << " s3 history info, and instance = " << proto_to_json(instance); + + val = instance.SerializeAsString(); + if (val.empty()) { + msg = "failed to serialize"; + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + return; + } + + txn->put(key, val); + LOG(INFO) << "put instance_id=" << instance_id << " instance_key=" << hex(key); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to commit kv txn, err={}", err); + LOG(WARNING) << msg; + } + LOG(INFO) << update_record.str(); +} + +void MetaServiceImpl::create_instance(google::protobuf::RpcController* controller, + const CreateInstanceRequest* request, + CreateInstanceResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(create_instance); + instance_id = request->instance_id(); + // Prepare data + auto& obj = request->obj_info(); + std::string plain_ak = obj.has_ak() ? obj.ak() : ""; + std::string plain_sk = obj.has_sk() ? obj.sk() : ""; + std::string bucket = obj.has_bucket() ? obj.bucket() : ""; + std::string prefix = obj.has_prefix() ? obj.prefix() : ""; + // format prefix, such as `/aa/bb/`, `aa/bb//`, `//aa/bb`, ` /aa/bb` -> `aa/bb` + prefix = trim(prefix); + std::string endpoint = obj.has_endpoint() ? obj.endpoint() : ""; + std::string external_endpoint = obj.has_external_endpoint() ? obj.external_endpoint() : ""; + std::string region = obj.has_region() ? obj.region() : ""; + + // ATTN: prefix may be empty + if (plain_ak.empty() || plain_sk.empty() || bucket.empty() || endpoint.empty() || + region.empty() || !obj.has_provider() || external_endpoint.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "s3 conf info err, please check it"; + return; + } + + if (request->has_ram_user()) { + auto& ram_user = request->ram_user(); + std::string ram_user_id = ram_user.has_user_id() ? ram_user.user_id() : ""; + std::string ram_user_ak = ram_user.has_ak() ? ram_user.ak() : ""; + std::string ram_user_sk = ram_user.has_sk() ? ram_user.sk() : ""; + if (ram_user_id.empty() || ram_user_ak.empty() || ram_user_sk.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "ram user info err, please check it"; + return; + } + } + + EncryptionInfoPB encryption_info; + AkSkPair cipher_ak_sk_pair; + if (encrypt_ak_sk_helper(plain_ak, plain_sk, &encryption_info, &cipher_ak_sk_pair, code, msg) != + 0) { + return; + } + InstanceInfoPB instance; + instance.set_instance_id(instance_id); + instance.set_user_id(request->has_user_id() ? request->user_id() : ""); + instance.set_name(request->has_name() ? request->name() : ""); + instance.set_status(InstanceInfoPB::NORMAL); + instance.set_sse_enabled(request->sse_enabled()); + auto obj_info = instance.add_obj_info(); + if (obj.has_user_id()) { + obj_info->set_user_id(obj.user_id()); + } + obj_info->set_ak(std::move(cipher_ak_sk_pair.first)); + obj_info->set_sk(std::move(cipher_ak_sk_pair.second)); + obj_info->mutable_encryption_info()->CopyFrom(encryption_info); + obj_info->set_bucket(bucket); + obj_info->set_prefix(prefix); + obj_info->set_endpoint(endpoint); + obj_info->set_external_endpoint(external_endpoint); + obj_info->set_region(region); + obj_info->set_provider(obj.provider()); + std::ostringstream oss; + // create instance's s3 conf, id = 1 + obj_info->set_id(std::to_string(1)); + auto now_time = std::chrono::system_clock::now(); + uint64_t time = + std::chrono::duration_cast(now_time.time_since_epoch()).count(); + obj_info->set_ctime(time); + obj_info->set_mtime(time); + obj_info->set_sse_enabled(instance.sse_enabled()); + if (request->has_ram_user()) { + auto& ram_user = request->ram_user(); + EncryptionInfoPB encryption_info; + AkSkPair cipher_ak_sk_pair; + if (encrypt_ak_sk_helper(ram_user.ak(), ram_user.sk(), &encryption_info, &cipher_ak_sk_pair, + code, msg) != 0) { + return; + } + RamUserPB new_ram_user; + new_ram_user.CopyFrom(ram_user); + new_ram_user.set_ak(std::move(cipher_ak_sk_pair.first)); + new_ram_user.set_sk(std::move(cipher_ak_sk_pair.second)); + new_ram_user.mutable_encryption_info()->CopyFrom(encryption_info); + instance.mutable_ram_user()->CopyFrom(new_ram_user); + } + + if (instance.instance_id().empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "instance id not set"; + return; + } + + InstanceKeyInfo key_info {request->instance_id()}; + std::string key; + std::string val = instance.SerializeAsString(); + instance_key(key_info, &key); + if (val.empty()) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + msg = "failed to serialize"; + LOG(ERROR) << msg; + return; + } + + LOG(INFO) << "xxx instance json=" << proto_to_json(instance); + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return; + } + + // Check existence before proceeding + err = txn->get(key, &val); + if (err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + std::stringstream ss; + ss << (err == TxnErrorCode::TXN_OK ? "instance already existed" + : "internal error failed to check instance") + << ", instance_id=" << request->instance_id(); + code = err == TxnErrorCode::TXN_OK ? MetaServiceCode::ALREADY_EXISTED + : cast_as(err); + msg = ss.str(); + LOG(WARNING) << msg << " err=" << err; + return; + } + + txn->put(key, val); + LOG(INFO) << "put instance_id=" << request->instance_id() << " instance_key=" << hex(key); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to commit kv txn, err={}", err); + LOG(WARNING) << msg; + } +} + +void MetaServiceImpl::alter_instance(google::protobuf::RpcController* controller, + const AlterInstanceRequest* request, + AlterInstanceResponse* response, + ::google::protobuf::Closure* done) { + StopWatch sw; + auto ctrl = static_cast(controller); + LOG(INFO) << __PRETTY_FUNCTION__ << " rpc from " << ctrl->remote_side() + << " request=" << request->ShortDebugString(); + brpc::ClosureGuard closure_guard(done); + MetaServiceCode code = MetaServiceCode::OK; + std::string msg = "OK"; + [[maybe_unused]] std::stringstream ss; + std::string instance_id = request->has_instance_id() ? request->instance_id() : ""; + std::unique_ptr> defer_status( + (int*)0x01, [&code, &msg, &response, &ctrl, &closure_guard, &sw, &instance_id](int*) { + response->mutable_status()->set_code(code); + response->mutable_status()->set_msg(msg); + LOG(INFO) << (code == MetaServiceCode::OK ? "succ to " : "failed to ") + << __PRETTY_FUNCTION__ << " " << ctrl->remote_side() << " " << msg; + closure_guard.reset(nullptr); + if (config::use_detailed_metrics && !instance_id.empty()) { + g_bvar_ms_alter_instance.put(instance_id, sw.elapsed_us()); + } + }); + + std::pair ret; + switch (request->op()) { + case AlterInstanceRequest::DROP: { + ret = alter_instance(request, [&request](InstanceInfoPB* instance) { + std::string msg; + // check instance doesn't have any cluster. + if (instance->clusters_size() != 0) { + msg = "failed to drop instance, instance has clusters"; + LOG(WARNING) << msg; + return std::make_pair(MetaServiceCode::INVALID_ARGUMENT, msg); + } + + instance->set_status(InstanceInfoPB::DELETED); + instance->set_mtime( + duration_cast(system_clock::now().time_since_epoch()).count()); + + std::string ret = instance->SerializeAsString(); + if (ret.empty()) { + msg = "failed to serialize"; + LOG(ERROR) << msg; + return std::make_pair(MetaServiceCode::PROTOBUF_SERIALIZE_ERR, msg); + } + LOG(INFO) << "put instance_id=" << request->instance_id() + << "drop instance json=" << proto_to_json(*instance); + return std::make_pair(MetaServiceCode::OK, ret); + }); + } break; + case AlterInstanceRequest::RENAME: { + ret = alter_instance(request, [&request](InstanceInfoPB* instance) { + std::string msg; + std::string name = request->has_name() ? request->name() : ""; + if (name.empty()) { + msg = "rename instance name, but not set"; + LOG(WARNING) << msg; + return std::make_pair(MetaServiceCode::INVALID_ARGUMENT, msg); + } + instance->set_name(name); + + std::string ret = instance->SerializeAsString(); + if (ret.empty()) { + msg = "failed to serialize"; + LOG(ERROR) << msg; + return std::make_pair(MetaServiceCode::PROTOBUF_SERIALIZE_ERR, msg); + } + LOG(INFO) << "put instance_id=" << request->instance_id() + << "rename instance json=" << proto_to_json(*instance); + return std::make_pair(MetaServiceCode::OK, ret); + }); + } break; + case AlterInstanceRequest::ENABLE_SSE: { + ret = alter_instance(request, [&request](InstanceInfoPB* instance) { + std::string msg; + if (instance->sse_enabled()) { + msg = "failed to enable sse, instance has enabled sse"; + LOG(WARNING) << msg; + return std::make_pair(MetaServiceCode::INVALID_ARGUMENT, msg); + } + instance->set_sse_enabled(true); + instance->set_mtime( + duration_cast(system_clock::now().time_since_epoch()).count()); + + for (auto& obj_info : *(instance->mutable_obj_info())) { + obj_info.set_sse_enabled(true); + } + std::string ret = instance->SerializeAsString(); + if (ret.empty()) { + msg = "failed to serialize"; + LOG(ERROR) << msg; + return std::make_pair(MetaServiceCode::PROTOBUF_SERIALIZE_ERR, msg); + } + LOG(INFO) << "put instance_id=" << request->instance_id() + << "instance enable sse json=" << proto_to_json(*instance); + return std::make_pair(MetaServiceCode::OK, ret); + }); + } break; + case AlterInstanceRequest::DISABLE_SSE: { + ret = alter_instance(request, [&request](InstanceInfoPB* instance) { + std::string msg; + if (!instance->sse_enabled()) { + msg = "failed to disable sse, instance has disabled sse"; + LOG(WARNING) << msg; + return std::make_pair(MetaServiceCode::INVALID_ARGUMENT, msg); + } + instance->set_sse_enabled(false); + instance->set_mtime( + duration_cast(system_clock::now().time_since_epoch()).count()); + + for (auto& obj_info : *(instance->mutable_obj_info())) { + obj_info.set_sse_enabled(false); + } + std::string ret = instance->SerializeAsString(); + if (ret.empty()) { + msg = "failed to serialize"; + LOG(ERROR) << msg; + return std::make_pair(MetaServiceCode::PROTOBUF_SERIALIZE_ERR, msg); + } + LOG(INFO) << "put instance_id=" << request->instance_id() + << "instance disable sse json=" << proto_to_json(*instance); + return std::make_pair(MetaServiceCode::OK, ret); + }); + } break; + case AlterInstanceRequest::REFRESH: { + ret = resource_mgr_->refresh_instance(request->instance_id()); + } break; + case AlterInstanceRequest::SET_OVERDUE: { + ret = alter_instance(request, [&request](InstanceInfoPB* instance) { + std::string msg; + + if (instance->status() == InstanceInfoPB::DELETED) { + msg = "can't set deleted instance to overdue, instance_id = " + + request->instance_id(); + LOG(WARNING) << msg; + return std::make_pair(MetaServiceCode::INVALID_ARGUMENT, msg); + } + if (instance->status() == InstanceInfoPB::OVERDUE) { + msg = "the instance has already set instance to overdue, instance_id = " + + request->instance_id(); + LOG(WARNING) << msg; + return std::make_pair(MetaServiceCode::INVALID_ARGUMENT, msg); + } + instance->set_status(InstanceInfoPB::OVERDUE); + instance->set_mtime( + duration_cast(system_clock::now().time_since_epoch()).count()); + + std::string ret = instance->SerializeAsString(); + if (ret.empty()) { + msg = "failed to serialize"; + LOG(WARNING) << msg; + return std::make_pair(MetaServiceCode::PROTOBUF_SERIALIZE_ERR, msg); + } + LOG(INFO) << "put instance_id=" << request->instance_id() + << "set instance overdue json=" << proto_to_json(*instance); + return std::make_pair(MetaServiceCode::OK, ret); + }); + } break; + case AlterInstanceRequest::SET_NORMAL: { + ret = alter_instance(request, [&request](InstanceInfoPB* instance) { + std::string msg; + + if (instance->status() == InstanceInfoPB::DELETED) { + msg = "can't set deleted instance to normal, instance_id = " + + request->instance_id(); + LOG(WARNING) << msg; + return std::make_pair(MetaServiceCode::INVALID_ARGUMENT, msg); + } + if (instance->status() == InstanceInfoPB::NORMAL) { + msg = "the instance is already normal, instance_id = " + request->instance_id(); + LOG(WARNING) << msg; + return std::make_pair(MetaServiceCode::INVALID_ARGUMENT, msg); + } + instance->set_status(InstanceInfoPB::NORMAL); + instance->set_mtime( + duration_cast(system_clock::now().time_since_epoch()).count()); + + std::string ret = instance->SerializeAsString(); + if (ret.empty()) { + msg = "failed to serialize"; + LOG(WARNING) << msg; + return std::make_pair(MetaServiceCode::PROTOBUF_SERIALIZE_ERR, msg); + } + LOG(INFO) << "put instance_id=" << request->instance_id() + << "set instance normal json=" << proto_to_json(*instance); + return std::make_pair(MetaServiceCode::OK, ret); + }); + } break; + default: { + ss << "invalid request op, op=" << request->op(); + ret = std::make_pair(MetaServiceCode::INVALID_ARGUMENT, ss.str()); + } + } + code = ret.first; + msg = ret.second; + + if (request->op() == AlterInstanceRequest::REFRESH) return; + + auto f = new std::function([instance_id = request->instance_id(), txn_kv = txn_kv_] { + notify_refresh_instance(txn_kv, instance_id); + }); + bthread_t bid; + if (bthread_start_background(&bid, nullptr, run_bthread_work, f) != 0) { + LOG(WARNING) << "notify refresh instance inplace, instance_id=" << request->instance_id(); + run_bthread_work(f); + } +} + +void MetaServiceImpl::get_instance(google::protobuf::RpcController* controller, + const GetInstanceRequest* request, GetInstanceResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(get_instance); + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + if (cloud_unique_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "cloud_unique_id must be given"; + return; + } + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << cloud_unique_id; + return; + } + RPC_RATE_LIMIT(get_instance); + InstanceKeyInfo key_info {instance_id}; + std::string key; + std::string val; + instance_key(key_info, &key); + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return; + } + err = txn->get(key, &val); + LOG(INFO) << "get instance_key=" << hex(key); + + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get instance, instance_id=" << instance_id << " err=" << err; + msg = ss.str(); + return; + } + + InstanceInfoPB instance; + if (!instance.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse InstanceInfoPB"; + return; + } + + response->mutable_instance()->CopyFrom(instance); +} + +std::pair MetaServiceImpl::alter_instance( + const cloud::AlterInstanceRequest* request, + std::function(InstanceInfoPB*)> action) { + MetaServiceCode code = MetaServiceCode::OK; + std::string msg = "OK"; + std::string instance_id = request->has_instance_id() ? request->instance_id() : ""; + if (instance_id.empty()) { + msg = "instance id not set"; + LOG(WARNING) << msg; + return std::make_pair(MetaServiceCode::INVALID_ARGUMENT, msg); + } + + InstanceKeyInfo key_info {instance_id}; + std::string key; + std::string val; + instance_key(key_info, &key); + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return std::make_pair(cast_as(err), msg); + } + + // Check existence before proceeding + err = txn->get(key, &val); + if (err != TxnErrorCode::TXN_OK) { + std::stringstream ss; + ss << (err == TxnErrorCode::TXN_KEY_NOT_FOUND ? "instance not existed" + : "internal error failed to check instance") + << ", instance_id=" << request->instance_id(); + // TODO(dx): fix CLUSTER_NOT_FOUND,VERSION_NOT_FOUND,TXN_LABEL_NOT_FOUND,etc to NOT_FOUND + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::CLUSTER_NOT_FOUND + : cast_as(err); + msg = ss.str(); + LOG(WARNING) << msg << " err=" << err; + return std::make_pair(code, msg); + } + LOG(INFO) << "alter instance key=" << hex(key); + InstanceInfoPB instance; + if (!instance.ParseFromString(val)) { + msg = "failed to parse InstanceInfoPB"; + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + LOG(WARNING) << msg; + return std::make_pair(code, msg); + } + auto r = action(&instance); + if (r.first != MetaServiceCode::OK) { + return r; + } + val = r.second; + txn->put(key, val); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to commit kv txn, err={}", err); + LOG(WARNING) << msg; + return std::make_pair(code, msg); + } + return std::make_pair(code, msg); +} + +void MetaServiceImpl::alter_cluster(google::protobuf::RpcController* controller, + const AlterClusterRequest* request, + AlterClusterResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(alter_cluster); + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + instance_id = request->has_instance_id() ? request->instance_id() : ""; + if (!cloud_unique_id.empty() && instance_id.empty()) { + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << cloud_unique_id; + return; + } + } + + if (instance_id.empty() || !request->has_cluster()) { + msg = "invalid request instance_id or cluster not given"; + code = MetaServiceCode::INVALID_ARGUMENT; + return; + } + + if (!request->has_op()) { + msg = "op not given"; + code = MetaServiceCode::INVALID_ARGUMENT; + return; + } + + LOG(INFO) << "alter cluster instance_id=" << instance_id << " op=" << request->op(); + ClusterInfo cluster; + cluster.cluster.CopyFrom(request->cluster()); + + switch (request->op()) { + case AlterClusterRequest::ADD_CLUSTER: { + auto r = resource_mgr_->add_cluster(instance_id, cluster); + code = r.first; + msg = r.second; + } break; + case AlterClusterRequest::DROP_CLUSTER: { + auto r = resource_mgr_->drop_cluster(instance_id, cluster); + code = r.first; + msg = r.second; + } break; + case AlterClusterRequest::UPDATE_CLUSTER_MYSQL_USER_NAME: { + msg = resource_mgr_->update_cluster( + instance_id, cluster, + [&](const ClusterPB& i) { return i.cluster_id() == cluster.cluster.cluster_id(); }, + [&](ClusterPB& c, std::set&) { + auto& mysql_user_names = cluster.cluster.mysql_user_name(); + c.mutable_mysql_user_name()->CopyFrom(mysql_user_names); + return ""; + }); + } break; + case AlterClusterRequest::ADD_NODE: { + resource_mgr_->check_cluster_params_valid(request->cluster(), &msg, false); + if (msg != "") { + LOG(INFO) << msg; + break; + } + std::vector to_add; + std::vector to_del; + for (auto& n : request->cluster().nodes()) { + NodeInfo node; + node.instance_id = request->instance_id(); + node.node_info = n; + node.cluster_id = request->cluster().cluster_id(); + node.cluster_name = request->cluster().cluster_name(); + node.role = + (request->cluster().type() == ClusterPB::SQL + ? Role::SQL_SERVER + : (request->cluster().type() == ClusterPB::COMPUTE ? Role::COMPUTE_NODE + : Role::UNDEFINED)); + node.node_info.set_status(NodeStatusPB::NODE_STATUS_RUNNING); + to_add.emplace_back(std::move(node)); + } + msg = resource_mgr_->modify_nodes(instance_id, to_add, to_del); + } break; + case AlterClusterRequest::DROP_NODE: { + resource_mgr_->check_cluster_params_valid(request->cluster(), &msg, false); + if (msg != "") { + LOG(INFO) << msg; + break; + } + std::vector to_add; + std::vector to_del; + for (auto& n : request->cluster().nodes()) { + NodeInfo node; + node.instance_id = request->instance_id(); + node.node_info = n; + node.cluster_id = request->cluster().cluster_id(); + node.cluster_name = request->cluster().cluster_name(); + node.role = + (request->cluster().type() == ClusterPB::SQL + ? Role::SQL_SERVER + : (request->cluster().type() == ClusterPB::COMPUTE ? Role::COMPUTE_NODE + : Role::UNDEFINED)); + to_del.emplace_back(std::move(node)); + } + msg = resource_mgr_->modify_nodes(instance_id, to_add, to_del); + } break; + case AlterClusterRequest::DECOMMISSION_NODE: { + resource_mgr_->check_cluster_params_valid(request->cluster(), &msg, false); + if (msg != "") { + LOG(INFO) << msg; + break; + } + + std::string be_unique_id = (request->cluster().nodes())[0].cloud_unique_id(); + std::vector nodes; + std::string err = resource_mgr_->get_node(be_unique_id, &nodes); + if (!err.empty()) { + LOG(INFO) << "failed to check instance info, err=" << err; + msg = err; + break; + } + + std::vector decomission_nodes; + for (auto& node : nodes) { + for (auto req_node : request->cluster().nodes()) { + bool ip_processed = false; + if (node.node_info.has_ip() && req_node.has_ip()) { + std::string endpoint = node.node_info.ip() + ":" + + std::to_string(node.node_info.heartbeat_port()); + std::string req_endpoint = + req_node.ip() + ":" + std::to_string(req_node.heartbeat_port()); + if (endpoint == req_endpoint) { + decomission_nodes.push_back(node); + node.node_info.set_status(NodeStatusPB::NODE_STATUS_DECOMMISSIONING); + } + ip_processed = true; + } + + if (!ip_processed && node.node_info.has_host() && req_node.has_host()) { + std::string endpoint = node.node_info.host() + ":" + + std::to_string(node.node_info.heartbeat_port()); + std::string req_endpoint = + req_node.host() + ":" + std::to_string(req_node.heartbeat_port()); + if (endpoint == req_endpoint) { + decomission_nodes.push_back(node); + node.node_info.set_status(NodeStatusPB::NODE_STATUS_DECOMMISSIONING); + } + } + } + } + + { + std::vector to_add; + std::vector& to_del = decomission_nodes; + msg = resource_mgr_->modify_nodes(instance_id, to_add, to_del); + } + { + std::vector& to_add = decomission_nodes; + std::vector to_del; + for (auto& node : to_add) { + node.node_info.set_status(NodeStatusPB::NODE_STATUS_DECOMMISSIONING); + LOG(INFO) << "decomission node, " + << "size: " << to_add.size() << " " << node.node_info.DebugString() << " " + << node.cluster_id << " " << node.cluster_name; + } + msg = resource_mgr_->modify_nodes(instance_id, to_add, to_del); + } + } break; + case AlterClusterRequest::NOTIFY_DECOMMISSIONED: { + resource_mgr_->check_cluster_params_valid(request->cluster(), &msg, false); + if (msg != "") { + LOG(INFO) << msg; + break; + } + + std::string be_unique_id = (request->cluster().nodes())[0].cloud_unique_id(); + std::vector nodes; + std::string err = resource_mgr_->get_node(be_unique_id, &nodes); + if (!err.empty()) { + LOG(INFO) << "failed to check instance info, err=" << err; + msg = err; + break; + } + + std::vector decomission_nodes; + for (auto& node : nodes) { + for (auto req_node : request->cluster().nodes()) { + bool ip_processed = false; + if (node.node_info.has_ip() && req_node.has_ip()) { + std::string endpoint = node.node_info.ip() + ":" + + std::to_string(node.node_info.heartbeat_port()); + std::string req_endpoint = + req_node.ip() + ":" + std::to_string(req_node.heartbeat_port()); + if (endpoint == req_endpoint) { + decomission_nodes.push_back(node); + } + ip_processed = true; + } + + if (!ip_processed && node.node_info.has_host() && req_node.has_host()) { + std::string endpoint = node.node_info.host() + ":" + + std::to_string(node.node_info.heartbeat_port()); + std::string req_endpoint = + req_node.host() + ":" + std::to_string(req_node.heartbeat_port()); + if (endpoint == req_endpoint) { + decomission_nodes.push_back(node); + } + } + } + } + + { + std::vector to_add; + std::vector& to_del = decomission_nodes; + msg = resource_mgr_->modify_nodes(instance_id, to_add, to_del); + } + { + std::vector& to_add = decomission_nodes; + std::vector to_del; + for (auto& node : to_add) { + node.node_info.set_status(NodeStatusPB::NODE_STATUS_DECOMMISSIONED); + LOG(INFO) << "notify node decomissioned, " + << " size: " << to_add.size() << " " << node.node_info.DebugString() + << " " << node.cluster_id << " " << node.cluster_name; + } + msg = resource_mgr_->modify_nodes(instance_id, to_add, to_del); + } + } break; + case AlterClusterRequest::RENAME_CLUSTER: { + msg = resource_mgr_->update_cluster( + instance_id, cluster, + [&](const ClusterPB& i) { return i.cluster_id() == cluster.cluster.cluster_id(); }, + [&](ClusterPB& c, std::set& cluster_names) { + std::string msg; + auto it = cluster_names.find(cluster.cluster.cluster_name()); + LOG(INFO) << "cluster.cluster.cluster_name(): " + << cluster.cluster.cluster_name(); + for (auto itt : cluster_names) { + LOG(INFO) << "itt : " << itt; + } + if (it != cluster_names.end()) { + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "failed to rename cluster, a cluster with the same name already " + "exists in this instance " + << proto_to_json(c); + msg = ss.str(); + return msg; + } + if (c.cluster_name() == cluster.cluster.cluster_name()) { + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "failed to rename cluster, name eq original name, original cluster " + "is " + << proto_to_json(c); + msg = ss.str(); + return msg; + } + c.set_cluster_name(cluster.cluster.cluster_name()); + return msg; + }); + } break; + case AlterClusterRequest::UPDATE_CLUSTER_ENDPOINT: { + msg = resource_mgr_->update_cluster( + instance_id, cluster, + [&](const ClusterPB& i) { return i.cluster_id() == cluster.cluster.cluster_id(); }, + [&](ClusterPB& c, std::set&) { + std::string msg; + if (!cluster.cluster.has_private_endpoint() || + cluster.cluster.private_endpoint().empty()) { + code = MetaServiceCode::CLUSTER_ENDPOINT_MISSING; + ss << "missing private endpoint"; + msg = ss.str(); + return msg; + } + + c.set_public_endpoint(cluster.cluster.public_endpoint()); + c.set_private_endpoint(cluster.cluster.private_endpoint()); + + return msg; + }); + } break; + case AlterClusterRequest::SET_CLUSTER_STATUS: { + msg = resource_mgr_->update_cluster( + instance_id, cluster, + [&](const ClusterPB& i) { return i.cluster_id() == cluster.cluster.cluster_id(); }, + [&](ClusterPB& c, std::set&) { + std::string msg; + if (c.cluster_status() == request->cluster().cluster_status()) { + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "failed to set cluster status, status eq original status, original " + "cluster is " + << print_cluster_status(c.cluster_status()); + msg = ss.str(); + return msg; + } + // status from -> to + std::set> + can_work_directed_edges { + {ClusterStatus::UNKNOWN, ClusterStatus::NORMAL}, + {ClusterStatus::NORMAL, ClusterStatus::SUSPENDED}, + {ClusterStatus::SUSPENDED, ClusterStatus::TO_RESUME}, + {ClusterStatus::TO_RESUME, ClusterStatus::NORMAL}, + {ClusterStatus::SUSPENDED, ClusterStatus::NORMAL}, + }; + auto from = c.cluster_status(); + auto to = request->cluster().cluster_status(); + if (can_work_directed_edges.count({from, to}) == 0) { + // can't find a directed edge in set, so refuse it + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "failed to set cluster status, original cluster is " + << print_cluster_status(from) << " and want set " + << print_cluster_status(to); + msg = ss.str(); + return msg; + } + c.set_cluster_status(request->cluster().cluster_status()); + return msg; + }); + } break; + default: { + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "invalid request op, op=" << request->op(); + msg = ss.str(); + return; + } + } + if (!msg.empty() && code == MetaServiceCode::OK) { + code = MetaServiceCode::UNDEFINED_ERR; + } + + if (code != MetaServiceCode::OK) return; + + auto f = new std::function([instance_id = request->instance_id(), txn_kv = txn_kv_] { + notify_refresh_instance(txn_kv, instance_id); + }); + bthread_t bid; + if (bthread_start_background(&bid, nullptr, run_bthread_work, f) != 0) { + LOG(WARNING) << "notify refresh instance inplace, instance_id=" << request->instance_id(); + run_bthread_work(f); + } +} // alter cluster + +void MetaServiceImpl::get_cluster(google::protobuf::RpcController* controller, + const GetClusterRequest* request, GetClusterResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(get_cluster); + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + std::string cluster_id = request->has_cluster_id() ? request->cluster_id() : ""; + std::string cluster_name = request->has_cluster_name() ? request->cluster_name() : ""; + std::string mysql_user_name = request->has_mysql_user_name() ? request->mysql_user_name() : ""; + + if (cloud_unique_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "cloud_unique_id must be given"; + return; + } + + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + if (request->has_instance_id()) { + instance_id = request->instance_id(); + // FIXME(gavin): this mechanism benifits debugging and + // administration, is it dangerous? + LOG(WARNING) << "failed to get instance_id with cloud_unique_id=" << cloud_unique_id + << " use the given instance_id=" << instance_id << " instead"; + } else { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << cloud_unique_id; + return; + } + } + RPC_RATE_LIMIT(get_cluster) + // ATTN: if the case that multiple conditions are satisfied, just use by this order: + // cluster_id -> cluster_name -> mysql_user_name + if (!cluster_id.empty()) { + cluster_name = ""; + mysql_user_name = ""; + } else if (!cluster_name.empty()) { + mysql_user_name = ""; + } + + bool get_all_cluster_info = false; + // if cluster_id、cluster_name、mysql_user_name all empty, get this instance's all cluster info. + if (cluster_id.empty() && cluster_name.empty() && mysql_user_name.empty()) { + get_all_cluster_info = true; + } + + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "failed to get instance_id with cloud_unique_id=" + cloud_unique_id; + return; + } + + InstanceKeyInfo key_info {instance_id}; + std::string key; + std::string val; + instance_key(key_info, &key); + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return; + } + err = txn->get(key, &val); + LOG(INFO) << "get instance_key=" << hex(key); + + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get instance, instance_id=" << instance_id << " err=" << err; + msg = ss.str(); + return; + } + + InstanceInfoPB instance; + if (!instance.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse InstanceInfoPB"; + return; + } + + auto get_cluster_mysql_user = [](const ClusterPB& c, std::set* mysql_users) { + for (int i = 0; i < c.mysql_user_name_size(); i++) { + mysql_users->emplace(c.mysql_user_name(i)); + } + }; + + if (get_all_cluster_info) { + response->mutable_cluster()->CopyFrom(instance.clusters()); + LOG_EVERY_N(INFO, 100) << "get all cluster info, " << msg; + } else { + for (int i = 0; i < instance.clusters_size(); ++i) { + auto& c = instance.clusters(i); + std::set mysql_users; + get_cluster_mysql_user(c, &mysql_users); + // The last wins if add_cluster() does not ensure uniqueness of + // cluster_id and cluster_name respectively + if ((c.has_cluster_name() && c.cluster_name() == cluster_name) || + (c.has_cluster_id() && c.cluster_id() == cluster_id) || + mysql_users.count(mysql_user_name)) { + // just one cluster + response->add_cluster()->CopyFrom(c); + LOG_EVERY_N(INFO, 100) << "found a cluster, instance_id=" << instance.instance_id() + << " cluster=" << msg; + } + } + } + + if (response->cluster().size() == 0) { + ss << "fail to get cluster with " << request->ShortDebugString(); + msg = ss.str(); + std::replace(msg.begin(), msg.end(), '\n', ' '); + code = MetaServiceCode::CLUSTER_NOT_FOUND; + } +} // get_cluster + +void MetaServiceImpl::create_stage(::google::protobuf::RpcController* controller, + const CreateStageRequest* request, CreateStageResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(create_stage); + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + if (cloud_unique_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "cloud unique id not set"; + return; + } + + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << cloud_unique_id; + return; + } + RPC_RATE_LIMIT(create_stage) + + if (!request->has_stage()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "stage not set"; + return; + } + auto stage = request->stage(); + + if (!stage.has_type()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "stage type not set"; + return; + } + + if (stage.name().empty() && stage.type() == StagePB::EXTERNAL) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "stage name not set"; + return; + } + if (stage.stage_id().empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "stage id not set"; + return; + } + + if (stage.type() == StagePB::INTERNAL) { + if (stage.mysql_user_name().empty() || stage.mysql_user_id().empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "internal stage must have a mysql user name and id must be given, name size=" + << stage.mysql_user_name_size() << " id size=" << stage.mysql_user_id_size(); + msg = ss.str(); + LOG(WARNING) << msg; + return; + } + } + + InstanceKeyInfo key_info {instance_id}; + std::string key; + std::string val; + instance_key(key_info, &key); + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return; + } + err = txn->get(key, &val); + LOG(INFO) << "get instance_key=" << hex(key); + + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get instance, instance_id=" << instance_id << " err=" << err; + msg = ss.str(); + return; + } + + InstanceInfoPB instance; + if (!instance.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse InstanceInfoPB"; + return; + } + + VLOG_DEBUG << "config stages num=" << config::max_num_stages; + if (instance.stages_size() >= config::max_num_stages) { + code = MetaServiceCode::UNDEFINED_ERR; + msg = "this instance has greater than config num stages"; + LOG(WARNING) << "can't create more than config num stages, and instance has " + << std::to_string(instance.stages_size()); + return; + } + + // check if the stage exists + for (int i = 0; i < instance.stages_size(); ++i) { + auto& s = instance.stages(i); + if (stage.type() == StagePB::INTERNAL) { + // check all internal stage format is right + if (s.type() == StagePB::INTERNAL && s.mysql_user_id_size() == 0) { + LOG(WARNING) << "impossible, internal stage must have at least one id instance=" + << proto_to_json(instance); + } + + if (s.type() == StagePB::INTERNAL && + (s.mysql_user_id(0) == stage.mysql_user_id(0) || + s.mysql_user_name(0) == stage.mysql_user_name(0))) { + code = MetaServiceCode::ALREADY_EXISTED; + msg = "stage already exist"; + ss << "stage already exist, req user_name=" << stage.mysql_user_name(0) + << " existed user_name=" << s.mysql_user_name(0) + << "req user_id=" << stage.mysql_user_id(0) + << " existed user_id=" << s.mysql_user_id(0); + return; + } + } + + if (stage.type() == StagePB::EXTERNAL) { + if (s.name() == stage.name()) { + code = MetaServiceCode::ALREADY_EXISTED; + msg = "stage already exist"; + return; + } + } + + if (s.stage_id() == stage.stage_id()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "stage id is duplicated"; + return; + } + } + + if (stage.type() == StagePB::INTERNAL) { + if (instance.obj_info_size() == 0) { + LOG(WARNING) << "impossible, instance must have at least one obj_info."; + code = MetaServiceCode::UNDEFINED_ERR; + msg = "impossible, instance must have at least one obj_info."; + return; + } + auto& lastest_obj = instance.obj_info()[instance.obj_info_size() - 1]; + // ${obj_prefix}/stage/{username}/{user_id} + std::string mysql_user_name = stage.mysql_user_name(0); + std::string prefix = fmt::format("{}/stage/{}/{}", lastest_obj.prefix(), mysql_user_name, + stage.mysql_user_id(0)); + auto as = instance.add_stages(); + as->mutable_obj_info()->set_prefix(prefix); + as->mutable_obj_info()->set_id(lastest_obj.id()); + as->add_mysql_user_name(mysql_user_name); + as->add_mysql_user_id(stage.mysql_user_id(0)); + as->set_stage_id(stage.stage_id()); + } else if (stage.type() == StagePB::EXTERNAL) { + if (!stage.has_obj_info()) { + instance.add_stages()->CopyFrom(stage); + } else { + StagePB tmp_stage; + tmp_stage.CopyFrom(stage); + auto obj_info = tmp_stage.mutable_obj_info(); + EncryptionInfoPB encryption_info; + AkSkPair cipher_ak_sk_pair; + if (encrypt_ak_sk_helper(obj_info->ak(), obj_info->sk(), &encryption_info, + &cipher_ak_sk_pair, code, msg) != 0) { + return; + } + obj_info->set_ak(std::move(cipher_ak_sk_pair.first)); + obj_info->set_sk(std::move(cipher_ak_sk_pair.second)); + obj_info->mutable_encryption_info()->CopyFrom(encryption_info); + instance.add_stages()->CopyFrom(tmp_stage); + } + } + val = instance.SerializeAsString(); + if (val.empty()) { + msg = "failed to serialize"; + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + return; + } + + txn->put(key, val); + LOG(INFO) << "put instance_id=" << instance_id << " instance_key=" << hex(key) + << " json=" << proto_to_json(instance); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to commit kv txn, err={}", err); + LOG(WARNING) << msg; + } +} + +void MetaServiceImpl::get_stage(google::protobuf::RpcController* controller, + const GetStageRequest* request, GetStageResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(get_stage); + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + if (cloud_unique_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "cloud unique id not set"; + return; + } + + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << cloud_unique_id; + return; + } + RPC_RATE_LIMIT(get_stage) + if (!request->has_type()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "stage type not set"; + return; + } + auto type = request->type(); + + InstanceKeyInfo key_info {instance_id}; + std::string key; + std::string val; + instance_key(key_info, &key); + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return; + } + err = txn->get(key, &val); + LOG(INFO) << "get instance_key=" << hex(key); + + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get instance, instance_id=" << instance_id << " err=" << err; + msg = ss.str(); + return; + } + + InstanceInfoPB instance; + if (!instance.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse InstanceInfoPB"; + return; + } + + if (type == StagePB::INTERNAL) { + auto mysql_user_name = request->has_mysql_user_name() ? request->mysql_user_name() : ""; + if (mysql_user_name.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "mysql user name not set"; + return; + } + auto mysql_user_id = request->has_mysql_user_id() ? request->mysql_user_id() : ""; + if (mysql_user_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "mysql user id not set"; + return; + } + + // check mysql user_name has been created internal stage + auto& stage = instance.stages(); + bool found = false; + if (instance.obj_info_size() == 0) { + LOG(WARNING) << "impossible, instance must have at least one obj_info."; + code = MetaServiceCode::UNDEFINED_ERR; + msg = "impossible, instance must have at least one obj_info."; + return; + } + + for (auto s : stage) { + if (s.type() != StagePB::INTERNAL) { + continue; + } + if (s.mysql_user_name().size() == 0 || s.mysql_user_id().size() == 0) { + LOG(WARNING) << "impossible here, internal stage must have at least one user, " + "invalid stage=" + << proto_to_json(s); + continue; + } + if (s.mysql_user_name(0) == mysql_user_name) { + StagePB stage_pb; + // internal stage id is user_id, if user_id not eq internal stage's user_id, del it. + // let fe create a new internal stage + if (s.mysql_user_id(0) != mysql_user_id) { + LOG(INFO) << "ABA user=" << mysql_user_name + << " internal stage original user_id=" << s.mysql_user_id()[0] + << " rpc user_id=" << mysql_user_id + << " stage info=" << proto_to_json(s); + code = MetaServiceCode::STATE_ALREADY_EXISTED_FOR_USER; + msg = "aba user, drop stage and create a new one"; + // response return to be dropped stage id. + stage_pb.CopyFrom(s); + response->add_stage()->CopyFrom(stage_pb); + return; + } + // find, use it stage prefix and id + found = true; + // get from internal stage + int idx = stoi(s.obj_info().id()); + if (idx > instance.obj_info().size() || idx < 1) { + LOG(WARNING) << "invalid idx: " << idx; + code = MetaServiceCode::UNDEFINED_ERR; + msg = "impossible, id invalid"; + return; + } + auto& old_obj = instance.obj_info()[idx - 1]; + + stage_pb.mutable_obj_info()->set_ak(old_obj.ak()); + stage_pb.mutable_obj_info()->set_sk(old_obj.sk()); + if (old_obj.has_encryption_info()) { + AkSkPair plain_ak_sk_pair; + int ret = decrypt_ak_sk_helper(old_obj.ak(), old_obj.sk(), + old_obj.encryption_info(), &plain_ak_sk_pair, + code, msg); + if (ret != 0) return; + stage_pb.mutable_obj_info()->set_ak(std::move(plain_ak_sk_pair.first)); + stage_pb.mutable_obj_info()->set_sk(std::move(plain_ak_sk_pair.second)); + } + stage_pb.mutable_obj_info()->set_bucket(old_obj.bucket()); + stage_pb.mutable_obj_info()->set_endpoint(old_obj.endpoint()); + stage_pb.mutable_obj_info()->set_external_endpoint(old_obj.external_endpoint()); + stage_pb.mutable_obj_info()->set_region(old_obj.region()); + stage_pb.mutable_obj_info()->set_provider(old_obj.provider()); + stage_pb.mutable_obj_info()->set_prefix(s.obj_info().prefix()); + stage_pb.set_stage_id(s.stage_id()); + stage_pb.set_type(s.type()); + response->add_stage()->CopyFrom(stage_pb); + return; + } + } + if (!found) { + LOG(INFO) << "user=" << mysql_user_name + << " not have a valid stage, rpc user_id=" << mysql_user_id; + code = MetaServiceCode::STAGE_NOT_FOUND; + msg = "stage not found, create a new one"; + return; + } + } + + // get all external stages for display, but don't show ak/sk, so there is no need to decrypt ak/sk. + if (type == StagePB::EXTERNAL && !request->has_stage_name()) { + for (int i = 0; i < instance.stages_size(); ++i) { + auto& s = instance.stages(i); + if (s.type() != StagePB::EXTERNAL) { + continue; + } + response->add_stage()->CopyFrom(s); + } + return; + } + + // get external stage with the specified stage name + for (int i = 0; i < instance.stages_size(); ++i) { + auto& s = instance.stages(i); + if (s.type() == type && s.name() == request->stage_name()) { + StagePB stage; + stage.CopyFrom(s); + if (!stage.has_access_type() || stage.access_type() == StagePB::AKSK) { + stage.set_access_type(StagePB::AKSK); + auto obj_info = stage.mutable_obj_info(); + if (obj_info->has_encryption_info()) { + AkSkPair plain_ak_sk_pair; + int ret = decrypt_ak_sk_helper(obj_info->ak(), obj_info->sk(), + obj_info->encryption_info(), &plain_ak_sk_pair, + code, msg); + if (ret != 0) return; + obj_info->set_ak(std::move(plain_ak_sk_pair.first)); + obj_info->set_sk(std::move(plain_ak_sk_pair.second)); + } + } else if (stage.access_type() == StagePB::BUCKET_ACL) { + if (!instance.has_ram_user()) { + ss << "instance does not have ram user"; + msg = ss.str(); + code = MetaServiceCode::INVALID_ARGUMENT; + return; + } + if (instance.ram_user().has_encryption_info()) { + AkSkPair plain_ak_sk_pair; + int ret = decrypt_ak_sk_helper( + instance.ram_user().ak(), instance.ram_user().sk(), + instance.ram_user().encryption_info(), &plain_ak_sk_pair, code, msg); + if (ret != 0) return; + stage.mutable_obj_info()->set_ak(std::move(plain_ak_sk_pair.first)); + stage.mutable_obj_info()->set_sk(std::move(plain_ak_sk_pair.second)); + } else { + stage.mutable_obj_info()->set_ak(instance.ram_user().ak()); + stage.mutable_obj_info()->set_sk(instance.ram_user().sk()); + } + } else if (stage.access_type() == StagePB::IAM) { + std::string val; + TxnErrorCode err = txn->get(system_meta_service_arn_info_key(), &val); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + // For compatibility, use arn_info of config + stage.mutable_obj_info()->set_ak(config::arn_ak); + stage.mutable_obj_info()->set_sk(config::arn_sk); + stage.set_external_id(instance_id); + } else if (err == TxnErrorCode::TXN_OK) { + RamUserPB iam_user; + if (!iam_user.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse RamUserPB"; + return; + } + AkSkPair plain_ak_sk_pair; + int ret = decrypt_ak_sk_helper(iam_user.ak(), iam_user.sk(), + iam_user.encryption_info(), &plain_ak_sk_pair, + code, msg); + if (ret != 0) return; + stage.mutable_obj_info()->set_ak(std::move(plain_ak_sk_pair.first)); + stage.mutable_obj_info()->set_sk(std::move(plain_ak_sk_pair.second)); + stage.set_external_id(instance_id); + } else { + code = cast_as(err); + ss << "failed to get arn_info_key, err=" << err; + msg = ss.str(); + return; + } + } + response->add_stage()->CopyFrom(stage); + return; + } + } + + ss << "stage not found with " << proto_to_json(*request); + msg = ss.str(); + code = MetaServiceCode::STAGE_NOT_FOUND; +} + +void MetaServiceImpl::drop_stage(google::protobuf::RpcController* controller, + const DropStageRequest* request, DropStageResponse* response, + ::google::protobuf::Closure* done) { + StopWatch sw; + auto ctrl = static_cast(controller); + LOG(INFO) << "rpc from " << ctrl->remote_side() << " request=" << request->DebugString(); + brpc::ClosureGuard closure_guard(done); + int ret = 0; + MetaServiceCode code = MetaServiceCode::OK; + std::string msg = "OK"; + std::string instance_id; + bool drop_request = false; + std::unique_ptr> defer_status( + (int*)0x01, [&ret, &code, &msg, &response, &ctrl, &closure_guard, &sw, &instance_id, + &drop_request](int*) { + response->mutable_status()->set_code(code); + response->mutable_status()->set_msg(msg); + LOG(INFO) << (ret == 0 ? "succ to " : "failed to ") << __PRETTY_FUNCTION__ << " " + << ctrl->remote_side() << " " << msg; + closure_guard.reset(nullptr); + if (config::use_detailed_metrics && !instance_id.empty() && !drop_request) { + g_bvar_ms_drop_stage.put(instance_id, sw.elapsed_us()); + } + }); + + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + if (cloud_unique_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "cloud unique id not set"; + return; + } + + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << cloud_unique_id; + return; + } + RPC_RATE_LIMIT(drop_stage) + + if (!request->has_type()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "stage type not set"; + return; + } + auto type = request->type(); + + if (type == StagePB::EXTERNAL && request->stage_name().empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "external stage but not set stage name"; + return; + } + + if (type == StagePB::INTERNAL && request->mysql_user_id().empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "internal stage but not set user id"; + return; + } + + InstanceKeyInfo key_info {instance_id}; + std::string key; + std::string val; + instance_key(key_info, &key); + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return; + } + + err = txn->get(key, &val); + LOG(INFO) << "get instance_key=" << hex(key); + std::stringstream ss; + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get instance, instance_id=" << instance_id << " err=" << err; + msg = ss.str(); + return; + } + + InstanceInfoPB instance; + if (!instance.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse InstanceInfoPB"; + return; + } + + StagePB stage; + int idx = -1; + for (int i = 0; i < instance.stages_size(); ++i) { + auto& s = instance.stages(i); + if ((type == StagePB::INTERNAL && s.type() == StagePB::INTERNAL && + s.mysql_user_id(0) == request->mysql_user_id()) || + (type == StagePB::EXTERNAL && s.type() == StagePB::EXTERNAL && + s.name() == request->stage_name())) { + idx = i; + stage = s; + break; + } + } + if (idx == -1) { + ss << "stage not found with " << proto_to_json(*request); + msg = ss.str(); + code = MetaServiceCode::STAGE_NOT_FOUND; + return; + } + + auto& stages = const_cast&>(instance.stages()); + stages.DeleteSubrange(idx, 1); // Remove it + val = instance.SerializeAsString(); + if (val.empty()) { + msg = "failed to serialize"; + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + return; + } + txn->put(key, val); + LOG(INFO) << "put instance_id=" << instance_id << " instance_key=" << hex(key) + << " json=" << proto_to_json(instance); + + std::string key1; + std::string val1; + if (type == StagePB::INTERNAL) { + RecycleStageKeyInfo recycle_stage_key_info {instance_id, stage.stage_id()}; + recycle_stage_key(recycle_stage_key_info, &key1); + RecycleStagePB recycle_stage; + recycle_stage.set_instance_id(instance_id); + recycle_stage.set_reason(request->reason()); + recycle_stage.mutable_stage()->CopyFrom(stage); + val1 = recycle_stage.SerializeAsString(); + if (val1.empty()) { + msg = "failed to serialize"; + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + return; + } + txn->put(key1, val1); + } + + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to commit kv txn, err={}", err); + LOG(WARNING) << msg; + } +} + +void MetaServiceImpl::get_iam(google::protobuf::RpcController* controller, + const GetIamRequest* request, GetIamResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(get_iam); + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + if (cloud_unique_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "cloud unique id not set"; + return; + } + + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << cloud_unique_id; + return; + } + RPC_RATE_LIMIT(get_iam) + + InstanceKeyInfo key_info {instance_id}; + std::string key; + std::string val; + instance_key(key_info, &key); + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return; + } + err = txn->get(key, &val); + LOG(INFO) << "get instance_key=" << hex(key); + + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get instance, instance_id=" << instance_id << " err=" << err; + msg = ss.str(); + return; + } + + InstanceInfoPB instance; + if (!instance.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse InstanceInfoPB"; + return; + } + + val.clear(); + err = txn->get(system_meta_service_arn_info_key(), &val); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + // For compatibility, use arn_info of config + RamUserPB iam_user; + iam_user.set_user_id(config::arn_id); + iam_user.set_external_id(instance_id); + iam_user.set_ak(config::arn_ak); + iam_user.set_sk(config::arn_sk); + response->mutable_iam_user()->CopyFrom(iam_user); + } else if (err == TxnErrorCode::TXN_OK) { + RamUserPB iam_user; + if (!iam_user.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse RamUserPB"; + return; + } + AkSkPair plain_ak_sk_pair; + int ret = decrypt_ak_sk_helper(iam_user.ak(), iam_user.sk(), iam_user.encryption_info(), + &plain_ak_sk_pair, code, msg); + if (ret != 0) return; + iam_user.set_external_id(instance_id); + iam_user.set_ak(std::move(plain_ak_sk_pair.first)); + iam_user.set_sk(std::move(plain_ak_sk_pair.second)); + response->mutable_iam_user()->CopyFrom(iam_user); + } else { + code = cast_as(err); + ss << "failed to get arn_info_key, err=" << err; + msg = ss.str(); + return; + } + + if (instance.has_ram_user()) { + RamUserPB ram_user; + ram_user.CopyFrom(instance.ram_user()); + if (ram_user.has_encryption_info()) { + AkSkPair plain_ak_sk_pair; + int ret = decrypt_ak_sk_helper(ram_user.ak(), ram_user.sk(), ram_user.encryption_info(), + &plain_ak_sk_pair, code, msg); + if (ret != 0) return; + ram_user.set_ak(std::move(plain_ak_sk_pair.first)); + ram_user.set_sk(std::move(plain_ak_sk_pair.second)); + } + response->mutable_ram_user()->CopyFrom(ram_user); + } +} + +void MetaServiceImpl::alter_iam(google::protobuf::RpcController* controller, + const AlterIamRequest* request, AlterIamResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(alter_iam); + std::string arn_id = request->has_account_id() ? request->account_id() : ""; + std::string arn_ak = request->has_ak() ? request->ak() : ""; + std::string arn_sk = request->has_sk() ? request->sk() : ""; + if (arn_id.empty() || arn_ak.empty() || arn_sk.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "invalid argument"; + return; + } + + RPC_RATE_LIMIT(alter_iam) + + std::string key = system_meta_service_arn_info_key(); + std::string val; + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return; + } + err = txn->get(key, &val); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = cast_as(err); + ss << "fail to arn_info_key, err=" << err; + msg = ss.str(); + return; + } + + bool is_add_req = err == TxnErrorCode::TXN_KEY_NOT_FOUND; + EncryptionInfoPB encryption_info; + AkSkPair cipher_ak_sk_pair; + if (encrypt_ak_sk_helper(arn_ak, arn_sk, &encryption_info, &cipher_ak_sk_pair, code, msg) != + 0) { + return; + } + const auto& [ak, sk] = cipher_ak_sk_pair; + RamUserPB iam_user; + std::string old_ak; + std::string old_sk; + if (!is_add_req) { + if (!iam_user.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse RamUserPB"; + msg = ss.str(); + return; + } + + if (arn_id == iam_user.user_id() && ak == iam_user.ak() && sk == iam_user.sk()) { + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "already has the same arn info"; + msg = ss.str(); + return; + } + old_ak = iam_user.ak(); + old_sk = iam_user.sk(); + } + iam_user.set_user_id(arn_id); + iam_user.set_ak(std::move(cipher_ak_sk_pair.first)); + iam_user.set_sk(std::move(cipher_ak_sk_pair.second)); + iam_user.mutable_encryption_info()->CopyFrom(encryption_info); + val = iam_user.SerializeAsString(); + if (val.empty()) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize"; + msg = ss.str(); + return; + } + txn->put(key, val); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "txn->commit failed() err=" << err; + msg = ss.str(); + return; + } + if (is_add_req) { + LOG(INFO) << "add new iam info, cipher ak: " << iam_user.ak() + << " cipher sk: " << iam_user.sk(); + } else { + LOG(INFO) << "alter iam info, old: cipher ak: " << old_ak << " cipher sk" << old_sk + << " new: cipher ak: " << iam_user.ak() << " cipher sk:" << iam_user.sk(); + } +} + +void MetaServiceImpl::alter_ram_user(google::protobuf::RpcController* controller, + const AlterRamUserRequest* request, + AlterRamUserResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(alter_ram_user); + instance_id = request->has_instance_id() ? request->instance_id() : ""; + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + return; + } + if (!request->has_ram_user() || request->ram_user().user_id().empty() || + request->ram_user().ak().empty() || request->ram_user().sk().empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "ram user info err " + proto_to_json(*request); + return; + } + auto& ram_user = request->ram_user(); + RPC_RATE_LIMIT(alter_ram_user) + InstanceKeyInfo key_info {instance_id}; + std::string key; + std::string val; + instance_key(key_info, &key); + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return; + } + err = txn->get(key, &val); + LOG(INFO) << "get instance_key=" << hex(key); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get instance, instance_id=" << instance_id << " err=" << err; + msg = ss.str(); + return; + } + InstanceInfoPB instance; + if (!instance.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse InstanceInfoPB"; + return; + } + if (instance.status() == InstanceInfoPB::DELETED) { + code = MetaServiceCode::CLUSTER_NOT_FOUND; + msg = "instance status has been set delete, plz check it"; + return; + } + if (instance.has_ram_user()) { + LOG(WARNING) << "instance has ram user. instance_id=" << instance_id + << ", ram_user_id=" << ram_user.user_id(); + } + EncryptionInfoPB encryption_info; + AkSkPair cipher_ak_sk_pair; + if (encrypt_ak_sk_helper(ram_user.ak(), ram_user.sk(), &encryption_info, &cipher_ak_sk_pair, + code, msg) != 0) { + return; + } + RamUserPB new_ram_user; + new_ram_user.CopyFrom(ram_user); + new_ram_user.set_user_id(ram_user.user_id()); + new_ram_user.set_ak(std::move(cipher_ak_sk_pair.first)); + new_ram_user.set_sk(std::move(cipher_ak_sk_pair.second)); + new_ram_user.mutable_encryption_info()->CopyFrom(encryption_info); + instance.mutable_ram_user()->CopyFrom(new_ram_user); + val = instance.SerializeAsString(); + if (val.empty()) { + msg = "failed to serialize"; + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + return; + } + txn->put(key, val); + LOG(INFO) << "put instance_id=" << instance_id << " instance_key=" << hex(key); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to commit kv txn, err={}", err); + LOG(WARNING) << msg; + } +} + +void MetaServiceImpl::begin_copy(google::protobuf::RpcController* controller, + const BeginCopyRequest* request, BeginCopyResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(begin_copy); + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + if (cloud_unique_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "cloud unique id not set"; + return; + } + + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << cloud_unique_id; + return; + } + RPC_RATE_LIMIT(begin_copy) + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return; + } + + // copy job key + CopyJobKeyInfo key_info {instance_id, request->stage_id(), request->table_id(), + request->copy_id(), request->group_id()}; + std::string key; + std::string val; + copy_job_key(key_info, &key); + // copy job value + CopyJobPB copy_job; + copy_job.set_stage_type(request->stage_type()); + copy_job.set_job_status(CopyJobPB::LOADING); + copy_job.set_start_time_ms(request->start_time_ms()); + copy_job.set_timeout_time_ms(request->timeout_time_ms()); + + std::vector> copy_files; + auto& object_files = request->object_files(); + int file_num = 0; + size_t file_size = 0; + size_t file_meta_size = 0; + for (auto i = 0; i < object_files.size(); ++i) { + auto& file = object_files.at(i); + // 1. get copy file kv to check if file is loading or loaded + CopyFileKeyInfo file_key_info {instance_id, request->stage_id(), request->table_id(), + file.relative_path(), file.etag()}; + std::string file_key; + copy_file_key(file_key_info, &file_key); + std::string file_val; + TxnErrorCode err = txn->get(file_key, &file_val); + if (err == TxnErrorCode::TXN_OK) { // found key + continue; + } else if (err != TxnErrorCode::TXN_KEY_NOT_FOUND) { // error + code = cast_as(err); + msg = fmt::format("failed to get copy file, err={}", err); + LOG(WARNING) << msg; + return; + } + // 2. check if reach any limit + ++file_num; + file_size += file.size(); + file_meta_size += file.ByteSizeLong(); + if (file_num > 1 && + ((request->file_num_limit() > 0 && file_num > request->file_num_limit()) || + (request->file_size_limit() > 0 && file_size > request->file_size_limit()) || + (request->file_meta_size_limit() > 0 && + file_meta_size > request->file_meta_size_limit()))) { + break; + } + // 3. put copy file kv + CopyFilePB copy_file; + copy_file.set_copy_id(request->copy_id()); + copy_file.set_group_id(request->group_id()); + std::string copy_file_val = copy_file.SerializeAsString(); + if (copy_file_val.empty()) { + msg = "failed to serialize"; + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + return; + } + copy_files.emplace_back(std::move(file_key), std::move(copy_file_val)); + // 3. add file to copy job value + copy_job.add_object_files()->CopyFrom(file); + response->add_filtered_object_files()->CopyFrom(file); + } + + if (file_num == 0) { + return; + } + + val = copy_job.SerializeAsString(); + if (val.empty()) { + msg = "failed to serialize"; + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + return; + } + // put copy job + txn->put(key, val); + LOG(INFO) << "put copy_job_key=" << hex(key); + // put copy file + for (const auto& [k, v] : copy_files) { + txn->put(k, v); + LOG(INFO) << "put copy_file_key=" << hex(k); + } + + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to commit kv txn, err={}", err); + LOG(WARNING) << msg; + } +} + +void MetaServiceImpl::finish_copy(google::protobuf::RpcController* controller, + const FinishCopyRequest* request, FinishCopyResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(finish_copy); + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + if (cloud_unique_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "cloud unique id not set"; + return; + } + + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << cloud_unique_id; + return; + } + RPC_RATE_LIMIT(finish_copy) + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return; + } + + // copy job key + CopyJobKeyInfo key_info {instance_id, request->stage_id(), request->table_id(), + request->copy_id(), request->group_id()}; + std::string key; + std::string val; + copy_job_key(key_info, &key); + err = txn->get(key, &val); + LOG(INFO) << "get copy_job_key=" << hex(key); + + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { // not found + code = MetaServiceCode::COPY_JOB_NOT_FOUND; + ss << "copy job does not found"; + msg = ss.str(); + return; + } else if (err != TxnErrorCode::TXN_OK) { // error + code = cast_as(err); + ss << "failed to get copy_job, instance_id=" << instance_id << " err=" << err; + msg = ss.str(); + return; + } + + CopyJobPB copy_job; + if (!copy_job.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse CopyJobPB"; + return; + } + + std::vector copy_files; + if (request->action() == FinishCopyRequest::COMMIT) { + // 1. update copy job status from Loading to Finish + copy_job.set_job_status(CopyJobPB::FINISH); + if (request->has_finish_time_ms()) { + copy_job.set_finish_time_ms(request->finish_time_ms()); + } + val = copy_job.SerializeAsString(); + if (val.empty()) { + msg = "failed to serialize"; + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + return; + } + txn->put(key, val); + LOG(INFO) << "put copy_job_key=" << hex(key); + } else if (request->action() == FinishCopyRequest::ABORT || + request->action() == FinishCopyRequest::REMOVE) { + // 1. remove copy job kv + // 2. remove copy file kvs + txn->remove(key); + LOG(INFO) << (request->action() == FinishCopyRequest::ABORT ? "abort" : "remove") + << " copy_job_key=" << hex(key); + for (const auto& file : copy_job.object_files()) { + // copy file key + CopyFileKeyInfo file_key_info {instance_id, request->stage_id(), request->table_id(), + file.relative_path(), file.etag()}; + std::string file_key; + copy_file_key(file_key_info, &file_key); + copy_files.emplace_back(std::move(file_key)); + } + for (const auto& k : copy_files) { + txn->remove(k); + LOG(INFO) << "remove copy_file_key=" << hex(k); + } + } else { + msg = "Unhandled action"; + code = MetaServiceCode::UNDEFINED_ERR; + return; + } + + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to commit kv txn, err={}", err); + LOG(WARNING) << msg; + } +} + +void MetaServiceImpl::get_copy_job(google::protobuf::RpcController* controller, + const GetCopyJobRequest* request, GetCopyJobResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(get_copy_job); + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + if (cloud_unique_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "cloud unique id not set"; + return; + } + + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << cloud_unique_id; + return; + } + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return; + } + + CopyJobKeyInfo key_info {instance_id, request->stage_id(), request->table_id(), + request->copy_id(), request->group_id()}; + std::string key; + copy_job_key(key_info, &key); + std::string val; + err = txn->get(key, &val); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { // not found key + return; + } else if (err != TxnErrorCode::TXN_OK) { // error + code = cast_as(err); + msg = fmt::format("failed to get copy job, err={}", err); + LOG(WARNING) << msg << " err=" << err; + return; + } + CopyJobPB copy_job; + if (!copy_job.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse CopyJobPB"; + return; + } + response->mutable_copy_job()->CopyFrom(copy_job); +} + +void MetaServiceImpl::get_copy_files(google::protobuf::RpcController* controller, + const GetCopyFilesRequest* request, + GetCopyFilesResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(get_copy_files); + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + if (cloud_unique_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "cloud unique id not set"; + return; + } + + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << cloud_unique_id; + return; + } + RPC_RATE_LIMIT(get_copy_files) + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return; + } + + CopyJobKeyInfo key_info0 {instance_id, request->stage_id(), request->table_id(), "", 0}; + CopyJobKeyInfo key_info1 {instance_id, request->stage_id(), request->table_id() + 1, "", 0}; + std::string key0; + std::string key1; + copy_job_key(key_info0, &key0); + copy_job_key(key_info1, &key1); + std::unique_ptr it; + do { + TxnErrorCode err = txn->get(key0, key1, &it); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to get copy jobs, err={}", err); + LOG(WARNING) << msg << " err=" << err; + return; + } + + while (it->has_next()) { + auto [k, v] = it->next(); + if (!it->has_next()) key0 = k; + CopyJobPB copy_job; + if (!copy_job.ParseFromArray(v.data(), v.size())) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse CopyJobPB"; + return; + } + // TODO check if job is timeout + for (const auto& file : copy_job.object_files()) { + response->add_object_files()->CopyFrom(file); + } + } + key0.push_back('\x00'); + } while (it->more()); +} + +void MetaServiceImpl::filter_copy_files(google::protobuf::RpcController* controller, + const FilterCopyFilesRequest* request, + FilterCopyFilesResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(filter_copy_files); + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + if (cloud_unique_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "cloud unique id not set"; + return; + } + + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << cloud_unique_id; + return; + } + RPC_RATE_LIMIT(filter_copy_files) + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return; + } + + std::vector filter_files; + for (auto i = 0; i < request->object_files().size(); ++i) { + auto& file = request->object_files().at(i); + // 1. get copy file kv to check if file is loading or loaded + CopyFileKeyInfo file_key_info {instance_id, request->stage_id(), request->table_id(), + file.relative_path(), file.etag()}; + std::string file_key; + copy_file_key(file_key_info, &file_key); + std::string file_val; + TxnErrorCode err = txn->get(file_key, &file_val); + if (err == TxnErrorCode::TXN_OK) { // found key + continue; + } else if (err != TxnErrorCode::TXN_KEY_NOT_FOUND) { // error + msg = fmt::format("failed to get copy file, err={}", err); + LOG(WARNING) << msg << " err=" << err; + return; + } else { + response->add_object_files()->CopyFrom(file); + } + } +} + +void MetaServiceImpl::get_cluster_status(google::protobuf::RpcController* controller, + const GetClusterStatusRequest* request, + GetClusterStatusResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(get_cluster_status); + if (request->instance_ids().empty() && request->cloud_unique_ids().empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "cloud_unique_ids or instance_ids must be given, instance_ids.size: " + + std::to_string(request->instance_ids().size()) + + " cloud_unique_ids.size: " + std::to_string(request->cloud_unique_ids().size()); + return; + } + + std::vector instance_ids; + instance_ids.reserve( + std::max(request->instance_ids().size(), request->cloud_unique_ids().size())); + + // priority use instance_ids + if (!request->instance_ids().empty()) { + std::for_each(request->instance_ids().begin(), request->instance_ids().end(), + [&](const auto& it) { instance_ids.emplace_back(it); }); + } else if (!request->cloud_unique_ids().empty()) { + std::for_each(request->cloud_unique_ids().begin(), request->cloud_unique_ids().end(), + [&](const auto& it) { + std::string instance_id = get_instance_id(resource_mgr_, it); + if (instance_id.empty()) { + LOG(INFO) << "cant get instance_id from cloud_unique_id : " << it; + return; + } + instance_ids.emplace_back(instance_id); + }); + } + + if (instance_ids.empty()) { + LOG(INFO) << "can't get valid instanceids"; + return; + } + bool has_filter = request->has_status(); + + RPC_RATE_LIMIT(get_cluster_status) + + auto get_clusters_info = [this, &request, &response, + &has_filter](const std::string& instance_id) { + InstanceKeyInfo key_info {instance_id}; + std::string key; + std::string val; + instance_key(key_info, &key); + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create txn err=" << err; + return; + } + err = txn->get(key, &val); + LOG(INFO) << "get instance_key=" << hex(key); + + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to get instance, instance_id=" << instance_id << " err=" << err; + return; + } + + InstanceInfoPB instance; + if (!instance.ParseFromString(val)) { + LOG(WARNING) << "failed to parse InstanceInfoPB"; + return; + } + GetClusterStatusResponse::GetClusterStatusResponseDetail detail; + detail.set_instance_id(instance_id); + for (auto& cluster : instance.clusters()) { + if (cluster.type() != ClusterPB::COMPUTE) { + continue; + } + ClusterPB pb; + pb.set_cluster_name(cluster.cluster_name()); + pb.set_cluster_id(cluster.cluster_id()); + if (has_filter && request->status() != cluster.cluster_status()) { + continue; + } + // for compatible + if (cluster.has_cluster_status()) { + pb.set_cluster_status(cluster.cluster_status()); + } else { + pb.set_cluster_status(ClusterStatus::NORMAL); + } + detail.add_clusters()->CopyFrom(pb); + } + if (detail.clusters().size() == 0) { + return; + } + response->add_details()->CopyFrom(detail); + }; + + std::for_each(instance_ids.begin(), instance_ids.end(), get_clusters_info); + + msg = proto_to_json(*response); +} + +void notify_refresh_instance(std::shared_ptr txn_kv, const std::string& instance_id) { + LOG(INFO) << "begin notify_refresh_instance"; + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create txn err=" << err; + return; + } + std::string key = system_meta_service_registry_key(); + std::string val; + err = txn->get(key, &val); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to get server registry" + << " err=" << err; + return; + } + std::string self_endpoint; + if (config::hostname.empty()) { + self_endpoint = fmt::format("{}:{}", butil::my_ip_cstr(), config::brpc_listen_port); + } else { + self_endpoint = fmt::format("{}:{}", config::hostname, config::brpc_listen_port); + } + ServiceRegistryPB reg; + reg.ParseFromString(val); + + brpc::ChannelOptions options; + options.connection_type = brpc::ConnectionType::CONNECTION_TYPE_SHORT; + + static std::unordered_map> stubs; + static std::mutex mtx; + + std::vector btids; + btids.reserve(reg.items_size()); + for (int i = 0; i < reg.items_size(); ++i) { + int ret = 0; + auto& e = reg.items(i); + std::string endpoint; + if (e.has_host()) { + endpoint = fmt::format("{}:{}", e.host(), e.port()); + } else { + endpoint = fmt::format("{}:{}", e.ip(), e.port()); + } + if (endpoint == self_endpoint) continue; + + // Prepare stub + std::shared_ptr stub; + do { + std::lock_guard l(mtx); + if (auto it = stubs.find(endpoint); it != stubs.end()) { + stub = it->second; + break; + } + auto channel = std::make_unique(); + ret = channel->Init(endpoint.c_str(), &options); + if (ret != 0) { + LOG(WARNING) << "fail to init brpc channel, endpoint=" << endpoint; + break; + } + stub = std::make_shared(channel.release(), + google::protobuf::Service::STUB_OWNS_CHANNEL); + } while (false); + if (ret != 0) continue; + + // Issue RPC + auto f = new std::function([instance_id, stub, endpoint] { + int num_try = 0; + bool succ = false; + while (num_try++ < 3) { + brpc::Controller cntl; + cntl.set_timeout_ms(3000); + AlterInstanceRequest req; + AlterInstanceResponse res; + req.set_instance_id(instance_id); + req.set_op(AlterInstanceRequest::REFRESH); + stub->alter_instance(&cntl, &req, &res, nullptr); + if (cntl.Failed()) { + LOG_WARNING("issue refresh instance rpc") + .tag("endpoint", endpoint) + .tag("num_try", num_try) + .tag("code", cntl.ErrorCode()) + .tag("msg", cntl.ErrorText()); + } else { + succ = res.status().code() == MetaServiceCode::OK; + LOG(INFO) << (succ ? "succ" : "failed") + << " to issue refresh_instance rpc, num_try=" << num_try + << " endpoint=" << endpoint << " response=" << proto_to_json(res); + if (succ) return; + } + bthread_usleep(300000); + } + if (succ) return; + LOG(WARNING) << "failed to refresh finally, it may left the system inconsistent," + << " tired=" << num_try; + }); + bthread_t bid; + ret = bthread_start_background(&bid, nullptr, run_bthread_work, f); + if (ret != 0) continue; + btids.emplace_back(bid); + } // for + for (auto& i : btids) bthread_join(i, nullptr); + LOG(INFO) << "finish notify_refresh_instance, num_items=" << reg.items_size(); +} + +} // namespace doris::cloud diff --git a/cloud/src/meta-service/meta_service_schema.cpp b/cloud/src/meta-service/meta_service_schema.cpp new file mode 100644 index 00000000000000..094512dfd3600b --- /dev/null +++ b/cloud/src/meta-service/meta_service_schema.cpp @@ -0,0 +1,122 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "meta-service/meta_service_schema.h" + +#include + +#include "common/logging.h" +#include "common/sync_point.h" +#include "common/util.h" +#include "meta-service/keys.h" +#include "meta-service/meta_service_helper.h" +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" + +namespace doris::cloud { +namespace config { +extern int16_t meta_schema_value_version; +} + +void put_schema_kv(MetaServiceCode& code, std::string& msg, Transaction* txn, + std::string_view schema_key, const doris::TabletSchemaCloudPB& schema) { + TxnErrorCode err = cloud::key_exists(txn, schema_key); + if (err == TxnErrorCode::TXN_OK) { // schema has already been saved + TEST_SYNC_POINT_RETURN_WITH_VOID("put_schema_kv:schema_key_exists_return"); + DCHECK([&] { + auto transform = [](std::string_view type) -> std::string_view { + if (type == "DECIMALV2") return "DECIMAL"; + if (type == "BITMAP") return "OBJECT"; + return type; + }; + ValueBuf buf; + auto err = cloud::get(txn, schema_key, &buf); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to get schema, err=" << err; + return false; + } + doris::TabletSchemaCloudPB saved_schema; + if (!buf.to_pb(&saved_schema)) { + LOG(WARNING) << "failed to parse schema value"; + return false; + } + if (saved_schema.column_size() != schema.column_size()) { + LOG(WARNING) << "saved_schema.column_size()=" << saved_schema.column_size() + << " schema.column_size()=" << schema.column_size(); + return false; + } + // Sort by column id + std::sort(saved_schema.mutable_column()->begin(), saved_schema.mutable_column()->end(), + [](auto& c1, auto& c2) { return c1.unique_id() < c2.unique_id(); }); + auto& schema_ref = const_cast(schema); + std::sort(schema_ref.mutable_column()->begin(), schema_ref.mutable_column()->end(), + [](auto& c1, auto& c2) { return c1.unique_id() < c2.unique_id(); }); + for (int i = 0; i < saved_schema.column_size(); ++i) { + auto& saved_column = saved_schema.column(i); + auto& column = schema.column(i); + if (saved_column.unique_id() != column.unique_id() || + transform(saved_column.type()) != transform(column.type())) { + LOG(WARNING) << "existed column: " << saved_column.DebugString() + << "\nto save column: " << column.DebugString(); + return false; + } + } + if (saved_schema.index_size() != schema.index_size()) { + LOG(WARNING) << "saved_schema.index_size()=" << saved_schema.column_size() + << " schema.index_size()=" << schema.column_size(); + return false; + } + // Sort by index id + std::sort(saved_schema.mutable_index()->begin(), saved_schema.mutable_index()->end(), + [](auto& i1, auto& i2) { return i1.index_id() < i2.index_id(); }); + std::sort(schema_ref.mutable_index()->begin(), schema_ref.mutable_index()->end(), + [](auto& i1, auto& i2) { return i1.index_id() < i2.index_id(); }); + for (int i = 0; i < saved_schema.index_size(); ++i) { + auto& saved_index = saved_schema.index(i); + auto& index = schema.index(i); + if (saved_index.index_id() != index.index_id() || + saved_index.index_type() != index.index_type()) { + LOG(WARNING) << "existed index: " << saved_index.DebugString() + << "\nto save index: " << index.DebugString(); + return false; + } + } + return true; + }()) << hex(schema_key) + << "\n to_save: " << schema.ShortDebugString(); + return; + } else if (err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + msg = "failed to check that key exists"; + code = cast_as(err); + return; + } + LOG_INFO("put schema kv").tag("key", hex(schema_key)); + uint8_t ver = config::meta_schema_value_version; + if (ver > 0) { + cloud::put(txn, schema_key, schema, ver); + } else { + auto schema_value = schema.SerializeAsString(); + txn->put(schema_key, schema_value); + } +} + +bool parse_schema_value(const ValueBuf& buf, doris::TabletSchemaCloudPB* schema) { + // TODO(plat1ko): Apply decompression based on value version + return buf.to_pb(schema); +} + +} // namespace doris::cloud diff --git a/cloud/src/meta-service/meta_service_schema.h b/cloud/src/meta-service/meta_service_schema.h new file mode 100644 index 00000000000000..44fabeafd73b28 --- /dev/null +++ b/cloud/src/meta-service/meta_service_schema.h @@ -0,0 +1,33 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include + +namespace doris::cloud { +class Transaction; +struct ValueBuf; + +void put_schema_kv(MetaServiceCode& code, std::string& msg, Transaction* txn, + std::string_view schema_key, const doris::TabletSchemaCloudPB& schema); + +// Return true if parse success +[[nodiscard]] bool parse_schema_value(const ValueBuf& buf, doris::TabletSchemaCloudPB* schema); + +} // namespace doris::cloud diff --git a/cloud/src/meta-service/meta_service_tablet_stats.cpp b/cloud/src/meta-service/meta_service_tablet_stats.cpp new file mode 100644 index 00000000000000..d5ba690d068e8a --- /dev/null +++ b/cloud/src/meta-service/meta_service_tablet_stats.cpp @@ -0,0 +1,130 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "meta-service/meta_service_tablet_stats.h" + +#include + +#include "common/logging.h" +#include "common/util.h" +#include "meta-service/keys.h" +#include "meta-service/meta_service_helper.h" +#include "meta-service/txn_kv.h" + +namespace doris::cloud { + +void internal_get_tablet_stats(MetaServiceCode& code, std::string& msg, Transaction* txn, + const std::string& instance_id, const TabletIndexPB& idx, + TabletStatsPB& stats, TabletStats& detached_stats, bool snapshot) { + auto begin_key = stats_tablet_key( + {instance_id, idx.table_id(), idx.index_id(), idx.partition_id(), idx.tablet_id()}); + auto end_key = stats_tablet_key( + {instance_id, idx.table_id(), idx.index_id(), idx.partition_id(), idx.tablet_id() + 1}); + std::unique_ptr it; + TxnErrorCode err = txn->get(begin_key, end_key, &it, snapshot); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to get tablet stats, err={} tablet_id={}", err, idx.tablet_id()); + return; + } + if (!it->has_next()) { + code = MetaServiceCode::TABLET_NOT_FOUND; + msg = fmt::format("tablet stats not found, tablet_id={}", idx.tablet_id()); + return; + } + auto [k, v] = it->next(); + // First key MUST be tablet stats key + DCHECK(k == begin_key) << hex(k) << " vs " << hex(begin_key); + if (!stats.ParseFromArray(v.data(), v.size())) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = fmt::format("marformed tablet stats value, key={}", hex(k)); + return; + } + // Parse split tablet stats + do { + if (!it->has_next()) { + break; + } + while (it->has_next()) { + auto [k, v] = it->next(); + if (!it->has_next() && it->more()) { + begin_key = k; + } + // 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "data_size" + auto k1 = k; + k1.remove_prefix(1); + std::vector, int, int>> out; + if (decode_key(&k1, &out) != 0) [[unlikely]] { + code = MetaServiceCode::UNDEFINED_ERR; + msg = fmt::format("failed to decode tablet stats key, key={}", hex(k)); + return; + } + if (out.size() != 8) [[unlikely]] { + code = MetaServiceCode::UNDEFINED_ERR; + msg = fmt::format("failed to decode tablet stats key, key={}", hex(k)); + return; + } + auto suffix = std::get_if(&std::get<0>(out.back())); + if (!suffix) [[unlikely]] { + code = MetaServiceCode::UNDEFINED_ERR; + msg = fmt::format("failed to decode tablet stats key, key={}", hex(k)); + return; + } + int64_t val = *reinterpret_cast(v.data()); + if (*suffix == STATS_KEY_SUFFIX_DATA_SIZE) { + detached_stats.data_size = val; + } else if (*suffix == STATS_KEY_SUFFIX_NUM_ROWS) { + detached_stats.num_rows = val; + } else if (*suffix == STATS_KEY_SUFFIX_NUM_ROWSETS) { + detached_stats.num_rowsets = val; + } else if (*suffix == STATS_KEY_SUFFIX_NUM_SEGS) { + detached_stats.num_segs = val; + } else { + VLOG_DEBUG << "unknown suffix=" << *suffix; + } + } + if (it->more()) { + begin_key.push_back('\x00'); // Update to next smallest key for iteration + err = txn->get(begin_key, end_key, &it, snapshot); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = fmt::format("failed to get tablet stats, err={} tablet_id={}", err, + idx.tablet_id()); + return; + } + } else { + break; + } + } while (true); +} + +void merge_tablet_stats(TabletStatsPB& stats, const TabletStats& detached_stats) { + stats.set_data_size(stats.data_size() + detached_stats.data_size); + stats.set_num_rows(stats.num_rows() + detached_stats.num_rows); + stats.set_num_rowsets(stats.num_rowsets() + detached_stats.num_rowsets); + stats.set_num_segments(stats.num_segments() + detached_stats.num_segs); +} + +void internal_get_tablet_stats(MetaServiceCode& code, std::string& msg, Transaction* txn, + const std::string& instance_id, const TabletIndexPB& idx, + TabletStatsPB& stats, bool snapshot) { + TabletStats detached_stats; + internal_get_tablet_stats(code, msg, txn, instance_id, idx, stats, detached_stats, snapshot); + merge_tablet_stats(stats, detached_stats); +} + +} // namespace doris::cloud diff --git a/cloud/src/meta-service/meta_service_tablet_stats.h b/cloud/src/meta-service/meta_service_tablet_stats.h new file mode 100644 index 00000000000000..f341887e3f1cd3 --- /dev/null +++ b/cloud/src/meta-service/meta_service_tablet_stats.h @@ -0,0 +1,49 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include + +namespace doris::cloud { +class Transaction; + +// Detached tablet stats +struct TabletStats { + int64_t data_size = 0; + int64_t num_rows = 0; + int64_t num_rowsets = 0; + int64_t num_segs = 0; +}; + +// Get tablet stats and detached tablet stats via `txn`. If an error occurs, `code` will be set to non OK. +// NOTE: this function returns original `TabletStatsPB` and detached tablet stats val stored in kv store, +// MUST call `merge_tablet_stats(stats, detached_stats)` to get the real tablet stats. +void internal_get_tablet_stats(MetaServiceCode& code, std::string& msg, Transaction* txn, + const std::string& instance_id, const TabletIndexPB& idx, + TabletStatsPB& stats, TabletStats& detached_stats, + bool snapshot = false); + +// Merge `detached_stats` `stats` to `stats`. +void merge_tablet_stats(TabletStatsPB& stats, const TabletStats& detached_stats); + +// Get merged tablet stats via `txn`. If an error occurs, `code` will be set to non OK. +void internal_get_tablet_stats(MetaServiceCode& code, std::string& msg, Transaction* txn, + const std::string& instance_id, const TabletIndexPB& idx, + TabletStatsPB& stats, bool snapshot = false); + +} // namespace doris::cloud diff --git a/cloud/src/meta-service/meta_service_txn.cpp b/cloud/src/meta-service/meta_service_txn.cpp new file mode 100644 index 00000000000000..a7ac3dc64e46fe --- /dev/null +++ b/cloud/src/meta-service/meta_service_txn.cpp @@ -0,0 +1,1756 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include + +#include "common/logging.h" +#include "common/sync_point.h" +#include "meta-service/doris_txn.h" +#include "meta-service/meta_service.h" +#include "meta-service/meta_service_helper.h" +#include "meta-service/meta_service_tablet_stats.h" +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" + +using namespace std::chrono; + +namespace doris::cloud { + +//TODO: we need move begin/commit etc txn to TxnManager +void MetaServiceImpl::begin_txn(::google::protobuf::RpcController* controller, + const BeginTxnRequest* request, BeginTxnResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(begin_txn); + if (!request->has_txn_info()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "invalid argument, missing txn info"; + return; + } + + auto& txn_info = const_cast(request->txn_info()); + std::string label = txn_info.has_label() ? txn_info.label() : ""; + int64_t db_id = txn_info.has_db_id() ? txn_info.db_id() : -1; + + if (label.empty() || db_id < 0 || txn_info.table_ids().empty() || !txn_info.has_timeout_ms()) { + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "invalid argument, label=" << label << " db_id=" << db_id; + msg = ss.str(); + return; + } + + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "cannot find instance_id with cloud_unique_id=" + << (cloud_unique_id.empty() ? "(empty)" : cloud_unique_id) << " label=" << label; + msg = ss.str(); + return; + } + + RPC_RATE_LIMIT(begin_txn) + //1. Generate version stamp for txn id + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "txn_kv_->create_txn() failed, err=" << err << " label=" << label + << " db_id=" << db_id; + msg = ss.str(); + return; + } + + const std::string label_key = txn_label_key({instance_id, db_id, label}); + std::string label_val; + err = txn->get(label_key, &label_val); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = cast_as(err); + ss << "txn->get failed(), err=" << err << " label=" << label; + msg = ss.str(); + return; + } + + LOG(INFO) << "txn->get label_key=" << hex(label_key) << " label=" << label << " err=" << err; + + // err == OK means label has previous txn ids. + if (err == TxnErrorCode::TXN_OK) { + label_val = label_val.substr(0, label_val.size() - VERSION_STAMP_LEN); + } + + //ret > 0, means label not exist previously. + txn->atomic_set_ver_value(label_key, label_val); + LOG(INFO) << "txn->atomic_set_ver_value label_key=" << hex(label_key); + + TEST_SYNC_POINT_CALLBACK("begin_txn:before:commit_txn:1", &label); + err = txn->commit(); + TEST_SYNC_POINT_CALLBACK("begin_txn:after:commit_txn:1", &label); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "txn->commit failed(), label=" << label << " err=" << err; + msg = ss.str(); + return; + } + //2. Get txn id from version stamp + txn.reset(); + + err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to create txn when get txn id, label=" << label << " err=" << err; + msg = ss.str(); + return; + } + + label_val.clear(); + err = txn->get(label_key, &label_val); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "txn->get() failed, label=" << label << " err=" << err; + msg = ss.str(); + return; + } + + LOG(INFO) << "txn->get label_key=" << hex(label_key) << " label=" << label << " err=" << err; + + // Generated by TxnKv system + int64_t txn_id = 0; + int ret = + get_txn_id_from_fdb_ts(std::string_view(label_val).substr( + label_val.size() - VERSION_STAMP_LEN, label_val.size()), + &txn_id); + if (ret != 0) { + code = MetaServiceCode::TXN_GEN_ID_ERR; + ss << "get_txn_id_from_fdb_ts() failed, label=" << label << " ret=" << ret; + msg = ss.str(); + return; + } + + LOG(INFO) << "get_txn_id_from_fdb_ts() label=" << label << " txn_id=" << txn_id + << " label_val.size()=" << label_val.size(); + + TxnLabelPB label_pb; + if (label_val.size() > VERSION_STAMP_LEN) { + //3. Check label + //label_val.size() > VERSION_STAMP_LEN means label has previous txn ids. + if (!label_pb.ParseFromArray(label_val.data(), label_val.size() - VERSION_STAMP_LEN)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "label_pb->ParseFromString() failed, txn_id=" << txn_id << " label=" << label; + msg = ss.str(); + return; + } + + // Check if label already used, by following steps + // 1. get all existing transactions + // 2. if there is a PREPARE transaction, check if this is a retry request. + // 3. if there is a non-aborted transaction, throw label already used exception. + + for (auto& cur_txn_id : label_pb.txn_ids()) { + const std::string cur_info_key = txn_info_key({instance_id, db_id, cur_txn_id}); + std::string cur_info_val; + err = txn->get(cur_info_key, &cur_info_val); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = cast_as(err); + ss << "txn->get() failed, cur_txn_id=" << cur_txn_id << " label=" << label + << " err=" << err; + msg = ss.str(); + return; + } + + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + //label_to_idx and txn info inconsistency. + code = MetaServiceCode::TXN_ID_NOT_FOUND; + ss << "txn->get() failed, cur_txn_id=" << cur_txn_id << " label=" << label + << " err=" << err; + msg = ss.str(); + return; + } + + TxnInfoPB cur_txn_info; + if (!cur_txn_info.ParseFromString(cur_info_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "cur_txn_info->ParseFromString() failed, cur_txn_id=" << cur_txn_id + << " label=" << label << " err=" << err; + msg = ss.str(); + return; + } + + VLOG_DEBUG << "cur_txn_info=" << cur_txn_info.ShortDebugString(); + if (cur_txn_info.status() == TxnStatusPB::TXN_STATUS_ABORTED) { + continue; + } + + if (cur_txn_info.status() == TxnStatusPB::TXN_STATUS_PREPARED || + cur_txn_info.status() == TxnStatusPB::TXN_STATUS_PRECOMMITTED) { + // clang-format off + if (cur_txn_info.has_request_id() && txn_info.has_request_id() && + ((cur_txn_info.request_id().hi() == txn_info.request_id().hi()) && + (cur_txn_info.request_id().lo() == txn_info.request_id().lo()))) { + + response->set_dup_txn_id(cur_txn_info.txn_id()); + code = MetaServiceCode::TXN_DUPLICATED_REQ; + ss << "db_id=" << db_id << " label=" << label << " txn_id=" << cur_txn_info.txn_id() << " dup begin txn request."; + msg = ss.str(); + return; + } + // clang-format on + } + code = MetaServiceCode::TXN_LABEL_ALREADY_USED; + ss << "Label [" << label << "] has already been used, relate to txn [" + << cur_txn_info.txn_id() << "]"; + msg = ss.str(); + return; + } + } + + // Update txn_info to be put into TxnKv + // Update txn_id in PB + txn_info.set_txn_id(txn_id); + // TODO: + // check initial status must be TXN_STATUS_PREPARED or TXN_STATUS_UNKNOWN + txn_info.set_status(TxnStatusPB::TXN_STATUS_PREPARED); + + auto now_time = system_clock::now(); + uint64_t prepare_time = duration_cast(now_time.time_since_epoch()).count(); + + txn_info.set_prepare_time(prepare_time); + //4. put txn info and db_tbl + const std::string info_key = txn_info_key({instance_id, db_id, txn_id}); + std::string info_val; + if (!txn_info.SerializeToString(&info_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize txn_info, label=" << label << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + + const std::string index_key = txn_index_key({instance_id, txn_id}); + std::string index_val; + TxnIndexPB index_pb; + index_pb.mutable_tablet_index()->set_db_id(db_id); + if (!index_pb.SerializeToString(&index_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize txn_index_pb " + << "label=" << label << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + + const std::string running_key = txn_running_key({instance_id, db_id, txn_id}); + std::string running_val; + TxnRunningPB running_pb; + running_pb.set_timeout_time(prepare_time + txn_info.timeout_ms()); + for (auto i : txn_info.table_ids()) { + running_pb.add_table_ids(i); + } + VLOG_DEBUG << "label=" << label << " txn_id=" << txn_id + << "running_pb=" << running_pb.ShortDebugString(); + if (!running_pb.SerializeToString(&running_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize running_pb label=" << label << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + + label_pb.add_txn_ids(txn_id); + VLOG_DEBUG << "label=" << label << " txn_id=" << txn_id + << "txn_label_pb=" << label_pb.ShortDebugString(); + if (!label_pb.SerializeToString(&label_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize txn_label_pb label=" << label << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + txn->atomic_set_ver_value(label_key, label_val); + LOG(INFO) << "txn->atomic_set_ver_value label_key=" << hex(label_key) << " label=" << label + << " txn_id=" << txn_id; + + txn->put(info_key, info_val); + txn->put(index_key, index_val); + txn->put(running_key, running_val); + LOG(INFO) << "xxx put info_key=" << hex(info_key) << " txn_id=" << txn_id; + LOG(INFO) << "xxx put running_key=" << hex(running_key) << " txn_id=" << txn_id; + LOG(INFO) << "xxx put index_key=" << hex(index_key) << " txn_id=" << txn_id; + + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to commit txn kv, label=" << label << " txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + return; + } + TEST_SYNC_POINT_CALLBACK("begin_txn:after:commit_txn:2", &txn_id); + response->set_txn_id(txn_id); +} + +void MetaServiceImpl::precommit_txn(::google::protobuf::RpcController* controller, + const PrecommitTxnRequest* request, + PrecommitTxnResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(precommit_txn); + int64_t txn_id = request->has_txn_id() ? request->txn_id() : -1; + int64_t db_id = request->has_db_id() ? request->db_id() : -1; + if ((txn_id < 0 && db_id < 0)) { + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "invalid argument, " + << "txn_id=" << txn_id << " db_id=" << db_id; + msg = ss.str(); + return; + } + + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "cannot find instance_id with cloud_unique_id=" + << (cloud_unique_id.empty() ? "(empty)" : cloud_unique_id) << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + RPC_RATE_LIMIT(precommit_txn); + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "txn_kv_->create_txn() failed, err=" << err << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + + //not provide db_id, we need read from disk. + if (db_id < 0) { + const std::string index_key = txn_index_key({instance_id, txn_id}); + std::string index_val; + err = txn->get(index_key, &index_val); + if (err != TxnErrorCode::TXN_OK) { + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::TXN_ID_NOT_FOUND + : cast_as(err); + ss << "failed to get db id with txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + return; + } + TxnIndexPB index_pb; + if (!index_pb.ParseFromString(index_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse txn_inf" + << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + DCHECK(index_pb.has_tablet_index() == true); + DCHECK(index_pb.tablet_index().has_db_id() == true); + db_id = index_pb.tablet_index().db_id(); + VLOG_DEBUG << " find db_id=" << db_id << " from index"; + } else { + db_id = request->db_id(); + } + + // Get txn info with db_id and txn_id + const std::string info_key = txn_info_key({instance_id, db_id, txn_id}); + std::string info_val; // Will be reused when saving updated txn + err = txn->get(info_key, &info_val); + if (err != TxnErrorCode::TXN_OK) { + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::TXN_ID_NOT_FOUND + : cast_as(err); + ss << "failed to get db id with db_id=" << db_id << " txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + return; + } + + TxnInfoPB txn_info; + if (!txn_info.ParseFromString(info_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse txn_inf db_id=" << db_id << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + + DCHECK(txn_info.txn_id() == txn_id); + if (txn_info.status() == TxnStatusPB::TXN_STATUS_ABORTED) { + code = MetaServiceCode::TXN_ALREADY_ABORTED; + ss << "transaction is already aborted: db_id=" << db_id << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + + if (txn_info.status() == TxnStatusPB::TXN_STATUS_VISIBLE) { + code = MetaServiceCode::TXN_ALREADY_VISIBLE; + ss << "transaction is already visible: db_id=" << db_id << " txn_id=" << txn_id; + msg = ss.str(); + } + + if (txn_info.status() == TxnStatusPB::TXN_STATUS_PRECOMMITTED) { + code = MetaServiceCode::TXN_ALREADY_PRECOMMITED; + ss << "transaction is already precommited: db_id=" << db_id << " txn_id=" << txn_id; + msg = ss.str(); + } + + LOG(INFO) << "before update txn_info=" << txn_info.ShortDebugString(); + + // Update txn_info + txn_info.set_status(TxnStatusPB::TXN_STATUS_PRECOMMITTED); + + auto now_time = system_clock::now(); + uint64_t precommit_time = duration_cast(now_time.time_since_epoch()).count(); + txn_info.set_precommit_time(precommit_time); + if (request->has_commit_attachment()) { + txn_info.mutable_commit_attachment()->CopyFrom(request->commit_attachment()); + } + LOG(INFO) << "after update txn_info=" << txn_info.ShortDebugString(); + + info_val.clear(); + if (!txn_info.SerializeToString(&info_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize txn_info when saving, txn_id=" << txn_id; + msg = ss.str(); + return; + } + + txn->put(info_key, info_val); + LOG(INFO) << "xxx put info_key=" << hex(info_key) << " txn_id=" << txn_id; + + const std::string running_key = txn_running_key({instance_id, db_id, txn_id}); + std::string running_val; + + TxnRunningPB running_pb; + running_pb.set_timeout_time(precommit_time + txn_info.precommit_timeout_ms()); + if (!running_pb.SerializeToString(&running_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize running_pb, txn_id=" << txn_id; + msg = ss.str(); + return; + } + + LOG(INFO) << "xxx put running_key=" << hex(running_key) << " txn_id=" << txn_id; + txn->put(running_key, running_val); + + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to commit txn kv, txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + return; + } +} + +/** + * 0. Extract txn_id from request + * 1. Get db id from TxnKv with txn_id + * 2. Get TxnInfo from TxnKv with db_id and txn_id + * 3. Get tmp rowset meta, there may be several or hundred of tmp rowsets + * 4. Get versions of each rowset + * 5. Put rowset meta, which will be visible to user + * 6. Put TxnInfo back into TxnKv with updated txn status (committed) + * 7. Update versions of each partition + * 8. Remove tmp rowset meta + * + * Note: getting version and all changes maded are in a single TxnKv transaction: + * step 5, 6, 7, 8 + */ +void MetaServiceImpl::commit_txn(::google::protobuf::RpcController* controller, + const CommitTxnRequest* request, CommitTxnResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(commit_txn); + if (!request->has_txn_id()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "invalid argument, missing txn id"; + return; + } + + int64_t txn_id = request->txn_id(); + + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << cloud_unique_id << " txn_id=" << txn_id; + return; + } + + RPC_RATE_LIMIT(commit_txn) + + // Create a readonly txn for scan tmp rowset + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "filed to create txn, txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + return; + } + + //Get db id with txn id + std::string index_val; + const std::string index_key = txn_index_key({instance_id, txn_id}); + err = txn->get(index_key, &index_val); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get db id, txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + return; + } + + TxnIndexPB index_pb; + if (!index_pb.ParseFromString(index_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse txn_index_pb, txn_id=" << txn_id; + msg = ss.str(); + return; + } + + DCHECK(index_pb.has_tablet_index() == true); + DCHECK(index_pb.tablet_index().has_db_id() == true); + int64_t db_id = index_pb.tablet_index().db_id(); + + // Get temporary rowsets involved in the txn + // This is a range scan + MetaRowsetTmpKeyInfo rs_tmp_key_info0 {instance_id, txn_id, 0}; + MetaRowsetTmpKeyInfo rs_tmp_key_info1 {instance_id, txn_id + 1, 0}; + std::string rs_tmp_key0; + std::string rs_tmp_key1; + meta_rowset_tmp_key(rs_tmp_key_info0, &rs_tmp_key0); + meta_rowset_tmp_key(rs_tmp_key_info1, &rs_tmp_key1); + // Get rowset meta that should be commited + // tmp_rowset_key -> rowset_meta + std::vector> tmp_rowsets_meta; + + int num_rowsets = 0; + std::unique_ptr> defer_log_range( + (int*)0x01, [rs_tmp_key0, rs_tmp_key1, &num_rowsets, &txn_id](int*) { + LOG(INFO) << "get tmp rowset meta, txn_id=" << txn_id + << " num_rowsets=" << num_rowsets << " range=[" << hex(rs_tmp_key0) << "," + << hex(rs_tmp_key1) << ")"; + }); + + std::unique_ptr it; + do { + err = txn->get(rs_tmp_key0, rs_tmp_key1, &it); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "internal error, failed to get tmp rowset while committing, txn_id=" << txn_id + << " err=" << err; + msg = ss.str(); + return; + } + + while (it->has_next()) { + auto [k, v] = it->next(); + LOG(INFO) << "range_get rowset_tmp_key=" << hex(k) << " txn_id=" << txn_id; + tmp_rowsets_meta.emplace_back(); + if (!tmp_rowsets_meta.back().second.ParseFromArray(v.data(), v.size())) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "malformed rowset meta, unable to initialize, txn_id=" << txn_id; + msg = ss.str(); + ss << " key=" << hex(k); + LOG(WARNING) << ss.str(); + return; + } + // Save keys that will be removed later + tmp_rowsets_meta.back().first = std::string(k.data(), k.size()); + ++num_rowsets; + if (!it->has_next()) rs_tmp_key0 = k; + } + rs_tmp_key0.push_back('\x00'); // Update to next smallest key for iteration + } while (it->more()); + + VLOG_DEBUG << "txn_id=" << txn_id << " tmp_rowsets_meta.size()=" << tmp_rowsets_meta.size(); + + // Create a read/write txn for guarantee consistency + txn.reset(); + err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "filed to create txn, txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + return; + } + + int64_t put_size = 0; + int64_t del_size = 0; + int num_put_keys = 0, num_del_keys = 0; + + // Get txn info with db_id and txn_id + std::string info_val; // Will be reused when saving updated txn + const std::string info_key = txn_info_key({instance_id, db_id, txn_id}); + err = txn->get(info_key, &info_val); + if (err != TxnErrorCode::TXN_OK) { + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::TXN_ID_NOT_FOUND + : cast_as(err); + ss << "failed to get txn_info, db_id=" << db_id << " txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + return; + } + + TxnInfoPB txn_info; + if (!txn_info.ParseFromString(info_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse txn_info, db_id=" << db_id << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + + // TODO: do more check like txn state, 2PC etc. + DCHECK(txn_info.txn_id() == txn_id); + if (txn_info.status() == TxnStatusPB::TXN_STATUS_ABORTED) { + code = MetaServiceCode::TXN_ALREADY_ABORTED; + ss << "transaction is already aborted: db_id=" << db_id << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + + if (txn_info.status() == TxnStatusPB::TXN_STATUS_VISIBLE) { + code = MetaServiceCode::TXN_ALREADY_VISIBLE; + if (request->has_is_2pc() && request->is_2pc()) { + ss << "transaction [" << txn_id << "] is already visible, not pre-committed."; + msg = ss.str(); + response->mutable_txn_info()->CopyFrom(txn_info); + return; + } + ss << "transaction is already visible: db_id=" << db_id << " txn_id=" << txn_id; + msg = ss.str(); + response->mutable_txn_info()->CopyFrom(txn_info); + return; + } + + if (request->has_is_2pc() && request->is_2pc() && + txn_info.status() == TxnStatusPB::TXN_STATUS_PREPARED) { + code = MetaServiceCode::TXN_INVALID_STATUS; + ss << "transaction is prepare, not pre-committed: db_id=" << db_id << " txn_id" << txn_id; + msg = ss.str(); + return; + } + + LOG(INFO) << "txn_id=" << txn_id << " txn_info=" << txn_info.ShortDebugString(); + + // Prepare rowset meta and new_versions + std::vector> rowsets; + std::map new_versions; + std::map tablet_stats; // tablet_id -> stats + std::map table_ids; // tablet_id -> {table/index/partition}_id + std::map> table_id_tablet_ids; // table_id -> tablets_ids + rowsets.reserve(tmp_rowsets_meta.size()); + for (auto& [_, i] : tmp_rowsets_meta) { + int64_t tablet_id = i.tablet_id(); + // Get version for the rowset + if (table_ids.count(tablet_id) == 0) { + MetaTabletIdxKeyInfo key_info {instance_id, tablet_id}; + auto [key, val] = std::make_tuple(std::string(""), std::string("")); + meta_tablet_idx_key(key_info, &key); + TxnErrorCode err = txn->get(key, &val); + if (err != TxnErrorCode::TXN_OK) { // Must be TXN_OK, an existing value + code = cast_as(err); + ss << "failed to get tablet table index ids," + << (err == TxnErrorCode::TXN_KEY_NOT_FOUND ? " not found" : " internal error") + << " tablet_id=" << tablet_id << " key=" << hex(key); + msg = ss.str(); + LOG(INFO) << msg << " err=" << err << " txn_id=" << txn_id; + return; + } + if (!table_ids[tablet_id].ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "malformed tablet index value tablet_id=" << tablet_id + << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + table_id_tablet_ids[table_ids[tablet_id].table_id()].push_back(tablet_id); + VLOG_DEBUG << "tablet_id:" << tablet_id + << " value:" << table_ids[tablet_id].ShortDebugString(); + } + + int64_t table_id = table_ids[tablet_id].table_id(); + int64_t partition_id = i.partition_id(); + + std::string ver_key = version_key({instance_id, db_id, table_id, partition_id}); + int64_t version = -1; + std::string ver_val_str; + int64_t new_version = -1; + VersionPB version_pb; + if (new_versions.count(ver_key) == 0) { + err = txn->get(ver_key, &ver_val_str); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = cast_as(err); + ss << "failed to get version, table_id=" << table_id + << "partition_id=" << partition_id << " key=" << hex(ver_key); + msg = ss.str(); + LOG(INFO) << msg << " txn_id=" << txn_id; + return; + } + + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + // Maybe first version + version = 1; + } else { + if (!version_pb.ParseFromString(ver_val_str)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse ver_val_str" + << " txn_id=" << txn_id << " key=" << hex(ver_key); + msg = ss.str(); + return; + } + version = version_pb.version(); + } + new_version = version + 1; + new_versions.insert({std::move(ver_key), new_version}); + } else { + new_version = new_versions[ver_key]; + } + + // Update rowset version + i.set_start_version(new_version); + i.set_end_version(new_version); + + std::string key = meta_rowset_key({instance_id, tablet_id, i.end_version()}); + std::string val; + if (!i.SerializeToString(&val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize rowset_meta, txn_id=" << txn_id; + msg = ss.str(); + return; + } + rowsets.emplace_back(std::move(key), std::move(val)); + + // Accumulate affected rows + auto& stats = tablet_stats[tablet_id]; + stats.data_size += i.data_disk_size(); + stats.num_rows += i.num_rows(); + ++stats.num_rowsets; + stats.num_segs += i.num_segments(); + } // for tmp_rowsets_meta + + // process mow table, check lock and remove pending key + for (auto table_id : request->mow_table_ids()) { + std::string lock_key = meta_delete_bitmap_update_lock_key({instance_id, table_id, -1}); + std::string lock_val; + err = txn->get(lock_key, &lock_val); + LOG(INFO) << "get delete bitmap update lock info, table_id=" << table_id + << " key=" << hex(lock_key) << " err=" << err; + if (err != TxnErrorCode::TXN_OK) { + ss << "failed to get delete bitmap update lock key info, instance_id=" << instance_id + << " table_id=" << table_id << " key=" << hex(lock_key) << " err=" << err; + msg = ss.str(); + code = cast_as(err); + return; + } + DeleteBitmapUpdateLockPB lock_info; + if (!lock_info.ParseFromString(lock_val)) [[unlikely]] { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse DeleteBitmapUpdateLockPB"; + return; + } + if (lock_info.lock_id() != request->txn_id()) { + msg = "lock is expired"; + code = MetaServiceCode::LOCK_EXPIRED; + return; + } + txn->remove(lock_key); + LOG(INFO) << "xxx remove delete bitmap lock, lock_key=" << hex(lock_key) + << " txn_id=" << txn_id; + + for (auto tablet_id : table_id_tablet_ids[table_id]) { + std::string pending_key = meta_pending_delete_bitmap_key({instance_id, tablet_id}); + txn->remove(pending_key); + LOG(INFO) << "xxx remove delete bitmap pending key, pending_key=" << hex(pending_key) + << " txn_id=" << txn_id; + } + } + + // Save rowset meta + num_put_keys += rowsets.size(); + for (auto& i : rowsets) { + size_t rowset_size = i.first.size() + i.second.size(); + txn->put(i.first, i.second); + put_size += rowset_size; + LOG(INFO) << "xxx put rowset_key=" << hex(i.first) << " txn_id=" << txn_id + << " rowset_size=" << rowset_size; + } + + // Save versions + num_put_keys += new_versions.size(); + for (auto& i : new_versions) { + std::string ver_val; + VersionPB version_pb; + version_pb.set_version(i.second); + if (!version_pb.SerializeToString(&ver_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize version_pb when saving, txn_id=" << txn_id; + msg = ss.str(); + return; + } + + txn->put(i.first, ver_val); + put_size += i.first.size() + ver_val.size(); + LOG(INFO) << "xxx put version_key=" << hex(i.first) << " version:" << i.second + << " txn_id=" << txn_id; + + std::string_view ver_key = i.first; + //VersionKeyInfo {instance_id, db_id, table_id, partition_id} + ver_key.remove_prefix(1); // Remove key space + std::vector, int, int>> out; + int ret = decode_key(&ver_key, &out); + if (ret != 0) [[unlikely]] { + // decode version key error means this is something wrong, + // we can not continue this txn + LOG(WARNING) << "failed to decode key, ret=" << ret << " key=" << hex(ver_key); + code = MetaServiceCode::UNDEFINED_ERR; + msg = "decode version key error"; + return; + } + + int64_t table_id = std::get(std::get<0>(out[4])); + int64_t partition_id = std::get(std::get<0>(out[5])); + VLOG_DEBUG << " table_id=" << table_id << " partition_id=" << partition_id; + + response->add_table_ids(table_id); + response->add_partition_ids(partition_id); + response->add_versions(i.second); + } + + LOG(INFO) << " before update txn_info=" << txn_info.ShortDebugString(); + + // Update txn_info + txn_info.set_status(TxnStatusPB::TXN_STATUS_VISIBLE); + + auto now_time = system_clock::now(); + uint64_t commit_time = duration_cast(now_time.time_since_epoch()).count(); + if ((txn_info.prepare_time() + txn_info.timeout_ms()) < commit_time) { + code = MetaServiceCode::UNDEFINED_ERR; + msg = fmt::format("txn is expired, not allow to commit txn_id={}", txn_id); + LOG(INFO) << msg << " prepare_time=" << txn_info.prepare_time() + << " timeout_ms=" << txn_info.timeout_ms() << " commit_time=" << commit_time; + return; + } + txn_info.set_commit_time(commit_time); + txn_info.set_finish_time(commit_time); + if (request->has_commit_attachment()) { + txn_info.mutable_commit_attachment()->CopyFrom(request->commit_attachment()); + } + LOG(INFO) << "after update txn_info=" << txn_info.ShortDebugString(); + info_val.clear(); + if (!txn_info.SerializeToString(&info_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize txn_info when saving, txn_id=" << txn_id; + msg = ss.str(); + return; + } + txn->put(info_key, info_val); + put_size += info_key.size() + info_val.size(); + ++num_put_keys; + LOG(INFO) << "xxx put info_key=" << hex(info_key) << " txn_id=" << txn_id; + + // Update stats of affected tablet + std::deque kv_pool; + std::function update_tablet_stats; + if (config::split_tablet_stats) { + update_tablet_stats = [&](const StatsTabletKeyInfo& info, const TabletStats& stats) { + if (stats.num_segs > 0) { + auto& data_size_key = kv_pool.emplace_back(); + stats_tablet_data_size_key(info, &data_size_key); + txn->atomic_add(data_size_key, stats.data_size); + auto& num_rows_key = kv_pool.emplace_back(); + stats_tablet_num_rows_key(info, &num_rows_key); + txn->atomic_add(num_rows_key, stats.num_rows); + auto& num_segs_key = kv_pool.emplace_back(); + stats_tablet_num_segs_key(info, &num_segs_key); + txn->atomic_add(num_segs_key, stats.num_segs); + put_size += data_size_key.size() + num_rows_key.size() + num_segs_key.size() + 24; + num_put_keys += 3; + } + auto& num_rowsets_key = kv_pool.emplace_back(); + stats_tablet_num_rowsets_key(info, &num_rowsets_key); + txn->atomic_add(num_rowsets_key, stats.num_rowsets); + put_size += num_rowsets_key.size() + 8; + ++num_put_keys; + }; + } else { + update_tablet_stats = [&](const StatsTabletKeyInfo& info, const TabletStats& stats) { + auto& key = kv_pool.emplace_back(); + stats_tablet_key(info, &key); + auto& val = kv_pool.emplace_back(); + TxnErrorCode err = txn->get(key, &val); + if (err != TxnErrorCode::TXN_OK) { + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::TABLET_NOT_FOUND + : cast_as(err); + msg = fmt::format("failed to get tablet stats, err={} tablet_id={}", err, + std::get<4>(info)); + return; + } + TabletStatsPB stats_pb; + if (!stats_pb.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = fmt::format("malformed tablet stats value, key={}", hex(key)); + return; + } + stats_pb.set_data_size(stats_pb.data_size() + stats.data_size); + stats_pb.set_num_rows(stats_pb.num_rows() + stats.num_rows); + stats_pb.set_num_rowsets(stats_pb.num_rowsets() + stats.num_rowsets); + stats_pb.set_num_segments(stats_pb.num_segments() + stats.num_segs); + stats_pb.SerializeToString(&val); + txn->put(key, val); + put_size += key.size() + val.size(); + ++num_put_keys; + }; + } + for (auto& [tablet_id, stats] : tablet_stats) { + DCHECK(table_ids.count(tablet_id)); + auto& tablet_idx = table_ids[tablet_id]; + StatsTabletKeyInfo info {instance_id, tablet_idx.table_id(), tablet_idx.index_id(), + tablet_idx.partition_id(), tablet_id}; + update_tablet_stats(info, stats); + if (code != MetaServiceCode::OK) return; + } + // Remove tmp rowset meta + num_del_keys += tmp_rowsets_meta.size(); + for (auto& [k, _] : tmp_rowsets_meta) { + txn->remove(k); + del_size += k.size(); + LOG(INFO) << "xxx remove tmp_rowset_key=" << hex(k) << " txn_id=" << txn_id; + } + + const std::string running_key = txn_running_key({instance_id, db_id, txn_id}); + LOG(INFO) << "xxx remove running_key=" << hex(running_key) << " txn_id=" << txn_id; + txn->remove(running_key); + del_size += running_key.size(); + ++num_del_keys; + + std::string recycle_val; + std::string recycle_key = recycle_txn_key({instance_id, db_id, txn_id}); + RecycleTxnPB recycle_pb; + recycle_pb.set_creation_time(commit_time); + recycle_pb.set_label(txn_info.label()); + + if (!recycle_pb.SerializeToString(&recycle_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize recycle_pb, txn_id=" << txn_id; + msg = ss.str(); + return; + } + txn->put(recycle_key, recycle_val); + put_size += recycle_key.size() + recycle_val.size(); + ++num_put_keys; + + if (txn_info.load_job_source_type() == + LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_ROUTINE_LOAD_TASK) { + if (!request->has_commit_attachment()) { + ss << "failed to get commit attachment from req, db_id=" << db_id + << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + + TxnCommitAttachmentPB txn_commit_attachment = request->commit_attachment(); + RLTaskTxnCommitAttachmentPB commit_attachment = + txn_commit_attachment.rl_task_txn_commit_attachment(); + int64_t job_id = commit_attachment.job_id(); + + std::string rl_progress_key; + std::string rl_progress_val; + bool prev_progress_existed = true; + RLJobProgressKeyInfo rl_progress_key_info {instance_id, db_id, job_id}; + rl_job_progress_key_info(rl_progress_key_info, &rl_progress_key); + TxnErrorCode err = txn->get(rl_progress_key, &rl_progress_val); + if (err != TxnErrorCode::TXN_OK) { + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + prev_progress_existed = false; + } else { + code = cast_as(err); + ss << "failed to get txn_info, db_id=" << db_id << " txn_id=" << txn_id + << " err=" << err; + msg = ss.str(); + return; + } + } + + RoutineLoadProgressPB prev_progress_info; + if (prev_progress_existed) { + if (!prev_progress_info.ParseFromString(rl_progress_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse txn_info, db_id=" << db_id << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + + int cal_row_num = 0; + for (auto const& elem : commit_attachment.progress().partition_to_offset()) { + if (elem.second >= 0) { + auto it = prev_progress_info.partition_to_offset().find(elem.first); + if (it != prev_progress_info.partition_to_offset().end() && it->second >= 0) { + cal_row_num += elem.second - it->second; + } else { + cal_row_num += elem.second + 1; + } + } + } + + LOG(INFO) << " calculated row num " << cal_row_num << " actual row num " + << commit_attachment.loaded_rows() << " prev progress " + << prev_progress_info.DebugString(); + + if (cal_row_num == 0) { + LOG(WARNING) << " repeated to load task in routine load, db_id=" << db_id + << " txn_id=" << txn_id << " calculated row num " << cal_row_num + << " actual row num " << commit_attachment.loaded_rows(); + return; + } + } + + std::string new_progress_val; + RoutineLoadProgressPB new_progress_info; + new_progress_info.CopyFrom(commit_attachment.progress()); + for (auto const& elem : prev_progress_info.partition_to_offset()) { + auto it = new_progress_info.partition_to_offset().find(elem.first); + if (it == new_progress_info.partition_to_offset().end()) { + new_progress_info.mutable_partition_to_offset()->insert(elem); + } + } + + if (!new_progress_info.SerializeToString(&new_progress_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize new progress val, txn_id=" << txn_info.txn_id(); + msg = ss.str(); + return; + } + txn->put(rl_progress_key, new_progress_val); + } + + LOG(INFO) << "xxx commit_txn put recycle_key key=" << hex(recycle_key) << " txn_id=" << txn_id; + LOG(INFO) << "commit_txn put_size=" << put_size << " del_size=" << del_size + << " num_put_keys=" << num_put_keys << " num_del_keys=" << num_del_keys + << " txn_id=" << txn_id; + + // Finally we are done... + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to commit kv txn, txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + return; + } + response->mutable_txn_info()->CopyFrom(txn_info); +} // end commit_txn + +void MetaServiceImpl::abort_txn(::google::protobuf::RpcController* controller, + const AbortTxnRequest* request, AbortTxnResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(abort_txn); + // Get txn id + int64_t txn_id = request->has_txn_id() ? request->txn_id() : -1; + std::string label = request->has_label() ? request->label() : ""; + int64_t db_id = request->has_db_id() ? request->db_id() : -1; + if (txn_id < 0 && (label.empty() || db_id < 0)) { + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "invalid txn id and label, db_id=" << db_id << " txn_id=" << txn_id + << " label=" << label; + msg = ss.str(); + return; + } + + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "cannot find instance_id with cloud_unique_id=" + << (cloud_unique_id.empty() ? "(empty)" : cloud_unique_id) << " label=" << label + << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + + RPC_RATE_LIMIT(abort_txn); + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "filed to txn_kv_->create_txn(), txn_id=" << txn_id << " label=" << label + << " err=" << err; + msg = ss.str(); + return; + } + + std::string info_key; // Will be used when saving updated txn + std::string info_val; // Will be reused when saving updated txn + TxnInfoPB txn_info; + //TODO: split with two function. + //there two ways to abort txn: + //1. abort txn by txn id + //2. abort txn by label and db_id + if (txn_id > 0) { + VLOG_DEBUG << "abort_txn by txn_id"; + //abort txn by txn id + // Get db id with txn id + + std::string index_key; + std::string index_val; + //not provide db_id, we need read from disk. + if (!request->has_db_id()) { + index_key = txn_index_key({instance_id, txn_id}); + err = txn->get(index_key, &index_val); + if (err != TxnErrorCode::TXN_OK) { + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::TXN_ID_NOT_FOUND + : cast_as(err); + ss << "failed to get db id, txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + return; + } + + TxnIndexPB index_pb; + if (!index_pb.ParseFromString(index_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse txn_index_val" + << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + DCHECK(index_pb.has_tablet_index() == true); + DCHECK(index_pb.tablet_index().has_db_id() == true); + db_id = index_pb.tablet_index().db_id(); + } else { + db_id = request->db_id(); + } + + // Get txn info with db_id and txn_id + info_key = txn_info_key({instance_id, db_id, txn_id}); + err = txn->get(info_key, &info_val); + if (err != TxnErrorCode::TXN_OK) { + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::TXN_ID_NOT_FOUND + : cast_as(err); + ss << "failed to get txn_info, db_id=" << db_id << "txn_id=" << txn_id << "err=" << err; + msg = ss.str(); + return; + } + + if (!txn_info.ParseFromString(info_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse txn_info db_id=" << db_id << "txn_id=" << txn_id; + msg = ss.str(); + return; + } + + DCHECK(txn_info.txn_id() == txn_id); + + //check state is valid. + if (txn_info.status() == TxnStatusPB::TXN_STATUS_ABORTED) { + code = MetaServiceCode::TXN_ALREADY_ABORTED; + ss << "transaction is already abort db_id=" << db_id << "txn_id=" << txn_id; + msg = ss.str(); + return; + } + if (txn_info.status() == TxnStatusPB::TXN_STATUS_VISIBLE) { + code = MetaServiceCode::TXN_ALREADY_VISIBLE; + ss << "transaction is already visible db_id=" << db_id << "txn_id=" << txn_id; + msg = ss.str(); + return; + } + } else { + VLOG_DEBUG << "abort_txn by db_id and txn label"; + //abort txn by label. + std::string label_key = txn_label_key({instance_id, db_id, label}); + std::string label_val; + err = txn->get(label_key, &label_val); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "txn->get() failed, label=" << label << " err=" << err; + msg = ss.str(); + return; + } + + //label index not exist + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = MetaServiceCode::TXN_LABEL_NOT_FOUND; + ss << "label not found, db_id=" << db_id << " label=" << label << " err=" << err; + msg = ss.str(); + return; + } + + TxnLabelPB label_pb; + DCHECK(label_val.size() > VERSION_STAMP_LEN); + if (!label_pb.ParseFromArray(label_val.data(), label_val.size() - VERSION_STAMP_LEN)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "txn_label_pb->ParseFromString() failed, label=" << label; + msg = ss.str(); + return; + } + + int64_t prepare_txn_id = 0; + //found prepare state txn for abort + for (auto& cur_txn_id : label_pb.txn_ids()) { + std::string cur_info_key = txn_info_key({instance_id, db_id, cur_txn_id}); + std::string cur_info_val; + err = txn->get(cur_info_key, &cur_info_val); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + std::stringstream ss; + ss << "txn->get() failed, cur_txn_id=" << cur_txn_id << " err=" << err; + msg = ss.str(); + return; + } + // ret == 0 + TxnInfoPB cur_txn_info; + if (!cur_txn_info.ParseFromString(cur_info_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + std::stringstream ss; + ss << "cur_txn_info->ParseFromString() failed, cur_txn_id=" << cur_txn_id; + msg = ss.str(); + return; + } + VLOG_DEBUG << "cur_txn_info=" << cur_txn_info.ShortDebugString(); + //TODO: 2pc else need to check TxnStatusPB::TXN_STATUS_PRECOMMITTED + if ((cur_txn_info.status() == TxnStatusPB::TXN_STATUS_PREPARED) || + (cur_txn_info.status() == TxnStatusPB::TXN_STATUS_PRECOMMITTED)) { + prepare_txn_id = cur_txn_id; + txn_info = std::move(cur_txn_info); + info_key = std::move(cur_info_key); + DCHECK_EQ(prepare_txn_id, txn_info.txn_id()) + << "prepare_txn_id=" << prepare_txn_id << " txn_id=" << txn_info.txn_id(); + break; + } + } + + if (prepare_txn_id == 0) { + code = MetaServiceCode::TXN_INVALID_STATUS; + std::stringstream ss; + ss << "running transaction not found, db_id=" << db_id << " label=" << label; + msg = ss.str(); + return; + } + } + + auto now_time = system_clock::now(); + uint64_t finish_time = duration_cast(now_time.time_since_epoch()).count(); + + // Update txn_info + txn_info.set_status(TxnStatusPB::TXN_STATUS_ABORTED); + txn_info.set_finish_time(finish_time); + request->has_reason() ? txn_info.set_reason(request->reason()) + : txn_info.set_reason("User Abort"); + + if (request->has_commit_attachment()) { + txn_info.mutable_commit_attachment()->CopyFrom(request->commit_attachment()); + } + + info_val.clear(); + if (!txn_info.SerializeToString(&info_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize txn_info when saving, txn_id=" << txn_info.txn_id(); + msg = ss.str(); + return; + } + LOG(INFO) << "check watermark conflict, txn_info=" << txn_info.ShortDebugString(); + txn->put(info_key, info_val); + LOG(INFO) << "xxx put info_key=" << hex(info_key) << " txn_id=" << txn_info.txn_id(); + + std::string running_key = txn_running_key({instance_id, db_id, txn_info.txn_id()}); + txn->remove(running_key); + LOG(INFO) << "xxx remove running_key=" << hex(running_key) << " txn_id=" << txn_info.txn_id(); + + std::string recycle_key = recycle_txn_key({instance_id, db_id, txn_info.txn_id()}); + std::string recycle_val; + RecycleTxnPB recycle_pb; + recycle_pb.set_creation_time(finish_time); + recycle_pb.set_label(txn_info.label()); + + if (!recycle_pb.SerializeToString(&recycle_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize recycle_pb, txn_id=" << txn_info.txn_id(); + msg = ss.str(); + return; + } + txn->put(recycle_key, recycle_val); + LOG(INFO) << "xxx put recycle_key=" << hex(recycle_key) << " txn_id=" << txn_info.txn_id(); + + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to commit kv txn, txn_id=" << txn_info.txn_id() << " err=" << err; + msg = ss.str(); + return; + } + response->mutable_txn_info()->CopyFrom(txn_info); +} + +void MetaServiceImpl::get_txn(::google::protobuf::RpcController* controller, + const GetTxnRequest* request, GetTxnResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(get_txn); + int64_t txn_id = request->has_txn_id() ? request->txn_id() : -1; + int64_t db_id = request->has_db_id() ? request->db_id() : -1; + if (txn_id < 0) { + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "invalid txn_id, it may be not given or set properly, txn_id=" << txn_id; + msg = ss.str(); + return; + } + + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "cannot find instance_id with cloud_unique_id=" + << (cloud_unique_id.empty() ? "(empty)" : cloud_unique_id); + msg = ss.str(); + return; + } + + RPC_RATE_LIMIT(get_txn) + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to create txn, txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + return; + } + + //not provide db_id, we need read from disk. + if (db_id < 0) { + const std::string index_key = txn_index_key({instance_id, txn_id}); + std::string index_val; + err = txn->get(index_key, &index_val); + if (err != TxnErrorCode::TXN_OK) { + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::TXN_ID_NOT_FOUND + : cast_as(err); + ss << "failed to get db id with txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + return; + } + + TxnIndexPB index_pb; + if (!index_pb.ParseFromString(index_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse txn_inf" + << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + DCHECK(index_pb.has_tablet_index() == true); + DCHECK(index_pb.tablet_index().has_db_id() == true); + db_id = index_pb.tablet_index().db_id(); + if (db_id <= 0) { + ss << "internal error: unexpected db_id " << db_id; + code = MetaServiceCode::UNDEFINED_ERR; + msg = ss.str(); + return; + } + } + + // Get txn info with db_id and txn_id + const std::string info_key = txn_info_key({instance_id, db_id, txn_id}); + std::string info_val; + err = txn->get(info_key, &info_val); + if (err != TxnErrorCode::TXN_OK) { + code = err == TxnErrorCode::TXN_KEY_NOT_FOUND ? MetaServiceCode::TXN_ID_NOT_FOUND + : cast_as(err); + ss << "failed to get db id with db_id=" << db_id << " txn_id=" << txn_id << " err=" << err; + msg = ss.str(); + return; + } + + TxnInfoPB txn_info; + if (!txn_info.ParseFromString(info_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse txn_info db_id=" << db_id << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + + VLOG_DEBUG << "txn_info=" << txn_info.ShortDebugString(); + DCHECK(txn_info.txn_id() == txn_id); + response->mutable_txn_info()->CopyFrom(txn_info); +} + +//To get current max txn id for schema change watermark etc. +void MetaServiceImpl::get_current_max_txn_id(::google::protobuf::RpcController* controller, + const GetCurrentMaxTxnRequest* request, + GetCurrentMaxTxnResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(get_current_max_txn_id); + // TODO: For auth + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); + return; + } + RPC_RATE_LIMIT(get_current_max_txn_id) + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + msg = "failed to create txn"; + code = cast_as(err); + return; + } + + const std::string key = "schema change"; + std::string val; + err = txn->get(key, &val); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = cast_as(err); + std::stringstream ss; + ss << "txn->get() failed, err=" << err; + msg = ss.str(); + return; + } + int64_t read_version = 0; + err = txn->get_read_version(&read_version); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + std::stringstream ss; + ss << "get read version failed, ret=" << err; + msg = ss.str(); + return; + } + + int64_t current_max_txn_id = read_version << 10; + VLOG_DEBUG << "read_version=" << read_version << " current_max_txn_id=" << current_max_txn_id; + response->set_current_max_txn_id(current_max_txn_id); +} + +void MetaServiceImpl::check_txn_conflict(::google::protobuf::RpcController* controller, + const CheckTxnConflictRequest* request, + CheckTxnConflictResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(check_txn_conflict); + if (!request->has_db_id() || !request->has_end_txn_id() || (request->table_ids_size() <= 0)) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "invalid db id, end txn id or table_ids."; + return; + } + // TODO: For auth + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "cannot find instance_id with cloud_unique_id=" + << (cloud_unique_id.empty() ? "(empty)" : cloud_unique_id); + msg = ss.str(); + return; + } + RPC_RATE_LIMIT(check_txn_conflict) + int64_t db_id = request->db_id(); + + std::string begin_running_key = txn_running_key({instance_id, db_id, 0}); + std::string end_running_key = txn_running_key({instance_id, db_id, request->end_txn_id()}); + LOG(INFO) << "begin_running_key:" << hex(begin_running_key) + << " end_running_key:" << hex(end_running_key); + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + msg = "failed to create txn"; + code = cast_as(err); + return; + } + + //TODO: use set to replace + std::vector src_table_ids(request->table_ids().begin(), request->table_ids().end()); + std::sort(src_table_ids.begin(), src_table_ids.end()); + std::unique_ptr it; + int64_t skip_timeout_txn_cnt = 0; + int total_iteration_cnt = 0; + do { + err = txn->get(begin_running_key, end_running_key, &it, true); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get txn running info. err=" << err; + msg = ss.str(); + LOG(WARNING) << msg; + return; + } + + VLOG_DEBUG << "begin_running_key=" << hex(begin_running_key) + << " end_running_key=" << hex(end_running_key) + << " it->has_next()=" << it->has_next(); + + auto now_time = system_clock::now(); + uint64_t check_time = duration_cast(now_time.time_since_epoch()).count(); + while (it->has_next()) { + total_iteration_cnt++; + auto [k, v] = it->next(); + LOG(INFO) << "check watermark conflict range_get txn_run_key=" << hex(k); + TxnRunningPB running_pb; + if (!running_pb.ParseFromArray(v.data(), v.size())) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "malformed txn running info"; + msg = ss.str(); + ss << " key=" << hex(k); + LOG(WARNING) << ss.str(); + return; + } + + if (running_pb.timeout_time() < check_time) { + skip_timeout_txn_cnt++; + break; + } + + LOG(INFO) << "check watermark conflict range_get txn_run_key=" << hex(k) + << " running_pb=" << running_pb.ShortDebugString(); + std::vector running_table_ids(running_pb.table_ids().begin(), + running_pb.table_ids().end()); + std::sort(running_table_ids.begin(), running_table_ids.end()); + std::vector result(std::min(running_table_ids.size(), src_table_ids.size())); + std::vector::iterator iter = std::set_intersection( + src_table_ids.begin(), src_table_ids.end(), running_table_ids.begin(), + running_table_ids.end(), result.begin()); + result.resize(iter - result.begin()); + if (result.size() > 0) { + response->set_finished(false); + LOG(INFO) << "skip timeout txn count: " << skip_timeout_txn_cnt + << " total iteration count: " << total_iteration_cnt; + return; + } + + if (!it->has_next()) { + begin_running_key = k; + } + } + begin_running_key.push_back('\x00'); // Update to next smallest key for iteration + } while (it->more()); + LOG(INFO) << "skip timeout txn count: " << skip_timeout_txn_cnt + << " total iteration count: " << total_iteration_cnt; + response->set_finished(true); +} + +/** + * @brief + * + * @param txn_kv + * @param instance_id + * @param db_id + * @param label_key + * @return TxnErrorCode + */ +TxnErrorCode internal_clean_label(std::shared_ptr txn_kv, const std::string_view instance_id, + int64_t db_id, const std::string_view label_key) { + std::string label_val; + TxnLabelPB label_pb; + + int64_t key_size = 0; + int64_t val_size = 0; + std::vector survival_txn_ids; + std::vector clean_txn_ids; + + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create txn. err=" << err << " db_id=" << db_id + << " label_key=" << hex(label_key); + return err; + } + + err = txn->get(label_key, &label_val); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + LOG(WARNING) << "failed to txn get err=" << err << " db_id=" << db_id + << " label_key=" << hex(label_key); + return err; + } + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + LOG(INFO) << "txn get err=" << err << " db_id=" << db_id << " label_key=" << hex(label_key); + return TxnErrorCode::TXN_OK; + } + + if (label_val.size() <= VERSION_STAMP_LEN) { + LOG(INFO) << "label_val.size()=" << label_val.size() << " db_id=" << db_id + << " label_key=" << hex(label_key); + return TxnErrorCode::TXN_OK; + } + + if (!label_pb.ParseFromArray(label_val.data(), label_val.size() - VERSION_STAMP_LEN)) { + LOG(WARNING) << "failed to parse txn label" + << " db_id=" << db_id << " label_key=" << hex(label_key) + << " label_val.size()=" << label_val.size(); + return TxnErrorCode::TXN_UNIDENTIFIED_ERROR; + } + + for (auto txn_id : label_pb.txn_ids()) { + const std::string recycle_key = recycle_txn_key({instance_id, db_id, txn_id}); + const std::string index_key = txn_index_key({instance_id, txn_id}); + const std::string info_key = txn_info_key({instance_id, db_id, txn_id}); + + std::string info_val; + err = txn->get(info_key, &info_val); + if (err != TxnErrorCode::TXN_OK) { + LOG_WARNING("info_key get failed") + .tag("info_key", hex(info_key)) + .tag("label_key", hex(label_key)) + .tag("db_id", db_id) + .tag("txn_id", txn_id) + .tag("err", err); + return err; + } + + TxnInfoPB txn_info; + if (!txn_info.ParseFromString(info_val)) { + LOG_WARNING("info_val parse failed") + .tag("info_key", hex(info_key)) + .tag("label_key", hex(label_key)) + .tag("db_id", db_id) + .tag("txn_id", txn_id) + .tag("size", info_val.size()); + return TxnErrorCode::TXN_UNIDENTIFIED_ERROR; + } + + std::string recycle_val; + if ((txn_info.status() != TxnStatusPB::TXN_STATUS_ABORTED) && + (txn_info.status() != TxnStatusPB::TXN_STATUS_VISIBLE)) { + // txn status is not final status + LOG(INFO) << "txn not final state, label_key=" << hex(label_key) + << " txn_id=" << txn_id; + survival_txn_ids.push_back(txn_id); + DCHECK_EQ(txn->get(recycle_key, &recycle_val), TxnErrorCode::TXN_KEY_NOT_FOUND); + continue; + } + + DCHECK_EQ(txn->get(recycle_key, &recycle_val), TxnErrorCode::TXN_OK); + DCHECK((txn_info.status() == TxnStatusPB::TXN_STATUS_ABORTED) || + (txn_info.status() == TxnStatusPB::TXN_STATUS_VISIBLE)); + + txn->remove(index_key); + key_size += index_key.size(); + + txn->remove(info_key); + key_size += info_key.size(); + + txn->remove(recycle_key); + key_size += recycle_key.size(); + clean_txn_ids.push_back(txn_id); + LOG(INFO) << "remove index_key=" << hex(index_key) << " info_key=" << hex(info_key) + << " recycle_key=" << hex(recycle_key); + } + if (label_pb.txn_ids().size() == clean_txn_ids.size()) { + txn->remove(label_key); + key_size += label_key.size(); + LOG(INFO) << "remove label_key=" << hex(label_key); + } else { + label_pb.clear_txn_ids(); + for (auto txn_id : survival_txn_ids) { + label_pb.add_txn_ids(txn_id); + } + LOG(INFO) << "rewrite label_pb=" << label_pb.ShortDebugString(); + label_val.clear(); + if (!label_pb.SerializeToString(&label_val)) { + LOG(INFO) << "failed to serialize label_pb=" << label_pb.ShortDebugString() + << " label_key=" << hex(label_key); + return TxnErrorCode::TXN_UNIDENTIFIED_ERROR; + } + txn->atomic_set_ver_value(label_key, label_val); + key_size += label_key.size(); + val_size += label_val.size(); + } + + err = txn->commit(); + TEST_SYNC_POINT_CALLBACK("internal_clean_label:err", &err); + if (err != TxnErrorCode::TXN_OK) { + LOG(INFO) << fmt::format( + "label_key={} key_size={} val_size={} label_pb={} clean_txn_ids={}", hex(label_key), + key_size, val_size, label_pb.ShortDebugString(), fmt::join(clean_txn_ids, " ")); + } + return err; +} + +void MetaServiceImpl::clean_txn_label(::google::protobuf::RpcController* controller, + const CleanTxnLabelRequest* request, + CleanTxnLabelResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(clean_txn_label); + if (!request->has_db_id()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "missing db id"; + LOG(WARNING) << msg; + return; + } + + std::string cloud_unique_id = request->has_cloud_unique_id() ? request->cloud_unique_id() : ""; + instance_id = get_instance_id(resource_mgr_, cloud_unique_id); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "cannot find instance_id with cloud_unique_id=" + << (cloud_unique_id.empty() ? "(empty)" : cloud_unique_id); + msg = ss.str(); + LOG(WARNING) << msg; + return; + } + RPC_RATE_LIMIT(clean_txn_label) + const int64_t db_id = request->db_id(); + + // clean label only by db_id + if (request->labels().empty()) { + std::string begin_label_key = txn_label_key({instance_id, db_id, ""}); + const std::string end_label_key = txn_label_key({instance_id, db_id + 1, ""}); + + std::unique_ptr it; + bool snapshot = true; + int limit = 1000; + TEST_SYNC_POINT_CALLBACK("clean_txn_label:limit", &limit); + do { + std::unique_ptr txn; + auto err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + msg = "failed to create txn"; + code = cast_as(err); + LOG(INFO) << msg << " err=" << err << " begin=" << hex(begin_label_key) + << " end=" << hex(end_label_key); + return; + } + + err = txn->get(begin_label_key, end_label_key, &it, snapshot, limit); + if (err != TxnErrorCode::TXN_OK) { + msg = "failed to txn range get"; + code = cast_as(err); + LOG(WARNING) << msg << " err=" << err << " begin=" << hex(begin_label_key) + << " end=" << hex(end_label_key); + return; + } + + if (!it->has_next()) { + LOG(INFO) << "no keys in the range. begin=" << hex(begin_label_key) + << " end=" << hex(end_label_key); + break; + } + while (it->has_next()) { + auto [k, v] = it->next(); + if (!it->has_next()) { + begin_label_key = k; + LOG(INFO) << "iterator has no more kvs. key=" << hex(k); + } + err = internal_clean_label(txn_kv_, instance_id, db_id, k); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to clean txn label."; + LOG(WARNING) << msg << " err=" << err << " db_id=" << db_id; + return; + } + } + begin_label_key.push_back('\x00'); + } while (it->more()); + } else { + const std::string& label = request->labels(0); + const std::string label_key = txn_label_key({instance_id, db_id, label}); + TxnErrorCode err = internal_clean_label(txn_kv_, instance_id, db_id, label_key); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + msg = "failed to clean txn label."; + LOG(WARNING) << msg << " err=" << err << " db_id=" << db_id + << " label_key=" << hex(label_key); + return; + } + } + + code = MetaServiceCode::OK; +} + +} // namespace doris::cloud diff --git a/cloud/src/meta-service/txn_kv.cpp b/cloud/src/meta-service/txn_kv.cpp new file mode 100644 index 00000000000000..54cec231f6f3f4 --- /dev/null +++ b/cloud/src/meta-service/txn_kv.cpp @@ -0,0 +1,544 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "txn_kv.h" + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "common/bvars.h" +#include "common/config.h" +#include "common/logging.h" +#include "common/stopwatch.h" +#include "common/sync_point.h" +#include "common/util.h" +#include "meta-service/txn_kv_error.h" + +// ============================================================================= +// FoundationDB implementation of TxnKv +// ============================================================================= + +#define RETURN_IF_ERROR(op) \ + do { \ + TxnErrorCode code = op; \ + if (code != TxnErrorCode::TXN_OK) return code; \ + } while (false) + +namespace doris::cloud { + +int FdbTxnKv::init() { + network_ = std::make_shared(FDBNetworkOption {}); + int ret = network_->init(); + if (ret != 0) { + LOG(WARNING) << "failed to init network"; + return ret; + } + database_ = std::make_shared(network_, config::fdb_cluster_file_path, + FDBDatabaseOption {}); + ret = database_->init(); + if (ret != 0) { + LOG(WARNING) << "failed to init database"; + return ret; + } + return 0; +} + +TxnErrorCode FdbTxnKv::create_txn(std::unique_ptr* txn) { + auto* t = new fdb::Transaction(database_); + txn->reset(t); + auto ret = t->init(); + if (ret != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to init txn, ret=" << ret; + } + return ret; +} + +} // namespace doris::cloud + +namespace doris::cloud::fdb { + +// Ref https://apple.github.io/foundationdb/api-error-codes.html#developer-guide-error-codes. +constexpr fdb_error_t FDB_ERROR_CODE_TIMED_OUT = 1004; +constexpr fdb_error_t FDB_ERROR_CODE_TXN_TOO_OLD = 1007; +constexpr fdb_error_t FDB_ERROR_CODE_TXN_CONFLICT = 1020; +constexpr fdb_error_t FDB_ERROR_CODE_TXN_TIMED_OUT = 1031; +constexpr fdb_error_t FDB_ERROR_CODE_INVALID_OPTION_VALUE = 2006; +constexpr fdb_error_t FDB_ERROR_CODE_INVALID_OPTION = 2007; +constexpr fdb_error_t FDB_ERROR_CODE_VERSION_INVALID = 2011; +constexpr fdb_error_t FDB_ERROR_CODE_RANGE_LIMITS_INVALID = 2012; +constexpr fdb_error_t FDB_ERROR_CODE_TXN_TOO_LARGE = 2101; +constexpr fdb_error_t FDB_ERROR_CODE_KEY_TOO_LARGE = 2102; +constexpr fdb_error_t FDB_ERROR_CODE_VALUE_TOO_LARGE = 2103; +constexpr fdb_error_t FDB_ERROR_CODE_CONNECTION_STRING_INVALID = 2104; + +static bool fdb_error_is_txn_conflict(fdb_error_t err) { + return err == FDB_ERROR_CODE_TXN_CONFLICT; +} + +static TxnErrorCode cast_as_txn_code(fdb_error_t err) { + switch (err) { + case 0: + return TxnErrorCode::TXN_OK; + case FDB_ERROR_CODE_INVALID_OPTION: + case FDB_ERROR_CODE_INVALID_OPTION_VALUE: + case FDB_ERROR_CODE_VERSION_INVALID: + case FDB_ERROR_CODE_RANGE_LIMITS_INVALID: + case FDB_ERROR_CODE_CONNECTION_STRING_INVALID: + return TxnErrorCode::TXN_INVALID_ARGUMENT; + case FDB_ERROR_CODE_TXN_TOO_LARGE: + return TxnErrorCode::TXN_BYTES_TOO_LARGE; + case FDB_ERROR_CODE_KEY_TOO_LARGE: + return TxnErrorCode::TXN_KEY_TOO_LARGE; + case FDB_ERROR_CODE_VALUE_TOO_LARGE: + return TxnErrorCode::TXN_VALUE_TOO_LARGE; + case FDB_ERROR_CODE_TIMED_OUT: + case FDB_ERROR_CODE_TXN_TIMED_OUT: + return TxnErrorCode::TXN_TIMEOUT; + case FDB_ERROR_CODE_TXN_TOO_OLD: + return TxnErrorCode::TXN_TOO_OLD; + case FDB_ERROR_CODE_TXN_CONFLICT: + return TxnErrorCode::TXN_CONFLICT; + } + + if (fdb_error_predicate(FDB_ERROR_PREDICATE_MAYBE_COMMITTED, err)) { + return TxnErrorCode::TXN_MAYBE_COMMITTED; + } + if (fdb_error_predicate(FDB_ERROR_PREDICATE_RETRYABLE_NOT_COMMITTED, err)) { + return TxnErrorCode::TXN_RETRYABLE_NOT_COMMITTED; + } + return TxnErrorCode::TXN_UNIDENTIFIED_ERROR; +} + +// ============================================================================= +// Impl of Network +// ============================================================================= +decltype(Network::working) Network::working {false}; + +int Network::init() { + // Globally once + bool expected = false; + if (!Network::working.compare_exchange_strong(expected, true)) return 1; + + fdb_error_t err = fdb_select_api_version(fdb_get_max_api_version()); + if (err) { + LOG(WARNING) << "failed to select api version, max api version: " + << fdb_get_max_api_version() << ", err: " << fdb_get_error(err); + return 1; + } + + // Setup network thread + // Optional setting + // FDBNetworkOption opt; + // fdb_network_set_option() + (void)opt_; + // ATTN: Network can be configured only once, + // even if fdb_stop_network() is called successfully + err = fdb_setup_network(); // Must be called only once before any + // other functions of C-API + if (err) { + LOG(WARNING) << "failed to setup fdb network, err: " << fdb_get_error(err); + return 1; + } + + // Network complete callback is optional, and useful for some cases + // std::function network_complete_callback = + // []() { std::cout << __PRETTY_FUNCTION__ << std::endl; }; + // err = fdb_add_network_thread_completion_hook(callback1, + // &network_complete_callback); + // std::cout << "fdb_add_network_thread_completion_hook error: " + // << fdb_get_error(err) << std::endl; + // if (err) std::exit(-1); + + // Run network in a separate thread + network_thread_ = std::shared_ptr( + new std::thread([] { + // Will not return until fdb_stop_network() called + auto err = fdb_run_network(); + LOG(WARNING) << "exit fdb_run_network" + << (err ? std::string(", error: ") + fdb_get_error(err) : ""); + }), + [](auto* p) { + auto err = fdb_stop_network(); + LOG(WARNING) << "fdb_stop_network" + << (err ? std::string(", error: ") + fdb_get_error(err) : ""); + p->join(); + delete p; + + // Another network will work only after this thread exits + bool expected = true; + Network::working.compare_exchange_strong(expected, false); + }); + + return 0; +} + +void Network::stop() { + network_thread_.reset(); +} + +// ============================================================================= +// Impl of Database +// ============================================================================= + +int Database::init() { + // TODO: process opt + fdb_error_t err = fdb_create_database(cluster_file_path_.c_str(), &db_); + if (err) { + LOG(WARNING) << __PRETTY_FUNCTION__ << " fdb_create_database error: " << fdb_get_error(err) + << " conf: " << cluster_file_path_; + return 1; + } + + return 0; +} + +// ============================================================================= +// Impl of Transaction +// ============================================================================= + +TxnErrorCode Transaction::init() { + // TODO: process opt + fdb_error_t err = fdb_database_create_transaction(db_->db(), &txn_); + TEST_SYNC_POINT_CALLBACK("transaction:init:create_transaction_err", &err); + if (err) { + LOG(WARNING) << __PRETTY_FUNCTION__ + << " fdb_database_create_transaction error:" << fdb_get_error(err); + return cast_as_txn_code(err); + } + + // FDB txn callback only guaranteed *at most once*, because the future might be set to `Never` + // by unexpected. In order to achieve *exactly once* semantic, a timeout must be set to force + // fdb cancel future and invoke callback eventually. + // + // See: + // - https://apple.github.io/foundationdb/api-c.html#fdb_future_set_callback. + // - https://forums.foundationdb.org/t/does-fdb-future-set-callback-guarantee-exactly-once-execution/1498/2 + static_assert(sizeof(config::fdb_txn_timeout_ms) == sizeof(int64_t)); + err = fdb_transaction_set_option(txn_, FDBTransactionOption::FDB_TR_OPTION_TIMEOUT, + (const uint8_t*)&config::fdb_txn_timeout_ms, + sizeof(config::fdb_txn_timeout_ms)); + if (err) { + LOG_WARNING("fdb_transaction_set_option error: ") + .tag("code", err) + .tag("msg", fdb_get_error(err)); + return cast_as_txn_code(err); + } + + return TxnErrorCode::TXN_OK; +} + +void Transaction::put(std::string_view key, std::string_view val) { + StopWatch sw; + fdb_transaction_set(txn_, (uint8_t*)key.data(), key.size(), (uint8_t*)val.data(), val.size()); + g_bvar_txn_kv_put << sw.elapsed_us(); +} + +// return 0 for success otherwise error +static TxnErrorCode bthread_fdb_future_block_until_ready(FDBFuture* fut) { + bthread::CountdownEvent event; + static auto callback = [](FDBFuture*, void* event) { + ((bthread::CountdownEvent*)event)->signal(); + }; + auto err = fdb_future_set_callback(fut, callback, &event); + if (err) [[unlikely]] { + LOG(WARNING) << "fdb_future_set_callback failed, err=" << fdb_get_error(err); + return cast_as_txn_code(err); + } + if (int ec = event.wait(); ec != 0) [[unlikely]] { + LOG(WARNING) << "CountdownEvent wait failed, err=" << std::strerror(ec); + return TxnErrorCode::TXN_UNIDENTIFIED_ERROR; + } + return TxnErrorCode::TXN_OK; +} + +// return TXN_OK for success otherwise error +static TxnErrorCode await_future(FDBFuture* fut) { + if (bthread_self() != 0) { + return bthread_fdb_future_block_until_ready(fut); + } + auto err = fdb_future_block_until_ready(fut); + TEST_SYNC_POINT_CALLBACK("fdb_future_block_until_ready_err", &err); + if (err) [[unlikely]] { + LOG(WARNING) << "fdb_future_block_until_ready failed: " << fdb_get_error(err); + return cast_as_txn_code(err); + } + return TxnErrorCode::TXN_OK; +} + +TxnErrorCode Transaction::get(std::string_view key, std::string* val, bool snapshot) { + StopWatch sw; + auto* fut = fdb_transaction_get(txn_, (uint8_t*)key.data(), key.size(), snapshot); + + auto release_fut = [fut, &sw](int*) { + fdb_future_destroy(fut); + g_bvar_txn_kv_get << sw.elapsed_us(); + }; + std::unique_ptr defer((int*)0x01, std::move(release_fut)); + RETURN_IF_ERROR(await_future(fut)); + auto err = fdb_future_get_error(fut); + TEST_SYNC_POINT_CALLBACK("transaction:get:get_err", &err); + if (err) { + LOG(WARNING) << __PRETTY_FUNCTION__ + << " failed to fdb_future_get_error err=" << fdb_get_error(err) + << " key=" << hex(key); + return cast_as_txn_code(err); + } + + fdb_bool_t found; + const uint8_t* ret; + int len; + err = fdb_future_get_value(fut, &found, &ret, &len); + + if (err) { + LOG(WARNING) << __PRETTY_FUNCTION__ + << " failed to fdb_future_get_value err=" << fdb_get_error(err) + << " key=" << hex(key); + return cast_as_txn_code(err); + } + + if (!found) return TxnErrorCode::TXN_KEY_NOT_FOUND; + *val = std::string((char*)ret, len); + return TxnErrorCode::TXN_OK; +} + +TxnErrorCode Transaction::get(std::string_view begin, std::string_view end, + std::unique_ptr* iter, bool snapshot, + int limit) { + StopWatch sw; + std::unique_ptr> defer( + (int*)0x01, [&sw](int*) { g_bvar_txn_kv_range_get << sw.elapsed_us(); }); + + FDBFuture* fut = fdb_transaction_get_range( + txn_, FDB_KEYSEL_FIRST_GREATER_OR_EQUAL((uint8_t*)begin.data(), begin.size()), + FDB_KEYSEL_FIRST_GREATER_OR_EQUAL((uint8_t*)end.data(), end.size()), limit, + 0 /*target_bytes, unlimited*/, FDBStreamingMode::FDB_STREAMING_MODE_WANT_ALL, + // FDBStreamingMode::FDB_STREAMING_MODE_ITERATOR, + 0 /*iteration*/, snapshot, false /*reverse*/); + + RETURN_IF_ERROR(await_future(fut)); + auto err = fdb_future_get_error(fut); + TEST_SYNC_POINT_CALLBACK("transaction:get_range:get_err", &err); + if (err) { + LOG(WARNING) << fdb_get_error(err); + return cast_as_txn_code(err); + } + + std::unique_ptr ret(new RangeGetIterator(fut)); + RETURN_IF_ERROR(ret->init()); + + *(iter) = std::move(ret); + + return TxnErrorCode::TXN_OK; +} + +void Transaction::atomic_set_ver_key(std::string_view key_prefix, std::string_view val) { + StopWatch sw; + std::unique_ptr key(new std::string(key_prefix)); + int prefix_size = key->size(); + // ATTN: + // 10 bytes for versiontimestamp must be 0, trailing 4 bytes is for prefix len + key->resize(key->size() + 14, '\0'); + std::memcpy(key->data() + (key->size() - 4), &prefix_size, 4); + + fdb_transaction_atomic_op(txn_, (uint8_t*)key->data(), key->size(), (uint8_t*)val.data(), + val.size(), + FDBMutationType::FDB_MUTATION_TYPE_SET_VERSIONSTAMPED_KEY); + + g_bvar_txn_kv_atomic_set_ver_key << sw.elapsed_us(); +} + +void Transaction::atomic_set_ver_value(std::string_view key, std::string_view value) { + StopWatch sw; + std::unique_ptr val(new std::string(value)); + int prefix_size = val->size(); + // ATTN: + // 10 bytes for versiontimestamp must be 0, trailing 4 bytes is for prefix len + val->resize(val->size() + 14, '\0'); + std::memcpy(val->data() + (val->size() - 4), &prefix_size, 4); + + fdb_transaction_atomic_op(txn_, (uint8_t*)key.data(), key.size(), (uint8_t*)val->data(), + val->size(), + FDBMutationType::FDB_MUTATION_TYPE_SET_VERSIONSTAMPED_VALUE); + + g_bvar_txn_kv_atomic_set_ver_value << sw.elapsed_us(); +} + +void Transaction::atomic_add(std::string_view key, int64_t to_add) { + StopWatch sw; + auto val = std::make_unique(sizeof(to_add), '\0'); + std::memcpy(val->data(), &to_add, sizeof(to_add)); + fdb_transaction_atomic_op(txn_, (uint8_t*)key.data(), key.size(), (uint8_t*)val->data(), + sizeof(to_add), FDBMutationType::FDB_MUTATION_TYPE_ADD); + + g_bvar_txn_kv_atomic_add << sw.elapsed_us(); +} + +void Transaction::remove(std::string_view key) { + StopWatch sw; + fdb_transaction_clear(txn_, (uint8_t*)key.data(), key.size()); + g_bvar_txn_kv_remove << sw.elapsed_us(); +} + +void Transaction::remove(std::string_view begin, std::string_view end) { + StopWatch sw; + fdb_transaction_clear_range(txn_, (uint8_t*)begin.data(), begin.size(), (uint8_t*)end.data(), + end.size()); + g_bvar_txn_kv_range_remove << sw.elapsed_us(); +} + +TxnErrorCode Transaction::commit() { + fdb_error_t err = 0; + TEST_SYNC_POINT_CALLBACK("transaction:commit:get_err", &err); + if (err == 0) [[likely]] { + StopWatch sw; + auto* fut = fdb_transaction_commit(txn_); + auto release_fut = [fut, &sw](int*) { + fdb_future_destroy(fut); + g_bvar_txn_kv_commit << sw.elapsed_us(); + }; + std::unique_ptr defer((int*)0x01, std::move(release_fut)); + RETURN_IF_ERROR(await_future(fut)); + err = fdb_future_get_error(fut); + } + + if (err) { + LOG(WARNING) << "fdb commit error, code=" << err << " msg=" << fdb_get_error(err); + fdb_error_is_txn_conflict(err) ? g_bvar_txn_kv_commit_conflict_counter << 1 + : g_bvar_txn_kv_commit_error_counter << 1; + return cast_as_txn_code(err); + } + return TxnErrorCode::TXN_OK; +} + +TxnErrorCode Transaction::get_read_version(int64_t* version) { + StopWatch sw; + auto fut = fdb_transaction_get_read_version(txn_); + std::unique_ptr> defer((int*)0x01, [fut, &sw](...) { + fdb_future_destroy(fut); + g_bvar_txn_kv_get_read_version << sw.elapsed_us(); + }); + RETURN_IF_ERROR(await_future(fut)); + auto err = fdb_future_get_error(fut); + TEST_SYNC_POINT_CALLBACK("transaction:get_read_version:get_err", &err); + if (err) { + LOG(WARNING) << "get read version: " << fdb_get_error(err); + return cast_as_txn_code(err); + } + err = fdb_future_get_int64(fut, version); + if (err) { + LOG(WARNING) << "get read version: " << fdb_get_error(err); + return cast_as_txn_code(err); + } + return TxnErrorCode::TXN_OK; +} + +TxnErrorCode Transaction::get_committed_version(int64_t* version) { + StopWatch sw; + auto err = fdb_transaction_get_committed_version(txn_, version); + if (err) { + LOG(WARNING) << "get committed version " << fdb_get_error(err); + g_bvar_txn_kv_get_committed_version << sw.elapsed_us(); + return cast_as_txn_code(err); + } + g_bvar_txn_kv_get_committed_version << sw.elapsed_us(); + return TxnErrorCode::TXN_OK; +} + +TxnErrorCode Transaction::abort() { + return TxnErrorCode::TXN_OK; +} + +TxnErrorCode RangeGetIterator::init() { + if (fut_ == nullptr) return TxnErrorCode::TXN_UNIDENTIFIED_ERROR; + idx_ = 0; + kvs_size_ = 0; + more_ = false; + kvs_ = nullptr; + auto err = fdb_future_get_keyvalue_array(fut_, &kvs_, &kvs_size_, &more_); + TEST_SYNC_POINT_CALLBACK("range_get_iterator:init:get_keyvalue_array_err", &err); + if (err) { + LOG(WARNING) << "fdb_future_get_keyvalue_array failed, err=" << fdb_get_error(err); + return cast_as_txn_code(err); + } + return TxnErrorCode::TXN_OK; +} + +TxnErrorCode Transaction::batch_get(std::vector>* res, + const std::vector& keys, + const BatchGetOptions& opts) { + if (keys.empty()) { + return TxnErrorCode::TXN_OK; + } + StopWatch sw; + std::vector futures; + futures.reserve(keys.size()); + for (const auto& k : keys) { + futures.push_back(fdb_transaction_get(txn_, (uint8_t*)k.data(), k.size(), opts.snapshot)); + } + + auto release_futures = [&futures, &sw](int*) { + std::for_each(futures.begin(), futures.end(), + [](FDBFuture* fut) { fdb_future_destroy(fut); }); + g_bvar_txn_kv_batch_get << sw.elapsed_us(); + }; + std::unique_ptr defer((int*)0x01, std::move(release_futures)); + + res->reserve(keys.size()); + DCHECK(keys.size() == futures.size()); + auto size = futures.size(); + for (auto i = 0; i < size; ++i) { + const auto& fut = futures[i]; + RETURN_IF_ERROR(await_future(fut)); + auto err = fdb_future_get_error(fut); + if (err) { + LOG(WARNING) << __PRETTY_FUNCTION__ + << " failed to fdb_future_get_error err=" << fdb_get_error(err) + << " key=" << hex(keys[i]); + return cast_as_txn_code(err); + } + fdb_bool_t found; + const uint8_t* ret; + int len; + err = fdb_future_get_value(fut, &found, &ret, &len); + + if (err) { + LOG(WARNING) << __PRETTY_FUNCTION__ + << " failed to fdb_future_get_value err=" << fdb_get_error(err) + << " key=" << hex(keys[i]); + return cast_as_txn_code(err); + } + if (!found) { + res->push_back(std::nullopt); + continue; + } + res->push_back(std::string((char*)ret, len)); + } + return TxnErrorCode::TXN_OK; +} + +} // namespace doris::cloud::fdb diff --git a/cloud/src/meta-service/txn_kv.h b/cloud/src/meta-service/txn_kv.h new file mode 100644 index 00000000000000..6cf225de3cc9d3 --- /dev/null +++ b/cloud/src/meta-service/txn_kv.h @@ -0,0 +1,463 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include "txn_kv_error.h" + +// ============================================================================= +// +// ============================================================================= + +namespace doris::cloud { + +class Transaction; +class RangeGetIterator; + +class TxnKv { +public: + TxnKv() = default; + virtual ~TxnKv() = default; + + /** + * Creates a transaction + * TODO: add options to create the txn + * + * @param txn output param + * @return TXN_OK for success + */ + virtual TxnErrorCode create_txn(std::unique_ptr* txn) = 0; + + virtual int init() = 0; +}; + +class Transaction { +public: + Transaction() = default; + virtual ~Transaction() = default; + + virtual void put(std::string_view key, std::string_view val) = 0; + + /** + * @param snapshot if true, `key` will not be included in txn conflict detection this time + * @return TXN_OK for success get a key, TXN_KEY_NOT_FOUND for key not found, otherwise for error + */ + virtual TxnErrorCode get(std::string_view key, std::string* val, bool snapshot = false) = 0; + + /** + * Closed-open range + * @param snapshot if true, key range will not be included in txn conflict detection this time + * @param limit if non-zero, indicates the maximum number of key-value pairs to return + * @return TXN_OK for success, otherwise for error + */ + virtual TxnErrorCode get(std::string_view begin, std::string_view end, + std::unique_ptr* iter, bool snapshot = false, + int limit = 10000) = 0; + + /** + * Put a key-value pair in which key will in the form of + * `key_prefix + versiontimestamp`. `versiontimestamp` is autogenerated by the + * system and it's 10-byte long and encoded in big-endian + * + * @param key_prefix prefix for key convertion, can be zero-length + * @param val value + */ + virtual void atomic_set_ver_key(std::string_view key_prefix, std::string_view val) = 0; + + /** + * Put a key-value pair in which key will in the form of + * `value + versiontimestamp`. `versiontimestamp` is autogenerated by the + * system and it's 10-byte long and encoded in big-endian + * + * @param key prefix for key convertion, can be zero-length + * @param val value + */ + virtual void atomic_set_ver_value(std::string_view key, std::string_view val) = 0; + + /** + * Adds a value to database + * @param to_add positive for addition, negative for substraction + * @return 0 for success otherwise error + */ + virtual void atomic_add(std::string_view key, int64_t to_add) = 0; + // TODO: min max or and xor cmp_and_clear set_ver_value + + virtual void remove(std::string_view key) = 0; + + /** + * Remove a closed-open range + */ + virtual void remove(std::string_view begin, std::string_view end) = 0; + + /** + * + *@return TXN_OK for success otherwise error + */ + virtual TxnErrorCode commit() = 0; + + /** + * Gets the read version used by the txn. + * Note that it does not make any sense we call this function before + * any `Transaction::get()` is called. + * + *@return TXN_OK for success otherwise error + */ + virtual TxnErrorCode get_read_version(int64_t* version) = 0; + + /** + * Gets the commited version used by the txn. + * Note that it does not make any sense we call this function before + * a successful call to `Transaction::commit()`. + * + *@return TXN_OK for success, TXN_CONFLICT for conflict, otherwise error + */ + virtual TxnErrorCode get_committed_version(int64_t* version) = 0; + + /** + * Aborts this transaction + * + * @return TXN_OK for success otherwise error + */ + virtual TxnErrorCode abort() = 0; + + struct BatchGetOptions { + BatchGetOptions() : snapshot(false) {}; + bool snapshot; + // TODO: Avoid consuming too many resources in one batch + // int limit = 1000; + }; + /** + * @brief batch get keys + * + * @param res + * @param keys + * @param opts + * @return If all keys are successfully retrieved, return TXN_OK. Otherwise, return the code of the first occurring error + */ + virtual TxnErrorCode batch_get(std::vector>* res, + const std::vector& keys, + const BatchGetOptions& opts = BatchGetOptions()) = 0; +}; + +class RangeGetIterator { +public: + RangeGetIterator() = default; + virtual ~RangeGetIterator() = default; + + /** + * Checks if we can call `next()` on this iterator. + */ + virtual bool has_next() = 0; + + /** + * Gets next element, this is usually called after a check of `has_next()` succeeds, + * If `has_next()` is not checked, the return value may be undefined. + * + * @return a kv pair + */ + virtual std::pair next() = 0; + + /** + * Repositions the offset to `pos` + */ + virtual void seek(size_t pos) = 0; + + /** + * Checks if there are more KVs to be get from the range, caller usually wants + * to issue another `get` with the last key of this iteration. + * + * @return if there are more kvs that this iterator cannot cover + */ + virtual bool more() = 0; + + /** + * + * Gets size of the range, some kinds of iterators may not support this function. + * + * @return size + */ + virtual int size() = 0; + + /** + * Resets to initial state, some kinds of iterators may not support this function. + */ + virtual void reset() = 0; + + /** + * Get the begin key of the next iterator if `more()` is true, otherwise returns empty string. + */ + virtual std::string next_begin_key() = 0; + + RangeGetIterator(const RangeGetIterator&) = delete; + RangeGetIterator& operator=(const RangeGetIterator&) = delete; +}; + +// ============================================================================= +// FoundationDB implementation of TxnKv +// ============================================================================= + +namespace fdb { +class Database; +class Transaction; +class Network; +} // namespace fdb + +class FdbTxnKv : public TxnKv { +public: + FdbTxnKv() = default; + ~FdbTxnKv() override = default; + + TxnErrorCode create_txn(std::unique_ptr* txn) override; + + int init() override; + +private: + std::shared_ptr network_; + std::shared_ptr database_; +}; + +namespace fdb { + +class Network { +public: + Network(FDBNetworkOption opt) : opt_(opt) {} + + /** + * @return 0 for success otherwise non-zero + */ + int init(); + + /** + * Notify the newwork thread to stop, this is an async. call, check + * Network::working to ensure the network exited finally. + * + * FIXME: may be we can implement it as a sync. function. + */ + void stop(); + + ~Network() = default; + +private: + std::shared_ptr network_thread_; + FDBNetworkOption opt_; + + // Global state, only one instance of Network is allowed + static std::atomic working; +}; + +class Database { +public: + Database(std::shared_ptr net, std::string cluster_file, FDBDatabaseOption opt) + : network_(std::move(net)), cluster_file_path_(std::move(cluster_file)), opt_(opt) {} + + /** + * + * @return 0 for success otherwise false + */ + int init(); + + ~Database() { + if (db_ != nullptr) fdb_database_destroy(db_); + } + + FDBDatabase* db() { return db_; }; + + std::shared_ptr create_txn(FDBTransactionOption opt); + +private: + std::shared_ptr network_; + std::string cluster_file_path_; + FDBDatabase* db_ = nullptr; + FDBDatabaseOption opt_; +}; + +class RangeGetIterator : public cloud::RangeGetIterator { +public: + /** + * Iterator takes the ownership of input future + */ + RangeGetIterator(FDBFuture* fut, bool owns = true) + : fut_(fut), owns_fut_(owns), kvs_(nullptr), kvs_size_(-1), more_(false), idx_(-1) {} + + RangeGetIterator(RangeGetIterator&& o) { + if (fut_ && owns_fut_) fdb_future_destroy(fut_); + fut_ = o.fut_; + owns_fut_ = o.owns_fut_; + kvs_ = o.kvs_; + kvs_size_ = o.kvs_size_; + more_ = o.more_; + idx_ = o.idx_; + + o.fut_ = nullptr; + o.kvs_ = nullptr; + o.idx_ = 0; + o.kvs_size_ = 0; + o.more_ = false; + } + + ~RangeGetIterator() override { + // Release all memory + if (fut_ && owns_fut_) fdb_future_destroy(fut_); + } + + TxnErrorCode init(); + + std::pair next() override { + if (idx_ < 0 || idx_ >= kvs_size_) return {}; + const auto& kv = kvs_[idx_++]; + return {{(char*)kv.key, (size_t)kv.key_length}, {(char*)kv.value, (size_t)kv.value_length}}; + } + + void seek(size_t pos) override { idx_ = pos; } + + bool has_next() override { return (idx_ < kvs_size_); } + + /** + * Check if there are more KVs to be get from the range, caller usually wants + * to issue a nother `get` with the last key of this iteration. + */ + bool more() override { return more_; } + + int size() override { return kvs_size_; } + + void reset() override { idx_ = 0; } + + std::string next_begin_key() override { + std::string k; + if (!more()) return k; + const auto& kv = kvs_[kvs_size_ - 1]; + k.reserve((size_t)kv.key_length + 1); + k.append((char*)kv.key, (size_t)kv.key_length); + k.push_back('\x00'); + return k; + } + + RangeGetIterator(const RangeGetIterator&) = delete; + RangeGetIterator& operator=(const RangeGetIterator&) = delete; + +private: + FDBFuture* fut_; + bool owns_fut_; + const FDBKeyValue* kvs_; + int kvs_size_; + fdb_bool_t more_; + int idx_; +}; + +class Transaction : public cloud::Transaction { +public: + friend class Database; + Transaction(std::shared_ptr db) : db_(std::move(db)) {} + + ~Transaction() override { + if (txn_) fdb_transaction_destroy(txn_); + } + + /** + * + * @return TxnErrorCode for success otherwise false + */ + TxnErrorCode init(); + + void put(std::string_view key, std::string_view val) override; + + using cloud::Transaction::get; + /** + * @param snapshot if true, `key` will not be included in txn conflict detection this time + * @return TXN_OK for success get a key, TXN_KEY_NOT_FOUND for key not found, otherwise for error + */ + TxnErrorCode get(std::string_view key, std::string* val, bool snapshot = false) override; + /** + * Closed-open range + * @param snapshot if true, key range will not be included in txn conflict detection this time + * @param limit if non-zero, indicates the maximum number of key-value pairs to return + * @return TXN_OK for success, otherwise for error + */ + TxnErrorCode get(std::string_view begin, std::string_view end, + std::unique_ptr* iter, bool snapshot = false, + int limit = 10000) override; + + /** + * Put a key-value pair in which key will in the form of + * `key_prefix + versiontimestamp`. `versiontimestamp` is autogenerated by the + * system and it's 10-byte long and encoded in big-endian + * + * @param key_prefix prefix for key convertion, can be zero-length + * @param val value + */ + void atomic_set_ver_key(std::string_view key_prefix, std::string_view val) override; + + /** + * Put a key-value pair in which key will in the form of + * `value + versiontimestamp`. `versiontimestamp` is autogenerated by the + * system and it's 10-byte long and encoded in big-endian + * + * @param key prefix for key convertion, can be zero-length + * @param val value + */ + void atomic_set_ver_value(std::string_view key, std::string_view val) override; + + /** + * Adds a value to database + * @param to_add positive for addition, negative for substraction + */ + void atomic_add(std::string_view key, int64_t to_add) override; + // TODO: min max or and xor cmp_and_clear set_ver_value + + void remove(std::string_view key) override; + + /** + * Remove a closed-open range + */ + void remove(std::string_view begin, std::string_view end) override; + + /** + * + *@return TXN_OK for success, TXN_CONFLICT for conflict, otherwise for error + */ + TxnErrorCode commit() override; + + TxnErrorCode get_read_version(int64_t* version) override; + TxnErrorCode get_committed_version(int64_t* version) override; + + TxnErrorCode abort() override; + + TxnErrorCode batch_get(std::vector>* res, + const std::vector& keys, + const BatchGetOptions& opts = BatchGetOptions()) override; + +private: + std::shared_ptr db_ {nullptr}; + bool commited_ = false; + bool aborted_ = false; + FDBTransaction* txn_ = nullptr; +}; + +} // namespace fdb +} // namespace doris::cloud diff --git a/cloud/src/meta-service/txn_kv_error.h b/cloud/src/meta-service/txn_kv_error.h new file mode 100644 index 00000000000000..c82de20701be9f --- /dev/null +++ b/cloud/src/meta-service/txn_kv_error.h @@ -0,0 +1,79 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include + +#include + +namespace doris::cloud { + +enum class [[nodiscard]] TxnErrorCode : int { + TXN_OK = 0, + TXN_KEY_NOT_FOUND = 1, + TXN_CONFLICT = -1, + TXN_TOO_OLD = -2, + TXN_MAYBE_COMMITTED = -3, + TXN_RETRYABLE_NOT_COMMITTED = -4, + TXN_TIMEOUT = -5, + TXN_INVALID_ARGUMENT = -6, + TXN_KEY_TOO_LARGE = -7, + TXN_VALUE_TOO_LARGE = -8, + TXN_BYTES_TOO_LARGE = -9, + // other unidentified errors. + TXN_UNIDENTIFIED_ERROR = -10, +}; + +inline const char* format_as(TxnErrorCode code) { + // clang-format off + switch (code) { + case TxnErrorCode::TXN_OK: return "Ok"; + case TxnErrorCode::TXN_KEY_NOT_FOUND: return "KeyNotFound"; + case TxnErrorCode::TXN_CONFLICT: return "Conflict"; + case TxnErrorCode::TXN_TOO_OLD: return "TxnTooOld"; + case TxnErrorCode::TXN_MAYBE_COMMITTED: return "MaybeCommitted"; + case TxnErrorCode::TXN_RETRYABLE_NOT_COMMITTED: return "RetryableNotCommitted"; + case TxnErrorCode::TXN_TIMEOUT: return "Timeout"; + case TxnErrorCode::TXN_INVALID_ARGUMENT: return "InvalidArgument"; + case TxnErrorCode::TXN_KEY_TOO_LARGE: return "Key length exceeds limit"; + case TxnErrorCode::TXN_VALUE_TOO_LARGE: return "Value length exceeds limit"; + case TxnErrorCode::TXN_BYTES_TOO_LARGE: return "Transaction exceeds byte limit"; + case TxnErrorCode::TXN_UNIDENTIFIED_ERROR: return "Unknown"; + } + return "NotImplemented"; + // clang-format on +} + +inline std::ostream& operator<<(std::ostream& out, TxnErrorCode code) { + out << format_as(code); + return out; +} + +} // namespace doris::cloud + +template <> +struct fmt::formatter { + constexpr auto parse(format_parse_context& ctx) -> format_parse_context::iterator { + return ctx.begin(); + } + + auto format(const doris::cloud::TxnErrorCode& code, format_context& ctx) const + -> format_context::iterator { + return fmt::format_to(ctx.out(), "{}", doris::cloud::format_as(code)); + } +}; diff --git a/cloud/src/rate-limiter/CMakeLists.txt b/cloud/src/rate-limiter/CMakeLists.txt new file mode 100644 index 00000000000000..631c90d656a14c --- /dev/null +++ b/cloud/src/rate-limiter/CMakeLists.txt @@ -0,0 +1,12 @@ + +# where to put generated libraries +set(LIBRARY_OUTPUT_PATH "${BUILD_DIR}/src/rate-limiter") + +# where to put generated binaries +set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/src/rate-limiter") + +set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -lfdb_c -L${THIRDPARTY_DIR}/lib") + +add_library(RateLimiter + rate_limiter.cpp +) diff --git a/cloud/src/rate-limiter/rate_limiter.cpp b/cloud/src/rate-limiter/rate_limiter.cpp new file mode 100644 index 00000000000000..8988ff0560b170 --- /dev/null +++ b/cloud/src/rate-limiter/rate_limiter.cpp @@ -0,0 +1,113 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "rate_limiter.h" + +#include + +#include +#include +#include + +#include "common/bvars.h" +#include "common/config.h" +#include "common/configbase.h" + +namespace doris::cloud { + +void RateLimiter::init(google::protobuf::Service* service) { + std::map rpc_name_to_max_qps_limit; + std::vector max_qps_limit_list; + butil::SplitString(config::specific_max_qps_limit, ';', &max_qps_limit_list); + for (const auto& v : max_qps_limit_list) { + auto p = v.find(':'); + if (p != std::string::npos && p != (v.size() - 1)) { + auto rpc_name = v.substr(0, p); + try { + int64_t max_qps_limit = std::stoll(v.substr(p + 1)); + if (max_qps_limit > 0) { + rpc_name_to_max_qps_limit[rpc_name] = max_qps_limit; + LOG(INFO) << "set rpc: " << rpc_name << " max_qps_limit: " << max_qps_limit; + } + } catch (...) { + LOG(WARNING) << "failed to set max_qps_limit to rpc: " << rpc_name + << " config: " << v; + } + } + } + auto method_size = service->GetDescriptor()->method_count(); + for (auto i = 0; i < method_size; ++i) { + std::string rpc_name = service->GetDescriptor()->method(i)->name(); + int64_t max_qps_limit = config::default_max_qps_limit; + + auto it = rpc_name_to_max_qps_limit.find(rpc_name); + if (it != rpc_name_to_max_qps_limit.end()) { + max_qps_limit = it->second; + } + limiters_[rpc_name] = std::make_shared(rpc_name, max_qps_limit); + } +} + +std::shared_ptr RateLimiter::get_rpc_rate_limiter(const std::string& rpc_name) { + // no need to be locked, because it is only modified during initialization + auto it = limiters_.find(rpc_name); + if (it == limiters_.end()) { + return nullptr; + } + return it->second; +} + +bool RpcRateLimiter::get_qps_token(const std::string& instance_id, + std::function& get_bvar_qps) { + if (!config::use_detailed_metrics || instance_id.empty()) { + return true; + } + std::shared_ptr qps_token = nullptr; + { + std::lock_guard l(mutex_); + auto it = qps_limiter_.find(instance_id); + + // new instance always can get token + if (it == qps_limiter_.end()) { + qps_token = std::make_shared(max_qps_limit_); + qps_limiter_[instance_id] = qps_token; + return true; + } + qps_token = it->second; + } + + return qps_token->get_token(get_bvar_qps); +} + +bool RpcRateLimiter::QpsToken::get_token(std::function& get_bvar_qps) { + using namespace std::chrono; + auto now = steady_clock::now(); + std::lock_guard l(mutex_); + // Todo: if current_qps_ > max_qps_limit_, always return false until the bvar qps is updated, + // maybe need to reduce the bvar's update interval. + ++access_count_; + auto duration_s = duration_cast(now - last_update_time_).count(); + if (duration_s > config::bvar_qps_update_second || + (duration_s != 0 && (access_count_ / duration_s > max_qps_limit_ / 2))) { + access_count_ = 0; + last_update_time_ = now; + current_qps_ = get_bvar_qps(); + } + return current_qps_ < max_qps_limit_; +} + +} // namespace doris::cloud \ No newline at end of file diff --git a/cloud/src/rate-limiter/rate_limiter.h b/cloud/src/rate-limiter/rate_limiter.h new file mode 100644 index 00000000000000..df441656aa45f3 --- /dev/null +++ b/cloud/src/rate-limiter/rate_limiter.h @@ -0,0 +1,86 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include + +#include +#include +#include +#include + +#include "common/config.h" + +namespace doris::cloud { + +class RpcRateLimiter; + +class RateLimiter { +public: + RateLimiter() = default; + ~RateLimiter() = default; + void init(google::protobuf::Service* service); + std::shared_ptr get_rpc_rate_limiter(const std::string& rpc_name); + +private: + // rpc_name -> RpcRateLimiter + std::unordered_map> limiters_; +}; + +class RpcRateLimiter { +public: + RpcRateLimiter(const std::string rpc_name, const int64_t max_qps_limit) + : rpc_name_(rpc_name), max_qps_limit_(max_qps_limit) {} + + ~RpcRateLimiter() = default; + + /** + * @brief Get the qps token by instance_id + * + * @param instance_id + * @param get_bvar_qps a function that cat get the qps + */ + bool get_qps_token(const std::string& instance_id, std::function& get_bvar_qps); + + // Todo: Recycle outdated instance_id + +private: + class QpsToken { + public: + QpsToken(const int64_t max_qps_limit) : max_qps_limit_(max_qps_limit) {} + + bool get_token(std::function& get_bvar_qps); + + private: + bthread::Mutex mutex_; + std::chrono::steady_clock::time_point last_update_time_; + int64_t access_count_ {0}; + int64_t current_qps_ {0}; + int64_t max_qps_limit_; + }; + +private: + bthread::Mutex mutex_; + // instance_id -> QpsToken + std::unordered_map> qps_limiter_; + std::string rpc_name_; + int64_t max_qps_limit_; +}; + +} // namespace doris::cloud \ No newline at end of file diff --git a/cloud/src/recycler/CMakeLists.txt b/cloud/src/recycler/CMakeLists.txt new file mode 100644 index 00000000000000..e1568d2fb19344 --- /dev/null +++ b/cloud/src/recycler/CMakeLists.txt @@ -0,0 +1,23 @@ + +# where to put generated libraries +set(LIBRARY_OUTPUT_PATH "${BUILD_DIR}/src/recycler") + +# where to put generated binaries +set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/src/recycler") + +set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -lfdb_c -L${THIRDPARTY_DIR}/lib") + +set (SRC_LIST + checker.cpp + recycler.cpp + recycler_service.cpp + s3_accessor.cpp + util.cpp + white_black_list.cpp +) + +if(BUILD_CHECK_META STREQUAL "ON") + set (SRC_LIST ${SRC_LIST} meta_checker.cpp) +endif () + +add_library(Recycler ${SRC_LIST}) diff --git a/cloud/src/recycler/checker.cpp b/cloud/src/recycler/checker.cpp new file mode 100644 index 00000000000000..3965b660c1300f --- /dev/null +++ b/cloud/src/recycler/checker.cpp @@ -0,0 +1,626 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "recycler/checker.h" + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include "common/bvars.h" +#include "common/config.h" +#include "common/encryption_util.h" +#include "common/sync_point.h" +#include "common/util.h" +#include "meta-service/keys.h" +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" +#include "recycler/s3_accessor.h" +#ifdef UNIT_TEST +#include "../test/mock_accessor.h" +#endif +#include "recycler/util.h" + +namespace doris::cloud { +namespace config { +extern int32_t brpc_listen_port; +extern int32_t scan_instances_interval_seconds; +extern int32_t recycle_job_lease_expired_ms; +extern int32_t recycle_concurrency; +extern std::vector recycle_whitelist; +extern std::vector recycle_blacklist; +extern bool enable_inverted_check; +} // namespace config + +Checker::Checker(std::shared_ptr txn_kv) : txn_kv_(std::move(txn_kv)) { + ip_port_ = std::string(butil::my_ip_cstr()) + ":" + std::to_string(config::brpc_listen_port); +} + +Checker::~Checker() { + if (!stopped()) { + stop(); + } +} + +int Checker::start() { + DCHECK(txn_kv_); + instance_filter_.reset(config::recycle_whitelist, config::recycle_blacklist); + + // launch instance scanner + auto scanner_func = [this]() { + while (!stopped()) { + std::vector instances; + get_all_instances(txn_kv_.get(), instances); + LOG(INFO) << "Checker get instances: " << [&instances] { + std::stringstream ss; + for (auto& i : instances) ss << ' ' << i.instance_id(); + return ss.str(); + }(); + if (!instances.empty()) { + // enqueue instances + std::lock_guard lock(mtx_); + for (auto& instance : instances) { + if (instance_filter_.filter_out(instance.instance_id())) continue; + if (instance.status() == InstanceInfoPB::DELETED) continue; + using namespace std::chrono; + auto enqueue_time_s = + duration_cast(system_clock::now().time_since_epoch()).count(); + auto [_, success] = + pending_instance_map_.insert({instance.instance_id(), enqueue_time_s}); + // skip instance already in pending queue + if (success) { + pending_instance_queue_.push_back(std::move(instance)); + } + } + pending_instance_cond_.notify_all(); + } + { + std::unique_lock lock(mtx_); + notifier_.wait_for(lock, + std::chrono::seconds(config::scan_instances_interval_seconds), + [&]() { return stopped(); }); + } + } + }; + workers_.emplace_back(scanner_func); + // Launch lease thread + workers_.emplace_back([this] { lease_check_jobs(); }); + // Launch inspect thread + workers_.emplace_back([this] { inspect_instance_check_interval(); }); + + // launch check workers + auto checker_func = [this]() { + while (!stopped()) { + // fetch instance to check + InstanceInfoPB instance; + long enqueue_time_s = 0; + { + std::unique_lock lock(mtx_); + pending_instance_cond_.wait( + lock, [&]() { return !pending_instance_queue_.empty() || stopped(); }); + if (stopped()) { + return; + } + instance = std::move(pending_instance_queue_.front()); + pending_instance_queue_.pop_front(); + enqueue_time_s = pending_instance_map_[instance.instance_id()]; + pending_instance_map_.erase(instance.instance_id()); + } + const auto& instance_id = instance.instance_id(); + { + std::lock_guard lock(mtx_); + // skip instance in recycling + if (working_instance_map_.count(instance_id)) continue; + } + auto checker = std::make_shared(txn_kv_, instance.instance_id()); + if (checker->init(instance) != 0) { + LOG(WARNING) << "failed to init instance checker, instance_id=" + << instance.instance_id(); + continue; + } + std::string check_job_key; + job_check_key({instance.instance_id()}, &check_job_key); + int ret = prepare_instance_recycle_job(txn_kv_.get(), check_job_key, + instance.instance_id(), ip_port_, + config::check_object_interval_seconds * 1000); + if (ret != 0) { // Prepare failed + continue; + } else { + std::lock_guard lock(mtx_); + working_instance_map_.emplace(instance_id, checker); + } + if (stopped()) return; + using namespace std::chrono; + auto ctime_ms = + duration_cast(system_clock::now().time_since_epoch()).count(); + g_bvar_checker_enqueue_cost_s.put(instance_id, ctime_ms / 1000 - enqueue_time_s); + ret = checker->do_check(); + if (config::enable_inverted_check) { + if (checker->do_inverted_check() != 0) ret = -1; + } + if (ret == -1) return; + // If instance checker has been aborted, don't finish this job + if (!checker->stopped()) { + finish_instance_recycle_job(txn_kv_.get(), check_job_key, instance.instance_id(), + ip_port_, ret == 0, ctime_ms); + } + { + std::lock_guard lock(mtx_); + working_instance_map_.erase(instance.instance_id()); + } + } + }; + int num_threads = config::recycle_concurrency; // FIXME: use a new config entry? + for (int i = 0; i < num_threads; ++i) { + workers_.emplace_back(checker_func); + } + return 0; +} + +void Checker::stop() { + stopped_ = true; + notifier_.notify_all(); + pending_instance_cond_.notify_all(); + { + std::lock_guard lock(mtx_); + for (auto& [_, checker] : working_instance_map_) { + checker->stop(); + } + } + for (auto& w : workers_) { + if (w.joinable()) w.join(); + } +} + +void Checker::lease_check_jobs() { + while (!stopped()) { + std::vector instances; + instances.reserve(working_instance_map_.size()); + { + std::lock_guard lock(mtx_); + for (auto& [id, _] : working_instance_map_) { + instances.push_back(id); + } + } + for (auto& i : instances) { + std::string check_job_key; + job_check_key({i}, &check_job_key); + int ret = lease_instance_recycle_job(txn_kv_.get(), check_job_key, i, ip_port_); + if (ret == 1) { + std::lock_guard lock(mtx_); + if (auto it = working_instance_map_.find(i); it != working_instance_map_.end()) { + it->second->stop(); + } + } + } + { + std::unique_lock lock(mtx_); + notifier_.wait_for(lock, + std::chrono::milliseconds(config::recycle_job_lease_expired_ms / 3), + [&]() { return stopped(); }); + } + } +} + +#define LOG_CHECK_INTERVAL_ALARM LOG(WARNING) << "Err for check interval: " +void Checker::do_inspect(const InstanceInfoPB& instance) { + std::string check_job_key = job_check_key({instance.instance_id()}); + std::unique_ptr txn; + std::string val; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG_CHECK_INTERVAL_ALARM << "failed to create txn"; + return; + } + err = txn->get(check_job_key, &val); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + LOG_CHECK_INTERVAL_ALARM << "failed to get kv, err=" << err + << " key=" << hex(check_job_key); + return; + } + auto checker = InstanceChecker(txn_kv_, instance.instance_id()); + if (checker.init(instance) != 0) { + LOG_CHECK_INTERVAL_ALARM << "failed to init instance checker, instance_id=" + << instance.instance_id(); + return; + } + int64_t bucket_lifecycle_days = 0; + if (checker.get_bucket_lifecycle(&bucket_lifecycle_days) != 0) { + LOG_CHECK_INTERVAL_ALARM << "failed to get bucket lifecycle, instance_id=" + << instance.instance_id(); + return; + } + DCHECK(bucket_lifecycle_days > 0); + int64_t last_ctime_ms = -1; + auto job_status = JobRecyclePB::IDLE; + auto has_last_ctime = [&]() { + JobRecyclePB job_info; + if (!job_info.ParseFromString(val)) { + LOG_CHECK_INTERVAL_ALARM << "failed to parse JobRecyclePB, key=" << hex(check_job_key); + } + DCHECK(job_info.instance_id() == instance.instance_id()); + if (!job_info.has_last_ctime_ms()) return false; + last_ctime_ms = job_info.last_ctime_ms(); + job_status = job_info.status(); + g_bvar_checker_last_success_time_ms.put(instance.instance_id(), + job_info.last_success_time_ms()); + return true; + }; + using namespace std::chrono; + auto now = duration_cast(system_clock::now().time_since_epoch()).count(); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND || !has_last_ctime()) { + // Use instance's ctime for instances that do not have job's last ctime + last_ctime_ms = instance.ctime(); + } + DCHECK(now - last_ctime_ms >= 0); + int64_t expiration_ms = + bucket_lifecycle_days > config::reserved_buffer_days + ? (bucket_lifecycle_days - config::reserved_buffer_days) * 86400000 + : bucket_lifecycle_days * 86400000; + TEST_SYNC_POINT_CALLBACK("Checker:do_inspect", &last_ctime_ms); + if (now - last_ctime_ms >= expiration_ms) { + TEST_SYNC_POINT("Checker.do_inspect1"); + LOG_CHECK_INTERVAL_ALARM << "check risks, instance_id: " << instance.instance_id() + << " last_ctime_ms: " << last_ctime_ms + << " job_status: " << job_status + << " bucket_lifecycle_days: " << bucket_lifecycle_days + << " reserved_buffer_days: " << config::reserved_buffer_days + << " expiration_ms: " << expiration_ms; + } +} +#undef LOG_CHECK_INTERVAL_ALARM +void Checker::inspect_instance_check_interval() { + while (!stopped()) { + LOG(INFO) << "start to inspect instance check interval"; + std::vector instances; + get_all_instances(txn_kv_.get(), instances); + for (const auto& instance : instances) { + if (instance_filter_.filter_out(instance.instance_id())) continue; + if (stopped()) return; + if (instance.status() == InstanceInfoPB::DELETED) continue; + do_inspect(instance); + } + { + std::unique_lock lock(mtx_); + notifier_.wait_for(lock, std::chrono::seconds(config::scan_instances_interval_seconds), + [&]() { return stopped(); }); + } + } +} + +// return 0 for success get a key, 1 for key not found, negative for error +int key_exist(TxnKv* txn_kv, std::string_view key) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to init txn, err=" << err; + return -1; + } + std::string val; + switch (txn->get(key, &val)) { + case TxnErrorCode::TXN_OK: + return 0; + case TxnErrorCode::TXN_KEY_NOT_FOUND: + return 1; + default: + return -1; + } +} + +InstanceChecker::InstanceChecker(std::shared_ptr txn_kv, const std::string& instance_id) + : txn_kv_(std::move(txn_kv)), instance_id_(instance_id) {} + +int InstanceChecker::init(const InstanceInfoPB& instance) { + for (const auto& obj_info : instance.obj_info()) { + S3Conf s3_conf; + s3_conf.ak = obj_info.ak(); + s3_conf.sk = obj_info.sk(); + if (obj_info.has_encryption_info()) { + AkSkPair plain_ak_sk_pair; + int ret = decrypt_ak_sk_helper(obj_info.ak(), obj_info.sk(), obj_info.encryption_info(), + &plain_ak_sk_pair); + if (ret != 0) { + LOG(WARNING) << "fail to decrypt ak sk. instance_id: " << instance_id_ + << " obj_info: " << proto_to_json(obj_info); + } else { + s3_conf.ak = std::move(plain_ak_sk_pair.first); + s3_conf.sk = std::move(plain_ak_sk_pair.second); + } + } + s3_conf.endpoint = obj_info.endpoint(); + s3_conf.region = obj_info.region(); + s3_conf.bucket = obj_info.bucket(); + s3_conf.prefix = obj_info.prefix(); +#ifdef UNIT_TEST + auto accessor = std::make_shared(s3_conf); +#else + auto accessor = std::make_shared(std::move(s3_conf)); +#endif + if (accessor->init() != 0) [[unlikely]] { + LOG(WARNING) << "failed to init s3 accessor, instance_id=" << instance.instance_id(); + return -1; + } + accessor_map_.emplace(obj_info.id(), std::move(accessor)); + } + return 0; +} + +int InstanceChecker::do_check() { + TEST_SYNC_POINT("InstanceChecker.do_check"); + LOG(INFO) << "begin to check instance objects instance_id=" << instance_id_; + long num_scanned = 0; + long num_scanned_with_segment = 0; + long num_check_failed = 0; + long instance_volume = 0; + using namespace std::chrono; + auto start_time = steady_clock::now(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + auto cost = duration(steady_clock::now() - start_time).count(); + LOG(INFO) << "check instance objects finished, cost=" << cost + << "s. instance_id=" << instance_id_ << " num_scanned=" << num_scanned + << " num_scanned_with_segment=" << num_scanned_with_segment + << " num_check_failed=" << num_check_failed + << " instance_volume=" << instance_volume; + g_bvar_checker_num_scanned.put(instance_id_, num_scanned); + g_bvar_checker_num_scanned_with_segment.put(instance_id_, num_scanned_with_segment); + g_bvar_checker_num_check_failed.put(instance_id_, num_check_failed); + g_bvar_checker_check_cost_s.put(instance_id_, static_cast(cost)); + // FIXME(plat1ko): What if some list operation failed? + g_bvar_checker_instance_volume.put(instance_id_, instance_volume); + }); + + struct TabletFiles { + int64_t tablet_id {0}; + std::unordered_set files; + }; + TabletFiles tablet_files_cache; + + auto check_rowset_objects = [&, this](const doris::RowsetMetaCloudPB& rs_meta, + std::string_view key) { + if (rs_meta.num_segments() == 0) return; + ++num_scanned_with_segment; + if (tablet_files_cache.tablet_id != rs_meta.tablet_id()) { + long tablet_volume = 0; + // Clear cache + tablet_files_cache.tablet_id = 0; + tablet_files_cache.files.clear(); + // Get all file paths under this tablet directory + for (auto& [_, accessor] : accessor_map_) { + std::vector files; + int ret = accessor->list(tablet_path_prefix(rs_meta.tablet_id()), &files); + if (ret != 0) { // No need to log, because S3Accessor has logged this error + ++num_check_failed; + return; + } + for (auto& file : files) { + tablet_files_cache.files.insert(std::move(file.path)); + tablet_volume += file.size; + } + } + tablet_files_cache.tablet_id = rs_meta.tablet_id(); + instance_volume += tablet_volume; + } + + for (int i = 0; i < rs_meta.num_segments(); ++i) { + auto path = segment_path(rs_meta.tablet_id(), rs_meta.rowset_id_v2(), i); + if (tablet_files_cache.files.count(path)) continue; + if (1 == key_exist(txn_kv_.get(), key)) { + // Rowset has been deleted instead of data loss + continue; + } + ++num_check_failed; + TEST_SYNC_POINT_CALLBACK("InstanceChecker.do_check1", &path); + LOG(WARNING) << "object not exist, path=" << path << " key=" << hex(key); + } + }; + + // scan visible rowsets + auto start_key = meta_rowset_key({instance_id_, 0, 0}); + auto end_key = meta_rowset_key({instance_id_, INT64_MAX, 0}); + + std::unique_ptr it; + do { + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to init txn, err=" << err; + return -1; + } + + err = txn->get(start_key, end_key, &it); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "internal error, failed to get rowset meta, err=" << err; + return -1; + } + num_scanned += it->size(); + + while (it->has_next() && !stopped()) { + auto [k, v] = it->next(); + if (!it->has_next()) start_key = k; + + doris::RowsetMetaCloudPB rs_meta; + if (!rs_meta.ParseFromArray(v.data(), v.size())) { + ++num_check_failed; + LOG(WARNING) << "malformed rowset meta. key=" << hex(k) << " val=" << hex(v); + continue; + } + check_rowset_objects(rs_meta, k); + } + start_key.push_back('\x00'); // Update to next smallest key for iteration + } while (it->more() && !stopped()); + return num_check_failed == 0 ? 0 : -2; +} + +int InstanceChecker::get_bucket_lifecycle(int64_t* lifecycle_days) { + // If there are multiple buckets, return the minimum lifecycle. + int64_t min_lifecycle_days = std::numeric_limits::max(); + int64_t tmp_liefcycle_days = 0; + for (const auto& [obj_info, accessor] : accessor_map_) { + if (accessor->check_bucket_versioning() != 0) return -1; + if (accessor->get_bucket_lifecycle(&tmp_liefcycle_days) != 0) return -1; + if (tmp_liefcycle_days < min_lifecycle_days) min_lifecycle_days = tmp_liefcycle_days; + } + *lifecycle_days = min_lifecycle_days; + return 0; +} + +int InstanceChecker::do_inverted_check() { + LOG(INFO) << "begin to inverted check objects instance_id=" << instance_id_; + long num_scanned = 0; + long num_check_failed = 0; + using namespace std::chrono; + auto start_time = steady_clock::now(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + auto cost = duration(steady_clock::now() - start_time).count(); + LOG(INFO) << "inverted check instance objects finished, cost=" << cost + << "s. instance_id=" << instance_id_ << " num_scanned=" << num_scanned + << " num_check_failed=" << num_check_failed; + }); + + struct TabletRowsets { + int64_t tablet_id {0}; + std::unordered_set rowset_ids; + }; + TabletRowsets tablet_rowsets_cache; + + auto check_object_key = [&](const std::string& obj_key) { + std::vector str; + butil::SplitString(obj_key, '/', &str); + // {prefix}/data/{tablet_id}/{rowset_id}_{seg_num}.dat + if (str.size() < 4) { + return -1; + } + int64_t tablet_id = atol((str.end() - 2)->c_str()); + if (tablet_id <= 0) { + LOG(WARNING) << "failed to parse tablet_id, key=" << obj_key; + return -1; + } + std::string rowset_id; + if (auto pos = str.back().find('_'); pos != std::string::npos) { + rowset_id = str.back().substr(0, pos); + } else { + LOG(WARNING) << "failed to parse rowset_id, key=" << obj_key; + return -1; + } + if (tablet_rowsets_cache.tablet_id == tablet_id) { + if (tablet_rowsets_cache.rowset_ids.count(rowset_id) > 0) { + return 0; + } else { + LOG(WARNING) << "rowset not exists, key=" << obj_key; + return -1; + } + } + // Get all rowset id of this tablet + tablet_rowsets_cache.tablet_id = tablet_id; + tablet_rowsets_cache.rowset_ids.clear(); + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create txn"; + return -1; + } + std::unique_ptr it; + auto begin = meta_rowset_key({instance_id_, tablet_id, 0}); + auto end = meta_rowset_key({instance_id_, tablet_id, INT64_MAX}); + do { + TxnErrorCode err = txn->get(begin, end, &it); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to get rowset kv, err=" << err; + return -1; + } + if (!it->has_next()) { + break; + } + while (it->has_next()) { + // recycle corresponding resources + auto [k, v] = it->next(); + doris::RowsetMetaCloudPB rowset; + if (!rowset.ParseFromArray(v.data(), v.size())) { + LOG(WARNING) << "malformed rowset meta value, key=" << hex(k); + return -1; + } + tablet_rowsets_cache.rowset_ids.insert(rowset.rowset_id_v2()); + if (!it->has_next()) { + begin = k; + begin.push_back('\x00'); // Update to next smallest key for iteration + break; + } + } + } while (it->more() && !stopped()); + if (tablet_rowsets_cache.rowset_ids.count(rowset_id) > 0) { + return 0; + } else { + LOG(WARNING) << "rowset not exists, key=" << obj_key; + return -1; + } + return 0; + }; + + // TODO(Xiaocc): Currently we haven't implemented one generator-like s3 accessor list function + // so we choose to skip here. + { + [[maybe_unused]] int tmp_ret = 0; + TEST_SYNC_POINT_RETURN_WITH_VALUE("InstanceChecker::do_inverted_check", &tmp_ret); + } + for (auto& [_, accessor] : accessor_map_) { + auto* s3_accessor = static_cast(accessor.get()); + auto client = s3_accessor->s3_client(); + const auto& conf = s3_accessor->conf(); + Aws::S3::Model::ListObjectsV2Request request; + request.WithBucket(conf.bucket).WithPrefix(conf.prefix + "/data/"); + bool is_truncated = false; + do { + auto outcome = client->ListObjectsV2(request); + if (!outcome.IsSuccess()) { + LOG(WARNING) << "failed to list objects, endpoint=" << conf.endpoint + << " bucket=" << conf.bucket << " prefix=" << request.GetPrefix(); + return -1; + } + LOG(INFO) << "get " << outcome.GetResult().GetContents().size() + << " objects, endpoint=" << conf.endpoint << " bucket=" << conf.bucket + << " prefix=" << request.GetPrefix(); + const auto& result = outcome.GetResult(); + num_scanned += result.GetContents().size(); + for (const auto& obj : result.GetContents()) { + if (check_object_key(obj.GetKey()) != 0) { + LOG(WARNING) << "failed to check object key, endpoint=" << conf.endpoint + << " bucket=" << conf.bucket << " key=" << obj.GetKey(); + ++num_check_failed; + } + } + is_truncated = result.GetIsTruncated(); + request.SetContinuationToken(result.GetNextContinuationToken()); + } while (is_truncated && !stopped()); + } + return num_check_failed == 0 ? 0 : -1; +} + +} // namespace doris::cloud diff --git a/cloud/src/recycler/checker.h b/cloud/src/recycler/checker.h new file mode 100644 index 00000000000000..5a9bbcc988ef6e --- /dev/null +++ b/cloud/src/recycler/checker.h @@ -0,0 +1,97 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include +#include +#include +#include +#include + +#include "recycler/white_black_list.h" + +namespace doris::cloud { +class ObjStoreAccessor; +class InstanceChecker; +class TxnKv; +class InstanceInfoPB; + +class Checker { +public: + explicit Checker(std::shared_ptr txn_kv); + ~Checker(); + + int start(); + + void stop(); + bool stopped() const { return stopped_.load(std::memory_order_acquire); } + +private: + void lease_check_jobs(); + void inspect_instance_check_interval(); + void do_inspect(const InstanceInfoPB& instance); + +private: + friend class RecyclerServiceImpl; + + std::shared_ptr txn_kv_; + std::atomic_bool stopped_{false}; + std::string ip_port_; + std::vector workers_; + + std::mutex mtx_; + // notify check workers + std::condition_variable pending_instance_cond_; + std::deque pending_instance_queue_; + // instance_id -> enqueue_timestamp + std::unordered_map pending_instance_map_; + std::unordered_map> working_instance_map_; + // notify instance scanner and lease thread + std::condition_variable notifier_; + + WhiteBlackList instance_filter_; + +}; + +class InstanceChecker { +public: + explicit InstanceChecker(std::shared_ptr txn_kv, const std::string& instance_id); + // Return 0 if success, otherwise error + int init(const InstanceInfoPB& instance); + // Check whether the objects in the object store of the instance belong to the visible rowsets. + // This function is used to verify that there is no garbage data leakage, should only be called in recycler test. + // Return 0 if success, otherwise failed + int do_inverted_check(); + // Return 0 if success, the definition of success is the absence of S3 access errors and data loss + // Return -1 if encountering the situation that need to abort checker. + // Return -2 if having S3 access errors or data loss + int do_check(); + // Return 0 if success, otherwise error + int get_bucket_lifecycle(int64_t* lifecycle); + void stop() { stopped_.store(true, std::memory_order_release); } + bool stopped() const { return stopped_.load(std::memory_order_acquire); } + +private: + std::atomic_bool stopped_ {false}; + std::shared_ptr txn_kv_; + std::string instance_id_; + std::unordered_map> accessor_map_; +}; + +} // namespace doris::cloud diff --git a/cloud/src/recycler/meta_checker.cpp b/cloud/src/recycler/meta_checker.cpp new file mode 100644 index 00000000000000..86a17019399053 --- /dev/null +++ b/cloud/src/recycler/meta_checker.cpp @@ -0,0 +1,445 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "recycler/meta_checker.h" + +#include +#include +#include +#include + +#include +#include + +#include "common/util.h" +#include "meta-service/keys.h" +#include "meta-service/txn_kv.h" + +namespace doris::cloud { + +MetaChecker::MetaChecker(std::shared_ptr txn_kv) : txn_kv_(std::move(txn_kv)) {} + +struct TabletInfo { + int64_t db_id; + int64_t table_id; + int64_t partition_id; + int64_t index_id; + int64_t tablet_id; + int64_t schema_version; + + std::string debug_string() const { + return "db id: " + std::to_string(db_id) + " table id: " + std::to_string(table_id) + + " partition id: " + std::to_string(partition_id) + + " index id: " + std::to_string(index_id) + + " tablet id: " + std::to_string(tablet_id) + + " schema version: " + std::to_string(schema_version); + } +}; + +struct PartitionInfo { + int64_t db_id; + int64_t table_id; + int64_t partition_id; + int64_t visible_version; +}; + +bool MetaChecker::check_fe_meta_by_fdb(MYSQL* conn) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to init txn"; + return false; + } + + std::string start_key; + std::string end_key; + meta_tablet_idx_key({instance_id_, 0}, &start_key); + meta_tablet_idx_key({instance_id_, std::numeric_limits::max()}, &end_key); + std::vector tablet_indexes; + + std::unique_ptr it; + do { + err = txn->get(start_key, end_key, &it); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to get tablet idx, ret=" << err; + return false; + } + + while (it->has_next()) { + auto [k, v] = it->next(); + TabletIndexPB tablet_idx; + if (!tablet_idx.ParseFromArray(v.data(), v.size())) [[unlikely]] { + LOG(WARNING) << "malformed tablet index value"; + return false; + } + + tablet_indexes.push_back(std::move(tablet_idx)); + if (!it->has_next()) start_key = k; + } + start_key.push_back('\x00'); + } while (it->more()); + + bool check_res = true; + for (const TabletIndexPB& tablet_idx : tablet_indexes) { + std::string sql_stmt = "show tablet " + std::to_string(tablet_idx.tablet_id()); + MYSQL_RES* result; + mysql_query(conn, sql_stmt.c_str()); + result = mysql_store_result(conn); + if (result) { + MYSQL_ROW row = mysql_fetch_row(result); + if (tablet_idx.table_id() != atoll(row[5])) { + LOG(WARNING) << "check failed, fdb meta: " << tablet_idx.ShortDebugString() + << " fe table_id: " << atoll(row[5]); + check_res = false; + } + if (tablet_idx.partition_id() != atoll(row[6])) { + LOG(WARNING) << "check failed, fdb meta: " << tablet_idx.ShortDebugString() + << " fe partition_id: " << atoll(row[6]); + check_res = false; + } + if (tablet_idx.index_id() != atoll(row[7])) { + LOG(WARNING) << "check failed, fdb meta: " << tablet_idx.ShortDebugString() + << " fe index_id: " << atoll(row[7]); + check_res = false; + } + } + mysql_free_result(result); + stat_info_.check_fe_tablet_num++; + } + LOG(INFO) << "check_fe_tablet_num: " << stat_info_.check_fe_tablet_num; + + return check_res; +} + +bool MetaChecker::check_fdb_by_fe_meta(MYSQL* conn) { + // get db info from FE + std::string sql_stmt = "show databases"; + MYSQL_RES* result; + mysql_query(conn, sql_stmt.c_str()); + result = mysql_store_result(conn); + std::map*> db_to_tables; + if (result) { + int num_row = mysql_num_rows(result); + for (int i = 0; i < num_row; ++i) { + MYSQL_ROW row = mysql_fetch_row(result); + if (strcmp(row[0], "__internal_schema") == 0 || + strcmp(row[0], "information_schema") == 0) { + continue; + } + db_to_tables.insert({row[0], new std::vector()}); + } + } + mysql_free_result(result); + + // get tables info from FE + for (const auto& elem : db_to_tables) { + std::string sql_stmt = "show tables from " + elem.first; + mysql_query(conn, sql_stmt.c_str()); + result = mysql_store_result(conn); + if (result) { + int num_row = mysql_num_rows(result); + for (int i = 0; i < num_row; ++i) { + MYSQL_ROW row = mysql_fetch_row(result); + elem.second->push_back(row[0]); + } + } + mysql_free_result(result); + } + + // get tablet info from FE + std::vector tablets; + for (const auto& elem : db_to_tables) { + for (const std::string& table : *elem.second) { + std::string sql_stmt = "show tablets from " + elem.first + "." + table; + mysql_query(conn, sql_stmt.c_str()); + result = mysql_store_result(conn); + if (result) { + int num_row = mysql_num_rows(result); + for (int i = 0; i < num_row; ++i) { + MYSQL_ROW row = mysql_fetch_row(result); + TabletInfo tablet_info; + tablet_info.tablet_id = atoll(row[0]); + tablet_info.schema_version = atoll(row[4]); + tablets.push_back(std::move(tablet_info)); + } + } + mysql_free_result(result); + } + } + + // get tablet info from FE + // get Partition info from FE + std::map partitions; + for (auto& tablet_info : tablets) { + std::string sql_stmt = "show tablet " + std::to_string(tablet_info.tablet_id); + mysql_query(conn, sql_stmt.c_str()); + result = mysql_store_result(conn); + if (result) { + int num_row = mysql_num_rows(result); + for (int i = 0; i < num_row; ++i) { + MYSQL_ROW row = mysql_fetch_row(result); + tablet_info.db_id = atoll(row[4]); + tablet_info.table_id = atoll(row[5]); + tablet_info.partition_id = atoll(row[6]); + tablet_info.index_id = atoll(row[7]); + + PartitionInfo partition_info; + partition_info.db_id = atoll(row[4]); + partition_info.table_id = atoll(row[5]); + partition_info.partition_id = atoll(row[6]); + partitions.insert({partition_info.partition_id, std::move(partition_info)}); + } + } + mysql_free_result(result); + } + + // get partition version from FE + for (const auto& elem : db_to_tables) { + for (const std::string& table : *elem.second) { + std::string sql_stmt = "show partitions from " + elem.first + "." + table; + mysql_query(conn, sql_stmt.c_str()); + result = mysql_store_result(conn); + if (result) { + int num_row = mysql_num_rows(result); + for (int i = 0; i < num_row; ++i) { + MYSQL_ROW row = mysql_fetch_row(result); + int partition_id = atoll(row[0]); + int visible_version = atoll(row[2]); + partitions[partition_id].visible_version = visible_version; + } + } + mysql_free_result(result); + } + } + + // check tablet idx + for (const auto& tablet_info : tablets) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to init txn"; + return false; + } + + std::string key, val; + meta_tablet_idx_key({instance_id_, tablet_info.tablet_id}, &key); + err = txn->get(key, &val); + if (err != TxnErrorCode::TXN_OK) { + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + LOG(WARNING) << "tablet not found, tablet id: " << tablet_info.tablet_id; + return false; + } else { + LOG(WARNING) << "failed to get tablet_idx, err: " << err + << " tablet id: " << tablet_info.tablet_id; + return false; + } + } + + TabletIndexPB tablet_idx; + if (!tablet_idx.ParseFromString(val)) [[unlikely]] { + LOG(WARNING) << "malformed tablet index value"; + return false; + } + + /* + if (tablet_info.db_id != tablet_idx.db_id()) [[unlikely]] { + LOG(WARNING) << "tablet idx check failed, fe db id: " << tablet_info.db_id + << " tablet idx db id: " << tablet_idx.db_id(); + return false; + } + */ + + if (tablet_info.table_id != tablet_idx.table_id()) [[unlikely]] { + LOG(WARNING) << "tablet idx check failed, fe table id: " << tablet_info.table_id + << " tablet idx table id: " << tablet_idx.table_id(); + return false; + } + + if (tablet_info.partition_id != tablet_idx.partition_id()) [[unlikely]] { + LOG(WARNING) << "tablet idx check failed, fe part id: " << tablet_info.partition_id + << " tablet idx part id: " << tablet_idx.partition_id(); + return false; + } + + if (tablet_info.index_id != tablet_idx.index_id()) [[unlikely]] { + LOG(WARNING) << "tablet idx check failed, fe index id: " << tablet_info.index_id + << " tablet idx index id: " << tablet_idx.index_id(); + return false; + } + + if (tablet_info.tablet_id != tablet_idx.tablet_id()) [[unlikely]] { + LOG(WARNING) << "tablet idx check failed, fe tablet id: " << tablet_info.tablet_id + << " tablet idx tablet id: " << tablet_idx.tablet_id(); + return false; + } + + stat_info_.check_fdb_tablet_idx_num++; + } + + // check tablet meta + for (const auto& tablet_info : tablets) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to init txn"; + return false; + } + + MetaTabletKeyInfo key_info1 {instance_id_, tablet_info.table_id, tablet_info.index_id, + tablet_info.partition_id, tablet_info.tablet_id}; + std::string key, val; + meta_tablet_key(key_info1, &key); + err = txn->get(key, &val); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + LOG(WARNING) << "tablet meta not found: " << tablet_info.tablet_id; + return false; + } else if (err != TxnErrorCode::TXN_OK) [[unlikely]] { + LOG(WARNING) << "failed to get tablet, err: " << err; + return false; + } + stat_info_.check_fdb_tablet_meta_num++; + } + + // check tablet schema + /* + for (const auto& tablet_info : tablets) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to init txn"; + return false; + } + + std::string schema_key, schema_val; + meta_schema_key({instance_id_, tablet_info.index_id, tablet_info.schema_version}, + &schema_key); + ValueBuf val_buf; + err = cloud::get(txn.get(), schema_key, &val_buf); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + LOG(WARNING) << "tablet schema not found: " << tablet_info.debug_string(); + return false; + } else if (err != TxnErrorCode::TXN_OK) [[unlikely]] { + LOG(WARNING) <<"failed to get tablet schema, err: " << err; + return false; + } + } + */ + + // check partition + for (const auto& elem : partitions) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to init txn"; + return false; + } + if (elem.second.visible_version == 0) { + continue; + } + + int64_t db_id = elem.second.db_id; + int64_t table_id = elem.second.table_id; + int64_t partition_id = elem.second.partition_id; + std::string ver_key = version_key({instance_id_, db_id, table_id, partition_id}); + std::string ver_val; + err = txn->get(ver_key, &ver_val); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + LOG(WARNING) << "version key not found, partition id: " << partition_id; + return false; + } else if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to get version: " << partition_id; + return false; + } + + VersionPB version_pb; + if (!version_pb.ParseFromString(ver_val)) { + LOG(WARNING) << "malformed version value"; + return false; + } + + if (version_pb.version() != elem.second.visible_version) { + LOG(WARNING) << "partition version check failed, FE partition version" + << elem.second.visible_version << " ms version: " << version_pb.version(); + return false; + } + stat_info_.check_fdb_partition_version_num++; + } + + LOG(INFO) << "check_fdb_table_idx_num: " << stat_info_.check_fdb_tablet_idx_num + << " check_fdb_table_meta_num: " << stat_info_.check_fdb_tablet_meta_num + << " check_fdb_partition_version_num: " << stat_info_.check_fdb_partition_version_num; + + return true; +} + +void MetaChecker::do_check(const std::string& host, const std::string& port, + const std::string& user, const std::string& password, + const std::string& instance_id, std::string& msg) { + LOG(INFO) << "meta check begin"; + instance_id_ = instance_id; + MYSQL* conn; + conn = mysql_init(nullptr); + if (!conn) { + msg = "mysql init failed"; + LOG(WARNING) << msg; + return; + } + conn = mysql_real_connect(conn, host.c_str(), user.c_str(), password.c_str(), "", stol(port), + nullptr, 0); + if (!conn) { + msg = "mysql init failed"; + LOG(WARNING) << msg; + return; + } + LOG(INFO) << "mysql conn succ "; + + using namespace std::chrono; + int64_t start = duration_cast(system_clock::now().time_since_epoch()).count(); + int64_t now; + + LOG(INFO) << "check_fe_meta_by_fdb begin"; + bool ret = false; + do { + ret = check_fe_meta_by_fdb(conn); + if (!ret) { + std::this_thread::sleep_for(seconds(10)); + } + now = duration_cast(system_clock::now().time_since_epoch()).count(); + } while (now - start <= 180 && !ret); + + if (!ret) { + LOG(WARNING) << "check_fe_meta_by_fdb failed, there may be data leak"; + msg = "meta leak err"; + } + now = duration_cast(system_clock::now().time_since_epoch()).count(); + LOG(INFO) << "check_fe_meta_by_fdb finish, cost(second): " << now - start; + + ret = check_fdb_by_fe_meta(conn); + if (!ret) { + LOG(WARNING) << "check_fdb_by_fe_meta failed, there may be data loss"; + msg = "meta loss err"; + return; + } + now = duration_cast(system_clock::now().time_since_epoch()).count(); + LOG(INFO) << "check_fdb_by_fe_meta finish, cost(second): " << now - start; + + mysql_close(conn); + + LOG(INFO) << "meta check finish"; +} + +} // namespace doris::cloud diff --git a/cloud/src/recycler/meta_checker.h b/cloud/src/recycler/meta_checker.h new file mode 100644 index 00000000000000..90479c71673bfd --- /dev/null +++ b/cloud/src/recycler/meta_checker.h @@ -0,0 +1,56 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include + +#include +#include +#include +#include +#include +#include + +#include "recycler/white_black_list.h" + +namespace doris::cloud { +class TxnKv; + +struct StatInfo { + int64_t check_fe_tablet_num = 0; + int64_t check_fdb_tablet_idx_num = 0; + int64_t check_fdb_tablet_meta_num = 0; + int64_t check_fdb_partition_version_num = 0; +}; + +class MetaChecker { +public: + explicit MetaChecker(std::shared_ptr txn_kv); + void do_check(const std::string& host, const std::string& port, const std::string& user, + const std::string& password, const std::string& instance_id, std::string& msg); + bool check_fe_meta_by_fdb(MYSQL* conn); + bool check_fdb_by_fe_meta(MYSQL* conn); + +private: + std::shared_ptr txn_kv_; + StatInfo stat_info_; + std::string instance_id_; +}; + +} // namespace doris::cloud diff --git a/cloud/src/recycler/recycler.cpp b/cloud/src/recycler/recycler.cpp new file mode 100644 index 00000000000000..288089334f6ab8 --- /dev/null +++ b/cloud/src/recycler/recycler.cpp @@ -0,0 +1,2363 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "recycler/recycler.h" + +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#include "common/stopwatch.h" +#include "meta-service/meta_service_schema.h" +#include "meta-service/txn_kv_error.h" +#include "recycler/checker.h" +#include "recycler/s3_accessor.h" +#ifdef UNIT_TEST +#include "../test/mock_accessor.h" +#endif +#include "common/config.h" +#include "common/encryption_util.h" +#include "common/logging.h" +#include "common/simple_thread_pool.h" +#include "common/sync_point.h" +#include "common/util.h" +#include "meta-service/keys.h" +#include "recycler/recycler_service.h" +#include "recycler/util.h" + +namespace doris::cloud { + +// return 0 for success get a key, 1 for key not found, negative for error +[[maybe_unused]] static int txn_get(TxnKv* txn_kv, std::string_view key, std::string& val) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + return -1; + } + switch (txn->get(key, &val, true)) { + case TxnErrorCode::TXN_OK: + return 0; + case TxnErrorCode::TXN_KEY_NOT_FOUND: + return 1; + default: + return -1; + }; +} + +// 0 for success, negative for error +static int txn_get(TxnKv* txn_kv, std::string_view begin, std::string_view end, + std::unique_ptr& it) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + return -1; + } + switch (txn->get(begin, end, &it, true)) { + case TxnErrorCode::TXN_OK: + return 0; + case TxnErrorCode::TXN_KEY_NOT_FOUND: + return 1; + default: + return -1; + }; +} + +// return 0 for success otherwise error +static int txn_remove(TxnKv* txn_kv, std::vector keys) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + return -1; + } + for (auto k : keys) { + txn->remove(k); + } + switch (txn->commit()) { + case TxnErrorCode::TXN_OK: + return 0; + case TxnErrorCode::TXN_CONFLICT: + return -1; + default: + return -1; + } +} + +// return 0 for success otherwise error +static int txn_remove(TxnKv* txn_kv, std::vector keys) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + return -1; + } + for (auto& k : keys) { + txn->remove(k); + } + switch (txn->commit()) { + case TxnErrorCode::TXN_OK: + return 0; + case TxnErrorCode::TXN_CONFLICT: + return -1; + default: + return -1; + } +} + +// return 0 for success otherwise error +[[maybe_unused]] static int txn_remove(TxnKv* txn_kv, std::string_view begin, + std::string_view end) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + return -1; + } + txn->remove(begin, end); + switch (txn->commit()) { + case TxnErrorCode::TXN_OK: + return 0; + case TxnErrorCode::TXN_CONFLICT: + return -1; + default: + return -1; + } +} + +Recycler::Recycler(std::shared_ptr txn_kv) : txn_kv_(std::move(txn_kv)) { + ip_port_ = std::string(butil::my_ip_cstr()) + ":" + std::to_string(config::brpc_listen_port); +} + +Recycler::~Recycler() { + if (!stopped()) { + stop(); + } +} + +void Recycler::instance_scanner_callback() { + while (!stopped()) { + std::vector instances; + get_all_instances(txn_kv_.get(), instances); + // TODO(plat1ko): delete job recycle kv of non-existent instances + LOG(INFO) << "Recycler get instances: " << [&instances] { + std::stringstream ss; + for (auto& i : instances) ss << ' ' << i.instance_id(); + return ss.str(); + }(); + if (!instances.empty()) { + // enqueue instances + std::lock_guard lock(mtx_); + for (auto& instance : instances) { + if (instance_filter_.filter_out(instance.instance_id())) continue; + auto [_, success] = pending_instance_set_.insert(instance.instance_id()); + // skip instance already in pending queue + if (success) { + pending_instance_queue_.push_back(std::move(instance)); + } + } + pending_instance_cond_.notify_all(); + } + { + std::unique_lock lock(mtx_); + notifier_.wait_for(lock, std::chrono::seconds(config::recycle_interval_seconds), + [&]() { return stopped(); }); + } + } +} + +void Recycler::recycle_callback() { + while (!stopped()) { + InstanceInfoPB instance; + { + std::unique_lock lock(mtx_); + pending_instance_cond_.wait( + lock, [&]() { return !pending_instance_queue_.empty() || stopped(); }); + if (stopped()) { + return; + } + instance = std::move(pending_instance_queue_.front()); + pending_instance_queue_.pop_front(); + pending_instance_set_.erase(instance.instance_id()); + } + auto& instance_id = instance.instance_id(); + { + std::lock_guard lock(mtx_); + // skip instance in recycling + if (recycling_instance_map_.count(instance_id)) continue; + } + auto instance_recycler = std::make_shared(txn_kv_, instance); + if (instance_recycler->init() != 0) { + LOG(WARNING) << "failed to init instance recycler, instance_id=" << instance_id; + continue; + } + std::string recycle_job_key; + job_recycle_key({instance_id}, &recycle_job_key); + int ret = prepare_instance_recycle_job(txn_kv_.get(), recycle_job_key, instance_id, + ip_port_, config::recycle_interval_seconds * 1000); + if (ret != 0) { // Prepare failed + continue; + } else { + std::lock_guard lock(mtx_); + recycling_instance_map_.emplace(instance_id, instance_recycler); + } + if (stopped()) return; + LOG_INFO("begin to recycle instance").tag("instance_id", instance_id); + using namespace std::chrono; + auto ctime_ms = duration_cast(system_clock::now().time_since_epoch()).count(); + ret = instance_recycler->do_recycle(); + // If instance recycler has been aborted, don't finish this job + if (!instance_recycler->stopped()) { + finish_instance_recycle_job(txn_kv_.get(), recycle_job_key, instance_id, ip_port_, + ret == 0, ctime_ms); + } + { + std::lock_guard lock(mtx_); + recycling_instance_map_.erase(instance_id); + } + LOG_INFO("finish recycle instance").tag("instance_id", instance_id); + } +} + +void Recycler::lease_recycle_jobs() { + while (!stopped()) { + std::vector instances; + instances.reserve(recycling_instance_map_.size()); + { + std::lock_guard lock(mtx_); + for (auto& [id, _] : recycling_instance_map_) { + instances.push_back(id); + } + } + for (auto& i : instances) { + std::string recycle_job_key; + job_recycle_key({i}, &recycle_job_key); + int ret = lease_instance_recycle_job(txn_kv_.get(), recycle_job_key, i, ip_port_); + if (ret == 1) { + std::lock_guard lock(mtx_); + if (auto it = recycling_instance_map_.find(i); + it != recycling_instance_map_.end()) { + it->second->stop(); + } + } + } + { + std::unique_lock lock(mtx_); + notifier_.wait_for(lock, + std::chrono::milliseconds(config::recycle_job_lease_expired_ms / 3), + [&]() { return stopped(); }); + } + } +} + +int Recycler::start(brpc::Server* server) { + instance_filter_.reset(config::recycle_whitelist, config::recycle_blacklist); + + if (config::enable_checker) { + checker_ = std::make_unique(txn_kv_); + int ret = checker_->start(); + std::string msg; + if (ret != 0) { + msg = "failed to start checker"; + LOG(ERROR) << msg; + std::cerr << msg << std::endl; + return ret; + } + msg = "checker started"; + LOG(INFO) << msg; + std::cout << msg << std::endl; + } + + if (server) { + // Add service + auto recycler_service = new RecyclerServiceImpl(txn_kv_, this, checker_.get()); + server->AddService(recycler_service, brpc::SERVER_OWNS_SERVICE); + } + + workers_.push_back(std::thread(std::bind(&Recycler::instance_scanner_callback, this))); + for (int i = 0; i < config::recycle_concurrency; ++i) { + workers_.push_back(std::thread(std::bind(&Recycler::recycle_callback, this))); + } + + workers_.push_back(std::thread(std::mem_fn(&Recycler::lease_recycle_jobs), this)); + return 0; +} + +void Recycler::stop() { + stopped_ = true; + notifier_.notify_all(); + pending_instance_cond_.notify_all(); + { + std::lock_guard lock(mtx_); + for (auto& [_, recycler] : recycling_instance_map_) { + recycler->stop(); + } + } + for (auto& w : workers_) { + if (w.joinable()) w.join(); + } + if (checker_) { + checker_->stop(); + } +} + +class InstanceRecycler::InvertedIndexIdCache { +public: + InvertedIndexIdCache(std::string instance_id, std::shared_ptr txn_kv) + : instance_id_(std::move(instance_id)), txn_kv_(std::move(txn_kv)) {} + + // Return 0 if success, 1 if schema kv not found, negative for error + int get(int64_t index_id, int32_t schema_version, std::vector& res) { + { + std::lock_guard lock(mtx_); + if (schemas_without_inverted_index_.count({index_id, schema_version})) { + return 0; + } + if (auto it = inverted_index_id_map_.find({index_id, schema_version}); + it != inverted_index_id_map_.end()) { + res = it->second; + return 0; + } + } + // Get schema from kv + // TODO(plat1ko): Single flight + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create txn, err=" << err; + return -1; + } + auto schema_key = meta_schema_key({instance_id_, index_id, schema_version}); + ValueBuf val_buf; + err = cloud::get(txn.get(), schema_key, &val_buf); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to get schema, err=" << err; + return static_cast(err); + } + doris::TabletSchemaCloudPB schema; + if (!parse_schema_value(val_buf, &schema)) { + LOG(WARNING) << "malformed schema value, key=" << hex(schema_key); + return -1; + } + if (schema.index_size() > 0) { + res.reserve(schema.index_size()); + for (auto& i : schema.index()) { + res.push_back(i.index_id()); + } + } + insert(index_id, schema_version, res); + return 0; + } + + // Empty `ids` means this schema has no inverted index + void insert(int64_t index_id, int32_t schema_version, const std::vector& ids) { + if (ids.empty()) { + TEST_SYNC_POINT_CALLBACK("InvertedIndexIdCache::insert1", nullptr); + std::lock_guard lock(mtx_); + schemas_without_inverted_index_.emplace(index_id, schema_version); + } else { + TEST_SYNC_POINT_CALLBACK("InvertedIndexIdCache::insert2", nullptr); + std::lock_guard lock(mtx_); + inverted_index_id_map_.try_emplace({index_id, schema_version}, ids); + } + } + +private: + std::string instance_id_; + std::shared_ptr txn_kv_; + + std::mutex mtx_; + using Key = std::pair; // + struct HashOfKey { + size_t operator()(const Key& key) const { + size_t seed = 0; + seed = std::hash {}(key.first); + seed = std::hash {}(key.second); + return seed; + } + }; + // -> inverted_index_ids + std::unordered_map, HashOfKey> inverted_index_id_map_; + // Store of schema which doesn't have inverted index + std::unordered_set schemas_without_inverted_index_; +}; + +InstanceRecycler::InstanceRecycler(std::shared_ptr txn_kv, const InstanceInfoPB& instance) + : txn_kv_(std::move(txn_kv)), + instance_id_(instance.instance_id()), + instance_info_(instance), + inverted_index_id_cache_(std::make_unique(instance_id_, txn_kv_)) {} + +InstanceRecycler::~InstanceRecycler() = default; + +int InstanceRecycler::init() { + for (auto& obj_info : instance_info_.obj_info()) { + S3Conf s3_conf; + s3_conf.ak = obj_info.ak(); + s3_conf.sk = obj_info.sk(); + if (obj_info.has_encryption_info()) { + AkSkPair plain_ak_sk_pair; + int ret = decrypt_ak_sk_helper(obj_info.ak(), obj_info.sk(), obj_info.encryption_info(), + &plain_ak_sk_pair); + if (ret != 0) { + LOG(WARNING) << "fail to decrypt ak sk. instance_id: " << instance_id_ + << " obj_info: " << proto_to_json(obj_info); + } else { + s3_conf.ak = std::move(plain_ak_sk_pair.first); + s3_conf.sk = std::move(plain_ak_sk_pair.second); + } + } + s3_conf.endpoint = obj_info.endpoint(); + s3_conf.region = obj_info.region(); + s3_conf.bucket = obj_info.bucket(); + s3_conf.prefix = obj_info.prefix(); +#ifdef UNIT_TEST + auto accessor = std::make_shared(s3_conf); +#else + auto accessor = std::make_shared(std::move(s3_conf)); +#endif + if (accessor->init() != 0) [[unlikely]] { + LOG(WARNING) << "failed to init object accessor, instance_id=" << instance_id_; + return -1; + } + accessor_map_.emplace(obj_info.id(), std::move(accessor)); + } + return 0; +} + +int InstanceRecycler::do_recycle() { + TEST_SYNC_POINT("InstanceRecycler.do_recycle"); + if (instance_info_.status() == InstanceInfoPB::DELETED) { + return recycle_deleted_instance(); + } else if (instance_info_.status() == InstanceInfoPB::NORMAL) { + int ret = recycle_indexes(); + if (recycle_partitions() != 0) ret = -1; + if (recycle_tmp_rowsets() != 0) ret = -1; + if (recycle_rowsets() != 0) ret = -1; + if (abort_timeout_txn() != 0) ret = -1; + if (recycle_expired_txn_label() != 0) ret = -1; + if (recycle_copy_jobs() != 0) ret = -1; + if (recycle_stage() != 0) ret = -1; + if (recycle_expired_stage_objects() != 0) ret = -1; + if (recycle_versions() != 0) ret = -1; + return ret; + } else { + LOG(WARNING) << "invalid instance status: " << instance_info_.status() + << " instance_id=" << instance_id_; + return -1; + } +} + +int InstanceRecycler::recycle_deleted_instance() { + LOG_INFO("begin to recycle deleted instance").tag("instance_id", instance_id_); + + int ret = 0; + using namespace std::chrono; + auto start_time = steady_clock::now(); + + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + auto cost = duration(steady_clock::now() - start_time).count(); + LOG(INFO) << (ret == 0 ? "successfully" : "failed to") + << " recycle deleted instance, cost=" << cost + << "s, instance_id=" << instance_id_; + }); + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create txn"; + ret = -1; + return -1; + } + LOG(INFO) << "begin to delete all kv, instance_id=" << instance_id_; + // delete kv before deleting objects to prevent the checker from misjudging data loss + std::string start_txn_key = txn_key_prefix(instance_id_); + std::string end_txn_key = txn_key_prefix(instance_id_ + '\x00'); + txn->remove(start_txn_key, end_txn_key); + // 0:instance_id 1:db_id 2:tbl_id 3:partition_id + std::string start_version_key = version_key({instance_id_, 0, 0, 0}); + std::string end_version_key = version_key({instance_id_, INT64_MAX, 0, 0}); + txn->remove(start_version_key, end_version_key); + std::string start_meta_key = meta_key_prefix(instance_id_); + std::string end_meta_key = meta_key_prefix(instance_id_ + '\x00'); + txn->remove(start_meta_key, end_meta_key); + std::string start_recycle_key = recycle_key_prefix(instance_id_); + std::string end_recycle_key = recycle_key_prefix(instance_id_ + '\x00'); + txn->remove(start_recycle_key, end_recycle_key); + std::string start_stats_tablet_key = stats_tablet_key({instance_id_, 0, 0, 0, 0}); + std::string end_stats_tablet_key = stats_tablet_key({instance_id_, INT64_MAX, 0, 0, 0}); + txn->remove(start_stats_tablet_key, end_stats_tablet_key); + std::string start_copy_key = copy_key_prefix(instance_id_); + std::string end_copy_key = copy_key_prefix(instance_id_ + '\x00'); + txn->remove(start_copy_key, end_copy_key); + // should not remove job key range, because we need to reserve job recycle kv + // 0:instance_id 1:table_id 2:index_id 3:part_id 4:tablet_id + std::string start_job_tablet_key = job_tablet_key({instance_id_, 0, 0, 0, 0}); + std::string end_job_tablet_key = job_tablet_key({instance_id_, INT64_MAX, 0, 0, 0}); + txn->remove(start_job_tablet_key, end_job_tablet_key); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to delete all kv, instance_id=" << instance_id_ << ", err=" << err; + ret = -1; + } + + for (auto& [_, accessor] : accessor_map_) { + if (stopped()) return ret; + LOG(INFO) << "begin to delete all objects in " << accessor->path(); + int del_ret = accessor->delete_objects_by_prefix(""); + if (del_ret == 0) { + LOG(INFO) << "successfully delete all objects in " << accessor->path(); + } else if (del_ret != 1) { // no need to log, because S3Accessor has logged this error + // If `del_ret == 1`, it can be considered that the object data has been recycled by cloud platform, + // so the recycling has been successful. + ret = -1; + } + } + + if (ret == 0) { + // remove instance kv + // ATTN: MUST ensure that cloud platform won't regenerate the same instance id + err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create txn"; + ret = -1; + return ret; + } + std::string key; + instance_key({instance_id_}, &key); + txn->remove(key); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to delete instance kv, instance_id=" << instance_id_ + << " err=" << err; + ret = -1; + } + } + return ret; +} + +int InstanceRecycler::recycle_indexes() { + int num_scanned = 0; + int num_expired = 0; + int num_recycled = 0; + + RecycleIndexKeyInfo index_key_info0 {instance_id_, 0}; + RecycleIndexKeyInfo index_key_info1 {instance_id_, INT64_MAX}; + std::string index_key0; + std::string index_key1; + recycle_index_key(index_key_info0, &index_key0); + recycle_index_key(index_key_info1, &index_key1); + + LOG_INFO("begin to recycle indexes").tag("instance_id", instance_id_); + + using namespace std::chrono; + auto start_time = steady_clock::now(); + + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + auto cost = duration(steady_clock::now() - start_time).count(); + LOG_INFO("recycle indexes finished, cost={}s", cost) + .tag("instance_id", instance_id_) + .tag("num_scanned", num_scanned) + .tag("num_expired", num_expired) + .tag("num_recycled", num_recycled); + }); + + auto calc_expiration = [](const RecycleIndexPB& index) { + int64_t expiration = index.expiration() > 0 ? index.expiration() : index.creation_time(); + int64_t retention_seconds = config::retention_seconds; + if (index.state() == RecycleIndexPB::DROPPED) { + retention_seconds = + std::min(config::dropped_index_retention_seconds, retention_seconds); + } + return expiration + retention_seconds; + }; + + // Elements in `index_keys` has the same lifetime as `it` in `scan_and_recycle` + std::vector index_keys; + auto recycle_func = [&, this](std::string_view k, std::string_view v) -> int { + ++num_scanned; + RecycleIndexPB index_pb; + if (!index_pb.ParseFromArray(v.data(), v.size())) { + LOG_WARNING("malformed recycle index value").tag("key", hex(k)); + return -1; + } + int64_t current_time = ::time(nullptr); + if (current_time < calc_expiration(index_pb)) { // not expired + return 0; + } + ++num_expired; + // decode index_id + auto k1 = k; + k1.remove_prefix(1); + std::vector, int, int>> out; + decode_key(&k1, &out); + // 0x01 "recycle" ${instance_id} "index" ${index_id} -> RecycleIndexPB + auto index_id = std::get(std::get<0>(out[3])); + LOG(INFO) << "begin to recycle index, instance_id=" << instance_id_ + << " table_id=" << index_pb.table_id() << " index_id=" << index_id + << " state=" << RecycleIndexPB::State_Name(index_pb.state()); + // Change state to RECYCLING + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG_WARNING("failed to create txn").tag("err", err); + return -1; + } + std::string val; + err = txn->get(k, &val); + if (err == + TxnErrorCode::TXN_KEY_NOT_FOUND) { // UNKNOWN, maybe recycled or committed, skip it + LOG_INFO("index {} has been recycled or committed", index_id); + return 0; + } + if (err != TxnErrorCode::TXN_OK) { + LOG_WARNING("failed to get kv").tag("key", hex(k)).tag("err", err); + return -1; + } + index_pb.Clear(); + if (!index_pb.ParseFromString(val)) { + LOG_WARNING("malformed recycle index value").tag("key", hex(k)); + return -1; + } + if (index_pb.state() != RecycleIndexPB::RECYCLING) { + index_pb.set_state(RecycleIndexPB::RECYCLING); + txn->put(k, index_pb.SerializeAsString()); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + LOG_WARNING("failed to commit txn").tag("err", err); + return -1; + } + } + if (recycle_tablets(index_pb.table_id(), index_id) != 0) { + LOG_WARNING("failed to recycle tablets under index") + .tag("table_id", index_pb.table_id()) + .tag("instance_id", instance_id_) + .tag("index_id", index_id); + return -1; + } + ++num_recycled; + index_keys.push_back(k); + return 0; + }; + + auto loop_done = [&index_keys, this]() -> int { + if (index_keys.empty()) return 0; + std::unique_ptr> defer((int*)0x01, + [&](int*) { index_keys.clear(); }); + if (0 != txn_remove(txn_kv_.get(), index_keys)) { + LOG(WARNING) << "failed to delete recycle index kv, instance_id=" << instance_id_; + return -1; + } + return 0; + }; + + return scan_and_recycle(index_key0, index_key1, std::move(recycle_func), std::move(loop_done)); +} + +int InstanceRecycler::recycle_partitions() { + int num_scanned = 0; + int num_expired = 0; + int num_recycled = 0; + + RecyclePartKeyInfo part_key_info0 {instance_id_, 0}; + RecyclePartKeyInfo part_key_info1 {instance_id_, INT64_MAX}; + std::string part_key0; + std::string part_key1; + recycle_partition_key(part_key_info0, &part_key0); + recycle_partition_key(part_key_info1, &part_key1); + + LOG_INFO("begin to recycle partitions").tag("instance_id", instance_id_); + + using namespace std::chrono; + auto start_time = steady_clock::now(); + + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + auto cost = duration(steady_clock::now() - start_time).count(); + LOG_INFO("recycle partitions finished, cost={}s", cost) + .tag("instance_id", instance_id_) + .tag("num_scanned", num_scanned) + .tag("num_expired", num_expired) + .tag("num_recycled", num_recycled); + }); + + auto calc_expiration = [](const RecyclePartitionPB& partition) { + int64_t expiration = + partition.expiration() > 0 ? partition.expiration() : partition.creation_time(); + int64_t retention_seconds = config::retention_seconds; + if (partition.state() == RecyclePartitionPB::DROPPED) { + retention_seconds = + std::min(config::dropped_partition_retention_seconds, retention_seconds); + } + return expiration + retention_seconds; + }; + + // Elements in `partition_keys` has the same lifetime as `it` in `scan_and_recycle` + std::vector partition_keys; + std::vector version_keys; + auto recycle_func = [&, this](std::string_view k, std::string_view v) -> int { + ++num_scanned; + RecyclePartitionPB part_pb; + if (!part_pb.ParseFromArray(v.data(), v.size())) { + LOG_WARNING("malformed recycle partition value").tag("key", hex(k)); + return -1; + } + int64_t current_time = ::time(nullptr); + if (current_time < calc_expiration(part_pb)) { // not expired + return 0; + } + ++num_expired; + // decode partition_id + auto k1 = k; + k1.remove_prefix(1); + std::vector, int, int>> out; + decode_key(&k1, &out); + // 0x01 "recycle" ${instance_id} "partition" ${partition_id} -> RecyclePartitionPB + auto partition_id = std::get(std::get<0>(out[3])); + LOG(INFO) << "begin to recycle partition, instance_id=" << instance_id_ + << " table_id=" << part_pb.table_id() << " partition_id=" << partition_id + << " state=" << RecyclePartitionPB::State_Name(part_pb.state()); + // Change state to RECYCLING + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG_WARNING("failed to create txn").tag("err", err); + return -1; + } + std::string val; + err = txn->get(k, &val); + if (err == + TxnErrorCode::TXN_KEY_NOT_FOUND) { // UNKNOWN, maybe recycled or committed, skip it + LOG_INFO("partition {} has been recycled or committed", partition_id); + return 0; + } + if (err != TxnErrorCode::TXN_OK) { + LOG_WARNING("failed to get kv"); + return -1; + } + part_pb.Clear(); + if (!part_pb.ParseFromString(val)) { + LOG_WARNING("malformed recycle partition value").tag("key", hex(k)); + return -1; + } + // Partitions with PREPARED state MUST have no data + bool is_empty_tablet = part_pb.state() == RecyclePartitionPB::PREPARED; + if (part_pb.state() != RecyclePartitionPB::RECYCLING) { + part_pb.set_state(RecyclePartitionPB::RECYCLING); + txn->put(k, part_pb.SerializeAsString()); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + LOG_WARNING("failed to commit txn: {}", err); + return -1; + } + } + int ret = 0; + for (int64_t index_id : part_pb.index_id()) { + if (recycle_tablets(part_pb.table_id(), index_id, partition_id, is_empty_tablet) != 0) { + LOG_WARNING("failed to recycle tablets under partition") + .tag("table_id", part_pb.table_id()) + .tag("instance_id", instance_id_) + .tag("index_id", index_id) + .tag("partition_id", partition_id); + ret = -1; + } + } + if (ret == 0) { + ++num_recycled; + partition_keys.push_back(k); + if (part_pb.db_id() > 0) { + version_keys.push_back(version_key( + {instance_id_, part_pb.db_id(), part_pb.table_id(), partition_id})); + } + } + return ret; + }; + + auto loop_done = [&partition_keys, &version_keys, this]() -> int { + if (partition_keys.empty()) return 0; + std::unique_ptr> defer((int*)0x01, [&](int*) { + partition_keys.clear(); + version_keys.clear(); + }); + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to delete recycle partition kv, instance_id=" << instance_id_; + return -1; + } + for (auto& k : partition_keys) { + txn->remove(k); + } + for (auto& k : version_keys) { + txn->remove(k); + } + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to delete recycle partition kv, instance_id=" << instance_id_ + << " err=" << err; + return -1; + } + return 0; + }; + + return scan_and_recycle(part_key0, part_key1, std::move(recycle_func), std::move(loop_done)); +} + +int InstanceRecycler::recycle_versions() { + int num_scanned = 0; + int num_recycled = 0; + + LOG_INFO("begin to recycle partition versions").tag("instance_id", instance_id_); + + using namespace std::chrono; + auto start_time = steady_clock::now(); + + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + auto cost = duration(steady_clock::now() - start_time).count(); + LOG_INFO("recycle partition versions finished, cost={}s", cost) + .tag("instance_id", instance_id_) + .tag("num_scanned", num_scanned) + .tag("num_recycled", num_recycled); + }); + + auto version_key_begin = version_key({instance_id_, 0, 0, 0}); + auto version_key_end = version_key({instance_id_, INT64_MAX, 0, 0}); + int64_t last_scanned_table_id = 0; + bool is_recycled = false; // Is last scanned kv recycled + auto recycle_func = [&num_scanned, &num_recycled, &last_scanned_table_id, &is_recycled, this]( + std::string_view k, std::string_view) { + ++num_scanned; + auto k1 = k; + k1.remove_prefix(1); + // 0x01 "version" ${instance_id} "partition" ${db_id} ${tbl_id} ${partition_id} + std::vector, int, int>> out; + decode_key(&k1, &out); + DCHECK_EQ(out.size(), 6) << k; + auto table_id = std::get(std::get<0>(out[4])); + if (table_id == last_scanned_table_id) { // Already handle kvs of this table + num_recycled += is_recycled; // Version kv of this table has been recycled + return 0; + } + last_scanned_table_id = table_id; + is_recycled = false; + auto tablet_key_begin = stats_tablet_key({instance_id_, table_id, 0, 0, 0}); + auto tablet_key_end = stats_tablet_key({instance_id_, table_id, INT64_MAX, 0, 0}); + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + return -1; + } + std::unique_ptr iter; + err = txn->get(tablet_key_begin, tablet_key_end, &iter, false, 1); + if (err != TxnErrorCode::TXN_OK) { + return -1; + } + if (iter->has_next()) { // Table is useful, should not recycle partiton versions + return 0; + } + // Remove all version kvs of this table + auto db_id = std::get(std::get<0>(out[3])); + auto table_version_key_begin = version_key({instance_id_, db_id, table_id, 0}); + auto table_version_key_end = version_key({instance_id_, db_id, table_id, INT64_MAX}); + txn->remove(table_version_key_begin, table_version_key_end); + LOG(WARNING) << "remove version kv, begin=" << hex(table_version_key_begin) + << " end=" << hex(table_version_key_end); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + return -1; + } + ++num_recycled; + is_recycled = true; + return 0; + }; + + return scan_and_recycle(version_key_begin, version_key_end, std::move(recycle_func)); +} + +int InstanceRecycler::recycle_tablets(int64_t table_id, int64_t index_id, int64_t partition_id, + bool is_empty_tablet) { + int num_scanned = 0; + int num_recycled = 0; + + std::string tablet_key_begin, tablet_key_end; + std::string stats_key_begin, stats_key_end; + std::string job_key_begin, job_key_end; + + if (partition_id > 0) { + // recycle tablets in a partition belonging to the index + meta_tablet_key({instance_id_, table_id, index_id, partition_id, 0}, &tablet_key_begin); + meta_tablet_key({instance_id_, table_id, index_id, partition_id + 1, 0}, &tablet_key_end); + stats_tablet_key({instance_id_, table_id, index_id, partition_id, 0}, &stats_key_begin); + stats_tablet_key({instance_id_, table_id, index_id, partition_id + 1, 0}, &stats_key_end); + job_tablet_key({instance_id_, table_id, index_id, partition_id, 0}, &job_key_begin); + job_tablet_key({instance_id_, table_id, index_id, partition_id + 1, 0}, &job_key_end); + } else { + // recycle tablets in the index + meta_tablet_key({instance_id_, table_id, index_id, 0, 0}, &tablet_key_begin); + meta_tablet_key({instance_id_, table_id, index_id + 1, 0, 0}, &tablet_key_end); + stats_tablet_key({instance_id_, table_id, index_id, 0, 0}, &stats_key_begin); + stats_tablet_key({instance_id_, table_id, index_id + 1, 0, 0}, &stats_key_end); + job_tablet_key({instance_id_, table_id, index_id, 0, 0}, &job_key_begin); + job_tablet_key({instance_id_, table_id, index_id + 1, 0, 0}, &job_key_end); + } + + LOG_INFO("begin to recycle tablets") + .tag("table_id", table_id) + .tag("index_id", index_id) + .tag("partition_id", partition_id); + + using namespace std::chrono; + auto start_time = steady_clock::now(); + + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + auto cost = duration(steady_clock::now() - start_time).count(); + LOG_INFO("recycle tablets finished, cost={}s", cost) + .tag("instance_id", instance_id_) + .tag("table_id", table_id) + .tag("index_id", index_id) + .tag("partition_id", partition_id) + .tag("num_scanned", num_scanned) + .tag("num_recycled", num_recycled); + }); + + // Elements in `tablet_keys` has the same lifetime as `it` in `scan_and_recycle` + std::vector tablet_keys; + std::vector tablet_idx_keys; + std::vector init_rs_keys; + bool use_range_remove = true; + auto recycle_func = [&, is_empty_tablet, this](std::string_view k, std::string_view v) -> int { + ++num_scanned; + doris::TabletMetaCloudPB tablet_meta_pb; + if (!tablet_meta_pb.ParseFromArray(v.data(), v.size())) { + LOG_WARNING("malformed tablet meta").tag("key", hex(k)); + use_range_remove = false; + return -1; + } + int64_t tablet_id = tablet_meta_pb.tablet_id(); + tablet_idx_keys.push_back(meta_tablet_idx_key({instance_id_, tablet_id})); + if (!is_empty_tablet) { + if (recycle_tablet(tablet_id) != 0) { + LOG_WARNING("failed to recycle tablet") + .tag("instance_id", instance_id_) + .tag("tablet_id", tablet_id); + use_range_remove = false; + return -1; + } + } else { + // Empty tablet only has a [0-1] init rowset + init_rs_keys.push_back(meta_rowset_key({instance_id_, tablet_id, 1})); + DCHECK([&]() { + std::unique_ptr txn; + if (TxnErrorCode err = txn_kv_->create_txn(&txn); err != TxnErrorCode::TXN_OK) { + LOG_ERROR("failed to create txn").tag("err", err); + return false; + } + auto rs_key_begin = meta_rowset_key({instance_id_, tablet_id, 2}); + auto rs_key_end = meta_rowset_key({instance_id_, tablet_id, INT64_MAX}); + std::unique_ptr iter; + if (TxnErrorCode err = txn->get(rs_key_begin, rs_key_end, &iter, true, 1); + err != TxnErrorCode::TXN_OK) { + LOG_ERROR("failed to get kv").tag("err", err); + return false; + } + if (iter->has_next()) { + LOG_ERROR("tablet is not empty").tag("tablet_id", tablet_id); + return false; + } + return true; + }()); + } + ++num_recycled; + tablet_keys.push_back(k); + return 0; + }; + + auto loop_done = [&, this]() -> int { + if (tablet_keys.empty() && tablet_idx_keys.empty()) return 0; + std::unique_ptr> defer((int*)0x01, [&](int*) { + tablet_keys.clear(); + tablet_idx_keys.clear(); + init_rs_keys.clear(); + use_range_remove = true; + }); + std::unique_ptr txn; + if (txn_kv_->create_txn(&txn) != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to delete tablet meta kv, instance_id=" << instance_id_; + return -1; + } + std::string tablet_key_end; + if (!tablet_keys.empty()) { + if (use_range_remove) { + tablet_key_end = std::string(tablet_keys.back()) + '\x00'; + txn->remove(tablet_keys.front(), tablet_key_end); + } else { + for (auto k : tablet_keys) { + txn->remove(k); + } + } + } + for (auto& k : tablet_idx_keys) { + txn->remove(k); + } + for (auto& k : init_rs_keys) { + txn->remove(k); + } + if (TxnErrorCode err = txn->commit(); err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to delete kvs related to tablets, instance_id=" << instance_id_ + << ", err=" << err; + return -1; + } + return 0; + }; + + int ret = scan_and_recycle(tablet_key_begin, tablet_key_end, std::move(recycle_func), + std::move(loop_done)); + + // directly remove tablet stats and tablet jobs of these dropped index or partition + std::unique_ptr txn; + if (txn_kv_->create_txn(&txn) != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to delete tablet job or stats key, instance_id=" << instance_id_; + return -1; + } + txn->remove(stats_key_begin, stats_key_end); + LOG(WARNING) << "remove stats kv, begin=" << hex(stats_key_begin) + << " end=" << hex(stats_key_end); + txn->remove(job_key_begin, job_key_end); + LOG(WARNING) << "remove job kv, begin=" << hex(job_key_begin) << " end=" << hex(job_key_end); + std::string schema_key_begin, schema_key_end; + if (partition_id <= 0) { + // Delete schema kv of this index + meta_schema_key({instance_id_, index_id, 0}, &schema_key_begin); + meta_schema_key({instance_id_, index_id + 1, 0}, &schema_key_end); + txn->remove(schema_key_begin, schema_key_end); + LOG(WARNING) << "remove schema kv, begin=" << hex(schema_key_begin) + << " end=" << hex(schema_key_end); + } + + TxnErrorCode err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to delete tablet job or stats key, instance_id=" << instance_id_ + << " err=" << err; + return -1; + } + + return ret; +} + +int InstanceRecycler::delete_rowset_data(const doris::RowsetMetaCloudPB& rs_meta_pb) { + int64_t num_segments = rs_meta_pb.num_segments(); + if (num_segments <= 0) return 0; + if (!rs_meta_pb.has_tablet_schema()) { + return delete_rowset_data(rs_meta_pb.resource_id(), rs_meta_pb.tablet_id(), + rs_meta_pb.rowset_id_v2()); + } + auto it = accessor_map_.find(rs_meta_pb.resource_id()); + if (it == accessor_map_.end()) { + LOG_WARNING("instance has no such resource id") + .tag("instance_id", instance_id_) + .tag("resource_id", rs_meta_pb.resource_id()); + return -1; + } + auto& accessor = it->second; + const auto& rowset_id = rs_meta_pb.rowset_id_v2(); + int64_t tablet_id = rs_meta_pb.tablet_id(); + // process inverted indexes + std::vector index_ids; + index_ids.reserve(rs_meta_pb.tablet_schema().index_size()); + for (auto& i : rs_meta_pb.tablet_schema().index()) { + index_ids.push_back(i.index_id()); + } + std::vector file_paths; + file_paths.reserve(num_segments * (1 + index_ids.size())); + for (int64_t i = 0; i < num_segments; ++i) { + file_paths.push_back(segment_path(tablet_id, rowset_id, i)); + for (int64_t index_id : index_ids) { + file_paths.push_back(inverted_index_path(tablet_id, rowset_id, i, index_id)); + } + } + // TODO(AlexYue): seems could do do batch + return accessor->delete_objects(file_paths); +} + +int InstanceRecycler::delete_rowset_data(const std::vector& rowsets) { + int ret = 0; + // resource_id -> file_paths + std::map> resource_file_paths; + for (auto& rs : rowsets) { + { + std::lock_guard lock(recycled_tablets_mtx_); + if (recycled_tablets_.count(rs.tablet_id())) { + continue; // Rowset data has already been deleted + } + } + + auto it = accessor_map_.find(rs.resource_id()); + if (it == accessor_map_.end()) [[unlikely]] { // impossible + LOG_WARNING("instance has no such resource id") + .tag("instance_id", instance_id_) + .tag("resource_id", rs.resource_id()); + ret = -1; + continue; + } + + auto& file_paths = resource_file_paths[rs.resource_id()]; + const auto& rowset_id = rs.rowset_id_v2(); + int64_t tablet_id = rs.tablet_id(); + int64_t num_segments = rs.num_segments(); + if (num_segments <= 0) continue; + + // process inverted indexes + std::vector index_ids; + if (rs.has_tablet_schema()) { + index_ids.reserve(rs.tablet_schema().index().size()); + for (auto& index_pb : rs.tablet_schema().index()) { + index_ids.push_back(index_pb.index_id()); + } + } else { // Detached schema + if (!rs.has_index_id() || !rs.has_schema_version()) { + LOG(WARNING) << "rowset must have either schema or schema_version and index_id, " + "instance_id=" + << instance_id_ << " tablet_id=" << tablet_id + << " rowset_id=" << rowset_id; + ret = -1; + continue; + } + int get_ret = + inverted_index_id_cache_->get(rs.index_id(), rs.schema_version(), index_ids); + if (get_ret != 0) { + if (get_ret == 1) { // Schema kv not found + // Check tablet existence + std::string tablet_idx_key, tablet_idx_val; + meta_tablet_idx_key({instance_id_, tablet_id}, &tablet_idx_key); + if (txn_get(txn_kv_.get(), tablet_idx_key, tablet_idx_val) == 1) { + // Tablet has been recycled, rowset data has already been deleted + std::lock_guard lock(recycled_tablets_mtx_); + recycled_tablets_.insert(tablet_id); + continue; + } + } + LOG(WARNING) << "failed to get schema kv for rowset, instance_id=" << instance_id_ + << " tablet_id=" << tablet_id << " rowset_id=" << rowset_id; + ret = -1; + continue; + } + } + for (int64_t i = 0; i < num_segments; ++i) { + file_paths.push_back(segment_path(tablet_id, rowset_id, i)); + for (int64_t index_id : index_ids) { + file_paths.push_back(inverted_index_path(tablet_id, rowset_id, i, index_id)); + } + } + } + for (auto& [resource_id, file_paths] : resource_file_paths) { + auto& accessor = accessor_map_[resource_id]; + DCHECK(accessor); + if (accessor->delete_objects(file_paths) != 0) { + ret = -1; + } + } + return ret; +} + +int InstanceRecycler::delete_rowset_data(const std::string& resource_id, int64_t tablet_id, + const std::string& rowset_id) { + auto it = accessor_map_.find(resource_id); + if (it == accessor_map_.end()) { + LOG_WARNING("instance has no such resource id") + .tag("instance_id", instance_id_) + .tag("resource_id", resource_id); + return -1; + } + auto& accessor = it->second; + return accessor->delete_objects_by_prefix(rowset_path_prefix(tablet_id, rowset_id)); +} + +int InstanceRecycler::recycle_tablet(int64_t tablet_id) { + LOG_INFO("begin to recycle rowsets in a dropped tablet") + .tag("instance_id", instance_id_) + .tag("tablet_id", tablet_id); + + using namespace std::chrono; + auto start_time = steady_clock::now(); + + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + auto cost = duration(steady_clock::now() - start_time).count(); + LOG_INFO("recycle rowsets finished, cost={}s", cost) + .tag("instance_id", instance_id_) + .tag("tablet_id", tablet_id); + }); + + // delete all rowset kv in this tablet + std::string rs_key0 = meta_rowset_key({instance_id_, tablet_id, 0}); + std::string rs_key1 = meta_rowset_key({instance_id_, tablet_id + 1, 0}); + std::string recyc_rs_key0 = recycle_rowset_key({instance_id_, tablet_id, ""}); + std::string recyc_rs_key1 = recycle_rowset_key({instance_id_, tablet_id + 1, ""}); + + int ret = 0; + std::unique_ptr txn; + if (txn_kv_->create_txn(&txn) != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to delete rowset kv of tablet " << tablet_id; + ret = -1; + } + txn->remove(rs_key0, rs_key1); + txn->remove(recyc_rs_key0, recyc_rs_key1); + + // remove delete bitmap for MoW table + std::string pending_key = meta_pending_delete_bitmap_key({instance_id_, tablet_id}); + txn->remove(pending_key); + std::string delete_bitmap_start = meta_delete_bitmap_key({instance_id_, tablet_id, "", 0, 0}); + std::string delete_bitmap_end = meta_delete_bitmap_key({instance_id_, tablet_id + 1, "", 0, 0}); + txn->remove(delete_bitmap_start, delete_bitmap_end); + + TxnErrorCode err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to delete rowset kv of tablet " << tablet_id << ", err=" << err; + ret = -1; + } + + // delete all rowset data in this tablet + for (auto& [_, accessor] : accessor_map_) { + if (accessor->delete_objects_by_prefix(tablet_path_prefix(tablet_id)) != 0) { + LOG(WARNING) << "failed to delete rowset data of tablet " << tablet_id + << " s3_path=" << accessor->path(); + ret = -1; + } + } + + if (ret == 0) { + // All object files under tablet have been deleted + std::lock_guard lock(recycled_tablets_mtx_); + recycled_tablets_.insert(tablet_id); + } + + return ret; +} + +int InstanceRecycler::recycle_rowsets() { + int num_scanned = 0; + int num_expired = 0; + int num_prepare = 0; + size_t total_rowset_size = 0; + size_t expired_rowset_size = 0; + std::atomic_int num_recycled = 0; + + RecycleRowsetKeyInfo recyc_rs_key_info0 {instance_id_, 0, ""}; + RecycleRowsetKeyInfo recyc_rs_key_info1 {instance_id_, INT64_MAX, ""}; + std::string recyc_rs_key0; + std::string recyc_rs_key1; + recycle_rowset_key(recyc_rs_key_info0, &recyc_rs_key0); + recycle_rowset_key(recyc_rs_key_info1, &recyc_rs_key1); + + LOG_INFO("begin to recycle rowsets").tag("instance_id", instance_id_); + + using namespace std::chrono; + auto start_time = steady_clock::now(); + + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + auto cost = duration(steady_clock::now() - start_time).count(); + LOG_INFO("recycle rowsets finished, cost={}s", cost) + .tag("instance_id", instance_id_) + .tag("num_scanned", num_scanned) + .tag("num_expired", num_expired) + .tag("num_recycled", num_recycled) + .tag("num_prepare", num_prepare) + .tag("total_rowset_meta_size", total_rowset_size) + .tag("expired_rowset_meta_size", expired_rowset_size); + }); + + std::vector rowset_keys; + std::vector rowsets; + + // Store keys of rowset recycled by background workers + std::mutex async_recycled_rowset_keys_mutex; + std::vector async_recycled_rowset_keys; + auto worker_pool = + std::make_unique(config::instance_recycler_worker_pool_size); + worker_pool->start(); + auto delete_rowset_data_by_prefix = [&](std::string key, const std::string& resource_id, + int64_t tablet_id, const std::string& rowset_id) { + // Try to delete rowset data in background thread + int ret = worker_pool->submit_with_timeout( + [&, resource_id, tablet_id, rowset_id, key]() mutable { + if (delete_rowset_data(resource_id, tablet_id, rowset_id) != 0) { + LOG(WARNING) << "failed to delete rowset data, key=" << hex(key); + return; + } + std::vector keys; + { + std::lock_guard lock(async_recycled_rowset_keys_mutex); + async_recycled_rowset_keys.push_back(std::move(key)); + if (async_recycled_rowset_keys.size() > 100) { + keys.swap(async_recycled_rowset_keys); + } + } + if (keys.empty()) return; + if (txn_remove(txn_kv_.get(), keys) != 0) { + LOG(WARNING) << "failed to delete recycle rowset kv, instance_id=" + << instance_id_; + } else { + num_recycled.fetch_add(keys.size(), std::memory_order_relaxed); + } + }, + 0); + if (ret == 0) return 0; + // Submit task failed, delete rowset data in current thread + if (delete_rowset_data(resource_id, tablet_id, rowset_id) != 0) { + LOG(WARNING) << "failed to delete rowset data, key=" << hex(key); + return -1; + } + rowset_keys.push_back(std::move(key)); + return 0; + }; + + auto calc_expiration = [](const RecycleRowsetPB& rs) { + // RecycleRowsetPB created by compacted or dropped rowset has no expiration time, and will be recycled when exceed retention time + int64_t expiration = rs.expiration() > 0 ? rs.expiration() : rs.creation_time(); + int64_t retention_seconds = config::retention_seconds; + if (rs.type() == RecycleRowsetPB::COMPACT || rs.type() == RecycleRowsetPB::DROP) { + retention_seconds = + std::min(config::compacted_rowset_retention_seconds, retention_seconds); + } + return expiration + retention_seconds; + }; + + auto handle_rowset_kv = [&](std::string_view k, std::string_view v) -> int { + ++num_scanned; + total_rowset_size += v.size(); + RecycleRowsetPB rowset; + if (!rowset.ParseFromArray(v.data(), v.size())) { + LOG_WARNING("malformed recycle rowset").tag("key", hex(k)); + return -1; + } + int64_t current_time = ::time(nullptr); + if (current_time < calc_expiration(rowset)) { // not expired + return 0; + } + ++num_expired; + expired_rowset_size += v.size(); + if (!rowset.has_type()) { // old version `RecycleRowsetPB` + if (!rowset.has_resource_id()) [[unlikely]] { // impossible + // in old version, keep this key-value pair and it needs to be checked manually + LOG_WARNING("rowset meta has empty resource id").tag("key", hex(k)); + return -1; + } + if (rowset.resource_id().empty()) [[unlikely]] { + // old version `RecycleRowsetPB` may has empty resource_id, just remove the kv. + LOG(INFO) << "delete the recycle rowset kv that has empty resource_id, key=" + << hex(k) << " value=" << proto_to_json(rowset); + rowset_keys.push_back(std::string(k)); + return -1; + } + // decode rowset_id + auto k1 = k; + k1.remove_prefix(1); + std::vector, int, int>> out; + decode_key(&k1, &out); + // 0x01 "recycle" ${instance_id} "rowset" ${tablet_id} ${rowset_id} -> RecycleRowsetPB + const auto& rowset_id = std::get(std::get<0>(out[4])); + LOG(INFO) << "delete rowset data, instance_id=" << instance_id_ + << " tablet_id=" << rowset.tablet_id() << " rowset_id=" << rowset_id; + if (delete_rowset_data_by_prefix(std::string(k), rowset.resource_id(), + rowset.tablet_id(), rowset_id) != 0) { + return -1; + } + return 0; + } + // TODO(plat1ko): check rowset not referenced + auto rowset_meta = rowset.mutable_rowset_meta(); + if (!rowset_meta->has_resource_id()) [[unlikely]] { // impossible + if (rowset.type() != RecycleRowsetPB::PREPARE && rowset_meta->num_segments() == 0) { + LOG_INFO("recycle rowset that has empty resource id"); + } else { + // other situations, keep this key-value pair and it needs to be checked manually + LOG_WARNING("rowset meta has empty resource id").tag("key", hex(k)); + return -1; + } + } + LOG(INFO) << "delete rowset data, instance_id=" << instance_id_ + << " tablet_id=" << rowset_meta->tablet_id() + << " rowset_id=" << rowset_meta->rowset_id_v2() << " version=[" + << rowset_meta->start_version() << '-' << rowset_meta->end_version() + << "] txn_id=" << rowset_meta->txn_id() + << " type=" << RecycleRowsetPB_Type_Name(rowset.type()) + << " rowset_meta_size=" << v.size() << " creation_time" + << rowset_meta->creation_time(); + if (rowset.type() == RecycleRowsetPB::PREPARE) { + // unable to calculate file path, can only be deleted by rowset id prefix + num_prepare += 1; + if (delete_rowset_data_by_prefix(std::string(k), rowset_meta->resource_id(), + rowset_meta->tablet_id(), + rowset_meta->rowset_id_v2()) != 0) { + return -1; + } + } else { + rowset_keys.push_back(std::string(k)); + if (rowset_meta->num_segments() > 0) { // Skip empty rowset + rowsets.push_back(std::move(*rowset_meta)); + } + } + return 0; + }; + + auto loop_done = [&]() -> int { + std::vector rowset_keys_to_delete; + std::vector rowsets_to_delete; + rowset_keys_to_delete.swap(rowset_keys); + rowsets_to_delete.swap(rowsets); + worker_pool->submit([&, rowset_keys_to_delete = std::move(rowset_keys_to_delete), + rowsets_to_delete = std::move(rowsets_to_delete)]() { + if (delete_rowset_data(rowsets_to_delete) != 0) { + LOG(WARNING) << "failed to delete rowset data, instance_id=" << instance_id_; + return; + } + if (txn_remove(txn_kv_.get(), rowset_keys_to_delete) != 0) { + LOG(WARNING) << "failed to delete recycle rowset kv, instance_id=" << instance_id_; + return; + } + num_recycled.fetch_add(rowset_keys.size(), std::memory_order_relaxed); + }); + return 0; + }; + + int ret = scan_and_recycle(recyc_rs_key0, recyc_rs_key1, std::move(handle_rowset_kv), + std::move(loop_done)); + worker_pool->stop(); + + if (!async_recycled_rowset_keys.empty()) { + if (txn_remove(txn_kv_.get(), async_recycled_rowset_keys) != 0) { + LOG(WARNING) << "failed to delete recycle rowset kv, instance_id=" << instance_id_; + return -1; + } else { + num_recycled.fetch_add(async_recycled_rowset_keys.size(), std::memory_order_relaxed); + } + } + return ret; +} + +int InstanceRecycler::recycle_tmp_rowsets() { + int num_scanned = 0; + int num_expired = 0; + int num_recycled = 0; + size_t expired_rowset_size = 0; + size_t total_rowset_size = 0; + + MetaRowsetTmpKeyInfo tmp_rs_key_info0 {instance_id_, 0, 0}; + MetaRowsetTmpKeyInfo tmp_rs_key_info1 {instance_id_, INT64_MAX, 0}; + std::string tmp_rs_key0; + std::string tmp_rs_key1; + meta_rowset_tmp_key(tmp_rs_key_info0, &tmp_rs_key0); + meta_rowset_tmp_key(tmp_rs_key_info1, &tmp_rs_key1); + + LOG_INFO("begin to recycle tmp rowsets").tag("instance_id", instance_id_); + + using namespace std::chrono; + auto start_time = steady_clock::now(); + + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + auto cost = duration(steady_clock::now() - start_time).count(); + LOG_INFO("recycle tmp rowsets finished, cost={}s", cost) + .tag("instance_id", instance_id_) + .tag("num_scanned", num_scanned) + .tag("num_expired", num_expired) + .tag("num_recycled", num_recycled) + .tag("total_rowset_meta_size", total_rowset_size) + .tag("expired_rowset_meta_size", expired_rowset_size); + }); + + // Elements in `tmp_rowset_keys` has the same lifetime as `it` + std::vector tmp_rowset_keys; + std::vector tmp_rowsets; + + auto handle_rowset_kv = [&num_scanned, &num_expired, &tmp_rowset_keys, &tmp_rowsets, + &expired_rowset_size, &total_rowset_size, + this](std::string_view k, std::string_view v) -> int { + ++num_scanned; + total_rowset_size += v.size(); + doris::RowsetMetaCloudPB rowset; + if (!rowset.ParseFromArray(v.data(), v.size())) { + LOG_WARNING("malformed rowset meta").tag("key", hex(k)); + return -1; + } + int64_t current_time = ::time(nullptr); + // ATTN: `txn_expiration` should > 0, however we use `creation_time` + a large `retention_time` (> 1 day in production environment) + // when `txn_expiration` <= 0 in some unexpected situation (usually when there are bugs). This is usually safe, coz loading + // duration or timeout always < `retention_time` in practice. + int64_t expiration = + rowset.txn_expiration() > 0 ? rowset.txn_expiration() : rowset.creation_time(); + if (current_time < expiration + config::retention_seconds) { + // not expired + return 0; + } + ++num_expired; + expired_rowset_size += v.size(); + if (!rowset.has_resource_id()) { + if (rowset.num_segments() > 0) [[unlikely]] { // impossible + LOG_WARNING("rowset meta has empty resource id").tag("key", k); + return -1; + } + // might be a delete pred rowset + tmp_rowset_keys.push_back(k); + return 0; + } + // TODO(plat1ko): check rowset not referenced + LOG(INFO) << "delete rowset data, instance_id=" << instance_id_ + << " tablet_id=" << rowset.tablet_id() << " rowset_id=" << rowset.rowset_id_v2() + << " version=[" << rowset.start_version() << '-' << rowset.end_version() + << "] txn_id=" << rowset.txn_id() << " rowset_meta_size=" << v.size() + << " creation_time" << rowset.creation_time(); + tmp_rowset_keys.push_back(k); + if (rowset.num_segments() > 0) { // Skip empty rowset + tmp_rowsets.push_back(std::move(rowset)); + } + return 0; + }; + + auto loop_done = [&tmp_rowset_keys, &tmp_rowsets, &num_recycled, this]() -> int { + std::unique_ptr> defer((int*)0x01, [&](int*) { + tmp_rowset_keys.clear(); + tmp_rowsets.clear(); + }); + if (delete_rowset_data(tmp_rowsets) != 0) { + LOG(WARNING) << "failed to delete tmp rowset data, instance_id=" << instance_id_; + return -1; + } + if (txn_remove(txn_kv_.get(), tmp_rowset_keys) != 0) { + LOG(WARNING) << "failed to delete tmp rowset kv, instance_id=" << instance_id_; + return -1; + } + num_recycled += tmp_rowset_keys.size(); + return 0; + }; + + return scan_and_recycle(tmp_rs_key0, tmp_rs_key1, std::move(handle_rowset_kv), + std::move(loop_done)); +} + +int InstanceRecycler::scan_and_recycle( + std::string begin, std::string_view end, + std::function recycle_func, + std::function loop_done) { + int ret = 0; + std::unique_ptr it; + do { + int get_ret = txn_get(txn_kv_.get(), begin, end, it); + if (get_ret != 0) { + LOG(WARNING) << "failed to get kv, key=" << begin << " ret=" << get_ret; + return -1; + } + VLOG_DEBUG << "fetch " << it->size() << " kv"; + if (!it->has_next()) { + VLOG_DEBUG << "no keys in the given range, begin=" << hex(begin) << " end=" << hex(end); + break; + } + while (it->has_next()) { + // recycle corresponding resources + auto [k, v] = it->next(); + if (!it->has_next()) { + begin = k; + VLOG_DEBUG << "iterator has no more kvs. key=" << hex(k); + } + if (recycle_func(k, v) != 0) ret = -1; + } + begin.push_back('\x00'); // Update to next smallest key for iteration + if (loop_done) { + if (loop_done() != 0) ret = -1; + } + } while (it->more() && !stopped()); + return ret; +} + +int InstanceRecycler::abort_timeout_txn() { + int num_scanned = 0; + int num_timeout = 0; + int num_abort = 0; + + TxnRunningKeyInfo txn_running_key_info0 {instance_id_, 0, 0}; + TxnRunningKeyInfo txn_running_key_info1 {instance_id_, INT64_MAX, INT64_MAX}; + std::string begin_txn_running_key; + std::string end_txn_running_key; + txn_running_key(txn_running_key_info0, &begin_txn_running_key); + txn_running_key(txn_running_key_info1, &end_txn_running_key); + + LOG_INFO("begin to abort timeout txn").tag("instance_id", instance_id_); + + using namespace std::chrono; + auto start_time = steady_clock::now(); + + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + auto cost = duration(steady_clock::now() - start_time).count(); + LOG_INFO("end to abort timeout txn, cost={}s", cost) + .tag("instance_id", instance_id_) + .tag("num_scanned", num_scanned) + .tag("num_timeout", num_timeout) + .tag("num_abort", num_abort); + }); + + int64_t current_time = + duration_cast(system_clock::now().time_since_epoch()).count(); + + auto handle_txn_running_kv = [&num_scanned, &num_timeout, &num_abort, ¤t_time, this]( + std::string_view k, std::string_view v) -> int { + ++num_scanned; + TxnRunningPB txn_running_pb; + if (!txn_running_pb.ParseFromArray(v.data(), v.size())) { + LOG_WARNING("malformed txn_running_pb").tag("key", hex(k)); + return -1; + } + if (txn_running_pb.timeout_time() > current_time) { + return 0; + } + ++num_timeout; + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG_ERROR("failed to create txn err={}", err).tag("key", hex(k)); + return -1; + } + std::string_view k1 = k; + //TxnRunningKeyInfo 0:instance_id 1:db_id 2:txn_id + k1.remove_prefix(1); // Remove key space + std::vector, int, int>> out; + if (decode_key(&k1, &out) != 0) { + LOG_ERROR("failed to decode key").tag("key", hex(k)); + return -1; + } + int64_t db_id = std::get(std::get<0>(out[3])); + int64_t txn_id = std::get(std::get<0>(out[4])); + VLOG_DEBUG << "instance_id=" << instance_id_ << " db_id=" << db_id << " txn_id=" << txn_id; + // Update txn_info + std::string txn_inf_key, txn_inf_val; + txn_info_key({instance_id_, db_id, txn_id}, &txn_inf_key); + err = txn->get(txn_inf_key, &txn_inf_val); + if (err != TxnErrorCode::TXN_OK) { + LOG_WARNING("failed to get txn info err={}", err).tag("key", hex(txn_inf_key)); + return -1; + } + TxnInfoPB txn_info; + if (!txn_info.ParseFromString(txn_inf_val)) { + LOG_WARNING("failed to parse txn info").tag("key", hex(k)); + return -1; + } + txn_info.set_status(TxnStatusPB::TXN_STATUS_ABORTED); + txn_info.set_finish_time(current_time); + txn_info.set_reason("timeout"); + VLOG_DEBUG << "txn_info=" << txn_info.DebugString(); + txn_inf_val.clear(); + if (!txn_info.SerializeToString(&txn_inf_val)) { + LOG_WARNING("failed to serialize txn info").tag("key", hex(k)); + return -1; + } + txn->put(txn_inf_key, txn_inf_val); + VLOG_DEBUG << "txn->put, txn_inf_key=" << hex(txn_inf_key); + // Put recycle txn key + std::string recyc_txn_key, recyc_txn_val; + recycle_txn_key({instance_id_, db_id, txn_id}, &recyc_txn_key); + RecycleTxnPB recycle_txn_pb; + recycle_txn_pb.set_creation_time(current_time); + recycle_txn_pb.set_label(txn_info.label()); + if (!recycle_txn_pb.SerializeToString(&recyc_txn_val)) { + LOG_WARNING("failed to serialize txn recycle info") + .tag("key", hex(k)) + .tag("db_id", db_id) + .tag("txn_id", txn_id); + return -1; + } + txn->put(recyc_txn_key, recyc_txn_val); + // Remove txn running key + txn->remove(k); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + LOG_WARNING("failed to commit txn err={}", err) + .tag("key", hex(k)) + .tag("db_id", db_id) + .tag("txn_id", txn_id); + return -1; + } + ++num_abort; + return 0; + }; + + return scan_and_recycle(begin_txn_running_key, end_txn_running_key, + std::move(handle_txn_running_kv)); +} + +int InstanceRecycler::recycle_expired_txn_label() { + int num_scanned = 0; + int num_expired = 0; + int num_recycled = 0; + + RecycleTxnKeyInfo recycle_txn_key_info0 {instance_id_, 0, 0}; + RecycleTxnKeyInfo recycle_txn_key_info1 {instance_id_, INT64_MAX, INT64_MAX}; + std::string begin_recycle_txn_key; + std::string end_recycle_txn_key; + recycle_txn_key(recycle_txn_key_info0, &begin_recycle_txn_key); + recycle_txn_key(recycle_txn_key_info1, &end_recycle_txn_key); + + LOG_INFO("begin to recycle expire txn").tag("instance_id", instance_id_); + + using namespace std::chrono; + auto start_time = steady_clock::now(); + + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + auto cost = duration(steady_clock::now() - start_time).count(); + LOG_INFO("end to recycle expired txn, cost={}s", cost) + .tag("instance_id", instance_id_) + .tag("num_scanned", num_scanned) + .tag("num_expired", num_expired) + .tag("num_recycled", num_recycled); + }); + + int64_t current_time = + duration_cast(system_clock::now().time_since_epoch()).count(); + + auto handle_recycle_txn_kv = [&num_scanned, &num_expired, &num_recycled, ¤t_time, this]( + std::string_view k, std::string_view v) -> int { + ++num_scanned; + RecycleTxnPB recycle_txn_pb; + if (!recycle_txn_pb.ParseFromArray(v.data(), v.size())) { + LOG_WARNING("malformed txn_running_pb").tag("key", hex(k)); + return -1; + } + if ((recycle_txn_pb.has_immediate() && recycle_txn_pb.immediate()) || + (recycle_txn_pb.creation_time() + config::label_keep_max_second <= current_time)) { + LOG_INFO("found recycle txn").tag("key", hex(k)); + num_expired++; + } else { + return 0; + } + std::string_view k1 = k; + //RecycleTxnKeyInfo 0:instance_id 1:db_id 2:txn_id + k1.remove_prefix(1); // Remove key space + std::vector, int, int>> out; + int ret = decode_key(&k1, &out); + if (ret != 0) { + LOG_ERROR("failed to decode key, ret={}", ret).tag("key", hex(k)); + return -1; + } + int64_t db_id = std::get(std::get<0>(out[3])); + int64_t txn_id = std::get(std::get<0>(out[4])); + VLOG_DEBUG << "instance_id=" << instance_id_ << " db_id=" << db_id << " txn_id=" << txn_id; + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG_ERROR("failed to create txn err={}", err).tag("key", hex(k)); + return -1; + } + // Remove txn index kv + auto index_key = txn_index_key({instance_id_, txn_id}); + txn->remove(index_key); + // Remove txn info kv + std::string info_key, info_val; + txn_info_key({instance_id_, db_id, txn_id}, &info_key); + err = txn->get(info_key, &info_val); + if (err != TxnErrorCode::TXN_OK) { + LOG_WARNING("failed to get txn info err={}", err).tag("key", hex(info_key)); + return -1; + } + TxnInfoPB txn_info; + if (!txn_info.ParseFromString(info_val)) { + LOG_WARNING("failed to parse txn info").tag("key", hex(info_key)); + return -1; + } + txn->remove(info_key); + // Update txn label + std::string label_key, label_val; + txn_label_key({instance_id_, db_id, txn_info.label()}, &label_key); + err = txn->get(label_key, &label_val); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to get txn label, txn_id=" << txn_id << " key=" << label_key + << " err=" << err; + return -1; + } + TxnLabelPB txn_label; + if (!txn_label.ParseFromArray(label_val.data(), label_val.size() - VERSION_STAMP_LEN)) { + LOG_WARNING("failed to parse txn label").tag("key", hex(label_key)); + return -1; + } + auto it = std::find(txn_label.txn_ids().begin(), txn_label.txn_ids().end(), txn_id); + if (it != txn_label.txn_ids().end()) { + txn_label.mutable_txn_ids()->erase(it); + } + if (txn_label.txn_ids().empty()) { + txn->remove(label_key); + } else { + if (!txn_label.SerializeToString(&label_val)) { + LOG(WARNING) << "failed to serialize txn label, key=" << hex(label_key); + return -1; + } + txn->atomic_set_ver_value(label_key, label_val); + } + // Remove recycle txn kv + txn->remove(k); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to delete expired txn, err=" << err << " key=" << hex(k); + return -1; + } + ++num_recycled; + LOG(INFO) << "recycle expired txn, key=" << hex(k); + return 0; + }; + + return scan_and_recycle(begin_recycle_txn_key, end_recycle_txn_key, + std::move(handle_recycle_txn_kv)); +} + +struct CopyJobIdTuple { + std::string instance_id; + std::string stage_id; + long table_id; + std::string copy_id; + std::string stage_path; +}; +struct BatchObjStoreAccessor { + BatchObjStoreAccessor(std::shared_ptr accessor, uint64_t& batch_count, + TxnKv* txn_kv) + : accessor_(std::move(accessor)), batch_count_(batch_count), txn_kv_(txn_kv) {}; + ~BatchObjStoreAccessor() { + if (!paths_.empty()) { + consume(); + } + } + + /** + * To implicitely do batch work and submit the batch delete task to s3 + * The s3 delete opreations would be done in batches, and then delete CopyJobPB key one by one + * + * @param copy_job The protubuf struct consists of the copy job files. + * @param key The copy job's key on fdb, the key is originally occupied by fdb range iterator, to make sure + * it would last until we finish the delete task, here we need pass one string value + * @param cope_job_id_tuple One tuple {log_trace instance_id, stage_id, table_id, query_id, stage_path} to print log + */ + void add(CopyJobPB copy_job, std::string key, const CopyJobIdTuple cope_job_id_tuple) { + auto& [instance_id, stage_id, table_id, copy_id, path] = cope_job_id_tuple; + auto& file_keys = copy_file_keys_[key]; + file_keys.log_trace = + fmt::format("instance_id={}, stage_id={}, table_id={}, query_id={}, path={}", + instance_id, stage_id, table_id, copy_id, path); + std::string_view log_trace = file_keys.log_trace; + for (const auto& file : copy_job.object_files()) { + auto relative_path = file.relative_path(); + paths_.push_back(relative_path); + file_keys.keys.push_back(copy_file_key( + {instance_id, stage_id, table_id, file.relative_path(), file.etag()})); + LOG_INFO(log_trace) + .tag("relative_path", relative_path) + .tag("batch_count", batch_count_); + } + LOG_INFO(log_trace) + .tag("objects_num", copy_job.object_files().size()) + .tag("batch_count", batch_count_); + // TODO(AlexYue): If the size is 1001, it would be one delete with 1000 objects and one delete request with only one object(**ATTN**: DOESN'T + // recommend using delete objects when objects num is less than 10) + if (paths_.size() < 1000) { + return; + } + consume(); + } + +private: + void consume() { + std::unique_ptr> defer((int*)0x01, [this](int*) { + paths_.clear(); + copy_file_keys_.clear(); + batch_count_++; + }); + LOG_INFO("begin to delete {} internal stage objects in batch {}", paths_.size(), + batch_count_); + StopWatch sw; + // TODO(yuejing): 在accessor的delete_objets的实现里可以考虑如果_paths数量不超过10个的话,就直接发10个delete objection operation而不是发post + if (0 != accessor_->delete_objects(paths_)) { + LOG_WARNING("failed to delete {} internal stage objects in batch {} and it takes {} us", + paths_.size(), batch_count_, sw.elapsed_us()); + return; + } + LOG_INFO("succeed to delete {} internal stage objects in batch {} and it takes {} us", + paths_.size(), batch_count_, sw.elapsed_us()); + // delete fdb's keys + for (auto& file_keys : copy_file_keys_) { + auto& [log_trace, keys] = file_keys.second; + std::unique_ptr txn; + if (txn_kv_->create_txn(&txn) != cloud::TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create txn"; + continue; + } + // FIXME: We have already limited the file num and file meta size when selecting file in FE. + // And if too many copy files, begin_copy failed commit too. So here the copy file keys are + // limited, should not cause the txn commit failed. + for (const auto& key : keys) { + txn->remove(key); + LOG_INFO("remove copy_file_key={}, {}", hex(key), log_trace); + } + txn->remove(file_keys.first); + if (auto ret = txn->commit(); ret != cloud::TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to commit txn ret is " << ret; + continue; + } + } + } + std::shared_ptr accessor_; + // the path of the s3 files to be deleted + std::vector paths_; + struct CopyFiles { + std::string log_trace; + std::vector keys; + }; + // pair> + // first: instance_id_ stage_id table_id query_id + // second: keys to be deleted + // > + std::unordered_map copy_file_keys_; + // used to distinguish different batch tasks, the task log consists of thread ID and batch number + // which can together uniquely identifies different tasks for tracing log + uint64_t& batch_count_; + TxnKv* txn_kv_; +}; + +int InstanceRecycler::recycle_copy_jobs() { + int num_scanned = 0; + int num_finished = 0; + int num_expired = 0; + int num_recycled = 0; + // Used for INTERNAL stage's copy jobs to tag each batch for log trace + uint64_t batch_count = 0; + + LOG_INFO("begin to recycle copy jobs").tag("instance_id", instance_id_); + + using namespace std::chrono; + auto start_time = steady_clock::now(); + + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + auto cost = duration(steady_clock::now() - start_time).count(); + LOG_INFO("recycle copy jobs finished, cost={}s", cost) + .tag("instance_id", instance_id_) + .tag("num_scanned", num_scanned) + .tag("num_finished", num_finished) + .tag("num_expired", num_expired) + .tag("num_recycled", num_recycled); + }); + + CopyJobKeyInfo key_info0 {instance_id_, "", 0, "", 0}; + CopyJobKeyInfo key_info1 {instance_id_, "\xff", 0, "", 0}; + std::string key0; + std::string key1; + copy_job_key(key_info0, &key0); + copy_job_key(key_info1, &key1); + std::unordered_map> stage_accessor_map; + auto recycle_func = [&num_scanned, &num_finished, &num_expired, &num_recycled, &batch_count, + &stage_accessor_map, this](std::string_view k, std::string_view v) -> int { + ++num_scanned; + CopyJobPB copy_job; + if (!copy_job.ParseFromArray(v.data(), v.size())) { + LOG_WARNING("malformed copy job").tag("key", hex(k)); + return -1; + } + + // decode copy job key + auto k1 = k; + k1.remove_prefix(1); + std::vector, int, int>> out; + decode_key(&k1, &out); + // 0x01 "copy" ${instance_id} "job" ${stage_id} ${table_id} ${copy_id} ${group_id} + // -> CopyJobPB + const auto& stage_id = std::get(std::get<0>(out[3])); + const auto& table_id = std::get(std::get<0>(out[4])); + const auto& copy_id = std::get(std::get<0>(out[5])); + + bool check_storage = true; + if (copy_job.job_status() == CopyJobPB::FINISH) { + ++num_finished; + + if (copy_job.stage_type() == StagePB::INTERNAL) { + auto it = stage_accessor_map.find(stage_id); + std::shared_ptr accessor; + std::string_view path; + if (it != stage_accessor_map.end()) { + accessor = it->second; + } else { + std::shared_ptr inner_accessor; + auto ret = init_copy_job_accessor(stage_id, copy_job.stage_type(), + &inner_accessor); + if (ret < 0) { // error + LOG_WARNING("Failed to init_copy_job_accessor due to error code {}", ret); + return -1; + } else if (ret == 0) { + path = inner_accessor->path(); + accessor = std::make_shared( + inner_accessor, batch_count, txn_kv_.get()); + stage_accessor_map.emplace(stage_id, accessor); + } else { // stage not found, skip check storage + check_storage = false; + } + } + if (check_storage) { + // TODO delete objects with key and etag is not supported + accessor->add(std::move(copy_job), std::string(k), + {instance_id_, stage_id, table_id, copy_id, std::string(path)}); + return 0; + } + } else if (copy_job.stage_type() == StagePB::EXTERNAL) { + int64_t current_time = + duration_cast(system_clock::now().time_since_epoch()).count(); + if (copy_job.finish_time_ms() > 0) { + if (current_time < + copy_job.finish_time_ms() + config::copy_job_max_retention_second * 1000) { + return 0; + } + } else { + // For compatibility, copy job does not contain finish time before 2.2.2, use start time + if (current_time < + copy_job.start_time_ms() + config::copy_job_max_retention_second * 1000) { + return 0; + } + } + } + } else if (copy_job.job_status() == CopyJobPB::LOADING) { + int64_t current_time = + duration_cast(system_clock::now().time_since_epoch()).count(); + // if copy job is timeout: delete all copy file kvs and copy job kv + if (current_time <= copy_job.timeout_time_ms()) { + return 0; + } + ++num_expired; + } + + // delete all copy files + std::vector copy_file_keys; + for (auto& file : copy_job.object_files()) { + copy_file_keys.push_back(copy_file_key( + {instance_id_, stage_id, table_id, file.relative_path(), file.etag()})); + } + std::unique_ptr txn; + if (txn_kv_->create_txn(&txn) != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create txn"; + return -1; + } + // FIXME: We have already limited the file num and file meta size when selecting file in FE. + // And if too many copy files, begin_copy failed commit too. So here the copy file keys are + // limited, should not cause the txn commit failed. + for (const auto& key : copy_file_keys) { + txn->remove(key); + LOG(INFO) << "remove copy_file_key=" << hex(key) << ", instance_id=" << instance_id_ + << ", stage_id=" << stage_id << ", table_id=" << table_id + << ", query_id=" << copy_id; + } + txn->remove(k); + TxnErrorCode err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to commit txn, err=" << err; + return -1; + } + + ++num_recycled; + return 0; + }; + + return scan_and_recycle(key0, key1, std::move(recycle_func)); +} + +int InstanceRecycler::init_copy_job_accessor(const std::string& stage_id, + const StagePB::StageType& stage_type, + std::shared_ptr* accessor) { +#ifdef UNIT_TEST + // In unit test, external use the same accessor as the internal stage + auto it = accessor_map_.find(stage_id); + if (it != accessor_map_.end()) { + *accessor = it->second; + } else { + std::cout << "UT can not find accessor with stage_id: " << stage_id << std::endl; + return 1; + } +#else + // init s3 accessor and add to accessor map + bool found = false; + ObjectStoreInfoPB object_store_info; + StagePB::StageAccessType stage_access_type = StagePB::AKSK; + for (auto& s : instance_info_.stages()) { + if (s.stage_id() == stage_id) { + object_store_info = s.obj_info(); + if (s.has_access_type()) { + stage_access_type = s.access_type(); + } + found = true; + break; + } + } + if (!found) { + LOG(INFO) << "Recycle nonexisted stage copy jobs. instance_id=" << instance_id_ + << ", stage_id=" << stage_id << ", stage_type=" << stage_type; + return 1; + } + S3Conf s3_conf; + if (stage_type == StagePB::EXTERNAL) { + s3_conf.endpoint = object_store_info.endpoint(); + s3_conf.region = object_store_info.region(); + s3_conf.bucket = object_store_info.bucket(); + s3_conf.prefix = object_store_info.prefix(); + if (stage_access_type == StagePB::AKSK) { + s3_conf.ak = object_store_info.ak(); + s3_conf.sk = object_store_info.sk(); + if (object_store_info.has_encryption_info()) { + AkSkPair plain_ak_sk_pair; + int ret = decrypt_ak_sk_helper(object_store_info.ak(), object_store_info.sk(), + object_store_info.encryption_info(), + &plain_ak_sk_pair); + if (ret != 0) { + LOG(WARNING) << "fail to decrypt ak sk. instance_id: " << instance_id_ + << " obj_info: " << proto_to_json(object_store_info); + return -1; + } + s3_conf.ak = std::move(plain_ak_sk_pair.first); + s3_conf.sk = std::move(plain_ak_sk_pair.second); + } + } else if (stage_access_type == StagePB::BUCKET_ACL) { + s3_conf.ak = instance_info_.ram_user().ak(); + s3_conf.sk = instance_info_.ram_user().sk(); + if (instance_info_.ram_user().has_encryption_info()) { + AkSkPair plain_ak_sk_pair; + int ret = decrypt_ak_sk_helper( + instance_info_.ram_user().ak(), instance_info_.ram_user().sk(), + instance_info_.ram_user().encryption_info(), &plain_ak_sk_pair); + if (ret != 0) { + LOG(WARNING) << "fail to decrypt ak sk. instance_id: " << instance_id_ + << " ram_user: " << proto_to_json(instance_info_.ram_user()); + return -1; + } + s3_conf.ak = std::move(plain_ak_sk_pair.first); + s3_conf.sk = std::move(plain_ak_sk_pair.second); + } + } else { + LOG(INFO) << "Unsupported stage access type=" << stage_access_type + << ", instance_id=" << instance_id_ << ", stage_id=" << stage_id; + return -1; + } + } else if (stage_type == StagePB::INTERNAL) { + int idx = stoi(object_store_info.id()); + if (idx > instance_info_.obj_info().size() || idx < 1) { + LOG(WARNING) << "invalid idx: " << idx; + return -1; + } + auto& old_obj = instance_info_.obj_info()[idx - 1]; + s3_conf.ak = old_obj.ak(); + s3_conf.sk = old_obj.sk(); + if (old_obj.has_encryption_info()) { + AkSkPair plain_ak_sk_pair; + int ret = decrypt_ak_sk_helper(old_obj.ak(), old_obj.sk(), old_obj.encryption_info(), + &plain_ak_sk_pair); + if (ret != 0) { + LOG(WARNING) << "fail to decrypt ak sk. instance_id: " << instance_id_ + << " obj_info: " << proto_to_json(old_obj); + return -1; + } + s3_conf.ak = std::move(plain_ak_sk_pair.first); + s3_conf.sk = std::move(plain_ak_sk_pair.second); + } + s3_conf.endpoint = old_obj.endpoint(); + s3_conf.region = old_obj.region(); + s3_conf.bucket = old_obj.bucket(); + s3_conf.prefix = object_store_info.prefix(); + } else { + LOG(WARNING) << "unknown stage type " << stage_type; + return -1; + } + *accessor = std::make_shared(std::move(s3_conf)); + auto ret = (*accessor)->init(); + if (ret != 0) { + LOG(WARNING) << "failed to init s3 accessor ret=" << ret; + return -1; + } +#endif + return 0; +} + +int InstanceRecycler::recycle_stage() { + int num_scanned = 0; + int num_recycled = 0; + + LOG_INFO("begin to recycle stage").tag("instance_id", instance_id_); + + using namespace std::chrono; + auto start_time = steady_clock::now(); + + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + auto cost = duration(steady_clock::now() - start_time).count(); + LOG_INFO("recycle stage, cost={}s", cost) + .tag("instance_id", instance_id_) + .tag("num_scanned", num_scanned) + .tag("num_recycled", num_recycled); + }); + + RecycleStageKeyInfo key_info0 {instance_id_, ""}; + RecycleStageKeyInfo key_info1 {instance_id_, "\xff"}; + std::string key0; + std::string key1; + recycle_stage_key(key_info0, &key0); + recycle_stage_key(key_info1, &key1); + + // Elements in `tmp_rowset_keys` has the same lifetime as `it` + std::vector stage_keys; + auto recycle_func = [&num_scanned, &num_recycled, &stage_keys, this]( + std::string_view k, std::string_view v) -> int { + ++num_scanned; + RecycleStagePB recycle_stage; + if (!recycle_stage.ParseFromArray(v.data(), v.size())) { + LOG_WARNING("malformed recycle stage").tag("key", hex(k)); + return -1; + } + + int idx = stoi(recycle_stage.stage().obj_info().id()); + if (idx > instance_info_.obj_info().size() || idx < 1) { + LOG(WARNING) << "invalid idx: " << idx; + return -1; + } + auto& old_obj = instance_info_.obj_info()[idx - 1]; + S3Conf s3_conf; + s3_conf.ak = old_obj.ak(); + s3_conf.sk = old_obj.sk(); + if (old_obj.has_encryption_info()) { + AkSkPair plain_ak_sk_pair; + int ret = decrypt_ak_sk_helper(old_obj.ak(), old_obj.sk(), old_obj.encryption_info(), + &plain_ak_sk_pair); + if (ret != 0) { + LOG(WARNING) << "fail to decrypt ak sk. instance_id: " << instance_id_ + << " obj_info: " << proto_to_json(old_obj); + return -1; + } + s3_conf.ak = std::move(plain_ak_sk_pair.first); + s3_conf.sk = std::move(plain_ak_sk_pair.second); + } + s3_conf.endpoint = old_obj.endpoint(); + s3_conf.region = old_obj.region(); + s3_conf.bucket = old_obj.bucket(); + s3_conf.prefix = recycle_stage.stage().obj_info().prefix(); + std::shared_ptr accessor = + std::make_shared(std::move(s3_conf)); + TEST_SYNC_POINT_CALLBACK("recycle_stage:get_accessor", &accessor); + auto ret = accessor->init(); + if (ret != 0) { + LOG(WARNING) << "failed to init s3 accessor ret=" << ret; + return -1; + } + + LOG_INFO("begin to delete objects of dropped internal stage") + .tag("instance_id", instance_id_) + .tag("stage_id", recycle_stage.stage().stage_id()) + .tag("user_name", recycle_stage.stage().mysql_user_name()[0]) + .tag("user_id", recycle_stage.stage().mysql_user_id()[0]) + .tag("obj_info_id", idx) + .tag("prefix", recycle_stage.stage().obj_info().prefix()); + ret = accessor->delete_objects_by_prefix(""); + if (ret != 0) { + LOG(WARNING) << "failed to delete objects of dropped internal stage. instance_id=" + << instance_id_ << ", stage_id=" << recycle_stage.stage().stage_id() + << ", prefix=" << recycle_stage.stage().obj_info().prefix() + << ", ret=" << ret; + return -1; + } + ++num_recycled; + stage_keys.push_back(k); + return 0; + }; + + auto loop_done = [&stage_keys, this]() -> int { + if (stage_keys.empty()) return 0; + std::unique_ptr> defer((int*)0x01, + [&](int*) { stage_keys.clear(); }); + if (0 != txn_remove(txn_kv_.get(), stage_keys)) { + LOG(WARNING) << "failed to delete recycle partition kv, instance_id=" << instance_id_; + return -1; + } + return 0; + }; + + return scan_and_recycle(key0, key1, std::move(recycle_func), std::move(loop_done)); +} + +int InstanceRecycler::recycle_expired_stage_objects() { + LOG_INFO("begin to recycle expired stage objects").tag("instance_id", instance_id_); + + using namespace std::chrono; + auto start_time = steady_clock::now(); + + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + auto cost = duration(steady_clock::now() - start_time).count(); + LOG_INFO("recycle expired stage objects, cost={}s", cost).tag("instance_id", instance_id_); + }); + int ret = 0; + for (const auto& stage : instance_info_.stages()) { + if (stopped()) break; + if (stage.type() == StagePB::EXTERNAL) { + continue; + } + int idx = stoi(stage.obj_info().id()); + if (idx > instance_info_.obj_info().size() || idx < 1) { + LOG(WARNING) << "invalid idx: " << idx << ", id: " << stage.obj_info().id(); + continue; + } + auto& old_obj = instance_info_.obj_info()[idx - 1]; + S3Conf s3_conf; + s3_conf.ak = old_obj.ak(); + s3_conf.sk = old_obj.sk(); + if (old_obj.has_encryption_info()) { + AkSkPair plain_ak_sk_pair; + int ret1 = decrypt_ak_sk_helper(old_obj.ak(), old_obj.sk(), old_obj.encryption_info(), + &plain_ak_sk_pair); + if (ret1 != 0) { + LOG(WARNING) << "fail to decrypt ak sk " + << "obj_info:" << proto_to_json(old_obj); + } else { + s3_conf.ak = std::move(plain_ak_sk_pair.first); + s3_conf.sk = std::move(plain_ak_sk_pair.second); + } + } + s3_conf.endpoint = old_obj.endpoint(); + s3_conf.region = old_obj.region(); + s3_conf.bucket = old_obj.bucket(); + s3_conf.prefix = stage.obj_info().prefix(); + std::shared_ptr accessor = + std::make_shared(std::move(s3_conf)); + auto ret1 = accessor->init(); + if (ret1 != 0) { + LOG(WARNING) << "failed to init s3 accessor ret=" << ret1; + ret = -1; + continue; + } + + LOG(INFO) << "recycle expired stage objects, instance_id=" << instance_id_ + << ", stage_id=" << stage.stage_id() + << ", user_name=" << stage.mysql_user_name().at(0) + << ", user_id=" << stage.mysql_user_id().at(0) + << ", prefix=" << stage.obj_info().prefix(); + int64_t expired_time = + duration_cast(system_clock::now().time_since_epoch()).count() - + config::internal_stage_objects_expire_time_second; + ret1 = accessor->delete_expired_objects("", expired_time); + if (ret1 != 0) { + LOG(WARNING) << "failed to recycle expired stage objects, instance_id=" << instance_id_ + << ", stage_id=" << stage.stage_id() << ", ret=" << ret1; + ret = -1; + continue; + } + } + return ret; +} + +} // namespace doris::cloud diff --git a/cloud/src/recycler/recycler.h b/cloud/src/recycler/recycler.h new file mode 100644 index 00000000000000..34139be6d1a3db --- /dev/null +++ b/cloud/src/recycler/recycler.h @@ -0,0 +1,200 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include "recycler/white_black_list.h" + +namespace brpc { +class Server; +} // namespace brpc + +namespace doris::cloud { +class TxnKv; +class InstanceRecycler; +class ObjStoreAccessor; +class Checker; + +class Recycler { +public: + explicit Recycler(std::shared_ptr txn_kv); + ~Recycler(); + + // returns 0 for success otherwise error + int start(brpc::Server* server); + + void stop(); + + bool stopped() const { return stopped_.load(std::memory_order_acquire); } + +private: + void recycle_callback(); + + void instance_scanner_callback(); + + void lease_recycle_jobs(); + +private: + friend class RecyclerServiceImpl; + + std::shared_ptr txn_kv_; + std::atomic_bool stopped_ {false}; + + std::vector workers_; + + std::mutex mtx_; + // notify recycle workers + std::condition_variable pending_instance_cond_; + std::deque pending_instance_queue_; + std::unordered_set pending_instance_set_; + std::unordered_map> recycling_instance_map_; + // notify instance scanner and lease thread + std::condition_variable notifier_; + + std::string ip_port_; + + WhiteBlackList instance_filter_; + std::unique_ptr checker_; +}; + +class InstanceRecycler { +public: + explicit InstanceRecycler(std::shared_ptr txn_kv, const InstanceInfoPB& instance); + ~InstanceRecycler(); + + int init(); + + void stop() { stopped_.store(true, std::memory_order_release); } + bool stopped() const { return stopped_.load(std::memory_order_acquire); } + + // returns 0 for success otherwise error + int do_recycle(); + + // remove all kv and data in this instance, ONLY be called when instance has been deleted + // returns 0 for success otherwise error + int recycle_deleted_instance(); + + // scan and recycle expired indexes + // returns 0 for success otherwise error + int recycle_indexes(); + + // scan and recycle expired partitions + // returns 0 for success otherwise error + int recycle_partitions(); + + // scan and recycle expired rowsets + // returns 0 for success otherwise error + int recycle_rowsets(); + + // scan and recycle expired tmp rowsets + // returns 0 for success otherwise error + int recycle_tmp_rowsets(); + + /** + * recycle all tablets belonging to the index specified by `index_id` + * + * @param partition_id if positive, only recycle tablets in this partition belonging to the specified index + * @param is_empty_tablet indicates whether the tablet has object files, can skip delete objects if tablet is empty + * @return 0 for success otherwise error + */ + int recycle_tablets(int64_t table_id, int64_t index_id, int64_t partition_id = -1, + bool is_empty_tablet = false); + + /** + * recycle all rowsets belonging to the tablet specified by `tablet_id` + * + * @return 0 for success otherwise error + */ + int recycle_tablet(int64_t tablet_id); + + // scan and recycle useless partition version kv + int recycle_versions(); + + // scan and abort timeout txn label + // returns 0 for success otherwise error + int abort_timeout_txn(); + + //scan and recycle expire txn label + // returns 0 for success otherwise error + int recycle_expired_txn_label(); + + // scan and recycle finished or timeout copy jobs + // returns 0 for success otherwise error + int recycle_copy_jobs(); + + // scan and recycle dropped internal stage + // returns 0 for success otherwise error + int recycle_stage(); + + // scan and recycle expired stage objects + // returns 0 for success otherwise error + int recycle_expired_stage_objects(); + +private: + /** + * Scan key-value pairs between [`begin`, `end`), and perform `recycle_func` on each key-value pair. + * + * @param recycle_func defines how to recycle resources corresponding to a key-value pair. Returns 0 if the recycling is successful. + * @param loop_done is called after `RangeGetIterator` has no next kv. Usually used to perform a batch recycling. Returns 0 if success. + * @return 0 if all corresponding resources are recycled successfully, otherwise non-zero + */ + int scan_and_recycle(std::string begin, std::string_view end, + std::function recycle_func, + std::function loop_done = nullptr); + // return 0 for success otherwise error + int delete_rowset_data(const doris::RowsetMetaCloudPB& rs_meta_pb); + // return 0 for success otherwise error + // NOTE: this function ONLY be called when the file paths cannot be calculated + int delete_rowset_data(const std::string& resource_id, int64_t tablet_id, + const std::string& rowset_id); + // return 0 for success otherwise error + int delete_rowset_data(const std::vector& rowsets); + + /** + * Get stage storage info from instance and init ObjStoreAccessor + * @return 0 if accessor is successfully inited, 1 if stage not found, negative for error + */ + int init_copy_job_accessor(const std::string& stage_id, const StagePB::StageType& stage_type, + std::shared_ptr* accessor); + +private: + std::atomic_bool stopped_ {false}; + std::shared_ptr txn_kv_; + std::string instance_id_; + InstanceInfoPB instance_info_; + std::unordered_map> accessor_map_; + + class InvertedIndexIdCache; + std::unique_ptr inverted_index_id_cache_; + + std::mutex recycled_tablets_mtx_; + // Store recycled tablets, we can skip deleting rowset data of these tablets because these data has already been deleted. + std::unordered_set recycled_tablets_; +}; + +} // namespace doris::cloud diff --git a/cloud/src/recycler/recycler_service.cpp b/cloud/src/recycler/recycler_service.cpp new file mode 100644 index 00000000000000..0dc523312df228 --- /dev/null +++ b/cloud/src/recycler/recycler_service.cpp @@ -0,0 +1,408 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "recycler/recycler_service.h" + +#include +#include +#include +#include +#include + +#include "common/config.h" +#include "common/logging.h" +#include "common/util.h" +#include "meta-service/keys.h" +#include "meta-service/txn_kv_error.h" +#include "recycler/checker.h" +#include "recycler/meta_checker.h" +#include "recycler/recycler.h" + +namespace doris::cloud { + +extern std::tuple convert_ms_code_to_http_code(MetaServiceCode ret); + +RecyclerServiceImpl::RecyclerServiceImpl(std::shared_ptr txn_kv, Recycler* recycler, + Checker* checker) + : txn_kv_(std::move(txn_kv)), recycler_(recycler), checker_(checker) {} + +RecyclerServiceImpl::~RecyclerServiceImpl() = default; + +void RecyclerServiceImpl::recycle_instance(::google::protobuf::RpcController* controller, + const ::doris::cloud::RecycleInstanceRequest* request, + ::doris::cloud::RecycleInstanceResponse* response, + ::google::protobuf::Closure* done) { + auto ctrl = static_cast(controller); + LOG(INFO) << "rpc from " << ctrl->remote_side() << " request=" << request->ShortDebugString(); + brpc::ClosureGuard closure_guard(done); + MetaServiceCode code = MetaServiceCode::OK; + std::string msg = "OK"; + std::unique_ptr> defer_status( + (int*)0x01, [&code, &msg, &response, &ctrl](int*) { + response->mutable_status()->set_code(code); + response->mutable_status()->set_msg(msg); + LOG(INFO) << (code == MetaServiceCode::OK ? "succ to " : "failed to ") + << "recycle_instance" + << " " << ctrl->remote_side() << " " << msg; + }); + + std::vector instances; + instances.reserve(request->instance_ids_size()); + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = MetaServiceCode::KV_TXN_CREATE_ERR; + msg = "failed to create txn"; + return; + } + + for (auto& id : request->instance_ids()) { + InstanceKeyInfo key_info {id}; + std::string key; + instance_key(key_info, &key); + std::string val; + err = txn->get(key, &val); + if (err != TxnErrorCode::TXN_OK) { + code = MetaServiceCode::KV_TXN_GET_ERR; + msg = fmt::format("failed to get instance, instance_id={}, err={}", id, err); + LOG_WARNING(msg); + continue; + } + InstanceInfoPB instance; + if (!instance.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = fmt::format("malformed instance info, key={}, val={}", hex(key), hex(val)); + LOG_WARNING(msg); + continue; + } + instances.push_back(std::move(instance)); + } + { + std::lock_guard lock(recycler_->mtx_); + for (auto& i : instances) { + auto [_, success] = recycler_->pending_instance_set_.insert(i.instance_id()); + // skip instance already in pending queue + if (success) { + // TODO(plat1ko): Support high priority + recycler_->pending_instance_queue_.push_back(std::move(i)); + } + } + recycler_->pending_instance_cond_.notify_all(); + } +} + +void RecyclerServiceImpl::check_instance(const std::string& instance_id, MetaServiceCode& code, + std::string& msg) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = MetaServiceCode::KV_TXN_CREATE_ERR; + msg = "failed to create txn"; + return; + } + std::string key; + instance_key({instance_id}, &key); + std::string val; + err = txn->get(key, &val); + if (err != TxnErrorCode::TXN_OK) { + code = MetaServiceCode::KV_TXN_GET_ERR; + msg = fmt::format("failed to get instance, instance_id={}, err={}", instance_id, err); + return; + } + InstanceInfoPB instance; + if (!instance.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = fmt::format("malformed instance info, key={}", hex(key)); + return; + } + { + std::lock_guard lock(checker_->mtx_); + using namespace std::chrono; + auto enqueue_time_s = + duration_cast(system_clock::now().time_since_epoch()).count(); + auto [_, success] = checker_->pending_instance_map_.insert({instance_id, enqueue_time_s}); + // skip instance already in pending queue + if (success) { + // TODO(plat1ko): Support high priority + checker_->pending_instance_queue_.push_back(std::move(instance)); + } + checker_->pending_instance_cond_.notify_all(); + } +} + +void recycle_copy_jobs(const std::shared_ptr& txn_kv, const std::string& instance_id, + MetaServiceCode& code, std::string& msg) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = MetaServiceCode::KV_TXN_CREATE_ERR; + msg = "failed to create txn"; + return; + } + std::string key; + instance_key({instance_id}, &key); + std::string val; + err = txn->get(key, &val); + if (err != TxnErrorCode::TXN_OK) { + code = MetaServiceCode::KV_TXN_GET_ERR; + msg = fmt::format("failed to get instance, instance_id={}, err={}", instance_id, err); + return; + } + InstanceInfoPB instance; + if (!instance.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = fmt::format("malformed instance info, key={}", hex(key)); + return; + } + static std::mutex s_worker_mtx; + static std::set s_worker; + { + std::lock_guard lock(s_worker_mtx); + if (s_worker.size() >= config::recycle_concurrency) { // use another config entry? + msg = "exceeded the concurrency limit"; + return; + } + auto [_, success] = s_worker.insert(instance_id); + if (!success) { + msg = "recycle_copy_jobs not yet finished on this instance"; + return; + } + } + auto recycler = std::make_unique(txn_kv, instance); + std::thread worker([recycler = std::move(recycler), instance_id] { + LOG(INFO) << "manually trigger recycle_copy_jobs on instance " << instance_id; + recycler->recycle_copy_jobs(); + std::lock_guard lock(s_worker_mtx); + s_worker.erase(instance_id); + }); + worker.detach(); +} + +void recycle_job_info(const std::shared_ptr& txn_kv, const std::string& instance_id, + std::string_view key, MetaServiceCode& code, std::string& msg) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = MetaServiceCode::KV_TXN_CREATE_ERR; + msg = "failed to create txn"; + return; + } + std::string val; + err = txn->get(key, &val); + JobRecyclePB job_info; + if (err != TxnErrorCode::TXN_OK) { + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { // Not found, check instance existence + std::string key, val; + instance_key({instance_id}, &key); + err = txn->get(key, &val); + if (err == TxnErrorCode::TXN_OK) { // Never performed a recycle on this instance before + job_info.set_status(JobRecyclePB::IDLE); + job_info.set_last_ctime_ms(0); + job_info.set_last_finish_time_ms(0); + job_info.set_instance_id(instance_id); + msg = proto_to_json(job_info); + return; + } + } + code = MetaServiceCode::KV_TXN_GET_ERR; + msg = fmt::format("failed to get recycle job info, instance_id={}, err={}", instance_id, + err); + return; + } + if (!job_info.ParseFromString(val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = fmt::format("malformed job recycle value, key={}", hex(key)); + return; + } + msg = proto_to_json(job_info); +} + +void check_meta(const std::shared_ptr& txn_kv, const std::string& instance_id, + const std::string& host, const std::string& port, const std::string& user, + const std::string& password, std::string& msg) { +#ifdef BUILD_CHECK_META + std::unique_ptr meta_checker = std::make_unique(txn_kv); + meta_checker->do_check(host, port, user, password, instance_id, msg); +#else + msg = "check meta not build, please export BUILD_CHECK_META=ON before build cloud"; +#endif +} + +void RecyclerServiceImpl::http(::google::protobuf::RpcController* controller, + const ::doris::cloud::MetaServiceHttpRequest* request, + ::doris::cloud::MetaServiceHttpResponse* response, + ::google::protobuf::Closure* done) { + auto cntl = static_cast(controller); + LOG(INFO) << "rpc from " << cntl->remote_side() << " request: " << request->DebugString(); + brpc::ClosureGuard closure_guard(done); + MetaServiceCode code = MetaServiceCode::OK; + int status_code = 200; + std::string msg = "OK"; + std::string req; + std::string response_body; + std::string request_body; + std::unique_ptr> defer_status( + (int*)0x01, [&code, &msg, &status_code, &response_body, &cntl, &req](int*) { + status_code = std::get<0>(convert_ms_code_to_http_code(code)); + LOG(INFO) << (code == MetaServiceCode::OK ? "succ to " : "failed to ") << "http" + << " " << cntl->remote_side() << " request=\n" + << req << "\n ret=" << code << " msg=" << msg; + cntl->http_response().set_status_code(status_code); + cntl->response_attachment().append(response_body); + cntl->response_attachment().append("\n"); + }); + + // Prepare input request info + auto unresolved_path = cntl->http_request().unresolved_path(); + auto uri = cntl->http_request().uri(); + std::stringstream ss; + ss << "\nuri_path=" << uri.path(); + ss << "\nunresolved_path=" << unresolved_path; + ss << "\nmethod=" << brpc::HttpMethod2Str(cntl->http_request().method()); + ss << "\nquery strings:"; + for (auto it = uri.QueryBegin(); it != uri.QueryEnd(); ++it) { + ss << "\n" << it->first << "=" << it->second; + } + ss << "\nheaders:"; + for (auto it = cntl->http_request().HeaderBegin(); it != cntl->http_request().HeaderEnd(); + ++it) { + ss << "\n" << it->first << ":" << it->second; + } + req = ss.str(); + ss.clear(); + request_body = cntl->request_attachment().to_string(); // Just copy + + // Auth + auto token = uri.GetQuery("token"); + if (token == nullptr || *token != config::http_token) { + msg = "incorrect token, token=" + (token == nullptr ? std::string("(not given)") : *token); + response_body = "incorrect token"; + status_code = 403; + return; + } + + if (unresolved_path == "recycle_instance") { + RecycleInstanceRequest req; + auto st = google::protobuf::util::JsonStringToMessage(request_body, &req); + if (!st.ok()) { + msg = "failed to RecycleInstanceRequest, error: " + st.message().ToString(); + response_body = msg; + LOG(WARNING) << msg; + return; + } + RecycleInstanceResponse res; + recycle_instance(cntl, &req, &res, nullptr); + code = res.status().code(); + msg = res.status().msg(); + response_body = msg; + return; + } + + if (unresolved_path == "recycle_copy_jobs") { + auto instance_id = uri.GetQuery("instance_id"); + if (instance_id == nullptr || instance_id->empty()) { + msg = "no instance id"; + response_body = msg; + status_code = 400; + return; + } + recycle_copy_jobs(txn_kv_, *instance_id, code, msg); + response_body = msg; + return; + } + + if (unresolved_path == "recycle_job_info") { + auto instance_id = uri.GetQuery("instance_id"); + if (instance_id == nullptr || instance_id->empty()) { + msg = "no instance id"; + response_body = msg; + status_code = 400; + return; + } + std::string key; + job_recycle_key({*instance_id}, &key); + recycle_job_info(txn_kv_, *instance_id, key, code, msg); + response_body = msg; + return; + } + + if (unresolved_path == "check_instance") { + auto instance_id = uri.GetQuery("instance_id"); + if (instance_id == nullptr || instance_id->empty()) { + msg = "no instance id"; + response_body = msg; + status_code = 400; + return; + } + if (!checker_) { + msg = "checker not enabled"; + response_body = msg; + status_code = 400; + return; + } + check_instance(*instance_id, code, msg); + response_body = msg; + return; + } + + if (unresolved_path == "check_job_info") { + auto instance_id = uri.GetQuery("instance_id"); + if (instance_id == nullptr || instance_id->empty()) { + msg = "no instance id"; + response_body = msg; + status_code = 400; + return; + } + std::string key; + job_check_key({*instance_id}, &key); + recycle_job_info(txn_kv_, *instance_id, key, code, msg); + response_body = msg; + return; + } + + if (unresolved_path == "check_meta") { + auto instance_id = uri.GetQuery("instance_id"); + auto host = uri.GetQuery("host"); + auto port = uri.GetQuery("port"); + auto user = uri.GetQuery("user"); + auto password = uri.GetQuery("password"); + LOG(INFO) << " host " << *host; + LOG(INFO) << " port " << *port; + LOG(INFO) << " user " << *user; + LOG(INFO) << " passwd " << *password; + LOG(INFO) << " instance " << *instance_id; + if (instance_id == nullptr || instance_id->empty() || host == nullptr || host->empty() || + port == nullptr || port->empty() || password == nullptr || user == nullptr || + user->empty()) { + msg = "no instance id or mysql conn str info"; + response_body = msg; + status_code = 400; + return; + } + check_meta(txn_kv_, *instance_id, *host, *port, *user, *password, msg); + status_code = 200; + response_body = msg; + return; + } + + status_code = 404; + msg = "not found"; + response_body = msg; +} + +} // namespace doris::cloud diff --git a/cloud/src/recycler/recycler_service.h b/cloud/src/recycler/recycler_service.h new file mode 100644 index 00000000000000..8f9faec1efeb7d --- /dev/null +++ b/cloud/src/recycler/recycler_service.h @@ -0,0 +1,53 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include + +#include "meta-service/txn_kv.h" + +namespace doris::cloud { + +class Recycler; +class Checker; + +class RecyclerServiceImpl : public cloud::RecyclerService { +public: + RecyclerServiceImpl(std::shared_ptr txn_kv, Recycler* recycler, Checker* checker); + ~RecyclerServiceImpl() override; + + void recycle_instance(::google::protobuf::RpcController* controller, + const ::doris::cloud::RecycleInstanceRequest* request, + ::doris::cloud::RecycleInstanceResponse* response, + ::google::protobuf::Closure* done) override; + + void http(::google::protobuf::RpcController* controller, + const ::doris::cloud::MetaServiceHttpRequest* request, + ::doris::cloud::MetaServiceHttpResponse* response, + ::google::protobuf::Closure* done) override; + +private: + void check_instance(const std::string& instance_id, MetaServiceCode& code, std::string& msg); + +private: + std::shared_ptr txn_kv_; + Recycler* recycler_; // Ref + Checker* checker_; // Ref +}; + +} // namespace doris::cloud diff --git a/cloud/src/recycler/s3_accessor.cpp b/cloud/src/recycler/s3_accessor.cpp new file mode 100644 index 00000000000000..f70023d0a72bd5 --- /dev/null +++ b/cloud/src/recycler/s3_accessor.cpp @@ -0,0 +1,424 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "recycler/s3_accessor.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include "common/logging.h" +#include "common/sync_point.h" + +namespace doris::cloud { +#ifndef UNIT_TEST +#define HELP_MACRO(ret, req, point_name) +#else +#define HELP_MACRO(ret, req, point_name) \ + do { \ + std::pair p {&ret, &req}; \ + [[maybe_unused]] auto ret_pair = [&p]() mutable { \ + TEST_SYNC_POINT_RETURN_WITH_VALUE(point_name, &p); \ + return p; \ + }(); \ + return ret; \ + } while (false) +#endif +#define SYNC_POINT_HOOK_RETURN_VALUE(expr, point_name, req) \ + [&]() mutable { \ + [[maybe_unused]] decltype((expr)) t; \ + HELP_MACRO(t, req, point_name); \ + return (expr); \ + }() + +class S3Environment { +public: + S3Environment() { Aws::InitAPI(aws_options_); } + + ~S3Environment() { Aws::ShutdownAPI(aws_options_); } + +private: + Aws::SDKOptions aws_options_; +}; + +S3Accessor::S3Accessor(S3Conf conf) : conf_(std::move(conf)) { + path_ = conf_.endpoint + '/' + conf_.bucket + '/' + conf_.prefix; +} + +S3Accessor::~S3Accessor() = default; + +std::string S3Accessor::get_key(const std::string& relative_path) const { + return conf_.prefix + '/' + relative_path; +} + +std::string S3Accessor::get_relative_path(const std::string& key) const { + return key.find(conf_.prefix + "/") != 0 ? "" : key.substr(conf_.prefix.length() + 1); +} + +int S3Accessor::init() { + static S3Environment s3_env; + Aws::Auth::AWSCredentials aws_cred(conf_.ak, conf_.sk); + Aws::Client::ClientConfiguration aws_config; + aws_config.endpointOverride = conf_.endpoint; + aws_config.region = conf_.region; + aws_config.retryStrategy = std::make_shared( + /*maxRetries = 10, scaleFactor = 25*/); + s3_client_ = std::make_shared( + std::move(aws_cred), std::move(aws_config), + Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, + true /* useVirtualAddressing */); + return 0; +} + +int S3Accessor::delete_objects_by_prefix(const std::string& relative_path) { + Aws::S3::Model::ListObjectsV2Request request; + auto prefix = get_key(relative_path); + request.WithBucket(conf_.bucket).WithPrefix(prefix); + + Aws::S3::Model::DeleteObjectsRequest delete_request; + delete_request.SetBucket(conf_.bucket); + bool is_truncated = false; + do { + auto outcome = SYNC_POINT_HOOK_RETURN_VALUE(s3_client_->ListObjectsV2(request), + "s3_client::list_objects_v2", request); + if (!outcome.IsSuccess()) { + LOG_WARNING("failed to list objects") + .tag("endpoint", conf_.endpoint) + .tag("bucket", conf_.bucket) + .tag("prefix", prefix) + .tag("responseCode", static_cast(outcome.GetError().GetResponseCode())) + .tag("error", outcome.GetError().GetMessage()); + if (outcome.GetError().GetResponseCode() == Aws::Http::HttpResponseCode::FORBIDDEN) { + return 1; + } + return -1; + } + const auto& result = outcome.GetResult(); + VLOG_DEBUG << "get " << result.GetContents().size() << " objects"; + Aws::Vector objects; + objects.reserve(result.GetContents().size()); + for (const auto& obj : result.GetContents()) { + objects.emplace_back().SetKey(obj.GetKey()); + LOG_INFO("delete object") + .tag("endpoint", conf_.endpoint) + .tag("bucket", conf_.bucket) + .tag("key", obj.GetKey()); + } + if (!objects.empty()) { + Aws::S3::Model::Delete del; + del.WithObjects(std::move(objects)).SetQuiet(true); + delete_request.SetDelete(std::move(del)); + auto delete_outcome = + SYNC_POINT_HOOK_RETURN_VALUE(s3_client_->DeleteObjects(delete_request), + "s3_client::delete_objects", delete_request); + if (!delete_outcome.IsSuccess()) { + LOG_WARNING("failed to delete objects") + .tag("endpoint", conf_.endpoint) + .tag("bucket", conf_.bucket) + .tag("prefix", prefix) + .tag("responseCode", static_cast(outcome.GetError().GetResponseCode())) + .tag("error", outcome.GetError().GetMessage()); + if (delete_outcome.GetError().GetResponseCode() == + Aws::Http::HttpResponseCode::FORBIDDEN) { + return 1; + } + return -2; + } + if (!delete_outcome.GetResult().GetErrors().empty()) { + const auto& e = delete_outcome.GetResult().GetErrors().front(); + LOG_WARNING("failed to delete object") + .tag("endpoint", conf_.endpoint) + .tag("bucket", conf_.bucket) + .tag("key", e.GetKey()) + .tag("responseCode", static_cast(outcome.GetError().GetResponseCode())) + .tag("error", e.GetMessage()); + return -3; + } + } + is_truncated = result.GetIsTruncated(); + request.SetContinuationToken(result.GetNextContinuationToken()); + } while (is_truncated); + return 0; +} + +int S3Accessor::delete_objects(const std::vector& relative_paths) { + if (relative_paths.empty()) { + return 0; + } + // `DeleteObjectsRequest` can only contain 1000 keys at most. + constexpr size_t max_delete_batch = 1000; + auto path_iter = relative_paths.begin(); + + Aws::S3::Model::DeleteObjectsRequest delete_request; + delete_request.SetBucket(conf_.bucket); + do { + Aws::S3::Model::Delete del; + Aws::Vector objects; + auto path_begin = path_iter; + for (; path_iter != relative_paths.end() && (path_iter - path_begin < max_delete_batch); + ++path_iter) { + auto key = get_key(*path_iter); + LOG_INFO("delete object") + .tag("endpoint", conf_.endpoint) + .tag("bucket", conf_.bucket) + .tag("key", key); + objects.emplace_back().SetKey(std::move(key)); + } + if (objects.empty()) { + return 0; + } + del.WithObjects(std::move(objects)).SetQuiet(true); + delete_request.SetDelete(std::move(del)); + auto delete_outcome = + SYNC_POINT_HOOK_RETURN_VALUE(s3_client_->DeleteObjects(delete_request), + "s3_client::delete_objects", delete_request); + if (!delete_outcome.IsSuccess()) { + LOG_WARNING("failed to delete objects") + .tag("endpoint", conf_.endpoint) + .tag("bucket", conf_.bucket) + .tag("key[0]", delete_request.GetDelete().GetObjects().front().GetKey()) + .tag("responseCode", + static_cast(delete_outcome.GetError().GetResponseCode())) + .tag("error", delete_outcome.GetError().GetMessage()); + return -1; + } + if (!delete_outcome.GetResult().GetErrors().empty()) { + const auto& e = delete_outcome.GetResult().GetErrors().front(); + LOG_WARNING("failed to delete object") + .tag("endpoint", conf_.endpoint) + .tag("bucket", conf_.bucket) + .tag("key", e.GetKey()) + .tag("responseCode", + static_cast(delete_outcome.GetError().GetResponseCode())) + .tag("error", e.GetMessage()); + return -2; + } + } while (path_iter != relative_paths.end()); + + return 0; +} + +int S3Accessor::delete_object(const std::string& relative_path) { + // TODO(cyx) + return 0; +} + +int S3Accessor::put_object(const std::string& relative_path, const std::string& content) { + Aws::S3::Model::PutObjectRequest request; + auto key = get_key(relative_path); + request.WithBucket(conf_.bucket).WithKey(key); + auto input = Aws::MakeShared("S3Accessor"); + *input << content; + request.SetBody(input); + auto outcome = SYNC_POINT_HOOK_RETURN_VALUE(s3_client_->PutObject(request), + "s3_client::put_object", request); + if (!outcome.IsSuccess()) { + LOG_WARNING("failed to put object") + .tag("endpoint", conf_.endpoint) + .tag("bucket", conf_.bucket) + .tag("key", key) + .tag("responseCode", static_cast(outcome.GetError().GetResponseCode())) + .tag("error", outcome.GetError().GetMessage()); + return -1; + } + return 0; +} + +int S3Accessor::list(const std::string& relative_path, std::vector* files) { + Aws::S3::Model::ListObjectsV2Request request; + auto prefix = get_key(relative_path); + request.WithBucket(conf_.bucket).WithPrefix(prefix); + + bool is_truncated = false; + do { + auto outcome = SYNC_POINT_HOOK_RETURN_VALUE(s3_client_->ListObjectsV2(request), + "s3_client::list_objects_v2", request); + ; + if (!outcome.IsSuccess()) { + LOG_WARNING("failed to list objects") + .tag("endpoint", conf_.endpoint) + .tag("bucket", conf_.bucket) + .tag("prefix", prefix) + .tag("responseCode", static_cast(outcome.GetError().GetResponseCode())) + .tag("error", outcome.GetError().GetMessage()); + return -1; + } + const auto& result = outcome.GetResult(); + VLOG_DEBUG << "get " << result.GetContents().size() << " objects"; + for (const auto& obj : result.GetContents()) { + files->push_back({obj.GetKey().substr(conf_.prefix.size() + 1), obj.GetSize()}); + } + is_truncated = result.GetIsTruncated(); + request.SetContinuationToken(result.GetNextContinuationToken()); + } while (is_truncated); + return 0; +} + +int S3Accessor::exist(const std::string& relative_path) { + Aws::S3::Model::HeadObjectRequest request; + auto key = get_key(relative_path); + request.WithBucket(conf_.bucket).WithKey(key); + auto outcome = SYNC_POINT_HOOK_RETURN_VALUE(s3_client_->HeadObject(request), + "s3_client::head_object", request); + ; + if (outcome.IsSuccess()) { + return 0; + } else if (outcome.GetError().GetResponseCode() == Aws::Http::HttpResponseCode::NOT_FOUND) { + return 1; + } else { + LOG_WARNING("failed to head object") + .tag("endpoint", conf_.endpoint) + .tag("bucket", conf_.bucket) + .tag("key", key) + .tag("responseCode", static_cast(outcome.GetError().GetResponseCode())) + .tag("error", outcome.GetError().GetMessage()); + return -1; + } +} + +int S3Accessor::delete_expired_objects(const std::string& relative_path, int64_t expired_time) { + Aws::S3::Model::ListObjectsV2Request request; + auto prefix = get_key(relative_path); + request.WithBucket(conf_.bucket).WithPrefix(prefix); + + bool is_truncated = false; + do { + auto outcome = SYNC_POINT_HOOK_RETURN_VALUE(s3_client_->ListObjectsV2(request), + "s3_client::list_objects_v2", request); + ; + if (!outcome.IsSuccess()) { + LOG_WARNING("failed to list objects") + .tag("endpoint", conf_.endpoint) + .tag("bucket", conf_.bucket) + .tag("prefix", prefix) + .tag("responseCode", static_cast(outcome.GetError().GetResponseCode())) + .tag("error", outcome.GetError().GetMessage()); + return -1; + } + const auto& result = outcome.GetResult(); + std::vector expired_keys; + for (const auto& obj : result.GetContents()) { + if (obj.GetLastModified().Seconds() < expired_time) { + auto relative_key = get_relative_path(obj.GetKey()); + if (relative_key.empty()) { + LOG_WARNING("failed get relative path") + .tag("prefix", conf_.prefix) + .tag("key", obj.GetKey()); + } else { + expired_keys.push_back(relative_key); + LOG_INFO("delete expired object") + .tag("prefix", conf_.prefix) + .tag("key", obj.GetKey()) + .tag("relative_key", relative_key) + .tag("lastModifiedTime", obj.GetLastModified().Seconds()) + .tag("expiredTime", expired_time); + } + } + } + + auto ret = delete_objects(expired_keys); + if (ret != 0) { + return ret; + } + LOG_INFO("delete expired objects") + .tag("endpoint", conf_.endpoint) + .tag("bucket", conf_.bucket) + .tag("prefix", conf_.prefix) + .tag("num_scanned", result.GetContents().size()) + .tag("num_recycled", expired_keys.size()); + is_truncated = result.GetIsTruncated(); + request.SetContinuationToken(result.GetNextContinuationToken()); + } while (is_truncated); + return 0; +} + +int S3Accessor::get_bucket_lifecycle(int64_t* expiration_days) { + Aws::S3::Model::GetBucketLifecycleConfigurationRequest request; + request.SetBucket(conf_.bucket); + + auto outcome = + SYNC_POINT_HOOK_RETURN_VALUE(s3_client_->GetBucketLifecycleConfiguration(request), + "s3_client::get_bucket_lifecycle_configuration", request); + bool has_lifecycle = false; + if (outcome.IsSuccess()) { + const auto& rules = outcome.GetResult().GetRules(); + for (const auto& rule : rules) { + if (rule.NoncurrentVersionExpirationHasBeenSet()) { + has_lifecycle = true; + *expiration_days = rule.GetNoncurrentVersionExpiration().GetNoncurrentDays(); + } + } + } else { + LOG_WARNING("Err for check interval: failed to get bucket lifecycle") + .tag("endpoint", conf_.endpoint) + .tag("bucket", conf_.bucket) + .tag("prefix", conf_.prefix) + .tag("responseCode", static_cast(outcome.GetError().GetResponseCode())) + .tag("error", outcome.GetError().GetMessage()); + return -1; + } + + if (!has_lifecycle) { + LOG_WARNING("Err for check interval: bucket doesn't have lifecycle configuration") + .tag("endpoint", conf_.endpoint) + .tag("bucket", conf_.bucket) + .tag("prefix", conf_.prefix); + return -1; + } + return 0; +} + +int S3Accessor::check_bucket_versioning() { + Aws::S3::Model::GetBucketVersioningRequest request; + request.SetBucket(conf_.bucket); + auto outcome = SYNC_POINT_HOOK_RETURN_VALUE(s3_client_->GetBucketVersioning(request), + "s3_client::get_bucket_versioning", request); + + if (outcome.IsSuccess()) { + const auto& versioning_configuration = outcome.GetResult().GetStatus(); + if (versioning_configuration != Aws::S3::Model::BucketVersioningStatus::Enabled) { + LOG_WARNING("Err for check interval: bucket doesn't enable bucket versioning") + .tag("endpoint", conf_.endpoint) + .tag("bucket", conf_.bucket) + .tag("prefix", conf_.prefix); + return -1; + } + } else { + LOG_WARNING("Err for check interval: failed to get status of bucket versioning") + .tag("endpoint", conf_.endpoint) + .tag("bucket", conf_.bucket) + .tag("prefix", conf_.prefix) + .tag("responseCode", static_cast(outcome.GetError().GetResponseCode())) + .tag("error", outcome.GetError().GetMessage()); + return -1; + } + return 0; +} + +#undef HELP_MACRO +} // namespace doris::cloud diff --git a/cloud/src/recycler/s3_accessor.h b/cloud/src/recycler/s3_accessor.h new file mode 100644 index 00000000000000..98866f8ba0166c --- /dev/null +++ b/cloud/src/recycler/s3_accessor.h @@ -0,0 +1,138 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include +#include + +namespace Aws::S3 { +class S3Client; +} // namespace Aws::S3 + +namespace doris::cloud { + +struct ObjectMeta { + std::string path; // Relative path + int64_t size {0}; +}; + +class ObjStoreAccessor { +public: + ObjStoreAccessor() = default; + virtual ~ObjStoreAccessor() = default; + + virtual const std::string& path() const = 0; + + // returns 0 for success otherwise error + virtual int init() = 0; + + // returns 0 for success otherwise error + virtual int delete_objects_by_prefix(const std::string& relative_path) = 0; + + // returns 0 for success otherwise error + virtual int delete_objects(const std::vector& relative_paths) = 0; + + // returns 0 for success otherwise error + virtual int delete_object(const std::string& relative_path) = 0; + + // for test + // returns 0 for success otherwise error + virtual int put_object(const std::string& relative_path, const std::string& content) = 0; + + // returns 0 for success otherwise error + virtual int list(const std::string& relative_path, std::vector* files) = 0; + + // return 0 if object exists, 1 if object is not found, negative for error + virtual int exist(const std::string& relative_path) = 0; + + // delete objects which last modified time is less than the input expired time and under the input relative path + // returns 0 for success otherwise error + virtual int delete_expired_objects(const std::string& relative_path, int64_t expired_time) = 0; + + // return 0 for success otherwise error + virtual int get_bucket_lifecycle(int64_t* expiration_days) = 0; + + // returns 0 for enabling bucket versioning, otherwise error + virtual int check_bucket_versioning() = 0; +}; + +struct S3Conf { + std::string ak; + std::string sk; + std::string endpoint; + std::string region; + std::string bucket; + std::string prefix; +}; + +class S3Accessor : public ObjStoreAccessor { +public: + explicit S3Accessor(S3Conf conf); + ~S3Accessor() override; + + const std::string& path() const override { return path_; } + + const std::shared_ptr& s3_client() const { return s3_client_; } + + const S3Conf& conf() const { return conf_; } + + // returns 0 for success otherwise error + int init() override; + + // returns 0 for success, returns 1 for http FORBIDDEN error, negative for other errors + int delete_objects_by_prefix(const std::string& relative_path) override; + + // returns 0 for success otherwise error + int delete_objects(const std::vector& relative_paths) override; + + // returns 0 for success otherwise error + int delete_object(const std::string& relative_path) override; + + // for test + // returns 0 for success otherwise error + int put_object(const std::string& relative_path, const std::string& content) override; + + // returns 0 for success otherwise error + int list(const std::string& relative_path, std::vector* ObjectMeta) override; + + // return 0 if object exists, 1 if object is not found, otherwise error + int exist(const std::string& relative_path) override; + + // delete objects which last modified time is less than the input expired time and under the input relative path + // returns 0 for success otherwise error + int delete_expired_objects(const std::string& relative_path, int64_t expired_time) override; + + // returns 0 for success otherwise error + int get_bucket_lifecycle(int64_t* expiration_days) override; + + // returns 0 for enabling bucket versioning, otherwise error + int check_bucket_versioning() override; + +private: + std::string get_key(const std::string& relative_path) const; + // return empty string if the input key does not start with the prefix of S3 conf + std::string get_relative_path(const std::string& key) const; + +private: + std::shared_ptr s3_client_; + S3Conf conf_; + std::string path_; +}; + +} // namespace doris::cloud diff --git a/cloud/src/recycler/util.cpp b/cloud/src/recycler/util.cpp new file mode 100644 index 00000000000000..123c3c15116cb3 --- /dev/null +++ b/cloud/src/recycler/util.cpp @@ -0,0 +1,224 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "recycler/util.h" + +#include + +#include + +#include "common/util.h" +#include "meta-service/keys.h" +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" + +namespace doris::cloud { +namespace config { +extern int32_t recycle_job_lease_expired_ms; +} // namespace config + +int get_all_instances(TxnKv* txn_kv, std::vector& res) { + InstanceKeyInfo key0_info {""}; + InstanceKeyInfo key1_info {"\xff"}; // instance id are human readable strings + std::string key0; + std::string key1; + instance_key(key0_info, &key0); + instance_key(key1_info, &key1); + + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(INFO) << "failed to init txn, err=" << err; + return -1; + } + + std::unique_ptr it; + do { + TxnErrorCode err = txn->get(key0, key1, &it); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to get instance, err=" << err; + return -1; + } + + while (it->has_next()) { + auto [k, v] = it->next(); + if (!it->has_next()) key0 = k; + + InstanceInfoPB instance_info; + if (!instance_info.ParseFromArray(v.data(), v.size())) { + LOG(WARNING) << "malformed instance info, key=" << hex(k); + return -1; + } + res.push_back(std::move(instance_info)); + } + key0.push_back('\x00'); // Update to next smallest key for iteration + } while (it->more()); + + return 0; +} + +int prepare_instance_recycle_job(TxnKv* txn_kv, std::string_view key, + const std::string& instance_id, const std::string& ip_port, + int64_t interval_ms) { + std::string val; + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create txn"; + return -1; + } + err = txn->get(key, &val); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + LOG(WARNING) << "failed to get kv, err=" << err << " key=" << hex(key); + return -1; + } + using namespace std::chrono; + auto now = duration_cast(system_clock::now().time_since_epoch()).count(); + JobRecyclePB job_info; + + auto is_expired = [&]() { + if (!job_info.ParseFromString(val)) { + LOG(WARNING) << "failed to parse JobRecyclePB, key=" << hex(key); + // if failed to parse, just recycle it. + return true; + } + DCHECK(job_info.instance_id() == instance_id); + bool lease_expired = + job_info.status() == JobRecyclePB::BUSY && job_info.expiration_time_ms() < now; + bool finish_expired = job_info.status() == JobRecyclePB::IDLE && + now - job_info.last_ctime_ms() > interval_ms; + return lease_expired || finish_expired; + }; + + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND || is_expired()) { + job_info.set_status(JobRecyclePB::BUSY); + job_info.set_instance_id(instance_id); + job_info.set_ip_port(ip_port); + job_info.set_expiration_time_ms(now + config::recycle_job_lease_expired_ms); + val = job_info.SerializeAsString(); + txn->put(key, val); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to commit, err=" << err << " key=" << hex(key); + return -1; + } + return 0; + } + return 1; +} + +void finish_instance_recycle_job(TxnKv* txn_kv, std::string_view key, + const std::string& instance_id, const std::string& ip_port, + bool success, int64_t ctime_ms) { + std::string val; + int retry_times = 0; + do { + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create txn"; + return; + } + err = txn->get(key, &val); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to get kv, err=" << err << " key=" << hex(key); + return; + } + + using namespace std::chrono; + auto now = duration_cast(system_clock::now().time_since_epoch()).count(); + JobRecyclePB job_info; + if (!job_info.ParseFromString(val)) { + LOG(WARNING) << "failed to parse JobRecyclePB, key=" << hex(key); + return; + } + DCHECK(job_info.instance_id() == instance_id); + if (job_info.ip_port() != ip_port) { + LOG(WARNING) << "job is doing at other machine: " << job_info.ip_port() + << " key=" << hex(key); + return; + } + if (job_info.status() != JobRecyclePB::BUSY) { + LOG(WARNING) << "job is not busy, key=" << hex(key); + return; + } + job_info.set_status(JobRecyclePB::IDLE); + job_info.set_instance_id(instance_id); + job_info.set_last_finish_time_ms(now); + job_info.set_last_ctime_ms(ctime_ms); + if (success) { + job_info.set_last_success_time_ms(now); + } + val = job_info.SerializeAsString(); + txn->put(key, val); + err = txn->commit(); + if (err == TxnErrorCode::TXN_OK) { + LOG(INFO) << "succ to commit to finish recycle job, key=" << hex(key); + return; + } + // maybe conflict with the commit of the leased thread + LOG(WARNING) << "failed to commit to finish recycle job, err=" << err << " key=" << hex(key) + << " retry_times=" << retry_times; + } while (retry_times++ < 3); + LOG(WARNING) << "finally failed to commit to finish recycle job, key=" << hex(key); +} + +int lease_instance_recycle_job(TxnKv* txn_kv, std::string_view key, const std::string& instance_id, + const std::string& ip_port) { + std::string val; + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create txn"; + return -1; + } + err = txn->get(key, &val); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to get kv, err=" << err << " key=" << hex(key); + return -1; + } + + using namespace std::chrono; + auto now = duration_cast(system_clock::now().time_since_epoch()).count(); + JobRecyclePB job_info; + if (!job_info.ParseFromString(val)) { + LOG(WARNING) << "failed to parse JobRecyclePB, key=" << hex(key); + return 1; + } + DCHECK(job_info.instance_id() == instance_id); + if (job_info.ip_port() != ip_port) { + LOG(WARNING) << "job is doing at other machine: " << job_info.ip_port() + << " key=" << hex(key); + return 1; + } + if (job_info.status() != JobRecyclePB::BUSY) { + LOG(WARNING) << "job is not busy, key=" << hex(key); + return 1; + } + job_info.set_expiration_time_ms(now + config::recycle_job_lease_expired_ms); + val = job_info.SerializeAsString(); + txn->put(key, val); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to commit, failed to lease recycle job, err=" << err + << " key=" << hex(key); + return -1; + } + return 0; +} + +} // namespace doris::cloud diff --git a/cloud/src/recycler/util.h b/cloud/src/recycler/util.h new file mode 100644 index 00000000000000..20ea66def8c0b7 --- /dev/null +++ b/cloud/src/recycler/util.h @@ -0,0 +1,72 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include + +#include + +namespace doris::cloud { + +class TxnKv; + +/** + * Get all instances, include DELETED instance + * @return 0 for success, otherwise error + */ +int get_all_instances(TxnKv* txn_kv, std::vector& res); + +/** + * + * @return 0 for success + */ +int prepare_instance_recycle_job(TxnKv* txn_kv, std::string_view key, + const std::string& instance_id, const std::string& ip_port, + int64_t interval_ms); + +void finish_instance_recycle_job(TxnKv* txn_kv, std::string_view key, + const std::string& instance_id, const std::string& ip_port, + bool success, int64_t ctime_ms); + +/** + * + * @return 0 for success, 1 if job should be aborted, negative for other errors + */ +int lease_instance_recycle_job(TxnKv* txn_kv, std::string_view key, const std::string& instance_id, + const std::string& ip_port); + +inline std::string segment_path(int64_t tablet_id, const std::string& rowset_id, + int64_t segment_id) { + return fmt::format("data/{}/{}_{}.dat", tablet_id, rowset_id, segment_id); +} + +inline std::string inverted_index_path(int64_t tablet_id, const std::string& rowset_id, + int64_t segment_id, int64_t index_id) { + return fmt::format("data/{}/{}_{}_{}.idx", tablet_id, rowset_id, segment_id, index_id); +} + +inline std::string rowset_path_prefix(int64_t tablet_id, const std::string& rowset_id) { + return fmt::format("data/{}/{}_", tablet_id, rowset_id); +} + +inline std::string tablet_path_prefix(int64_t tablet_id) { + return fmt::format("data/{}/", tablet_id); +} + +} // namespace doris::cloud diff --git a/cloud/src/recycler/white_black_list.cpp b/cloud/src/recycler/white_black_list.cpp new file mode 100644 index 00000000000000..b7dc4a8099e635 --- /dev/null +++ b/cloud/src/recycler/white_black_list.cpp @@ -0,0 +1,44 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "recycler/white_black_list.h" + +namespace doris::cloud { + +void WhiteBlackList::reset(const std::vector& whitelist, + const std::vector& blacklist) { + blacklist_.clear(); + whitelist_.clear(); + if (!whitelist.empty()) { + for (const auto& str : whitelist) { + whitelist_.insert(str); + } + } else { + for (const auto& str : blacklist) { + blacklist_.insert(str); + } + } +} + +bool WhiteBlackList::filter_out(const std::string& instance_id) const { + if (whitelist_.empty()) { + return blacklist_.count(instance_id); + } + return !whitelist_.count(instance_id); +} + +} // namespace doris::cloud diff --git a/cloud/src/recycler/white_black_list.h b/cloud/src/recycler/white_black_list.h new file mode 100644 index 00000000000000..f90744f4b4b6e7 --- /dev/null +++ b/cloud/src/recycler/white_black_list.h @@ -0,0 +1,37 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include +#include + +namespace doris::cloud { + +class WhiteBlackList { +public: + void reset(const std::vector& whitelist, + const std::vector& blacklist); + bool filter_out(const std::string& name) const; + +private: + std::set whitelist_; + std::set blacklist_; +}; + +} // namespace doris::cloud diff --git a/cloud/src/resource-manager/CMakeLists.txt b/cloud/src/resource-manager/CMakeLists.txt new file mode 100644 index 00000000000000..dbd6d3539756ee --- /dev/null +++ b/cloud/src/resource-manager/CMakeLists.txt @@ -0,0 +1,12 @@ + +# where to put generated libraries +set(LIBRARY_OUTPUT_PATH "${BUILD_DIR}/src/resource-manager") + +# where to put generated binaries +set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/src/resource-manager") + +set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -lfdb_c -L${THIRDPARTY_DIR}/lib") + +add_library(ResourceManager + resource_manager.cpp +) diff --git a/cloud/src/resource-manager/resource_manager.cpp b/cloud/src/resource-manager/resource_manager.cpp new file mode 100644 index 00000000000000..b2573b12603471 --- /dev/null +++ b/cloud/src/resource-manager/resource_manager.cpp @@ -0,0 +1,981 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "resource_manager.h" + +#include + +#include + +#include "common/logging.h" +#include "common/sync_point.h" +#include "common/util.h" +#include "meta-service/keys.h" +#include "meta-service/meta_service_helper.h" +#include "meta-service/txn_kv_error.h" + +namespace doris::cloud { + +static std::atomic_int64_t seq = 0; + +int ResourceManager::init() { + // Scan all instances + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(INFO) << "failed to init txn, err=" << err; + return -1; + } + + InstanceKeyInfo key0_info {""}; + InstanceKeyInfo key1_info {"\xff"}; // instance id are human readable strings + std::string key0; + std::string key1; + instance_key(key0_info, &key0); + instance_key(key1_info, &key1); + + std::unique_ptr it; + + int num_instances = 0; + std::unique_ptr> defer_log_range( + (int*)0x01, [key0, key1, &num_instances](int*) { + LOG(INFO) << "get instances, num_instances=" << num_instances << " range=[" + << hex(key0) << "," << hex(key1) << "]"; + }); + + // instance_id instance + std::vector> instances; + int limit = 10000; + TEST_SYNC_POINT_CALLBACK("ResourceManager:init:limit", &limit); + do { + TxnErrorCode err = txn->get(key0, key1, &it, false, limit); + TEST_SYNC_POINT_CALLBACK("ResourceManager:init:get_err", &err); + if (err == TxnErrorCode::TXN_TOO_OLD) { + LOG(WARNING) << "failed to get instance, err=txn too old, " + << " already read " << instances.size() << " instance, " + << " now fallback to non snapshot scan"; + err = txn_kv_->create_txn(&txn); + if (err == TxnErrorCode::TXN_OK) { + err = txn->get(key0, key1, &it); + } + } + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "internal error, failed to get instance, err=" << err; + return -1; + } + + while (it->has_next()) { + auto [k, v] = it->next(); + if (!it->has_next()) key0 = k; + LOG(INFO) << "range get instance_key=" << hex(k); + instances.emplace_back("", InstanceInfoPB {}); + auto& [instance_id, inst] = instances.back(); + + if (!inst.ParseFromArray(v.data(), v.size())) { + LOG(WARNING) << "malformed instance, unable to deserialize, key=" << hex(k); + return -1; + } + // 0x01 "instance" ${instance_id} -> InstanceInfoPB + k.remove_prefix(1); // Remove key space + std::vector, int, int>> out; + int ret = decode_key(&k, &out); + if (ret != 0) { + LOG(WARNING) << "failed to decode key, ret=" << ret; + return -2; + } + if (out.size() != 2) { + LOG(WARNING) << "decoded size no match, expect 2, given=" << out.size(); + } + instance_id = std::get(std::get<0>(out[1])); + + LOG(INFO) << "get an instance, instance_id=" << instance_id + << " instance json=" << proto_to_json(inst); + + ++num_instances; + } + key0.push_back('\x00'); // Update to next smallest key for iteration + } while (it->more()); + + for (auto& [inst_id, inst] : instances) { + for (auto& c : inst.clusters()) { + add_cluster_to_index(inst_id, c); + } + } + + return 0; +} + +std::string ResourceManager::get_node(const std::string& cloud_unique_id, + std::vector* nodes) { + // FIXME(gavin): refresh the all instance if there is a miss? + // Or we can refresh all instances regularly to reduce + // read amplification. + std::shared_lock l(mtx_); + auto [s, e] = node_info_.equal_range(cloud_unique_id); + if (s == node_info_.end() || s->first != cloud_unique_id) { + LOG(INFO) << "cloud unique id not found cloud_unique_id=" << cloud_unique_id; + return "cloud_unique_id not found"; + } + nodes->reserve(nodes->size() + node_info_.count(cloud_unique_id)); + for (auto i = s; i != e; ++i) { + nodes->emplace_back(i->second); // Just copy, it's cheap + } + return ""; +} + +bool ResourceManager::check_cluster_params_valid(const ClusterPB& cluster, std::string* err, + bool check_master_num) { + // check + if (!cluster.has_type()) { + *err = "cluster must have type arg"; + return false; + } + + std::stringstream ss; + bool no_err = true; + int master_num = 0; + for (auto& n : cluster.nodes()) { + if (ClusterPB::SQL == cluster.type() && n.has_edit_log_port() && n.edit_log_port() && + n.has_node_type() && + (n.node_type() == NodeInfoPB_NodeType_FE_MASTER || + n.node_type() == NodeInfoPB_NodeType_FE_OBSERVER)) { + master_num += n.node_type() == NodeInfoPB_NodeType_FE_MASTER ? 1 : 0; + continue; + } + if (ClusterPB::COMPUTE == cluster.type() && n.has_heartbeat_port() && n.heartbeat_port()) { + continue; + } + ss << "check cluster params failed, node : " << proto_to_json(n); + *err = ss.str(); + no_err = false; + break; + } + // ATTN: add_cluster check must have only a master node + // add_node doesn't check it + if (check_master_num && ClusterPB::SQL == cluster.type() && master_num != 1) { + no_err = false; + ss << "cluster is SQL type, must have only one master node, now master count: " + << master_num; + *err = ss.str(); + } + return no_err; +} + +std::pair ResourceManager::add_cluster(const std::string& instance_id, + const ClusterInfo& cluster) { + std::string msg; + std::stringstream ss; + + std::unique_ptr> defer( + (int*)0x01, [&msg](int*) { LOG(INFO) << "add_cluster err=" << msg; }); + + if (!check_cluster_params_valid(cluster.cluster, &msg, true)) { + LOG(WARNING) << msg; + return std::make_pair(MetaServiceCode::INVALID_ARGUMENT, msg); + } + + // FIXME(gavin): ensure atomicity of the entire process of adding cluster. + // Inserting a placeholer to node_info_ before persistence + // and updating it after persistence for the cloud_unique_id + // to add is a fairly fine solution. + { + std::shared_lock l(mtx_); + // Check uniqueness of cloud_unique_id to add, cloud unique ids are not + // shared between instances + for (auto& i : cluster.cluster.nodes()) { + // check cloud_unique_id in the same instance + auto [start, end] = node_info_.equal_range(i.cloud_unique_id()); + if (start == node_info_.end() || start->first != i.cloud_unique_id()) continue; + for (auto it = start; it != end; ++it) { + if (it->second.instance_id != instance_id) { + // different instance, but has same cloud_unique_id + ss << "cloud_unique_id is already occupied by an instance," + << " instance_id=" << it->second.instance_id + << " cluster_name=" << it->second.cluster_name + << " cluster_id=" << it->second.cluster_id + << " cloud_unique_id=" << it->first; + msg = ss.str(); + LOG(INFO) << msg; + return std::make_pair(MetaServiceCode::ALREADY_EXISTED, msg); + } + } + } + } + + std::unique_ptr txn0; + TxnErrorCode err = txn_kv_->create_txn(&txn0); + if (err != TxnErrorCode::TXN_OK) { + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return std::make_pair(cast_as(err), msg); + } + + std::shared_ptr txn(txn0.release()); + InstanceInfoPB instance; + auto [c0, m0] = get_instance(txn, instance_id, &instance); + if (c0 != TxnErrorCode::TXN_OK) { + msg = "failed to get instance, info " + m0; + LOG(WARNING) << msg << " err=" << c0; + return std::make_pair(cast_as(c0), msg); + } + + if (instance.status() == InstanceInfoPB::DELETED) { + msg = "instance status has been set delete, plz check it"; + return std::make_pair(MetaServiceCode::CLUSTER_NOT_FOUND, msg); + } + + LOG(INFO) << "cluster to add json=" << proto_to_json(cluster.cluster); + LOG(INFO) << "json=" << proto_to_json(instance); + + // Check id and name, they need to be unique + // One cluster id per name, name is alias of cluster id + for (auto& i : instance.clusters()) { + if (i.cluster_id() == cluster.cluster.cluster_id()) { + ss << "try to add a existing cluster id," + << " existing_cluster_id=" << i.cluster_id(); + msg = ss.str(); + return std::make_pair(MetaServiceCode::ALREADY_EXISTED, msg); + } + + if (i.cluster_name() == cluster.cluster.cluster_name()) { + ss << "try to add a existing cluster name," + << " existing_cluster_name=" << i.cluster_name(); + msg = ss.str(); + return std::make_pair(MetaServiceCode::ALREADY_EXISTED, msg); + } + } + + // TODO(gavin): Check duplicated nodes, one node cannot deploy on multiple clusters + auto now_time = std::chrono::system_clock::now(); + uint64_t time = + std::chrono::duration_cast(now_time.time_since_epoch()).count(); + for (auto& n : cluster.cluster.nodes()) { + auto& node = const_cast&>(n); + node.set_ctime(time); + node.set_mtime(time); + } + + auto to_add_cluster = instance.add_clusters(); + to_add_cluster->CopyFrom(cluster.cluster); + // create compute cluster, set it status normal as default value + if (cluster.cluster.type() == ClusterPB::COMPUTE) { + to_add_cluster->set_cluster_status(ClusterStatus::NORMAL); + } + LOG(INFO) << "instance " << instance_id << " has " << instance.clusters().size() << " clusters"; + + InstanceKeyInfo key_info {instance_id}; + std::string key; + std::string val; + instance_key(key_info, &key); + + val = instance.SerializeAsString(); + if (val.empty()) { + msg = "failed to serialize"; + return std::make_pair(MetaServiceCode::PROTOBUF_SERIALIZE_ERR, msg); + } + + txn->put(key, val); + LOG(INFO) << "put instance_key=" << hex(key); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + msg = "failed to commit kv txn"; + LOG(WARNING) << msg << " err=" << err; + return std::make_pair(cast_as(err), msg); + } + + add_cluster_to_index(instance_id, cluster.cluster); + + return std::make_pair(MetaServiceCode::OK, ""); +} + +std::pair ResourceManager::drop_cluster( + const std::string& instance_id, const ClusterInfo& cluster) { + std::stringstream ss; + std::string msg; + + std::string cluster_id = cluster.cluster.has_cluster_id() ? cluster.cluster.cluster_id() : ""; + if (cluster_id.empty()) { + ss << "missing cluster_id=" << cluster_id; + msg = ss.str(); + LOG(INFO) << msg; + return std::make_pair(MetaServiceCode::INVALID_ARGUMENT, msg); + } + + std::unique_ptr txn0; + TxnErrorCode err = txn_kv_->create_txn(&txn0); + if (err != TxnErrorCode::TXN_OK) { + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return std::make_pair(cast_as(err), msg); + } + + std::shared_ptr txn(txn0.release()); + InstanceInfoPB instance; + auto [c0, m0] = get_instance(txn, instance_id, &instance); + if (c0 == TxnErrorCode::TXN_KEY_NOT_FOUND) { + msg = m0; + LOG(WARNING) << msg; + return std::make_pair(MetaServiceCode::CLUSTER_NOT_FOUND, msg); + } + if (c0 != TxnErrorCode::TXN_OK) { + msg = m0; + LOG(WARNING) << msg; + return std::make_pair(cast_as(c0), msg); + } + + if (instance.status() == InstanceInfoPB::DELETED) { + msg = "instance status has been set delete, plz check it"; + return std::make_pair(MetaServiceCode::CLUSTER_NOT_FOUND, msg); + } + + bool found = false; + int idx = -1; + ClusterPB to_del; + // Check id and name, they need to be unique + // One cluster id per name, name is alias of cluster id + for (auto& i : instance.clusters()) { + ++idx; + if (i.cluster_id() == cluster.cluster.cluster_id()) { + to_del.CopyFrom(i); + LOG(INFO) << "found a cluster to drop," + << " instance_id=" << instance_id << " cluster_id=" << i.cluster_id() + << " cluster_name=" << i.cluster_name() << " cluster=" << proto_to_json(i); + found = true; + break; + } + } + + if (!found) { + ss << "failed to find cluster to drop," + << " instance_id=" << instance_id << " cluster_id=" << cluster.cluster.cluster_id() + << " cluster_name=" << cluster.cluster.cluster_name(); + msg = ss.str(); + return std::make_pair(MetaServiceCode::CLUSTER_NOT_FOUND, msg); + } + + InstanceInfoPB new_instance(instance); + new_instance.mutable_clusters()->DeleteSubrange(idx, 1); // Remove it + + InstanceKeyInfo key_info {instance_id}; + std::string key; + std::string val; + instance_key(key_info, &key); + + val = new_instance.SerializeAsString(); + if (val.empty()) { + msg = "failed to serialize"; + LOG(WARNING) << msg; + return std::make_pair(MetaServiceCode::PROTOBUF_SERIALIZE_ERR, msg); + } + + txn->put(key, val); + LOG(INFO) << "put instance_key=" << hex(key); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + msg = "failed to commit kv txn"; + LOG(WARNING) << msg << " err=" << err; + return std::make_pair(cast_as(err), msg); + } + + remove_cluster_from_index(instance_id, to_del); + + return std::make_pair(MetaServiceCode::OK, ""); +} + +std::string ResourceManager::update_cluster( + const std::string& instance_id, const ClusterInfo& cluster, + std::function filter, + std::function& cluster_names)> action) { + std::stringstream ss; + std::string msg; + + std::string cluster_id = cluster.cluster.has_cluster_id() ? cluster.cluster.cluster_id() : ""; + std::string cluster_name = + cluster.cluster.has_cluster_name() ? cluster.cluster.cluster_name() : ""; + if (cluster_id.empty()) { + ss << "missing cluster_id=" << cluster_id; + msg = ss.str(); + LOG(INFO) << msg; + return msg; + } + + std::unique_ptr txn0; + TxnErrorCode err = txn_kv_->create_txn(&txn0); + if (err != TxnErrorCode::TXN_OK) { + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return msg; + } + + std::shared_ptr txn(txn0.release()); + InstanceInfoPB instance; + auto [c0, m0] = get_instance(txn, instance_id, &instance); + if (c0 != TxnErrorCode::TXN_OK) { + msg = m0; + return msg; + } + + if (instance.status() == InstanceInfoPB::DELETED) { + msg = "instance status has been set delete, plz check it"; + return msg; + } + + std::set cluster_names; + // collect cluster_names + for (auto& i : instance.clusters()) { + cluster_names.emplace(i.cluster_name()); + } + + bool found = false; + int idx = -1; + // Check id and name, they need to be unique + // One cluster id per name, name is alias of cluster id + for (auto& i : instance.clusters()) { + ++idx; + if (filter(i)) { + LOG(INFO) << "found a cluster to update," + << " instance_id=" << instance_id << " cluster_id=" << i.cluster_id() + << " cluster_name=" << i.cluster_name() << " cluster=" << proto_to_json(i); + found = true; + break; + } + } + + if (!found) { + ss << "failed to find cluster to update," + << " instance_id=" << instance_id << " cluster_id=" << cluster.cluster.cluster_id() + << " cluster_name=" << cluster.cluster.cluster_name(); + msg = ss.str(); + return msg; + } + + auto& clusters = const_cast&>(instance.clusters()); + + // do update + ClusterPB original = clusters[idx]; + msg = action(clusters[idx], cluster_names); + if (!msg.empty()) { + return msg; + } + ClusterPB now = clusters[idx]; + LOG(INFO) << "before update cluster original: " << proto_to_json(original) + << " after update now: " << proto_to_json(now); + + InstanceKeyInfo key_info {instance_id}; + std::string key; + std::string val; + instance_key(key_info, &key); + + val = instance.SerializeAsString(); + if (val.empty()) { + msg = "failed to serialize"; + return msg; + } + + txn->put(key, val); + LOG(INFO) << "put instanace_key=" << hex(key); + TxnErrorCode err_code = txn->commit(); + if (err_code != TxnErrorCode::TXN_OK) { + msg = "failed to commit kv txn"; + LOG(WARNING) << msg << " err=" << msg; + return msg; + } + + LOG(INFO) << "update cluster instance_id=" << instance_id + << " instance json=" << proto_to_json(instance); + + update_cluster_to_index(instance_id, original, now); + + return msg; +} + +void ResourceManager::update_cluster_to_index(const std::string& instance_id, + const ClusterPB& original, const ClusterPB& now) { + std::lock_guard l(mtx_); + remove_cluster_from_index_no_lock(instance_id, original); + add_cluster_to_index_no_lock(instance_id, now); +} + +void ResourceManager::add_cluster_to_index_no_lock(const std::string& instance_id, + const ClusterPB& c) { + auto type = c.has_type() ? c.type() : -1; + Role role = (type == ClusterPB::SQL + ? Role::SQL_SERVER + : (type == ClusterPB::COMPUTE ? Role::COMPUTE_NODE : Role::UNDEFINED)); + LOG(INFO) << "add cluster to index, instance_id=" << instance_id << " cluster_type=" << type + << " cluster_name=" << c.cluster_name() << " cluster_id=" << c.cluster_id(); + + for (auto& i : c.nodes()) { + bool existed = node_info_.count(i.cloud_unique_id()); + NodeInfo n {.role = role, + .instance_id = instance_id, + .cluster_name = c.cluster_name(), + .cluster_id = c.cluster_id(), + .node_info = i}; + LOG(WARNING) << (existed ? "duplicated cloud_unique_id " : "") + << "instance_id=" << instance_id << " cloud_unique_id=" << i.cloud_unique_id() + << " node_info=" << proto_to_json(i); + node_info_.insert({i.cloud_unique_id(), std::move(n)}); + } +} + +void ResourceManager::add_cluster_to_index(const std::string& instance_id, const ClusterPB& c) { + std::lock_guard l(mtx_); + add_cluster_to_index_no_lock(instance_id, c); +} + +void ResourceManager::remove_cluster_from_index_no_lock(const std::string& instance_id, + const ClusterPB& c) { + std::string cluster_name = c.cluster_name(); + std::string cluster_id = c.cluster_id(); + int cnt = 0; + for (auto it = node_info_.begin(); it != node_info_.end();) { + auto& [_, n] = *it; + if (n.instance_id != instance_id || n.cluster_id != cluster_id || + n.cluster_name != cluster_name) { + ++it; + continue; + } + ++cnt; + LOG(INFO) << "remove node from index, instance_id=" << instance_id + << " role=" << static_cast(n.role) << " cluster_name=" << n.cluster_name + << " cluster_id=" << n.cluster_id << " node_info=" << proto_to_json(n.node_info); + it = node_info_.erase(it); + } + LOG(INFO) << cnt << " nodes removed from index, cluster_id=" << cluster_id + << " cluster_name=" << cluster_name << " instance_id=" << instance_id; +} + +void ResourceManager::remove_cluster_from_index(const std::string& instance_id, + const ClusterPB& c) { + std::lock_guard l(mtx_); + remove_cluster_from_index_no_lock(instance_id, c); +} + +std::pair ResourceManager::get_instance(std::shared_ptr txn, + const std::string& instance_id, + InstanceInfoPB* inst_pb) { + std::pair ec {TxnErrorCode::TXN_OK, ""}; + [[maybe_unused]] auto& [code, msg] = ec; + std::stringstream ss; + + InstanceKeyInfo key_info {instance_id}; + std::string key; + std::string val; + instance_key(key_info, &key); + + if (txn == nullptr) { + std::unique_ptr txn0; + TxnErrorCode err = txn_kv_->create_txn(&txn0); + if (err != TxnErrorCode::TXN_OK) { + code = err; + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return ec; + } + txn.reset(txn0.release()); + } + + TxnErrorCode err = txn->get(key, &val); + LOG(INFO) << "get instance_key=" << hex(key); + + if (err != TxnErrorCode::TXN_OK) { + code = err; + ss << "failed to get instance, instance_id=" << instance_id << " err=" << err; + msg = ss.str(); + return ec; + } + + if (!inst_pb->ParseFromString(val)) { + code = TxnErrorCode::TXN_UNIDENTIFIED_ERROR; + msg = "failed to parse InstanceInfoPB"; + return ec; + } + + return ec; +} + +std::string ResourceManager::modify_nodes(const std::string& instance_id, + const std::vector& to_add, + const std::vector& to_del) { + std::string msg; + std::stringstream ss; + std::unique_ptr> defer( + (int*)0x01, [&msg](int*) { LOG(INFO) << "modify_nodes err=" << msg; }); + + if ((to_add.size() && to_del.size()) || (!to_add.size() && !to_del.size())) { + msg = "to_add and to_del both empty or both not empty"; + LOG(WARNING) << msg; + return msg; + } + + std::unique_ptr txn0; + TxnErrorCode err = txn_kv_->create_txn(&txn0); + if (err != TxnErrorCode::TXN_OK) { + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << msg; + return msg; + } + + std::shared_ptr txn(txn0.release()); + InstanceInfoPB instance; + auto [c0, m0] = get_instance(txn, instance_id, &instance); + { + TEST_SYNC_POINT_CALLBACK("modify_nodes:get_instance", &c0); + TEST_SYNC_POINT_CALLBACK("modify_nodes:get_instance_ret", &instance); + } + if (c0 != TxnErrorCode::TXN_OK) { + msg = m0; + return msg; + } + + if (instance.status() == InstanceInfoPB::DELETED) { + msg = "instance status has been set delete, plz check it"; + LOG(WARNING) << msg; + return msg; + } + + LOG(INFO) << "instance json=" << proto_to_json(instance); + std::vector> vec; + using modify_impl_func = std::function; + using check_func = std::function; + auto modify_func = [&](const NodeInfo& node, check_func check, + modify_impl_func action) -> std::string { + std::string cluster_id = node.cluster_id; + std::string cluster_name = node.cluster_name; + + { + std::shared_lock l(mtx_); + msg = check(node); + if (msg != "") { + return msg; + } + } + + LOG(INFO) << "node to modify json=" << proto_to_json(node.node_info); + + for (auto& c : instance.clusters()) { + if ((c.has_cluster_name() && c.cluster_name() == cluster_name) || + (c.has_cluster_id() && c.cluster_id() == cluster_id)) { + msg = action(c, node); + if (msg != "") { + return msg; + } + } + } + return ""; + }; + + check_func check_to_add = [&](const NodeInfo& n) -> std::string { + std::string err; + std::stringstream s; + auto [start, end] = node_info_.equal_range(n.node_info.cloud_unique_id()); + if (start == node_info_.end() || start->first != n.node_info.cloud_unique_id()) { + return ""; + } + for (auto it = start; it != end; ++it) { + if (it->second.instance_id != n.instance_id) { + // different instance, but has same cloud_unique_id + s << "cloud_unique_id is already occupied by an instance," + << " instance_id=" << it->second.instance_id + << " cluster_name=" << it->second.cluster_name + << " cluster_id=" << it->second.cluster_id + << " cloud_unique_id=" << n.node_info.cloud_unique_id(); + err = s.str(); + LOG(INFO) << err; + return err; + } + } + return ""; + }; + + modify_impl_func modify_to_add = [&](const ClusterPB& c, const NodeInfo& n) -> std::string { + std::string err; + std::stringstream s; + ClusterPB copied_original_cluster; + ClusterPB copied_cluster; + bool is_compute_node = n.node_info.has_heartbeat_port(); + for (auto it = c.nodes().begin(); it != c.nodes().end(); ++it) { + if (it->has_ip() && n.node_info.has_ip()) { + std::string c_endpoint = it->ip() + ":" + + (is_compute_node ? std::to_string(it->heartbeat_port()) + : std::to_string(it->edit_log_port())); + std::string n_endpoint = + n.node_info.ip() + ":" + + (is_compute_node ? std::to_string(n.node_info.heartbeat_port()) + : std::to_string(n.node_info.edit_log_port())); + if (c_endpoint == n_endpoint) { + // replicate request, do nothing + return ""; + } + } + + if (it->has_host() && n.node_info.has_host()) { + std::string c_endpoint_host = + it->host() + ":" + + (is_compute_node ? std::to_string(it->heartbeat_port()) + : std::to_string(it->edit_log_port())); + std::string n_endpoint_host = + n.node_info.host() + ":" + + (is_compute_node ? std::to_string(n.node_info.heartbeat_port()) + : std::to_string(n.node_info.edit_log_port())); + if (c_endpoint_host == n_endpoint_host) { + // replicate request, do nothing + return ""; + } + } + } + + // add ctime and mtime + auto& node = const_cast&>(n.node_info); + auto now_time = std::chrono::system_clock::now(); + uint64_t time = + std::chrono::duration_cast(now_time.time_since_epoch()) + .count(); + if (!node.has_ctime()) { + node.set_ctime(time); + } + node.set_mtime(time); + copied_original_cluster.CopyFrom(c); + auto& change_cluster = const_cast&>(c); + change_cluster.add_nodes()->CopyFrom(node); + copied_cluster.CopyFrom(change_cluster); + vec.emplace_back(std::move(copied_original_cluster), std::move(copied_cluster)); + return ""; + }; + + for (auto& it : to_add) { + msg = modify_func(it, check_to_add, modify_to_add); + if (msg != "") { + LOG(WARNING) << msg; + return msg; + } + } + + check_func check_to_del = [&](const NodeInfo& n) -> std::string { + std::string err; + std::stringstream s; + auto [start, end] = node_info_.equal_range(n.node_info.cloud_unique_id()); + if (start == node_info_.end() || start->first != n.node_info.cloud_unique_id()) { + s << "cloud_unique_id can not find to drop node," + << " instance_id=" << n.instance_id << " cluster_name=" << n.cluster_name + << " cluster_id=" << n.cluster_id + << " cloud_unique_id=" << n.node_info.cloud_unique_id(); + err = s.str(); + LOG(WARNING) << err; + return err; + } + + bool found = false; + for (auto it = start; it != end; ++it) { + const auto& m_node = it->second.node_info; + if (m_node.has_ip() && n.node_info.has_ip()) { + std::string m_endpoint = + m_node.ip() + ":" + + (m_node.has_heartbeat_port() ? std::to_string(m_node.heartbeat_port()) + : std::to_string(m_node.edit_log_port())); + + std::string n_endpoint = n.node_info.ip() + ":" + + (n.node_info.has_heartbeat_port() + ? std::to_string(n.node_info.heartbeat_port()) + : std::to_string(n.node_info.edit_log_port())); + + if (m_endpoint == n_endpoint) { + found = true; + break; + } + } + + if (m_node.has_host() && n.node_info.has_host()) { + std::string m_endpoint_host = + m_node.host() + ":" + + (m_node.has_heartbeat_port() ? std::to_string(m_node.heartbeat_port()) + : std::to_string(m_node.edit_log_port())); + + std::string n_endpoint_host = + n.node_info.host() + ":" + + (n.node_info.has_heartbeat_port() + ? std::to_string(n.node_info.heartbeat_port()) + : std::to_string(n.node_info.edit_log_port())); + + if (m_endpoint_host == n_endpoint_host) { + found = true; + break; + } + } + } + if (!found) { + s << "cloud_unique_id can not find to drop node," + << " instance_id=" << n.instance_id << " cluster_name=" << n.cluster_name + << " cluster_id=" << n.cluster_id + << " cloud_unique_id=" << n.node_info.cloud_unique_id(); + err = s.str(); + LOG(WARNING) << err; + return err; + } + return ""; + }; + + modify_impl_func modify_to_del = [&](const ClusterPB& c, const NodeInfo& n) -> std::string { + std::string err; + std::stringstream s; + ClusterPB copied_original_cluster; + ClusterPB copied_cluster; + + bool found = false; + int idx = -1; + const auto& ni = n.node_info; + for (auto& cn : c.nodes()) { + idx++; + if (cn.has_ip() && ni.has_ip()) { + std::string cn_endpoint = + cn.ip() + ":" + + (cn.has_heartbeat_port() ? std::to_string(cn.heartbeat_port()) + : std::to_string(cn.edit_log_port())); + + std::string ni_endpoint = + ni.ip() + ":" + + (ni.has_heartbeat_port() ? std::to_string(ni.heartbeat_port()) + : std::to_string(ni.edit_log_port())); + + if (ni.cloud_unique_id() == cn.cloud_unique_id() && cn_endpoint == ni_endpoint) { + found = true; + break; + } + } + + if (cn.has_host() && ni.has_host()) { + std::string cn_endpoint_host = + cn.host() + ":" + + (cn.has_heartbeat_port() ? std::to_string(cn.heartbeat_port()) + : std::to_string(cn.edit_log_port())); + + std::string ni_endpoint_host = + ni.host() + ":" + + (ni.has_heartbeat_port() ? std::to_string(ni.heartbeat_port()) + : std::to_string(ni.edit_log_port())); + + if (ni.cloud_unique_id() == cn.cloud_unique_id() && + cn_endpoint_host == ni_endpoint_host) { + found = true; + break; + } + } + } + + if (!found) { + s << "failed to find node to drop," + << " instance_id=" << instance.instance_id() << " cluster_id=" << c.cluster_id() + << " cluster_name=" << c.cluster_name() << " cluster=" << proto_to_json(c); + err = s.str(); + LOG(WARNING) << err; + // not found return ok. + return ""; + } + copied_original_cluster.CopyFrom(c); + auto& change_nodes = const_cast&>(c.nodes()); + change_nodes.DeleteSubrange(idx, 1); // Remove it + copied_cluster.CopyFrom(c); + vec.emplace_back(std::move(copied_original_cluster), std::move(copied_cluster)); + return ""; + }; + + for (auto& it : to_del) { + msg = modify_func(it, check_to_del, modify_to_del); + if (msg != "") { + LOG(WARNING) << msg; + // not found, just return OK to cloud control + return ""; + } + } + + LOG(INFO) << "instance " << instance_id << " info: " << instance.DebugString(); + + InstanceKeyInfo key_info {instance_id}; + std::string key; + std::string val; + instance_key(key_info, &key); + + val = instance.SerializeAsString(); + if (val.empty()) { + msg = "failed to serialize"; + return msg; + } + + txn->put(key, val); + LOG(INFO) << "put instance_key=" << hex(key); + TxnErrorCode err_code = txn->commit(); + if (err_code != TxnErrorCode::TXN_OK) { + msg = "failed to commit kv txn"; + LOG(WARNING) << msg << " err=" << err_code; + return msg; + } + + for (auto& it : vec) { + update_cluster_to_index(instance_id, it.first, it.second); + } + + return ""; +} + +std::pair ResourceManager::refresh_instance( + const std::string& instance_id) { + LOG(INFO) << "begin to refresh instance, instance_id=" << instance_id << " seq=" << ++seq; + std::pair ret0 {MetaServiceCode::OK, "OK"}; + auto& [code, msg] = ret0; + std::unique_ptr> defer_log( + (int*)0x01, [&ret0, &instance_id](int*) { + LOG(INFO) << (std::get<0>(ret0) == MetaServiceCode::OK ? "succ to " : "failed to ") + << "refresh_instance, instance_id=" << instance_id + << " code=" << std::get<0>(ret0) << " msg=" << std::get<1>(ret0); + }); + + std::unique_ptr txn0; + TxnErrorCode err = txn_kv_->create_txn(&txn0); + if (err != TxnErrorCode::TXN_OK) { + code = MetaServiceCode::KV_TXN_CREATE_ERR; + msg = "failed to create txn"; + LOG(WARNING) << msg << " err=" << err; + return ret0; + } + std::shared_ptr txn(txn0.release()); + InstanceInfoPB instance; + auto [c0, m0] = get_instance(txn, instance_id, &instance); + if (c0 != TxnErrorCode::TXN_OK) { + code = cast_as(c0); + msg = m0; + return ret0; + } + std::vector clusters; + clusters.reserve(instance.clusters_size()); + + std::lock_guard l(mtx_); + for (auto i = node_info_.begin(); i != node_info_.end();) { + if (i->second.instance_id != instance_id) { + ++i; + continue; + } + i = node_info_.erase(i); + } + for (int i = 0; i < instance.clusters_size(); ++i) { + add_cluster_to_index_no_lock(instance_id, instance.clusters(i)); + } + LOG(INFO) << "finish refreshing instance, instance_id=" << instance_id << " seq=" << seq; + return ret0; +} + +} // namespace doris::cloud diff --git a/cloud/src/resource-manager/resource_manager.h b/cloud/src/resource-manager/resource_manager.h new file mode 100644 index 00000000000000..5000764dee8a0b --- /dev/null +++ b/cloud/src/resource-manager/resource_manager.h @@ -0,0 +1,148 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include + +#include +#include +#include + +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" + +namespace doris::cloud { + +enum class Role : int { + UNDEFINED, + SQL_SERVER, + COMPUTE_NODE, +}; + +struct NodeInfo { + Role role; + std::string instance_id; + std::string cluster_name; + std::string cluster_id; + NodeInfoPB node_info; +}; + +struct ClusterInfo { + ClusterPB cluster; +}; + +/** + * This class manages resources referenced by cloud cloud. + * It manage a in-memory + */ +class ResourceManager { +public: + ResourceManager(std::shared_ptr txn_kv) : txn_kv_(txn_kv) {}; + virtual ~ResourceManager() = default; + /** + * Loads all instance into memory and build an index + * + * @return 0 for success, non-zero for failure + */ + virtual int init(); + + /** + * Gets nodes with given cloud unique id + * + * @param cloud_unique_id the cloud_unique_id attached to the node when it was added to the + * instance + * @param node output param + * @return empty string for success, otherwise failure reason returned + */ + virtual std::string get_node(const std::string& cloud_unique_id, std::vector* nodes); + + virtual std::pair add_cluster(const std::string& instance_id, + const ClusterInfo& cluster); + + /** + * Drops a cluster + * + * @param cluster cluster to drop, only cluster name and cluster id are concered + * @return empty string for success, otherwise failure reason returned + */ + virtual std::pair drop_cluster(const std::string& instance_id, + const ClusterInfo& cluster); + + /** + * Update a cluster + * + * @param cluster cluster to update, only cluster name and cluster id are concered + * @param action update operation code snippet + * @filter filter condition + * @return empty string for success, otherwise failure reason returned + */ + virtual std::string update_cluster( + const std::string& instance_id, const ClusterInfo& cluster, + std::function filter, + std::function& cluster_names)> action); + + /** + * Get instance from underlying storage with given transaction. + * + * @param txn if txn is not given, get with a new txn inside this function + * + * @return a pair, code == TXN_OK for success, otherwise error + */ + virtual std::pair get_instance(std::shared_ptr txn, + const std::string& instance_id, + InstanceInfoPB* inst_pb); + // return err msg + virtual std::string modify_nodes(const std::string& instance_id, + const std::vector& to_add, + const std::vector& to_del); + + bool check_cluster_params_valid(const ClusterPB& cluster, std::string* err, + bool check_master_num); + + /** + * Refreshes the cache of given instance. This process removes the instance in cache + * and then replaces it with persisted instance state read from underlying KV storage. + * + * @param instance_id instance to manipulate + * @return a pair of code and msg + */ + virtual std::pair refresh_instance( + const std::string& instance_id); + +private: + void add_cluster_to_index(const std::string& instance_id, const ClusterPB& cluster); + + void remove_cluster_from_index(const std::string& instance_id, const ClusterPB& cluster); + + void update_cluster_to_index(const std::string& instance_id, const ClusterPB& original, + const ClusterPB& now); + + void remove_cluster_from_index_no_lock(const std::string& instance_id, + const ClusterPB& cluster); + + void add_cluster_to_index_no_lock(const std::string& instance_id, const ClusterPB& cluster); + +private: + std::shared_mutex mtx_; + // cloud_unique_id -> NodeInfo + std::multimap node_info_; + + std::shared_ptr txn_kv_; +}; + +} // namespace doris::cloud diff --git a/cloud/src/s3_accessor_test.cpp b/cloud/src/s3_accessor_test.cpp new file mode 100644 index 00000000000000..8ff4b52a06136a --- /dev/null +++ b/cloud/src/s3_accessor_test.cpp @@ -0,0 +1,827 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "recycler/s3_accessor.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "common/configbase.h" +#include "common/logging.h" +#include "common/sync_point.h" +#include "mock_accessor.h" + +std::unique_ptr _mock_fs; + +class S3ClientInterface { +public: + S3ClientInterface() = default; + virtual ~S3ClientInterface() = default; + virtual Aws::S3::Model::ListObjectsV2Outcome ListObjectsV2( + const Aws::S3::Model::ListObjectsV2Request& req) = 0; + virtual Aws::S3::Model::DeleteObjectsOutcome DeleteObjects( + const Aws::S3::Model::DeleteObjectsRequest& req) = 0; + virtual Aws::S3::Model::PutObjectOutcome PutObject( + const Aws::S3::Model::PutObjectRequest& req) = 0; + virtual Aws::S3::Model::HeadObjectOutcome HeadObject( + const Aws::S3::Model::HeadObjectRequest& req) = 0; + virtual Aws::S3::Model::GetBucketLifecycleConfigurationOutcome GetBucketLifecycleConfiguration( + const Aws::S3::Model::GetBucketLifecycleConfigurationRequest& req) = 0; + virtual Aws::S3::Model::GetBucketVersioningOutcome GetBucketVersioning( + const Aws::S3::Model::GetBucketVersioningRequest& req) = 0; +}; + +static bool list_object_v2_with_expire_time = false; +static int64_t expire_time = 0; +static bool set_bucket_lifecycle = false; +static bool set_bucket_versioning_status_error = false; + +class S3Client : public S3ClientInterface { +public: + S3Client() = default; + ~S3Client() override = default; + Aws::S3::Model::ListObjectsV2Outcome ListObjectsV2( + const Aws::S3::Model::ListObjectsV2Request& req) override { + auto prefix = req.GetPrefix(); + auto continuation_token = + req.ContinuationTokenHasBeenSet() ? req.GetContinuationToken() : ""; + bool truncated = true; + std::vector files; + size_t num = 0; + do { + _mock_fs->list(prefix, &files); + if (num == files.size()) { + truncated = false; + break; + } + num = files.size(); + auto path1 = files.back().path; + prefix = path1.back() += 1; + } while (files.size() <= 1000); + Aws::S3::Model::ListObjectsV2Result result; + result.SetIsTruncated(truncated); + std::vector objects; + std::for_each(files.begin(), files.end(), [&](const cloud::ObjectMeta& file) { + Aws::S3::Model::Object obj; + obj.SetKey(file.path); + Aws::Utils::DateTime date; + if (list_object_v2_with_expire_time) { + date = Aws::Utils::DateTime(expire_time); + } + obj.SetLastModified(date); + objects.emplace_back(std::move(obj)); + }); + result.SetContents(std::move(objects)); + return Aws::S3::Model::ListObjectsV2Outcome(std::move(result)); + } + + Aws::S3::Model::DeleteObjectsOutcome DeleteObjects( + const Aws::S3::Model::DeleteObjectsRequest& req) override { + Aws::S3::Model::DeleteObjectsResult result; + const auto& deletes = req.GetDelete(); + for (const auto& obj : deletes.GetObjects()) { + _mock_fs->delete_object(obj.GetKey()); + } + return Aws::S3::Model::DeleteObjectsOutcome(std::move(result)); + } + + Aws::S3::Model::PutObjectOutcome PutObject( + const Aws::S3::Model::PutObjectRequest& req) override { + Aws::S3::Model::PutObjectResult result; + const auto& key = req.GetKey(); + _mock_fs->put_object(key, ""); + return Aws::S3::Model::PutObjectOutcome(std::move(result)); + } + + Aws::S3::Model::HeadObjectOutcome HeadObject( + const Aws::S3::Model::HeadObjectRequest& req) override { + Aws::S3::Model::HeadObjectResult result; + const auto& key = req.GetKey(); + auto v = _mock_fs->exist(key); + if (v == 1) { + auto err = Aws::Client::AWSError( + Aws::S3::S3Errors::RESOURCE_NOT_FOUND, false); + err.SetResponseCode(Aws::Http::HttpResponseCode::NOT_FOUND); + + return Aws::S3::Model::HeadObjectOutcome(std::move(err)); + } + return Aws::S3::Model::HeadObjectOutcome(std::move(result)); + } + + Aws::S3::Model::GetBucketLifecycleConfigurationOutcome GetBucketLifecycleConfiguration( + const Aws::S3::Model::GetBucketLifecycleConfigurationRequest& req) override { + Aws::S3::Model::GetBucketLifecycleConfigurationResult result; + Aws::Vector rules; + if (set_bucket_lifecycle) { + Aws::S3::Model::LifecycleRule rule; + Aws::S3::Model::NoncurrentVersionExpiration expiration; + expiration.SetNoncurrentDays(1000); + rule.SetNoncurrentVersionExpiration(expiration); + rules.emplace_back(std::move(rule)); + } + result.SetRules(std::move(rules)); + return Aws::S3::Model::GetBucketLifecycleConfigurationOutcome(std::move(result)); + } + + Aws::S3::Model::GetBucketVersioningOutcome GetBucketVersioning( + const Aws::S3::Model::GetBucketVersioningRequest& req) override { + Aws::S3::Model::GetBucketVersioningResult result; + if (set_bucket_versioning_status_error) { + result.SetStatus(Aws::S3::Model::BucketVersioningStatus::Suspended); + } else { + result.SetStatus(Aws::S3::Model::BucketVersioningStatus::Enabled); + } + return Aws::S3::Model::GetBucketVersioningOutcome(std::move(result)); + } +}; + +static bool return_error_for_error_s3_client = false; +static bool delete_objects_return_part_error = false; + +class ErrorS3Client : public S3ClientInterface { +public: + ErrorS3Client() : _correct_impl(std::make_unique()) {} + ~ErrorS3Client() override = default; + Aws::S3::Model::ListObjectsV2Outcome ListObjectsV2( + const Aws::S3::Model::ListObjectsV2Request& req) override { + if (!return_error_for_error_s3_client) { + return _correct_impl->ListObjectsV2(req); + } + auto err = Aws::Client::AWSError(Aws::S3::S3Errors::RESOURCE_NOT_FOUND, + false); + err.SetResponseCode(Aws::Http::HttpResponseCode::NOT_FOUND); + return Aws::S3::Model::ListObjectsV2Outcome(std::move(err)); + } + + Aws::S3::Model::DeleteObjectsOutcome DeleteObjects( + const Aws::S3::Model::DeleteObjectsRequest& req) override { + if (!delete_objects_return_part_error) { + Aws::S3::Model::DeleteObjectsResult result; + Aws::Vector errors; + Aws::S3::Model::Error error; + errors.emplace_back(std::move(error)); + result.SetErrors(std::move(errors)); + return Aws::S3::Model::DeleteObjectsOutcome(std::move(result)); + } + auto err = Aws::Client::AWSError(Aws::S3::S3Errors::RESOURCE_NOT_FOUND, + false); + err.SetResponseCode(Aws::Http::HttpResponseCode::NOT_FOUND); + // return -1 + return Aws::S3::Model::DeleteObjectsOutcome(std::move(err)); + } + + Aws::S3::Model::PutObjectOutcome PutObject( + const Aws::S3::Model::PutObjectRequest& req) override { + if (!return_error_for_error_s3_client) { + return _correct_impl->PutObject(req); + } + auto err = Aws::Client::AWSError(Aws::S3::S3Errors::RESOURCE_NOT_FOUND, + false); + err.SetResponseCode(Aws::Http::HttpResponseCode::NOT_FOUND); + return Aws::S3::Model::PutObjectOutcome(std::move(err)); + } + + Aws::S3::Model::HeadObjectOutcome HeadObject( + const Aws::S3::Model::HeadObjectRequest& req) override { + if (!return_error_for_error_s3_client) { + return _correct_impl->HeadObject(req); + } + auto err = Aws::Client::AWSError(Aws::S3::S3Errors::RESOURCE_NOT_FOUND, + false); + err.SetResponseCode(Aws::Http::HttpResponseCode::INTERNAL_SERVER_ERROR); + return Aws::S3::Model::HeadObjectOutcome(std::move(err)); + } + + Aws::S3::Model::GetBucketLifecycleConfigurationOutcome GetBucketLifecycleConfiguration( + const Aws::S3::Model::GetBucketLifecycleConfigurationRequest& req) override { + if (!return_error_for_error_s3_client) { + return _correct_impl->GetBucketLifecycleConfiguration(req); + } + auto err = Aws::Client::AWSError(Aws::S3::S3Errors::RESOURCE_NOT_FOUND, + false); + err.SetResponseCode(Aws::Http::HttpResponseCode::INTERNAL_SERVER_ERROR); + return Aws::S3::Model::GetBucketLifecycleConfigurationOutcome(std::move(err)); + } + + Aws::S3::Model::GetBucketVersioningOutcome GetBucketVersioning( + const Aws::S3::Model::GetBucketVersioningRequest& req) override { + if (!return_error_for_error_s3_client) { + return _correct_impl->GetBucketVersioning(req); + } + auto err = Aws::Client::AWSError(Aws::S3::S3Errors::RESOURCE_NOT_FOUND, + false); + err.SetResponseCode(Aws::Http::HttpResponseCode::INTERNAL_SERVER_ERROR); + return Aws::S3::Model::GetBucketVersioningOutcome(std::move(err)); + } + +private: + std::unique_ptr _correct_impl; +}; + +class MockS3Client { +public: + MockS3Client(std::unique_ptr impl = std::make_unique()) + : _impl(std::move(impl)) {} + auto ListObjectsV2(const Aws::S3::Model::ListObjectsV2Request& req) { + return _impl->ListObjectsV2(req); + } + + auto DeleteObjects(const Aws::S3::Model::DeleteObjectsRequest& req) { + return _impl->DeleteObjects(req); + } + + auto PutObject(const Aws::S3::Model::PutObjectRequest& req) { return _impl->PutObject(req); } + + auto HeadObject(const Aws::S3::Model::HeadObjectRequest& req) { return _impl->HeadObject(req); } + + auto GetBucketLifecycleConfiguration( + const Aws::S3::Model::GetBucketLifecycleConfigurationRequest& req) { + return _impl->GetBucketLifecycleConfiguration(req); + } + + auto GetBucketVersioning(const Aws::S3::Model::GetBucketVersioningRequest& req) { + return _impl->GetBucketVersioning(req); + } + +private: + std::unique_ptr _impl; +}; + +std::unique_ptr _mock_client; + +struct MockCallable { + std::string point_name; + std::function func; +}; + +static auto callbacks = std::array { + MockCallable {"s3_client::list_objects_v2", + [](void* p) { + auto pair = *(std::pair*)p; + *pair.first = (*_mock_client).ListObjectsV2(*pair.second); + }}, + MockCallable {"s3_client::delete_objects", + [](void* p) { + auto pair = *(std::pair*)p; + *pair.first = (*_mock_client).DeleteObjects(*pair.second); + }}, + MockCallable {"s3_client::put_object", + [](void* p) { + auto pair = *(std::pair*)p; + *pair.first = (*_mock_client).PutObject(*pair.second); + }}, + MockCallable {"s3_client::head_object", + [](void* p) { + auto pair = *(std::pair*)p; + *pair.first = (*_mock_client).HeadObject(*pair.second); + }}, + MockCallable { + "s3_client::get_bucket_lifecycle_configuration", + [](void* p) { + auto pair = + *(std::pair*)p; + *pair.first = (*_mock_client).GetBucketLifecycleConfiguration(*pair.second); + }}, + MockCallable {"s3_client::get_bucket_versioning", [](void* p) { + auto pair = *(std::pair*)p; + *pair.first = (*_mock_client).GetBucketVersioning(*pair.second); + }}}; + +int main(int argc, char** argv) { + const std::string conf_file = "doris_cloud.conf"; + if (!cloud::config::init(conf_file.c_str(), true)) { + std::cerr << "failed to init config file, conf=" << conf_file << std::endl; + return -1; + } + + if (!cloud::init_glog("s3_accessor_test")) { + std::cerr << "failed to init glog" << std::endl; + return -1; + } + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + +namespace cloud { + +std::string get_key(const std::string& relative_path) { + return fmt::format("/{}", relative_path); +} + +void create_file_under_prefix(std::string_view prefix, size_t file_nums) { + for (size_t i = 0; i < file_nums; i++) { + _mock_fs->put_object(get_key(fmt::format("{}{}", prefix, i)), ""); + } +} + +TEST(S3AccessorTest, init) { + auto accessor = std::make_unique(S3Conf {}); + ASSERT_EQ(0, accessor->init()); +} + +TEST(S3AccessorTest, check_bucket_versioning) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + set_bucket_versioning_status_error = false; + }); + { ASSERT_EQ(0, accessor->check_bucket_versioning()); } + { + set_bucket_versioning_status_error = true; + ASSERT_EQ(-1, accessor->check_bucket_versioning()); + } +} + +TEST(S3AccessorTest, check_bucket_versioning_error) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(std::make_unique()); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + return_error_for_error_s3_client = true; + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + return_error_for_error_s3_client = false; + }); + ASSERT_EQ(-1, accessor->check_bucket_versioning()); +} + +TEST(S3AccessorTest, get_bucket_lifecycle) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + set_bucket_lifecycle = false; + }); + { + int64_t expiration_time = 0; + ASSERT_EQ(-1, accessor->get_bucket_lifecycle(&expiration_time)); + } + { + set_bucket_lifecycle = true; + int64_t expiration_time = 0; + ASSERT_EQ(0, accessor->get_bucket_lifecycle(&expiration_time)); + } +} + +TEST(S3AccessorTest, get_bucket_lifecycle_error) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(std::make_unique()); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + return_error_for_error_s3_client = true; + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + return_error_for_error_s3_client = false; + }); + int64_t expiration_time = 0; + ASSERT_EQ(-1, accessor->get_bucket_lifecycle(&expiration_time)); +} + +TEST(S3AccessorTest, list) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + }); + create_file_under_prefix("test_list", 300); + std::vector files; + ASSERT_EQ(0, accessor->list("test_list", &files)); + ASSERT_EQ(300, files.size()); +} + +TEST(S3AccessorTest, list_error) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(std::make_unique()); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + return_error_for_error_s3_client = true; + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + return_error_for_error_s3_client = false; + }); + create_file_under_prefix("test_list", 300); + std::vector files; + ASSERT_EQ(-1, accessor->list("test_list", &files)); +} + +TEST(S3AccessorTest, put) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + }); + std::string prefix = "test_put"; + for (size_t i = 0; i < 300; i++) { + ASSERT_EQ(0, accessor->put_object(fmt::format("{}{}", prefix, i), "")); + } + std::vector files; + ASSERT_EQ(0, accessor->list("test_put", &files)); + ASSERT_EQ(300, files.size()); +} + +TEST(S3AccessorTest, put_error) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(std::make_unique()); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + return_error_for_error_s3_client = false; + }); + std::string prefix = "test_put_error"; + for (size_t i = 0; i < 300; i++) { + if (i % 2) { + return_error_for_error_s3_client = true; + ASSERT_EQ(-1, accessor->put_object(fmt::format("{}{}", prefix, i), "")); + return_error_for_error_s3_client = false; + break; + } + ASSERT_EQ(0, accessor->put_object(fmt::format("{}{}", prefix, i), "")); + } + std::vector files; + ASSERT_EQ(0, accessor->list("test_put_error", &files)); +} + +TEST(S3AccessorTest, exist) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + }); + std::string prefix = "test_exist"; + ASSERT_EQ(1, accessor->exist(prefix)); + ASSERT_EQ(0, accessor->put_object(prefix, "")); + ASSERT_EQ(0, accessor->exist(prefix)); +} + +TEST(S3AccessorTest, exist_error) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(std::make_unique()); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + return_error_for_error_s3_client = false; + }); + std::string prefix = "test_exist_error"; + ASSERT_EQ(1, accessor->exist(prefix)); + ASSERT_EQ(0, accessor->put_object(prefix, "")); + return_error_for_error_s3_client = true; + ASSERT_EQ(-1, accessor->exist(prefix)); +} + +// function is not implemented +TEST(S3AccessorTest, DISABLED_delete_object) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + }); + std::string prefix = "test_delete_object"; + create_file_under_prefix(prefix, 200); + for (size_t i = 0; i < 200; i++) { + auto path = fmt::format("{}{}", prefix, i); + ASSERT_EQ(0, accessor->delete_object(path)); + ASSERT_EQ(1, accessor->exist(path)); + } +} + +TEST(S3AccessorTest, delete_objects) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + }); + std::string prefix = "test_delete_objects"; + std::vector paths; + size_t num = 300; + for (size_t i = 0; i < num; i++) { + auto path = fmt::format("{}{}", prefix, i); + _mock_fs->put_object(path, ""); + paths.emplace_back(std::move(path)); + } + ASSERT_EQ(0, accessor->delete_objects(paths)); + for (size_t i = 0; i < num; i++) { + auto path = fmt::format("{}{}", prefix, i); + ASSERT_EQ(1, accessor->exist(path)); + } +} + +TEST(S3AccessorTest, delete_objects_error) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(std::make_unique()); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + return_error_for_error_s3_client = false; + delete_objects_return_part_error = false; + }); + std::string prefix = "test_delete_objects"; + std::vector paths_first_half; + std::vector paths_second_half; + size_t num = 300; + for (size_t i = 0; i < num; i++) { + auto path = fmt::format("{}{}", prefix, i); + _mock_fs->put_object(path, ""); + if (i < 150) { + paths_first_half.emplace_back(std::move(path)); + } else { + paths_second_half.emplace_back(std::move(path)); + } + } + std::vector empty; + ASSERT_EQ(0, accessor->delete_objects(empty)); + return_error_for_error_s3_client = true; + delete_objects_return_part_error = true; + ASSERT_EQ(-1, accessor->delete_objects(paths_first_half)); + delete_objects_return_part_error = false; + ASSERT_EQ(-2, accessor->delete_objects(paths_second_half)); +} + +TEST(S3AccessorTest, delete_expired_objects) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + }); + { + std::string prefix = "atest_delete_expired_objects"; + size_t num = 2000; + create_file_under_prefix(prefix, num); + list_object_v2_with_expire_time = true; + expire_time = 50; + ASSERT_EQ(0, accessor->delete_expired_objects(prefix, 100)); + for (size_t i = 0; i < num; i++) { + auto path = fmt::format("{}{}", prefix, i); + ASSERT_EQ(1, accessor->exist(path)); + } + } + { + std::string prefix = "btest_delete_expired_objects"; + size_t num = 2000; + create_file_under_prefix(prefix, num); + list_object_v2_with_expire_time = true; + expire_time = 150; + ASSERT_EQ(0, accessor->delete_expired_objects(prefix, 100)); + for (size_t i = 0; i < num; i++) { + auto path = fmt::format("{}{}", prefix, i); + ASSERT_EQ(1, accessor->exist(path)); + } + } + { + std::string prefix = "ctest_delete_expired_objects"; + size_t num = 2000; + create_file_under_prefix(prefix, num); + list_object_v2_with_expire_time = true; + expire_time = 150; + return_error_for_error_s3_client = true; + std::unique_ptr> defer( + (int*)0x01, [&](int*) { return_error_for_error_s3_client = false; }); + ASSERT_EQ(0, accessor->delete_expired_objects(prefix, 100)); + } +} + +TEST(S3AccessorTest, delete_object_by_prefix) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + }); + std::string prefix = "test_delete_objects_by_prefix"; + size_t num = 2000; + create_file_under_prefix(prefix, num); + ASSERT_EQ(0, accessor->delete_objects_by_prefix(prefix)); + for (size_t i = 0; i < num; i++) { + auto path = fmt::format("{}{}", prefix, i); + ASSERT_EQ(1, accessor->exist(path)); + } +} + +TEST(S3AccessorTest, delete_object_by_prefix_error) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(std::make_unique()); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + return_error_for_error_s3_client = false; + delete_objects_return_part_error = false; + }); + std::string prefix = "test_delete_objects_by_prefix"; + size_t num = 2000; + create_file_under_prefix(prefix, num); + delete_objects_return_part_error = true; + return_error_for_error_s3_client = true; + ASSERT_EQ(-1, accessor->delete_objects_by_prefix(prefix)); + return_error_for_error_s3_client = false; + ASSERT_EQ(-2, accessor->delete_objects_by_prefix(prefix)); + delete_objects_return_part_error = false; + ASSERT_EQ(-3, accessor->delete_objects_by_prefix(prefix)); +} + +} // namespace cloud diff --git a/cloud/test/CMakeLists.txt b/cloud/test/CMakeLists.txt new file mode 100644 index 00000000000000..10cf255d42a38b --- /dev/null +++ b/cloud/test/CMakeLists.txt @@ -0,0 +1,121 @@ +# where to put generated libraries +set(LIBRARY_OUTPUT_PATH "${BUILD_DIR}/test") + +# where to put generated libraries +set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/test") + +set(TEST_LINK_LIBS ResourceManager ${TEST_LINK_LIBS}) + +add_executable(sync_point_test sync_point_test.cpp) + +add_executable(codec_test codec_test.cpp) + +add_executable(keys_test keys_test.cpp) + +add_executable(doris_txn_test doris_txn_test.cpp) + +add_executable(txn_kv_test txn_kv_test.cpp) + +add_executable(recycler_test recycler_test.cpp) + +add_executable(mem_txn_kv_test mem_txn_kv_test.cpp) + +add_executable(meta_service_test + meta_service_test.cpp + meta_service_job_test.cpp + meta_service_http_test.cpp + schema_kv_test.cpp +) + +add_executable(meta_server_test meta_server_test.cpp) + +add_executable(rate_limiter_test rate_limiter_test.cpp) + +add_executable(encryption_test encryption_test.cpp) + +add_executable(metric_test metric_test.cpp) + +add_executable(log_test log_test.cpp) + +add_executable(resource_test resource_test.cpp) + +add_executable(http_encode_key_test http_encode_key_test.cpp) + +add_executable(fdb_injection_test fdb_injection_test.cpp) + +add_executable(s3_accessor_test s3_accessor_test.cpp) + +add_executable(stopwatch_test stopwatch_test.cpp) + +message("Meta-service test dependencies: ${TEST_LINK_LIBS}") +target_link_libraries(sync_point_test ${TEST_LINK_LIBS}) + +target_link_libraries(codec_test ${TEST_LINK_LIBS}) + +target_link_libraries(keys_test ${TEST_LINK_LIBS}) + +target_link_libraries(meta_service_test ${TEST_LINK_LIBS}) + +target_link_libraries(meta_server_test ${TEST_LINK_LIBS}) + +target_link_libraries(rate_limiter_test ${TEST_LINK_LIBS}) + +target_link_libraries(encryption_test ${TEST_LINK_LIBS}) + +target_link_libraries(metric_test ${TEST_LINK_LIBS}) + +target_link_libraries(log_test ${TEST_LINK_LIBS}) + +target_link_libraries(resource_test ${TEST_LINK_LIBS}) + +target_link_libraries(http_encode_key_test ${TEST_LINK_LIBS}) + +target_link_libraries(s3_accessor_test ${TEST_LINK_LIBS}) + +target_link_libraries(stopwatch_test ${TEST_LINK_LIBS}) + +# FDB related tests need to be linked with libfdb_c +set(FDB_LINKER_FLAGS "-lfdb_c -L${THIRDPARTY_DIR}/lib") + +target_link_libraries(doris_txn_test + ${FDB_LINKER_FLAGS} + ${TEST_LINK_LIBS} +) + +target_link_libraries(txn_kv_test + ${FDB_LINKER_FLAGS} + ${TEST_LINK_LIBS} +) + +target_link_libraries(mem_txn_kv_test + ${FDB_LINKER_FLAGS} + ${TEST_LINK_LIBS} +) + +target_link_libraries(recycler_test + ${FDB_LINKER_FLAGS} + ${TEST_LINK_LIBS} +) + +target_link_libraries(fdb_injection_test + ${FDB_LINKER_FLAGS} + ${TEST_LINK_LIBS}) + +install(FILES + ${BASE_DIR}/script/run_all_tests.sh + ${BASE_DIR}/conf/fdb.cluster + ${BASE_DIR}/conf/doris_cloud.conf + ${BASE_DIR}/conf/lsan_suppression.conf + ${BASE_DIR}/test/fdb_metric_example.json + PERMISSIONS OWNER_READ OWNER_WRITE OWNER_EXECUTE + GROUP_READ GROUP_WRITE GROUP_EXECUTE + WORLD_READ WORLD_EXECUTE + DESTINATION ${BUILD_DIR}/test) + +install(FILES + ${THIRDPARTY_DIR}/lib/libfdb_c.so + PERMISSIONS OWNER_READ OWNER_WRITE OWNER_EXECUTE + GROUP_READ GROUP_WRITE GROUP_EXECUTE + WORLD_READ WORLD_EXECUTE + DESTINATION ${BUILD_DIR}/test) + diff --git a/cloud/test/codec_test.cpp b/cloud/test/codec_test.cpp new file mode 100644 index 00000000000000..86702efbd8fa4c --- /dev/null +++ b/cloud/test/codec_test.cpp @@ -0,0 +1,202 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "meta-service/codec.h" + +#include + +#include +#include + +#include "common/util.h" + +using namespace doris; + +int main(int argc, char** argv) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + +TEST(CodecTest, StringCodecTest) { + std::mt19937 gen(std::random_device("/dev/urandom")()); + const int max_len = (2 << 16) + 10086; + std::uniform_int_distribution rd_len(0, max_len); + std::uniform_int_distribution rd_char(std::numeric_limits::min(), + std::numeric_limits::max()); + + int ret = -1; + + // Correctness test + { + int case_count = 50; + std::string str1; + std::string str2; + str1.reserve(max_len); + str2.reserve(max_len); + std::string b1; + std::string b2; + std::string d1; + std::string d2; + b1.reserve(1 + max_len * 2 + 2); + b2.reserve(1 + max_len * 2 + 2); + d1.reserve(max_len); + d2.reserve(max_len); + while (case_count--) { + str1.clear(); + str2.clear(); + b1.clear(); + b2.clear(); + d1.clear(); + d2.clear(); + int len1 = rd_len(gen); + int len2 = rd_len(gen); + int zero_count1 = 0; + int zero_count2 = 0; + while (len1--) { + str1.push_back(rd_char(gen)); + str1.back() == 0x00 ? ++zero_count1 : zero_count1 += 0; + } + while (len2--) { + str2.push_back(rd_char(gen)); + str2.back() == 0x00 ? ++zero_count2 : zero_count2 += 0; + } + cloud::encode_bytes(str1, &b1); + cloud::encode_bytes(str2, &b2); + int sequence = std::memcmp(&str1[0], &str2[0], + str1.size() > str2.size() ? str2.size() : str1.size()); + int sequence_decoded = + std::memcmp(&b1[0], &b2[0], b1.size() > b2.size() ? b2.size() : b1.size()); + ASSERT_TRUE((sequence * sequence_decoded > 0) || + (sequence == 0 && sequence_decoded == 0)); + ASSERT_TRUE(b1[0] == cloud::EncodingTag::BYTES_TAG); + ASSERT_TRUE(b2[0] == cloud::EncodingTag::BYTES_TAG); + // Check encoded value size, marker + zero_escape + terminator + ASSERT_TRUE(b1.size() == (str1.size() + 1 + zero_count1 + 2)); + ASSERT_TRUE(b2.size() == (str2.size() + 1 + zero_count2 + 2)); + + // Decoding test + b1 += "cloud is good"; + b2 += "cloud will be better"; + std::string_view b1_sv(b1); + ret = cloud::decode_bytes(&b1_sv, &d1); + ASSERT_TRUE(ret == 0); + ASSERT_TRUE(d1 == str1); + std::string_view b2_sv(b2); + ret = cloud::decode_bytes(&b2_sv, &d2); + ASSERT_TRUE(ret == 0); + ASSERT_TRUE(d2 == str2); + ASSERT_TRUE(b1_sv == "cloud is good"); + ASSERT_TRUE(b2_sv == "cloud will be better"); + } + } + + // Boundary tests + { + std::vector strs; + std::vector expected; + + int zeroes = 1 * 1024 * 1024; + strs.emplace_back(zeroes, static_cast(0x00)); + expected.push_back(""); + expected.back().push_back(cloud::EncodingTag::BYTES_TAG); + while (zeroes--) { + expected.back().push_back(cloud::EncodingTag::BYTE_ESCAPE); + expected.back().push_back(cloud::EncodingTag::ESCAPED_00); + } + expected.back().push_back(cloud::EncodingTag::BYTE_ESCAPE); + expected.back().push_back(cloud::EncodingTag::BYTES_ENDING); + + ASSERT_TRUE(strs.size() == expected.size()); + for (int i = 0; i < strs.size(); ++i) { + std::string b1; + std::string d1; + std::string_view sv(strs[i]); + cloud::encode_bytes(sv, &b1); + ASSERT_TRUE(b1.size() == expected[i].size()); + for (int j = 0; j < b1.size(); ++j) { + ASSERT_TRUE(expected[i][j] == b1[j]); + } + std::string_view b1_sv(b1); + ret = cloud::decode_bytes(&b1_sv, &d1); + ASSERT_EQ(ret, 0); + ASSERT_EQ(b1_sv.size(), 0); + ASSERT_EQ(d1.size(), strs[i].size()); + ASSERT_TRUE(d1 == strs[i]); + } + } + + // Other tests + { + std::string str1 = "This is"; + std::string str2 = "tHIS IS"; + // Append something strange + str1.push_back(static_cast(0x00)); + str1 += "an string"; + str1.push_back(static_cast(0xff)); + str2.push_back(static_cast(0x00)); + str2 += "AN STRING"; + str2.push_back(static_cast(0xff)); + + // Output byte array + std::string b1; + std::string b2; + + cloud::encode_bytes(str1, &b1); + cloud::encode_bytes(str2, &b2); + ASSERT_TRUE(std::memcmp(&b1[0], &b2[0], b1.size() > b2.size() ? b2.size() : b1.size()) < 0); + + std::string str11; + std::string str22; + std::string_view b1_sv(b1); + std::string_view b2_sv(b2); + cloud::decode_bytes(&b1_sv, &str11); + cloud::decode_bytes(&b2_sv, &str22); + ASSERT_TRUE(str1 == str11); + ASSERT_TRUE(str2 == str22); + } +} + +TEST(CodecTest, Int64CodecTest) { + using namespace doris::cloud; + int ret = 0; + + // Basic test + { + std::string out1; + cloud::encode_int64(10086, &out1); + ASSERT_EQ(out1[0], cloud::EncodingTag::POSITIVE_FIXED_INT_TAG); + std::cout << hex(out1) << std::endl; + int64_t val1 = 10010; + std::string_view in(out1); + ret = cloud::decode_int64(&in, &val1); + ASSERT_EQ(ret, 0); + ASSERT_EQ(val1, 10086); + + std::string out2; + cloud::encode_int64(-1001011, &out2); + ASSERT_EQ(out2[0], cloud::EncodingTag::NEGATIVE_FIXED_INT_TAG); + std::cout << hex(out2) << std::endl; + int64_t val2 = 10086; + in = out2; + ret = cloud::decode_int64(&in, &val2); + ASSERT_EQ(ret, 0); + ASSERT_EQ(val2, -1001011); + + // Compare lexical order + ASSERT_LT(out2, out1); + } +} diff --git a/cloud/test/doris_txn_test.cpp b/cloud/test/doris_txn_test.cpp new file mode 100644 index 00000000000000..ca2fde09d6cfef --- /dev/null +++ b/cloud/test/doris_txn_test.cpp @@ -0,0 +1,100 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "meta-service/doris_txn.h" + +#include +#include +#include + +#include "common/config.h" +#include "common/util.h" +#include "meta-service/meta_service.h" +#include "meta-service/txn_kv.h" + +int main(int argc, char** argv) { + doris::cloud::config::init(nullptr, true); + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + +TEST(TxnIdConvert, TxnIdTest) { + using namespace doris::cloud; + + // Correctness test + { + // 00000182a5ed173f0012 + std::string ts("\x00\x00\x01\x82\xa5\xed\x17\x3f\x00\x12", 10); + ASSERT_EQ(ts.size(), 10); + int64_t txn_id0; + int ret = get_txn_id_from_fdb_ts(ts, &txn_id0); + ASSERT_EQ(ret, 0); + std::string str((char*)&txn_id0, sizeof(txn_id0)); + std::cout << "fdb_ts0: " << hex(ts) << " " + << "txn_id0: " << txn_id0 << " hex: " << hex(str) << std::endl; + + // 00000182a5ed173f0013 + ts = std::string("\x00\x00\x01\x82\xa5\xed\x17\x3f\x00\x13", 10); + ASSERT_EQ(ts.size(), 10); + int64_t txn_id1; + ret = get_txn_id_from_fdb_ts(ts, &txn_id1); + ASSERT_EQ(ret, 0); + ASSERT_GT(txn_id1, txn_id0); + str = std::string((char*)&txn_id1, sizeof(txn_id1)); + std::cout << "fdb_ts1: " << hex(ts) << " " + << "txn_id1: " << txn_id1 << " hex: " << hex(str) << std::endl; + + // 00000182a5ed174f0013 + ts = std::string("\x00\x00\x01\x82\xa5\xed\x17\x4f\x00\x13", 10); + ASSERT_EQ(ts.size(), 10); + int64_t txn_id2; + ret = get_txn_id_from_fdb_ts(ts, &txn_id2); + ASSERT_EQ(ret, 0); + ASSERT_GT(txn_id2, txn_id1); + str = std::string((char*)&txn_id2, sizeof(txn_id2)); + std::cout << "fdb_ts2: " << hex(ts) << " " + << "txn_id2: " << txn_id2 << " hex: " << hex(str) << std::endl; + } + + // Boundary test + { + // 1024 + // 00000182a5ed174f0400 + std::string ts("\x00\x00\x01\x82\xa5\xed\x17\x4f\x04\x00", 10); + ASSERT_EQ(ts.size(), 10); + int64_t txn_id; + int ret = get_txn_id_from_fdb_ts(ts, &txn_id); + ASSERT_EQ(ret, 2); // Exceed max seq + + // 1023 + // 00000182a5ed174f03ff + ts = std::string("\x00\x00\x01\x82\xa5\xed\x17\x4f\x03\xff", 10); + ret = get_txn_id_from_fdb_ts(ts, &txn_id); + ASSERT_EQ(ret, 0); + + // 0000 + // 00000182a5ed174f0000 + ts = std::string("\x00\x00\x01\x82\xa5\xed\x17\x4f\x03\x00", 10); + ret = get_txn_id_from_fdb_ts(ts, &txn_id); + ASSERT_EQ(ret, 0); + + // Insufficient length + ts = std::string("\x00\x00\x01\x82\xa5\xed\x17\x4f\x03\x00", 9); + ret = get_txn_id_from_fdb_ts(ts, &txn_id); + ASSERT_EQ(ret, 1); + } +} diff --git a/cloud/test/encryption_test.cpp b/cloud/test/encryption_test.cpp new file mode 100644 index 00000000000000..4e1d1bd18151df --- /dev/null +++ b/cloud/test/encryption_test.cpp @@ -0,0 +1,371 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include +#include + +#include "common/config.h" +#include "common/encryption_util.h" +#include "common/logging.h" +#include "common/sync_point.h" +#include "common/util.h" +#include "meta-service/keys.h" +#include "meta-service/mem_txn_kv.h" +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" + +using namespace doris; + +int main(int argc, char** argv) { + auto conf_file = "doris_cloud.conf"; + if (!cloud::config::init(conf_file, true)) { + std::cerr << "failed to init config file, conf=" << conf_file << std::endl; + return -1; + } + if (!cloud::init_glog("encrypt")) { + std::cerr << "failed to init glog" << std::endl; + return -1; + } + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + +TEST(EncryptionTest, EncryptTest) { + std::string mock_ak = "AKIDOsfsagadsgdfaadgdsgdf"; + std::string mock_sk = "Hx60p12123af234541nsVsffdfsdfghsdfhsdf34t"; + cloud::config::encryption_method = "AES_256_ECB"; + // "selectdbselectdbselectdbselectdb" -> "c2VsZWN0ZGJzZWxlY3RkYnNlbGVjdGRic2VsZWN0ZGI=" + cloud::config::encryption_key = "c2VsZWN0ZGJzZWxlY3RkYnNlbGVjdGRic2VsZWN0ZGI="; + { + std::string decoded_text(cloud::config::encryption_key.length(), '0'); + int decoded_text_len = + cloud::base64_decode(cloud::config::encryption_key.c_str(), + cloud::config::encryption_key.length(), decoded_text.data()); + ASSERT_TRUE(decoded_text_len > 0); + decoded_text.assign(decoded_text.data(), decoded_text_len); + std::cout << "decoded_string: " << decoded_text << std::endl; + ASSERT_EQ(decoded_text, "selectdbselectdbselectdbselectdb"); + int ret; + cloud::AkSkPair cipher_ak_sk_pair; + ret = cloud::encrypt_ak_sk({mock_ak, mock_sk}, cloud::config::encryption_method, + decoded_text, &cipher_ak_sk_pair); + ASSERT_EQ(ret, 0); + std::cout << "cipher ak: " << cipher_ak_sk_pair.first << std::endl; + std::cout << "cipher sk: " << cipher_ak_sk_pair.second << std::endl; + cloud::AkSkPair plain_ak_sk_pair; + ret = cloud::decrypt_ak_sk(cipher_ak_sk_pair, cloud::config::encryption_method, + decoded_text, &plain_ak_sk_pair); + ASSERT_EQ(ret, 0); + std::cout << "plain ak: " << plain_ak_sk_pair.first << std::endl; + std::cout << "plain sk: " << plain_ak_sk_pair.second << std::endl; + ASSERT_EQ(mock_ak, plain_ak_sk_pair.first); + ASSERT_EQ(mock_sk, plain_ak_sk_pair.second); + } +} + +TEST(EncryptionTest, RootKeyTestWithoutKms) { + using namespace doris::cloud; + + config::enable_kms = false; + // generate new root key + global_encryption_key_info_map.clear(); + auto mem_kv = std::make_shared(); + ASSERT_EQ(mem_kv->init(), 0); + auto ret = init_global_encryption_key_info_map(mem_kv.get()); + ASSERT_EQ(ret, 0); + ASSERT_EQ(global_encryption_key_info_map.size(), 1); + std::string key = system_meta_service_encryption_key_info_key(); + std::string val; + std::unique_ptr txn; + TxnErrorCode err = mem_kv->create_txn(&txn); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + err = txn->get(key, &val); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + EncryptionKeyInfoPB key_info; + key_info.ParseFromString(val); + ASSERT_EQ(key_info.items_size(), 1); + std::cout << proto_to_json(key_info) << std::endl; + const auto& item = key_info.items().at(0); + ASSERT_EQ(item.key_id(), 1); + std::string decoded_string(item.key().length(), '0'); + int decoded_text_len = + base64_decode(item.key().c_str(), item.key().length(), decoded_string.data()); + decoded_string.assign(decoded_string.data(), decoded_text_len); + ASSERT_EQ(global_encryption_key_info_map.at(item.key_id()), decoded_string); + + // get existed root key + global_encryption_key_info_map.clear(); + ret = init_global_encryption_key_info_map(mem_kv.get()); + ASSERT_EQ(ret, 0); + ASSERT_EQ(global_encryption_key_info_map.size(), 1); + ASSERT_EQ(global_encryption_key_info_map.at(item.key_id()), decoded_string); +} + +TEST(EncryptionTest, DecryptKmsInfo) { + std::string plaintext = "AKIDsZHqgyhdDSRBpfaNtHPHN0Wnpci"; + std::string encryption_key = "cloud"; + // use `echo -n "$plaintext" | openssl enc -aes-256-ecb -e -a -pbkdf2 -pass pass:"$encryption_key" -p -nosalt` + // to create hex_encryption_key and ciphertext + // then use `echo -n "$hex_encryption_key" | xxd -r -p | base64` to create encoded_encryption_key + std::string ciphertext = "DRdlYbJmyEPJ9q1KggTCjBErv/9GzyjTFKXBgGR7X4I="; + std::string encoded_encryption_key = "uwPXjGTuFJXyDZJBuYG52kdMxrWB24952HkXSa2v3Vw="; + std::string decoded_key(encoded_encryption_key.length(), '0'); + int decoded_key_len = cloud::base64_decode(encoded_encryption_key.c_str(), + encoded_encryption_key.length(), decoded_key.data()); + ASSERT_TRUE(decoded_key_len > 0); + decoded_key.assign(decoded_key.data(), decoded_key_len); + std::cout << "decoded_string: " << cloud::hex(decoded_key) << std::endl; + + cloud::AkSkPair out; + auto ret = cloud::decrypt_ak_sk({"", ciphertext}, "AES_256_ECB", decoded_key, &out); + ASSERT_EQ(ret, 0); + ASSERT_EQ(out.second, plaintext); +} + +TEST(EncryptionTest, RootKeyTestWithKms1) { + using namespace doris::cloud; + config::enable_kms = true; + + // incorrect kms conf + + // case1: empty param + config::kms_ak = ""; + config::kms_sk = "DRdlYbJmyEPJ9q1KggTCjBErv/9GzyjTFKXBgGR7X4I="; + config::kms_cmk = "2"; + config::kms_endpoint = "3"; + config::kms_region = "4"; + config::kms_provider = "ali"; + config::kms_info_encryption_key = "uwPXjGTuFJXyDZJBuYG52kdMxrWB24952HkXSa2v3Vw="; + + global_encryption_key_info_map.clear(); + auto mem_kv = std::make_shared(); + ASSERT_EQ(mem_kv->init(), 0); + auto ret = init_global_encryption_key_info_map(mem_kv.get()); + ASSERT_EQ(ret, -1); + ASSERT_TRUE(global_encryption_key_info_map.empty()); + + // case2: only support ali cloud + config::kms_ak = "DRdlYbJmyEPJ9q1KggTCjBErv/9GzyjTFKXBgGR7X4I="; + config::kms_provider = "tx"; // only support ali cloud + ret = init_global_encryption_key_info_map(mem_kv.get()); + ASSERT_EQ(ret, -1); + ASSERT_TRUE(global_encryption_key_info_map.empty()); +} + +TEST(EncryptionTest, RootKeyTestWithKms2) { + using namespace doris::cloud; + config::enable_kms = true; + config::kms_ak = "DRdlYbJmyEPJ9q1KggTCjBErv/9GzyjTFKXBgGR7X4I="; + config::kms_sk = "DRdlYbJmyEPJ9q1KggTCjBErv/9GzyjTFKXBgGR7X4I="; + config::kms_cmk = "2"; + config::kms_endpoint = "3"; + config::kms_region = "4"; + config::kms_provider = "ali"; + config::kms_info_encryption_key = "uwPXjGTuFJXyDZJBuYG52kdMxrWB24952HkXSa2v3Vw="; + + auto mem_kv = std::make_shared(); + ASSERT_EQ(mem_kv->init(), 0); + // clear for generating new root key + global_encryption_key_info_map.clear(); + + // Generate data key failed + { + // mock falied to generate key + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("alikms::generate_data_key::ret::pred", + [](void* pred) { *reinterpret_cast(pred) = true; }); + sp->set_call_back("alikms::generate_data_key::ret", [](void* p) { *((int*)p) = -1; }); + sp->enable_processing(); + + auto ret = init_global_encryption_key_info_map(mem_kv.get()); + ASSERT_EQ(ret, -1); + ASSERT_TRUE(global_encryption_key_info_map.empty()); + } + + // "selectdbselectdbselectdbselectdb" -> "c2VsZWN0ZGJzZWxlY3RkYnNlbGVjdGRic2VsZWN0ZGI=" + std::string plaintext = "selectdbselectdbselectdbselectdb"; + std::string mock_encoded_plaintext = "c2VsZWN0ZGJzZWxlY3RkYnNlbGVjdGRic2VsZWN0ZGI="; + std::string mock_encoded_ciphertext = mock_encoded_plaintext; + // Generate data key succeeded + { + // mock succ to generate key + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("alikms::generate_data_key::plaintext", + [&](void* p) { *((std::string*)p) = mock_encoded_plaintext; }); + sp->set_call_back("alikms::generate_data_key::ciphertext", + [&](void* p) { *((std::string*)p) = mock_encoded_ciphertext; }); + sp->set_call_back("alikms::generate_data_key::ret::pred", + [](void* pred) { *reinterpret_cast(pred) = true; }); + sp->set_call_back("alikms::generate_data_key::ret", [](void* p) { *((int*)p) = 0; }); + sp->enable_processing(); + auto ret = init_global_encryption_key_info_map(mem_kv.get()); + ASSERT_EQ(ret, 0); + ASSERT_EQ(global_encryption_key_info_map.size(), 1); + + std::string key = system_meta_service_encryption_key_info_key(); + std::string val; + std::unique_ptr txn; + TxnErrorCode err = mem_kv->create_txn(&txn); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + err = txn->get(key, &val); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + ASSERT_EQ(ret, 0); + EncryptionKeyInfoPB key_info; + key_info.ParseFromString(val); + ASSERT_EQ(key_info.items_size(), 1); + std::cout << proto_to_json(key_info) << std::endl; + const auto& item = key_info.items().at(0); + ASSERT_EQ(item.key_id(), 1); + ASSERT_EQ(global_encryption_key_info_map.at(item.key_id()), plaintext); + } + + // Decryption failed + { + // clear for getting existed root key from memkv + // do not need to mock kms encryption + global_encryption_key_info_map.clear(); + + // mock abnormal decryption + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("alikms::decrypt::ret::pred", + [](void* pred) { *reinterpret_cast(pred) = true; }); + sp->set_call_back("alikms::decrypt::ret", [](void* p) { *((int*)p) = -1; }); + sp->enable_processing(); + + // memkv already has key info + auto ret = init_global_encryption_key_info_map(mem_kv.get()); + ASSERT_EQ(ret, -1); + ASSERT_TRUE(global_encryption_key_info_map.empty()); + } + + // Decryption succeeded + { + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("alikms::decrypt::output", + [&](void* p) { *((std::string*)p) = mock_encoded_plaintext; }); + sp->set_call_back("alikms::decrypt::ret::pred", + [](void* pred) { *reinterpret_cast(pred) = true; }); + sp->set_call_back("alikms::decrypt::ret", [](void* p) { *((int*)p) = 0; }); + sp->enable_processing(); + auto ret = init_global_encryption_key_info_map(mem_kv.get()); + ASSERT_EQ(ret, 0); + ASSERT_EQ(global_encryption_key_info_map.size(), 1); + ASSERT_EQ(global_encryption_key_info_map.at(1), plaintext); + } +} + +TEST(EncryptionTest, RootKeyTestWithKms3) { + // test focus to add kms data key + using namespace doris::cloud; + config::enable_kms = false; + + config::enable_kms = false; + // generate new root key without kms + global_encryption_key_info_map.clear(); + auto mem_kv = std::make_shared(); + ASSERT_EQ(mem_kv->init(), 0); + auto ret = init_global_encryption_key_info_map(mem_kv.get()); + ASSERT_EQ(ret, 0); + ASSERT_EQ(global_encryption_key_info_map.size(), 1); + + // enable kms but not focus to add kms data key + config::enable_kms = true; + config::focus_add_kms_data_key = false; + config::kms_ak = "DRdlYbJmyEPJ9q1KggTCjBErv/9GzyjTFKXBgGR7X4I="; + config::kms_sk = "DRdlYbJmyEPJ9q1KggTCjBErv/9GzyjTFKXBgGR7X4I="; + config::kms_cmk = "2"; + config::kms_endpoint = "3"; + config::kms_region = "4"; + config::kms_provider = "ali"; + config::kms_info_encryption_key = "uwPXjGTuFJXyDZJBuYG52kdMxrWB24952HkXSa2v3Vw="; + global_encryption_key_info_map.clear(); + ret = init_global_encryption_key_info_map(mem_kv.get()); + ASSERT_EQ(ret, 0); + ASSERT_EQ(global_encryption_key_info_map.size(), 1); + + // enable kms and focus to add kms data key + config::focus_add_kms_data_key = true; + std::string plaintext = "test123456test"; + std::string mock_encoded_plaintext = "dGVzdDEyMzQ1NnRlc3Q="; + std::string mock_encoded_ciphertext = mock_encoded_plaintext; + // mock succ to generate key + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("alikms::generate_data_key::plaintext", + [&](void* p) { *((std::string*)p) = mock_encoded_plaintext; }); + sp->set_call_back("alikms::generate_data_key::ciphertext", + [&](void* p) { *((std::string*)p) = mock_encoded_ciphertext; }); + sp->set_call_back("alikms::generate_data_key::ret::pred", + [](void* pred) { *reinterpret_cast(pred) = true; }); + sp->set_call_back("alikms::generate_data_key::ret", [](void* p) { *((int*)p) = 0; }); + + sp->set_call_back("alikms::decrypt::output", + [&](void* p) { *((std::string*)p) = mock_encoded_plaintext; }); + sp->set_call_back("alikms::decrypt::ret::pred", + [](void* pred) { *reinterpret_cast(pred) = true; }); + sp->set_call_back("alikms::decrypt::ret", [](void* p) { *((int*)p) = 0; }); + sp->enable_processing(); + + global_encryption_key_info_map.clear(); + ret = init_global_encryption_key_info_map(mem_kv.get()); + ASSERT_EQ(ret, 0); + ASSERT_EQ(global_encryption_key_info_map.size(), 2); + + // get again + global_encryption_key_info_map.clear(); + ret = init_global_encryption_key_info_map(mem_kv.get()); + ASSERT_EQ(global_encryption_key_info_map.size(), 2); + + // finally check kv + std::string key = system_meta_service_encryption_key_info_key(); + std::string val; + std::unique_ptr txn; + TxnErrorCode err = mem_kv->create_txn(&txn); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + err = txn->get(key, &val); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + EncryptionKeyInfoPB key_info; + key_info.ParseFromString(val); + ASSERT_EQ(key_info.items_size(), 2); + std::cout << proto_to_json(key_info) << std::endl; + const auto& item = key_info.items().at(0); + ASSERT_EQ(item.key_id(), 1); + std::string decoded_string(item.key().length(), '0'); + int decoded_text_len = + base64_decode(item.key().c_str(), item.key().length(), decoded_string.data()); + decoded_string.assign(decoded_string.data(), decoded_text_len); + ASSERT_EQ(decoded_string, "selectdbselectdbselectdbselectdb"); // from config + ASSERT_EQ(global_encryption_key_info_map.at(item.key_id()), decoded_string); + + const auto& item2 = key_info.items().at(1); + ASSERT_EQ(item2.key_id(), 2); + std::string decoded_string2(item.key().length(), '0'); + int decoded_text_len2 = + base64_decode(item2.key().c_str(), item2.key().length(), decoded_string2.data()); + decoded_string2.assign(decoded_string2.data(), decoded_text_len2); + ASSERT_EQ(decoded_string2, plaintext); + ASSERT_EQ(global_encryption_key_info_map.at(item2.key_id()), decoded_string2); +} \ No newline at end of file diff --git a/cloud/test/fdb_injection_test.cpp b/cloud/test/fdb_injection_test.cpp new file mode 100644 index 00000000000000..c59a302d347169 --- /dev/null +++ b/cloud/test/fdb_injection_test.cpp @@ -0,0 +1,838 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#include "common/bvars.h" +#include "common/config.h" +#include "common/logging.h" +#include "common/sync_point.h" +#include "meta-service/meta_service.h" +#include "meta-service/txn_kv.h" + +using namespace doris; + +static std::unique_ptr create_meta_service() { + std::shared_ptr txn_kv = std::make_unique(); + [&]() { ASSERT_EQ(txn_kv->init(), 0); }(); + + auto rate_limiter = std::make_shared(); + auto rc_mgr = std::make_shared(txn_kv); + [&]() { ASSERT_EQ(rc_mgr->init(), 0); }(); + + auto meta_service_impl = std::make_unique(txn_kv, rc_mgr, rate_limiter); + return std::make_unique(std::move(meta_service_impl)); +} + +static std::unique_ptr meta_service; + +int main(int argc, char** argv) { + const std::string conf_file = "doris_cloud.conf"; + if (!cloud::config::init(conf_file.c_str(), true)) { + std::cerr << "failed to init config file, conf=" << conf_file << std::endl; + return -1; + } + + if (!cloud::init_glog("fdb_injection_test")) { + std::cerr << "failed to init glog" << std::endl; + return -1; + } + + ::testing::InitGoogleTest(&argc, argv); + + cloud::config::enable_txn_store_retry = true; + cloud::config::txn_store_retry_times = 100; + cloud::config::fdb_cluster_file_path = "fdb.cluster"; + cloud::config::write_schema_kv = true; + + auto sp = cloud::SyncPoint::get_instance(); + sp->enable_processing(); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_id", + [](void* p) { *reinterpret_cast(p) = 1; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->set_call_back("MetaServiceProxy::call_impl_duration_ms", + [](void* raw) { *reinterpret_cast(raw) = 0; }); + sp->set_call_back("put_schema_kv:schema_key_exists_return::pred", + [](void* pred) { *reinterpret_cast(pred) = true; }); + + meta_service = create_meta_service(); + + int ret = RUN_ALL_TESTS(); + if (ret != 0) { + std::cerr << "run first round of tests failed" << std::endl; + return ret; + } + + std::vector sync_points { + "transaction:init:create_transaction_err", + "transaction:commit:get_err", + "transaction:get:get_err", + "transaction:get_range:get_err", + "transaction:get_read_version:get_err", + "range_get_iterator:init:get_keyvalue_array_err", + }; + + // See + // 1. https://apple.github.io/foundationdb/api-error-codes.html#developer-guide-error-codes + // 2. FDB source code: flow/include/flow/error_definitions.h + std::vector retryable_not_committed { + // too old + 1007, + // future version + 1009, + // process_behind + 1037, + // database locked + 1038, + // commit_proxy_memory_limit_exceeded + 1042, + // batch_transaction_throttled + 1051, + // grv_proxy_memory_limit_exceeded + 1078, + // tag_throttled + 1213, + // proxy_tag_throttled + 1223, + }; + + for (auto err : retryable_not_committed) { + if (!fdb_error_predicate(FDB_ERROR_PREDICATE_RETRYABLE_NOT_COMMITTED, err)) { + LOG_WARNING("skip unknown err").tag("err", err).tag("msg", fdb_get_error(err)); + continue; + } + for (auto&& name : sync_points) { + for (auto&& clear_name : sync_points) { + sp->clear_call_back(clear_name); + } + + auto count = std::make_shared>(0); + auto inject_at = std::make_shared>(0); + sp->set_call_back(name, [=](void* raw) mutable { + size_t n = count->fetch_add(1); + if (n == *inject_at) { + *reinterpret_cast(raw) = err; + } + }); + sp->set_call_back("MetaServiceProxy::call_impl:1", [=](void*) { + // For each RPC invoking, inject every fdb txn kv call. + count->store(0); + inject_at->store(0); + }); + sp->set_call_back("MetaServiceProxy::call_impl:2", [=](void*) { + count->store(0); + inject_at->fetch_add(1); + }); + + ret = RUN_ALL_TESTS(); + if (ret != 0) { + std::cerr << "run test failed, sync_point=" << name << ", err=" << err + << ", msg=" << fdb_get_error(err) << std::endl; + return ret; + } + } + } + + meta_service.reset(); + + return 0; +} + +namespace doris::cloud { + +using Status = MetaServiceResponseStatus; + +static std::string cloud_unique_id(const std::string& instance_id) { + // degraded format + return fmt::format("1:{}:unique_id", instance_id); +} + +static int create_instance(MetaService* service, const std::string& instance_id) { + CreateInstanceRequest req; + CreateInstanceResponse resp; + req.set_instance_id(instance_id); + req.set_user_id("user_id"); + req.set_name(fmt::format("instance-{}", instance_id)); + req.set_sse_enabled(false); + + auto* obj_info = req.mutable_obj_info(); + obj_info->set_ak("access-key"); + obj_info->set_sk("secret-key"); + obj_info->set_bucket("cloud-test-bucket"); + obj_info->set_prefix("cloud-test"); + obj_info->set_endpoint("endpoint"); + obj_info->set_external_endpoint("endpoint"); + obj_info->set_region("region"); + obj_info->set_provider(ObjectStoreInfoPB_Provider_COS); + + brpc::Controller ctrl; + service->create_instance(&ctrl, &req, &resp, nullptr); + if (ctrl.Failed()) { + LOG_ERROR("create instance") + .tag("instance_id", instance_id) + .tag("code", ctrl.ErrorCode()) + .tag("msg", ctrl.ErrorText()); + return -1; + } + + auto code = resp.status().code(); + if (code != cloud::MetaServiceCode::OK && code != cloud::MetaServiceCode::ALREADY_EXISTED) { + LOG_ERROR("create instance") + .tag("instance_id", instance_id) + .tag("code", code) + .tag("msg", resp.status().msg()); + return -1; + } + + return 0; +} + +static int remove_instance(MetaService* service, const std::string& instance_id) { + AlterInstanceRequest req; + AlterInstanceResponse resp; + req.set_instance_id(instance_id); + req.set_op(AlterInstanceRequest_Operation_DROP); + + brpc::Controller ctrl; + service->alter_instance(&ctrl, &req, &resp, nullptr); + if (ctrl.Failed()) { + LOG_ERROR("alter_instance") + .tag("instance_id", instance_id) + .tag("code", ctrl.ErrorCode()) + .tag("msg", ctrl.ErrorText()); + return -1; + } + + auto code = resp.status().code(); + if (code != cloud::MetaServiceCode::OK) { + LOG_ERROR("create instance") + .tag("instance_id", instance_id) + .tag("code", code) + .tag("msg", resp.status().msg()); + return -1; + } + + return 0; +} + +static int add_cluster(MetaService* service, const std::string& instance_id) { + bool retry = false; + while (true) { + brpc::Controller ctrl; + cloud::AlterClusterRequest req; + cloud::AlterClusterResponse resp; + + req.set_instance_id(instance_id); + req.set_op(cloud::AlterClusterRequest_Operation::AlterClusterRequest_Operation_ADD_CLUSTER); + auto* cluster = req.mutable_cluster(); + auto name = fmt::format("instance-{}-cluster", instance_id); + cluster->set_cluster_id(name); + cluster->set_cluster_name(name); + cluster->set_type(cloud::ClusterPB_Type::ClusterPB_Type_SQL); + cluster->set_desc("cluster description"); + auto* node = cluster->add_nodes(); + node->set_ip("0.0.0.0"); + node->set_node_type(cloud::NodeInfoPB_NodeType::NodeInfoPB_NodeType_FE_MASTER); + node->set_cloud_unique_id(cloud_unique_id(instance_id)); + node->set_edit_log_port(123); + node->set_heartbeat_port(456); + node->set_name("default_node"); + + service->alter_cluster(&ctrl, &req, &resp, nullptr); + if (ctrl.Failed()) { + LOG_ERROR("alter cluster") + .tag("instance_id", instance_id) + .tag("code", ctrl.ErrorCode()) + .tag("msg", ctrl.ErrorText()); + return -1; + } + + auto code = resp.status().code(); + if (code == cloud::MetaServiceCode::OK) { + return 0; + } else if (code == cloud::MetaServiceCode::ALREADY_EXISTED) { + if (!retry) { + retry = true; + req.set_op(cloud::AlterClusterRequest_Operation:: + AlterClusterRequest_Operation_DROP_CLUSTER); + ctrl.Reset(); + service->alter_cluster(&ctrl, &req, &resp, nullptr); + } + return 0; + } else { + LOG_ERROR("add default cluster") + .tag("instance_id", instance_id) + .tag("code", code) + .tag("msg", resp.status().msg()); + return -1; + } + } +} + +static int drop_cluster(MetaService* service, const std::string& instance_id) { + AlterClusterRequest req; + AlterClusterResponse resp; + req.set_instance_id(instance_id); + req.set_op(AlterClusterRequest_Operation_DROP_CLUSTER); + auto cluster = req.mutable_cluster(); + cluster->set_cluster_id(fmt::format("instance-{}-cluster", instance_id)); + + brpc::Controller ctrl; + service->alter_cluster(&ctrl, &req, &resp, nullptr); + if (ctrl.Failed()) { + LOG_ERROR("drop cluster") + .tag("instance_id", instance_id) + .tag("code", ctrl.ErrorCode()) + .tag("msg", ctrl.ErrorText()); + return -1; + } + if (resp.status().code() != cloud::MetaServiceCode::OK) { + LOG_ERROR("drop cluster") + .tag("instance_id", instance_id) + .tag("code", resp.status().code()) + .tag("msg", resp.status().msg()); + return -1; + } + return 0; +} + +static doris::TabletMetaCloudPB add_tablet(int64_t table_id, int64_t index_id, int64_t partition_id, + int64_t tablet_id) { + doris::TabletMetaCloudPB tablet; + tablet.set_table_id(table_id); + tablet.set_index_id(index_id); + tablet.set_partition_id(partition_id); + tablet.set_tablet_id(tablet_id); + auto schema = tablet.mutable_schema(); + schema->set_schema_version(1); + auto first_rowset = tablet.add_rs_metas(); + first_rowset->set_rowset_id(0); // required + first_rowset->set_rowset_id_v2(std::to_string(1)); + first_rowset->set_start_version(0); + first_rowset->set_end_version(1); + first_rowset->mutable_tablet_schema()->CopyFrom(*schema); + return tablet; +} + +static int create_tablet(MetaService* meta_service, const std::string& instance_id, + int64_t table_id, int64_t index_id, int64_t partition_id, + int64_t tablet_id) { + if (tablet_id < 0) { + LOG_ERROR("invalid tablet id").tag("id", tablet_id); + return -1; + } + + brpc::Controller ctrl; + cloud::CreateTabletsRequest req; + cloud::CreateTabletsResponse resp; + req.set_cloud_unique_id(cloud_unique_id(instance_id)); + req.add_tablet_metas()->CopyFrom(add_tablet(table_id, index_id, partition_id, tablet_id)); + meta_service->create_tablets(&ctrl, &req, &resp, nullptr); + if (ctrl.Failed()) { + LOG_ERROR("create_tablets") + .tag("instance_id", instance_id) + .tag("tablet_id", tablet_id) + .tag("code", ctrl.ErrorCode()) + .tag("msg", ctrl.ErrorText()); + return -1; + } + if (resp.status().code() != cloud::MetaServiceCode::OK) { + LOG_ERROR("create tablet") + .tag("instance_id", instance_id) + .tag("tablet_id", tablet_id) + .tag("code", resp.status().code()) + .tag("msg", resp.status().msg()); + return -1; + } + return 0; +} + +static Status begin_txn(MetaService* service, const std::string& instance_id, std::string label, + int64_t db_id, const std::vector& tablet_ids, int64_t* txn_id) { + brpc::Controller ctrl; + cloud::BeginTxnRequest req; + cloud::BeginTxnResponse resp; + req.set_cloud_unique_id(cloud_unique_id(instance_id)); + auto* txn_info = req.mutable_txn_info(); + txn_info->set_label(label); + txn_info->set_db_id(db_id); + txn_info->mutable_table_ids()->Add(tablet_ids.begin(), tablet_ids.end()); + txn_info->set_timeout_ms(1000 * 60 * 60); + + service->begin_txn(&ctrl, &req, &resp, nullptr); + if (ctrl.Failed()) { + LOG_ERROR("begin_txn") + .tag("instance_id", instance_id) + .tag("table_ids", fmt::format("{}", fmt::join(tablet_ids, ","))) + .tag("label", label) + .tag("code", ctrl.ErrorCode()) + .tag("msg", ctrl.ErrorText()); + Status status; + status.set_code(MetaServiceCode::UNDEFINED_ERR); + status.set_msg(ctrl.ErrorText()); + return status; + } + + *txn_id = resp.txn_id(); + return resp.status(); +} + +static Status commit_txn(MetaService* service, const std::string& instance_id, int64_t txn_id, + int64_t db_id) { + brpc::Controller ctrl; + cloud::CommitTxnRequest req; + cloud::CommitTxnResponse resp; + req.set_cloud_unique_id(cloud_unique_id(instance_id)); + req.set_db_id(db_id); + req.set_txn_id(txn_id); + + service->commit_txn(&ctrl, &req, &resp, nullptr); + if (ctrl.Failed()) { + LOG_ERROR("commit_txn") + .tag("instance_id", instance_id) + .tag("txn_id", txn_id) + .tag("code", ctrl.ErrorCode()) + .tag("msg", ctrl.ErrorText()); + Status status; + status.set_code(MetaServiceCode::UNDEFINED_ERR); + status.set_msg(ctrl.ErrorText()); + return status; + } + + return resp.status(); +} + +static doris::RowsetMetaCloudPB create_rowset(int64_t tablet_id, std::string rowset_id, + int64_t txn_id, int64_t index_id, + int64_t partition_id, int64_t version = -1, + int num_rows = 100) { + doris::RowsetMetaCloudPB rowset; + rowset.set_rowset_id(0); // required + rowset.set_rowset_id_v2(rowset_id); + rowset.set_index_id(index_id); + rowset.set_partition_id(partition_id); + rowset.set_tablet_id(tablet_id); + rowset.set_txn_id(txn_id); + if (version >= 0) { + rowset.set_start_version(version); + rowset.set_end_version(version); + } + rowset.set_num_segments(1); + rowset.set_num_rows(num_rows); + rowset.set_data_disk_size(num_rows * 100); + rowset.set_txn_expiration(10000); + rowset.set_schema_version(1); + + // auto* schema = rowset.mutable_tablet_schema(); + // schema->set_schema_version(1); + // schema->set_disable_auto_compaction(true); + + auto* bound = rowset.add_segments_key_bounds(); + bound->set_min_key("min_key"); + bound->set_max_key("max_key"); + + return rowset; +} + +static Status prepare_rowset(MetaService* service, const std::string& instance_id, int64_t txn_id, + int64_t tablet_id, std::string rowset_id, int64_t index_id, + int64_t partition_id) { + brpc::Controller ctrl; + cloud::CreateRowsetRequest req; + cloud::CreateRowsetResponse resp; + req.set_cloud_unique_id(cloud_unique_id(instance_id)); + req.set_temporary(true); + req.mutable_rowset_meta()->CopyFrom( + create_rowset(tablet_id, rowset_id, txn_id, index_id, partition_id)); + + service->prepare_rowset(&ctrl, &req, &resp, nullptr); + if (ctrl.Failed()) { + LOG_ERROR("prepare_rowset") + .tag("instance_id", instance_id) + .tag("txn_id", txn_id) + .tag("tablet_id", tablet_id) + .tag("rowset_id", rowset_id) + .tag("code", ctrl.ErrorCode()) + .tag("msg", ctrl.ErrorText()); + Status status; + status.set_code(MetaServiceCode::UNDEFINED_ERR); + status.set_msg(ctrl.ErrorText()); + return status; + } + return resp.status(); +} + +static Status commit_rowset(MetaService* service, const std::string& instance_id, int64_t txn_id, + int64_t tablet_id, std::string rowset_id, int64_t index_id, + int64_t partition_id) { + brpc::Controller ctrl; + cloud::CreateRowsetRequest req; + cloud::CreateRowsetResponse resp; + req.set_cloud_unique_id(cloud_unique_id(instance_id)); + req.set_temporary(true); + req.mutable_rowset_meta()->CopyFrom( + create_rowset(tablet_id, rowset_id, txn_id, index_id, partition_id)); + LOG_INFO("send commit rowset request"); + + service->commit_rowset(&ctrl, &req, &resp, nullptr); + if (ctrl.Failed()) { + LOG_ERROR("commit_rowset") + .tag("instance_id", instance_id) + .tag("txn_id", txn_id) + .tag("tablet_id", tablet_id) + .tag("rowset_id", rowset_id) + .tag("code", ctrl.ErrorCode()) + .tag("msg", ctrl.ErrorText()); + + Status status; + status.set_code(MetaServiceCode::UNDEFINED_ERR); + status.set_msg(ctrl.ErrorText()); + return status; + } + return resp.status(); +} + +static Status insert_rowset(MetaService* service, const std::string& instance_id, int64_t txn_id, + int64_t tablet_id, std::string rowset_id, int64_t index_id, + int64_t partition_id) { + auto status = prepare_rowset(service, instance_id, txn_id, tablet_id, rowset_id, index_id, + partition_id); + if (status.code() != cloud::MetaServiceCode::OK) { + LOG_ERROR("prepare_rowset") + .tag("instance_id", instance_id) + .tag("txn_id", txn_id) + .tag("tablet_id", tablet_id) + .tag("rowset_id", rowset_id) + .tag("code", status.code()) + .tag("msg", status.msg()); + if (status.code() != cloud::MetaServiceCode::ALREADY_EXISTED) { + return status; + } + } + + LOG_INFO("prepare rowset").tag("code", status.code()).tag("msg", status.msg()); + status = commit_rowset(service, instance_id, txn_id, tablet_id, rowset_id, index_id, + partition_id); + if (status.code() != cloud::MetaServiceCode::OK) { + LOG_ERROR("commit_rowset") + .tag("instance_id", instance_id) + .tag("txn_id", txn_id) + .tag("tablet_id", tablet_id) + .tag("rowset_id", rowset_id) + .tag("code", status.code()) + .tag("msg", status.msg()); + return status; + } + + return Status(); +} + +static Status copy_into(MetaService* service, const std::string& instance_id, std::string label, + int64_t db_id, int64_t index_id, int64_t partition_id, + const std::vector& tablet_ids) { + int64_t txn_id = 0; + auto status = begin_txn(service, instance_id, label, db_id, tablet_ids, &txn_id); + if (status.code() != cloud::MetaServiceCode::OK) { + LOG_ERROR("begin_txn") + .tag("instance_id", instance_id) + .tag("label", label) + .tag("code", status.code()) + .tag("msg", status.msg()); + return status; + } + + for (uint64_t tablet_id : tablet_ids) { + auto rowset_id = fmt::format("rowset_{}_{}", label, tablet_id); + status = insert_rowset(service, instance_id, txn_id, tablet_id, rowset_id, index_id, + partition_id); + if (status.code() != cloud::MetaServiceCode::OK) { + return status; + } + } + + status = commit_txn(service, instance_id, txn_id, db_id); + if (status.code() != cloud::MetaServiceCode::OK) { + LOG_ERROR("commit_txn") + .tag("instance_id", instance_id) + .tag("label", label) + .tag("code", status.code()) + .tag("msg", status.msg()); + return status; + } + + return Status(); +} + +static Status get_read_version(MetaService* service, const std::string& instance_id, + int64_t* version) { + brpc::Controller ctrl; + cloud::GetCurrentMaxTxnRequest req; + cloud::GetCurrentMaxTxnResponse resp; + req.set_cloud_unique_id(cloud_unique_id(instance_id)); + service->get_current_max_txn_id(&ctrl, &req, &resp, nullptr); + + if (ctrl.Failed()) { + LOG_ERROR("get read version") + .tag("instance_id", instance_id) + .tag("code", ctrl.ErrorCode()) + .tag("msg", ctrl.ErrorText()); + Status status; + status.set_code(MetaServiceCode::UNDEFINED_ERR); + status.set_msg(ctrl.ErrorText()); + return status; + } + + auto status = resp.status(); + if (status.code() != cloud::MetaServiceCode::OK) { + LOG_ERROR("get read version") + .tag("instance_id", instance_id) + .tag("code", status.code()) + .tag("msg", status.msg()); + return status; + } + + *version = resp.current_max_txn_id(); + + return Status(); +} + +static Status get_version(MetaService* service, const std::string& instance_id, int64_t db_id, + int64_t partition_id, int64_t table_id, int64_t* version) { + brpc::Controller ctrl; + cloud::GetVersionRequest req; + cloud::GetVersionResponse resp; + req.set_cloud_unique_id(cloud_unique_id(instance_id)); + req.set_db_id(db_id); + req.set_partition_id(partition_id); + req.set_table_id(table_id); + + service->get_version(&ctrl, &req, &resp, nullptr); + if (ctrl.Failed()) { + LOG_ERROR("get version") + .tag("instance_id", instance_id) + .tag("db_id", db_id) + .tag("partition_id", partition_id) + .tag("table_id", table_id) + .tag("code", ctrl.ErrorCode()) + .tag("msg", ctrl.ErrorText()); + Status status; + status.set_code(MetaServiceCode::UNDEFINED_ERR); + status.set_msg(ctrl.ErrorText()); + return status; + } + + auto status = resp.status(); + if (status.code() != cloud::MetaServiceCode::OK) { + LOG_ERROR("get version") + .tag("instance_id", instance_id) + .tag("db_id", db_id) + .tag("partition_id", partition_id) + .tag("table_id", table_id) + .tag("code", status.code()) + .tag("msg", status.msg()); + return status; + } + + *version = resp.version(); + + return Status(); +} + +static Status get_tablet_meta(MetaService* service, const std::string& instance_id, + int64_t tablet_id) { + brpc::Controller ctrl; + cloud::GetTabletRequest req; + cloud::GetTabletResponse resp; + req.set_cloud_unique_id(cloud_unique_id(instance_id)); + req.set_tablet_id(tablet_id); + service->get_tablet(&ctrl, &req, &resp, nullptr); + if (ctrl.Failed()) { + LOG_ERROR("get_tablet") + .tag("instance_id", instance_id) + .tag("tablet_id", tablet_id) + .tag("code", ctrl.ErrorCode()) + .tag("msg", ctrl.ErrorText()); + Status status; + status.set_code(MetaServiceCode::UNDEFINED_ERR); + status.set_msg(ctrl.ErrorText()); + return status; + } + + auto status = resp.status(); + if (status.code() != cloud::MetaServiceCode::OK) { + LOG_ERROR("get_tablet") + .tag("instance_id", instance_id) + .tag("tablet_id", tablet_id) + .tag("code", status.code()) + .tag("msg", status.msg()); + return status; + } + + return Status(); +} + +static Status get_tablet_stats(MetaService* service, const std::string& instance_id, + int64_t tablet_id, cloud::TabletStatsPB* stats) { + brpc::Controller ctrl; + cloud::GetTabletStatsRequest req; + cloud::GetTabletStatsResponse resp; + req.set_cloud_unique_id(cloud_unique_id(instance_id)); + auto* tablet_idx = req.mutable_tablet_idx()->Add(); + tablet_idx->set_tablet_id(tablet_id); + service->get_tablet_stats(&ctrl, &req, &resp, nullptr); + if (ctrl.Failed()) { + LOG_ERROR("get tablet stats") + .tag("instance_id", instance_id) + .tag("tablet_id", tablet_id) + .tag("code", ctrl.ErrorCode()) + .tag("msg", ctrl.ErrorText()); + Status status; + status.set_code(MetaServiceCode::UNDEFINED_ERR); + status.set_msg(ctrl.ErrorText()); + return status; + } + + auto status = resp.status(); + if (status.code() != cloud::MetaServiceCode::OK) { + LOG_ERROR("get_tablet_stats") + .tag("instance_id", instance_id) + .tag("tablet_id", tablet_id) + .tag("code", status.code()) + .tag("msg", status.msg()); + return status; + } + + stats->CopyFrom(resp.tablet_stats().at(0)); + + return Status(); +} + +static Status get_rowset_meta(MetaService* service, const std::string& instance_id, + int64_t tablet_id, int64_t version, + const cloud::TabletStatsPB& stats) { + brpc::Controller ctrl; + cloud::GetRowsetRequest req; + cloud::GetRowsetResponse resp; + req.set_cloud_unique_id(cloud_unique_id(instance_id)); + req.mutable_idx()->set_tablet_id(tablet_id); + req.set_start_version(0); + req.set_end_version(version); + req.set_base_compaction_cnt(stats.base_compaction_cnt()); + req.set_cumulative_compaction_cnt(stats.cumulative_compaction_cnt()); + req.set_cumulative_point(stats.cumulative_point()); + service->get_rowset(&ctrl, &req, &resp, nullptr); + if (ctrl.Failed()) { + LOG_ERROR("get_rowset") + .tag("instance_id", instance_id) + .tag("tablet_id", tablet_id) + .tag("code", ctrl.ErrorCode()) + .tag("msg", ctrl.ErrorText()); + Status status; + status.set_code(MetaServiceCode::UNDEFINED_ERR); + status.set_msg(ctrl.ErrorText()); + return status; + } + + auto status = resp.status(); + if (status.code() != cloud::MetaServiceCode::OK) { + LOG_ERROR("get_rowset") + .tag("instance_id", instance_id) + .tag("tablet_id", tablet_id) + .tag("code", status.code()) + .tag("msg", status.msg()); + return status; + } + + return Status(); +} + +TEST(FdbInjectionTest, AllInOne) { + auto now = std::chrono::high_resolution_clock::now(); + auto ns = std::chrono::duration_cast(now.time_since_epoch()); + std::string instance_id = fmt::format("fdb_injection_test_{}", ns.count()); + int ret = create_instance(meta_service.get(), instance_id); + ASSERT_EQ(ret, 0); + + ret = add_cluster(meta_service.get(), instance_id); + ASSERT_EQ(ret, 0); + + int64_t db_id = 1; + int64_t table_id = 1; + int64_t index_id = 1; + int64_t partition_id = 1; + std::vector tablet_ids; + for (int64_t tablet_id = 1; tablet_id <= 10; ++tablet_id) { + ret = create_tablet(meta_service.get(), instance_id, table_id, index_id, partition_id, + tablet_id); + ASSERT_EQ(ret, 0) << tablet_id; + tablet_ids.push_back(tablet_id); + } + + // Run copy into. + std::string label = fmt::format("{}-label", instance_id); + auto status = copy_into(meta_service.get(), instance_id, label, db_id, index_id, partition_id, + tablet_ids); + ASSERT_EQ(status.code(), MetaServiceCode::OK) << status.msg(); + + // Get version + int64_t version = 0; + status = get_version(meta_service.get(), instance_id, db_id, partition_id, table_id, &version); + ASSERT_EQ(status.code(), MetaServiceCode::OK) << status.msg(); + + // Get tablet meta & stats + TabletStatsPB stats; + status = get_tablet_stats(meta_service.get(), instance_id, tablet_ids.back(), &stats); + ASSERT_EQ(status.code(), MetaServiceCode::OK) << status.msg(); + + status = get_tablet_meta(meta_service.get(), instance_id, tablet_ids.back()); + ASSERT_EQ(status.code(), MetaServiceCode::OK) << status.msg(); + + // Get rowset metas. + status = get_rowset_meta(meta_service.get(), instance_id, tablet_ids.back(), version, stats); + ASSERT_EQ(status.code(), MetaServiceCode::OK) << status.msg(); + + int64_t max_txn_id = 0; + status = get_read_version(meta_service.get(), instance_id, &max_txn_id); + ASSERT_EQ(status.code(), MetaServiceCode::OK) << status.msg(); + ASSERT_GE(max_txn_id, version); + + ret = drop_cluster(meta_service.get(), instance_id); + ASSERT_EQ(ret, 0); + + ret = remove_instance(meta_service.get(), instance_id); + ASSERT_EQ(ret, 0); +} + +} // namespace doris::cloud diff --git a/cloud/test/fdb_metric_example.json b/cloud/test/fdb_metric_example.json new file mode 100644 index 00000000000000..19679b33c2bf6c --- /dev/null +++ b/cloud/test/fdb_metric_example.json @@ -0,0 +1,3058 @@ +{ + "client":{ + "cluster_file":{ + "path":"/etc/foundationdb/fdb.cluster", + "up_to_date":true + }, + "coordinators":{ + "coordinators":[ + { + "address":"10.29.0.115:4500", + "protocol":"0fdb00b071010000", + "reachable":true + }, + { + "address":"10.29.0.116:4500", + "protocol":"0fdb00b071010000", + "reachable":true + }, + { + "address":"10.29.0.117:4500", + "protocol":"0fdb00b071010000", + "reachable":true + } + ], + "quorum_reachable":true + }, + "database_status":{ + "available":true, + "healthy":true + }, + "messages":[ + + ], + "timestamp":1676535571 + }, + "cluster":{ + "active_primary_dc":"", + "active_tss_count":0, + "bounce_impact":{ + "can_clean_bounce":true + }, + "clients":{ + "count":8, + "supported_versions":[ + { + "client_version":"7.1.23", + "connected_clients":[ + { + "address":"10.29.0.115:36948", + "log_group":"default" + }, + { + "address":"10.29.0.115:44048", + "log_group":"default" + }, + { + "address":"10.29.0.115:49938", + "log_group":"default" + }, + { + "address":"10.29.0.115:52586", + "log_group":"default" + }, + { + "address":"10.29.0.116:33688", + "log_group":"default" + }, + { + "address":"10.29.0.116:37854", + "log_group":"default" + }, + { + "address":"10.29.0.117:37552", + "log_group":"default" + }, + { + "address":"10.29.0.117:48750", + "log_group":"default" + } + ], + "count":8, + "max_protocol_clients":[ + { + "address":"10.29.0.115:36948", + "log_group":"default" + }, + { + "address":"10.29.0.115:44048", + "log_group":"default" + }, + { + "address":"10.29.0.115:49938", + "log_group":"default" + }, + { + "address":"10.29.0.115:52586", + "log_group":"default" + }, + { + "address":"10.29.0.116:33688", + "log_group":"default" + }, + { + "address":"10.29.0.116:37854", + "log_group":"default" + }, + { + "address":"10.29.0.117:37552", + "log_group":"default" + }, + { + "address":"10.29.0.117:48750", + "log_group":"default" + } + ], + "max_protocol_count":8, + "protocol_version":"fdb00b071010000", + "source_version":"dd2e68b9b3175667914673539f06b8e1071c07d6" + } + ] + }, + "cluster_controller_timestamp":1676535571, + "configuration":{ + "backup_worker_enabled":0, + "blob_granules_enabled":0, + "coordinators_count":3, + "excluded_servers":[ + + ], + "log_spill":2, + "perpetual_storage_wiggle":0, + "perpetual_storage_wiggle_locality":"0", + "redundancy_mode":"double", + "storage_engine":"ssd-2", + "storage_migration_type":"disabled", + "tenant_mode":"disabled", + "usable_regions":1 + }, + "connection_string":"1y4c3E4b:SEXl2czWgZccifbXVaQ3vg6PoQ3UAgDF@10.29.0.115:4500,10.29.0.116:4500,10.29.0.117:4500", + "data":{ + "average_partition_size_bytes":25910030, + "least_operating_space_bytes_log_server":462795328579, + "least_operating_space_bytes_storage_server":462795328121, + "moving_data":{ + "highest_priority":0, + "in_flight_bytes":0, + "in_queue_bytes":0, + "total_written_bytes":619396980 + }, + "partitions_count":150, + "state":{ + "healthy":true, + "min_replicas_remaining":2, + "name":"healthy" + }, + "system_kv_size_bytes":0, + "team_trackers":[ + { + "in_flight_bytes":0, + "primary":true, + "state":{ + "healthy":true, + "min_replicas_remaining":2, + "name":"healthy" + }, + "unhealthy_servers":0 + } + ], + "total_disk_used_bytes":11492724736, + "total_kv_size_bytes":4316447722 + }, + "database_available":true, + "database_lock_state":{ + "locked":false + }, + "datacenter_lag":{ + "seconds":0, + "versions":0 + }, + "degraded_processes":0, + "fault_tolerance":{ + "max_zone_failures_without_losing_availability":1, + "max_zone_failures_without_losing_data":1 + }, + "full_replication":true, + "generation":25, + "incompatible_connections":[ + + ], + "latency_probe":{ + "batch_priority_transaction_start_seconds":0.00022602100000000002, + "commit_seconds":0.0014426700000000001, + "immediate_priority_transaction_start_seconds":0.00028300300000000004, + "read_seconds":0.00012659999999999999, + "transaction_start_seconds":0.00032758700000000001 + }, + "layers":{ + "_valid":true, + "backup":{ + "blob_recent_io":{ + "bytes_per_second":0, + "bytes_sent":0, + "requests_failed":0, + "requests_successful":0 + }, + "instances":{ + "16a0c56c38aec57f5009ac79299db14b":{ + "blob_stats":{ + "recent":{ + "bytes_per_second":0, + "bytes_sent":0, + "requests_failed":0, + "requests_successful":0 + }, + "total":{ + "bytes_sent":0, + "requests_failed":0, + "requests_successful":0 + } + }, + "configured_workers":10, + "id":"16a0c56c38aec57f5009ac79299db14b", + "last_updated":1676535553.0570624, + "main_thread_cpu_seconds":5061.7274200000002, + "memory_usage":588800000, + "process_cpu_seconds":5704.9799029999995, + "resident_size":126873600, + "version":"7.1.23" + }, + "2844cf9305901a25130e26c6c9f77739":{ + "blob_stats":{ + "recent":{ + "bytes_per_second":0, + "bytes_sent":0, + "requests_failed":0, + "requests_successful":0 + }, + "total":{ + "bytes_sent":0, + "requests_failed":0, + "requests_successful":0 + } + }, + "configured_workers":10, + "id":"2844cf9305901a25130e26c6c9f77739", + "last_updated":1676535546.8574159, + "main_thread_cpu_seconds":4783.8304990000006, + "memory_usage":580411392, + "process_cpu_seconds":5413.4282490000005, + "resident_size":116719616, + "version":"7.1.23" + }, + "555c929aefa1c9c9824b003a6ed29336":{ + "blob_stats":{ + "recent":{ + "bytes_per_second":0, + "bytes_sent":0, + "requests_failed":0, + "requests_successful":0 + }, + "total":{ + "bytes_sent":0, + "requests_failed":0, + "requests_successful":0 + } + }, + "configured_workers":10, + "id":"555c929aefa1c9c9824b003a6ed29336", + "last_updated":1676535552.9703305, + "main_thread_cpu_seconds":5046.9261430000006, + "memory_usage":579833856, + "process_cpu_seconds":5694.2449750000005, + "resident_size":123768832, + "version":"7.1.23" + } + }, + "instances_running":3, + "last_updated":1676535553.0570624, + "paused":false, + "tags":{ + "2022-12-14":{ + "current_container":"file:///mnt/fdbbackup/backup-2022-12-14-10-52-04.891403", + "current_status":"has been completed", + "last_restorable_seconds_behind":5549884.9560989998, + "last_restorable_version":2809214799283, + "mutation_log_bytes_written":760, + "mutation_stream_id":"589e8c6678f5fb4d7c0a2d7250080fe1", + "range_bytes_written":5135069254, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2022-12-15":{ + "current_container":"file:///mnt/fdbbackup/backup-2022-12-15-01-10-01.132541", + "current_status":"has been completed", + "last_restorable_seconds_behind":5498408.9221609998, + "last_restorable_version":2860690833221, + "mutation_log_bytes_written":760, + "mutation_stream_id":"f8da064c30c581ab24f4792a9052f586", + "range_bytes_written":5135069254, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2022-12-16":{ + "current_container":"file:///mnt/fdbbackup/backup-2022-12-16-01-10-01.925131", + "current_status":"has been completed", + "last_restorable_seconds_behind":5412008.2295059999, + "last_restorable_version":2947091525876, + "mutation_log_bytes_written":760, + "mutation_stream_id":"d8479657681a3e5f4701ed95ae954f48", + "range_bytes_written":4377982969, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2022-12-17":{ + "current_container":"file:///mnt/fdbbackup/backup-2022-12-17-01-10-01.338838", + "current_status":"has been completed", + "last_restorable_seconds_behind":5325607.7601429997, + "last_restorable_version":3033491995239, + "mutation_log_bytes_written":760, + "mutation_stream_id":"60aeb05eb8a8f1f0dcace6a845ccda1f", + "range_bytes_written":4261283286, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2022-12-18":{ + "current_container":"file:///mnt/fdbbackup/backup-2022-12-18-01-10-01.107178", + "current_status":"has been completed", + "last_restorable_seconds_behind":5239208.2455810001, + "last_restorable_version":3119891509801, + "mutation_log_bytes_written":760, + "mutation_stream_id":"31c6498b3fc2a919b12d512cce8f05ac", + "range_bytes_written":4261283286, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2022-12-19":{ + "current_container":"file:///mnt/fdbbackup/backup-2022-12-19-01-10-01.307658", + "current_status":"has been completed", + "last_restorable_seconds_behind":5152808.6855769996, + "last_restorable_version":3206291069805, + "mutation_log_bytes_written":760, + "mutation_stream_id":"d0523f3c12842039dd7b1e95a7f5be23", + "range_bytes_written":4261078593, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2022-12-20":{ + "current_container":"file:///mnt/fdbbackup/backup-2022-12-20-01-10-01.111076", + "current_status":"has been completed", + "last_restorable_seconds_behind":5066409.3317639995, + "last_restorable_version":3292690423618, + "mutation_log_bytes_written":760, + "mutation_stream_id":"9c2e5fd4ffa20e427ae44179f56cc21a", + "range_bytes_written":4261082453, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2022-12-21":{ + "current_container":"file:///mnt/fdbbackup/backup-2022-12-21-01-10-01.370828", + "current_status":"has been completed", + "last_restorable_seconds_behind":4980008.5864390004, + "last_restorable_version":3379091168943, + "mutation_log_bytes_written":760, + "mutation_stream_id":"9f01def6266df1c078d5b0ec0d3abea3", + "range_bytes_written":4261082453, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2022-12-22":{ + "current_container":"file:///mnt/fdbbackup/backup-2022-12-22-01-10-01.317463", + "current_status":"has been completed", + "last_restorable_seconds_behind":4893609.2052539997, + "last_restorable_version":3465490550128, + "mutation_log_bytes_written":760, + "mutation_stream_id":"59d895c4f9703b77f6e198d122fc3f5f", + "range_bytes_written":4262079296, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2022-12-23":{ + "current_container":"file:///mnt/fdbbackup/backup-2022-12-23-01-10-01.069940", + "current_status":"has been completed", + "last_restorable_seconds_behind":4807209.3429380003, + "last_restorable_version":3551890412444, + "mutation_log_bytes_written":760, + "mutation_stream_id":"b47514280012f171af9e02a58e89a03b", + "range_bytes_written":4262119487, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2022-12-24":{ + "current_container":"file:///mnt/fdbbackup/backup-2022-12-24-01-10-01.268872", + "current_status":"has been completed", + "last_restorable_seconds_behind":4720808.9462400004, + "last_restorable_version":3638290809142, + "mutation_log_bytes_written":760, + "mutation_stream_id":"ecccde69471c796a2bacd482d0ff5d82", + "range_bytes_written":4262119487, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2022-12-25":{ + "current_container":"file:///mnt/fdbbackup/backup-2022-12-25-01-10-01.104972", + "current_status":"has been completed", + "last_restorable_seconds_behind":4634409.4123339998, + "last_restorable_version":3724690343048, + "mutation_log_bytes_written":760, + "mutation_stream_id":"a3cd9647d1941106e14e944bae0f1008", + "range_bytes_written":4262066724, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2022-12-26":{ + "current_container":"file:///mnt/fdbbackup/backup-2022-12-26-01-10-01.337209", + "current_status":"has been completed", + "last_restorable_seconds_behind":4548009.3938379996, + "last_restorable_version":3811090361544, + "mutation_log_bytes_written":760, + "mutation_stream_id":"4909863a42ee46b93790da964437c74d", + "range_bytes_written":4262066724, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2022-12-27":{ + "current_container":"file:///mnt/fdbbackup/backup-2022-12-27-01-10-01.215872", + "current_status":"has been completed", + "last_restorable_seconds_behind":4461609.2065960001, + "last_restorable_version":3897490548786, + "mutation_log_bytes_written":760, + "mutation_stream_id":"ee0d889de84b32675e66793798fac27a", + "range_bytes_written":4264800034, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2022-12-28":{ + "current_container":"file:///mnt/fdbbackup/backup-2022-12-28-01-10-01.074156", + "current_status":"has been completed", + "last_restorable_seconds_behind":4375209.0099649997, + "last_restorable_version":3983890745417, + "mutation_log_bytes_written":760, + "mutation_stream_id":"90537554113b4733ffdf1e42c857279a", + "range_bytes_written":4272830718, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2022-12-29":{ + "current_container":"file:///mnt/fdbbackup/backup-2022-12-29-01-10-01.444742", + "current_status":"has been completed", + "last_restorable_seconds_behind":4288809.039601, + "last_restorable_version":4070290715781, + "mutation_log_bytes_written":760, + "mutation_stream_id":"28fcfb48e2194073d5dc570ebb297424", + "range_bytes_written":4274923073, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2022-12-30":{ + "current_container":"file:///mnt/fdbbackup/backup-2022-12-30-01-10-01.632344", + "current_status":"has been completed", + "last_restorable_seconds_behind":4201707.8215589998, + "last_restorable_version":4157391933823, + "mutation_log_bytes_written":1520, + "mutation_stream_id":"20a6214408801b6484023ebbd1cf78bb", + "range_bytes_written":4291976239, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2022-12-31":{ + "current_container":"file:///mnt/fdbbackup/backup-2022-12-31-01-10-01.076062", + "current_status":"has been completed", + "last_restorable_seconds_behind":4115328.919187, + "last_restorable_version":4243770836195, + "mutation_log_bytes_written":760, + "mutation_stream_id":"e88fb5ee3492673484f0d018eb2bd740", + "range_bytes_written":4291968028, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-01":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-01-01-10-01.432229", + "current_status":"has been completed", + "last_restorable_seconds_behind":4028928.6582820001, + "last_restorable_version":4330171097100, + "mutation_log_bytes_written":760, + "mutation_stream_id":"c978bc853b8b2fc23a50c42e9f91f837", + "range_bytes_written":4291769800, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-02":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-02-01-10-01.154425", + "current_status":"has been completed", + "last_restorable_seconds_behind":3942528.510549, + "last_restorable_version":4416571244833, + "mutation_log_bytes_written":760, + "mutation_stream_id":"d9c282afac01ae7bd7f24e2c6754de3c", + "range_bytes_written":4291693646, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-03":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-03-01-10-01.436776", + "current_status":"has been completed", + "last_restorable_seconds_behind":3856128.7394719999, + "last_restorable_version":4502971015910, + "mutation_log_bytes_written":760, + "mutation_stream_id":"bb447e7d2d542c1b482c4cef8d97b768", + "range_bytes_written":4288484952, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-04":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-04-01-10-01.119107", + "current_status":"has been completed", + "last_restorable_seconds_behind":3769728.743117, + "last_restorable_version":4589371012265, + "mutation_log_bytes_written":760, + "mutation_stream_id":"eabdf89f8ababb43860432e6149aa4f8", + "range_bytes_written":4286153449, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-05":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-05-01-10-01.418406", + "current_status":"has been completed", + "last_restorable_seconds_behind":3683328.5473710001, + "last_restorable_version":4675771208011, + "mutation_log_bytes_written":760, + "mutation_stream_id":"db8f65ff9efe30567c693bc4a5f72190", + "range_bytes_written":4286169319, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-06":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-06-01-10-01.267298", + "current_status":"has been completed", + "last_restorable_seconds_behind":3596928.884381, + "last_restorable_version":4762170871001, + "mutation_log_bytes_written":760, + "mutation_stream_id":"e6cf48cf1ca2c6576bad1eee5ec2b0d2", + "range_bytes_written":4286210325, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-07":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-07-01-10-01.391358", + "current_status":"has been completed", + "last_restorable_seconds_behind":3510528.9311390002, + "last_restorable_version":4848570824243, + "mutation_log_bytes_written":760, + "mutation_stream_id":"68556001f91a4a9342c159de634da2b7", + "range_bytes_written":4293243557, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-08":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-08-01-10-01.105407", + "current_status":"has been completed", + "last_restorable_seconds_behind":3424129.1710580001, + "last_restorable_version":4934970584324, + "mutation_log_bytes_written":760, + "mutation_stream_id":"99012216caf25173f51eaae93f8736fa", + "range_bytes_written":4292934984, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-09":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-09-01-10-01.053136", + "current_status":"has been completed", + "last_restorable_seconds_behind":3337729.1719499999, + "last_restorable_version":5021370583432, + "mutation_log_bytes_written":760, + "mutation_stream_id":"2ba601d9a677f759cd0b315c991d598a", + "range_bytes_written":4295846592, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-10":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-10-01-10-01.174134", + "current_status":"has been completed", + "last_restorable_seconds_behind":3251329.5688740001, + "last_restorable_version":5107770186508, + "mutation_log_bytes_written":760, + "mutation_stream_id":"69a7bea99565ccd261f6612d78065509", + "range_bytes_written":4294391063, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-11":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-11-01-10-01.118114", + "current_status":"has been completed", + "last_restorable_seconds_behind":3164929.6722710002, + "last_restorable_version":5194170083111, + "mutation_log_bytes_written":760, + "mutation_stream_id":"f5e2c579bf71c08a2d1b9751331b10a4", + "range_bytes_written":4294448476, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-12":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-12-01-10-01.128022", + "current_status":"has been completed", + "last_restorable_seconds_behind":3078529.6608529999, + "last_restorable_version":5280570094529, + "mutation_log_bytes_written":760, + "mutation_stream_id":"23beb18bad920b85aa9fa93c55fb6680", + "range_bytes_written":4294327032, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-13":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-13-01-10-01.230184", + "current_status":"has been completed", + "last_restorable_seconds_behind":2992129.350728, + "last_restorable_version":5366970404654, + "mutation_log_bytes_written":760, + "mutation_stream_id":"5599028fbf8fba1e2a55ae66c3db1d25", + "range_bytes_written":4294332200, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-14":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-14-01-10-01.076103", + "current_status":"has been completed", + "last_restorable_seconds_behind":2905728.823942, + "last_restorable_version":5453370931440, + "mutation_log_bytes_written":760, + "mutation_stream_id":"c8866995627f062a9e1fe6e7579f2d1f", + "range_bytes_written":4294316267, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-15":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-15-01-10-01.399305", + "current_status":"has been completed", + "last_restorable_seconds_behind":2819329.6009249999, + "last_restorable_version":5539770154457, + "mutation_log_bytes_written":760, + "mutation_stream_id":"6dd6a2ed8ab93f9c2eafb1631a5f3115", + "range_bytes_written":4294250636, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-16":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-16-01-10-01.395629", + "current_status":"has been completed", + "last_restorable_seconds_behind":2732929.3388140001, + "last_restorable_version":5626170416568, + "mutation_log_bytes_written":760, + "mutation_stream_id":"b4feafbaaa3324b9b704afba8b8a8c18", + "range_bytes_written":4294250636, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-17":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-17-01-10-01.243579", + "current_status":"has been completed", + "last_restorable_seconds_behind":2646529.3404799998, + "last_restorable_version":5712570414902, + "mutation_log_bytes_written":760, + "mutation_stream_id":"f5285b7f47d6e1f99d2a169da83aa5b0", + "range_bytes_written":4294275392, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-18":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-18-01-10-01.377913", + "current_status":"has been completed", + "last_restorable_seconds_behind":2560129.7150039999, + "last_restorable_version":5798970040378, + "mutation_log_bytes_written":760, + "mutation_stream_id":"a8063f706d9779600700cc8af0a812e7", + "range_bytes_written":4296382397, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-19":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-19-01-10-01.134316", + "current_status":"has been completed", + "last_restorable_seconds_behind":2473730.0732510001, + "last_restorable_version":5885369682131, + "mutation_log_bytes_written":760, + "mutation_stream_id":"bff7d8dd6a6a071f1065d9ae94fe61d9", + "range_bytes_written":4297305641, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-20":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-20-01-10-01.707693", + "current_status":"has been completed", + "last_restorable_seconds_behind":2387329.0857159998, + "last_restorable_version":5971770669666, + "mutation_log_bytes_written":760, + "mutation_stream_id":"de77351fa616eea2bc08e533f915f89c", + "range_bytes_written":4301210515, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-21":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-21-01-10-01.410291", + "current_status":"has been completed", + "last_restorable_seconds_behind":2300929.2810960002, + "last_restorable_version":6058170474286, + "mutation_log_bytes_written":760, + "mutation_stream_id":"a65465b0264e3268de6e3dca3cc7de34", + "range_bytes_written":4301157206, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-22":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-22-01-10-01.727037", + "current_status":"has been completed", + "last_restorable_seconds_behind":2214529.500213, + "last_restorable_version":6144570255169, + "mutation_log_bytes_written":760, + "mutation_stream_id":"7132ebe06f1cd17f4c2e5a923e45b686", + "range_bytes_written":4300950033, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-23":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-23-01-10-01.333121", + "current_status":"has been completed", + "last_restorable_seconds_behind":2128129.7239370001, + "last_restorable_version":6230970031445, + "mutation_log_bytes_written":760, + "mutation_stream_id":"8ae9a5cdef913b8536a8fe253ecbf422", + "range_bytes_written":4300874445, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-24":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-24-01-10-01.315104", + "current_status":"has been completed", + "last_restorable_seconds_behind":2041729.1929049999, + "last_restorable_version":6317370562477, + "mutation_log_bytes_written":760, + "mutation_stream_id":"6ba7f957b77aad33d095d7e45ce59096", + "range_bytes_written":4300874445, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-25":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-25-01-10-01.426599", + "current_status":"has been completed", + "last_restorable_seconds_behind":1955329.848027, + "last_restorable_version":6403769907355, + "mutation_log_bytes_written":760, + "mutation_stream_id":"7077f6ca748fc632639046399c992e81", + "range_bytes_written":4300874445, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-26":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-26-01-10-01.146545", + "current_status":"has been completed", + "last_restorable_seconds_behind":1868930.1424189999, + "last_restorable_version":6490169612963, + "mutation_log_bytes_written":760, + "mutation_stream_id":"84802ebded240f780073a139a94b7a53", + "range_bytes_written":4300874445, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-27":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-27-01-10-01.611161", + "current_status":"has been completed", + "last_restorable_seconds_behind":1782529.7207849999, + "last_restorable_version":6576570034597, + "mutation_log_bytes_written":760, + "mutation_stream_id":"14d73ea49bddaf9204ca6b46701536cc", + "range_bytes_written":4300874445, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-28":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-28-01-10-01.177020", + "current_status":"has been completed", + "last_restorable_seconds_behind":1696130.464136, + "last_restorable_version":6662969291246, + "mutation_log_bytes_written":760, + "mutation_stream_id":"b4d4c022a4bd8a06a04d776c1a1beb5c", + "range_bytes_written":4300874445, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-29":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-29-01-10-01.057106", + "current_status":"has been completed", + "last_restorable_seconds_behind":1609730.4922249999, + "last_restorable_version":6749369263157, + "mutation_log_bytes_written":760, + "mutation_stream_id":"6d3dbab67b367bc4d5529455f797f911", + "range_bytes_written":4301636487, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-30":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-30-01-10-01.344900", + "current_status":"has been completed", + "last_restorable_seconds_behind":1523329.794611, + "last_restorable_version":6835769960771, + "mutation_log_bytes_written":760, + "mutation_stream_id":"00c462ca775fe67bed22fc93b2989d69", + "range_bytes_written":4311275241, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-01-31":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-01-31-01-10-01.696560", + "current_status":"has been completed", + "last_restorable_seconds_behind":1436929.9665270001, + "last_restorable_version":6922169788855, + "mutation_log_bytes_written":760, + "mutation_stream_id":"d041a9e54f674029039e0857b7d0831f", + "range_bytes_written":4312922675, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-02-01":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-02-01-01-10-01.331049", + "current_status":"has been completed", + "last_restorable_seconds_behind":1350529.9150429999, + "last_restorable_version":7008569840339, + "mutation_log_bytes_written":760, + "mutation_stream_id":"931cbfdf479813b81af7831723aba685", + "range_bytes_written":4313374370, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-02-02":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-02-02-01-10-01.964075", + "current_status":"has been completed", + "last_restorable_seconds_behind":1264129.4106340001, + "last_restorable_version":7094970344748, + "mutation_log_bytes_written":760, + "mutation_stream_id":"3e04e6916ce931ce656dc2b51b6a6212", + "range_bytes_written":4304980192, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-02-03":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-02-03-01-10-01.362489", + "current_status":"has been completed", + "last_restorable_seconds_behind":1177729.763886, + "last_restorable_version":7181369991496, + "mutation_log_bytes_written":760, + "mutation_stream_id":"fecfe2e2246dd84d329b10bcfc43c137", + "range_bytes_written":4305334381, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-02-04":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-02-04-01-10-01.960478", + "current_status":"has been completed", + "last_restorable_seconds_behind":1091329.904421, + "last_restorable_version":7267769850961, + "mutation_log_bytes_written":760, + "mutation_stream_id":"76d05dfc1088e2cf3e1b8f201a87f6c4", + "range_bytes_written":4324785864, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-02-05":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-02-05-01-10-01.785117", + "current_status":"has been completed", + "last_restorable_seconds_behind":1004929.492231, + "last_restorable_version":7354170263151, + "mutation_log_bytes_written":760, + "mutation_stream_id":"20da64c05f2f6cda7ee6304243a71bb6", + "range_bytes_written":4324715145, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-02-06":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-02-06-01-10-01.175772", + "current_status":"has been completed", + "last_restorable_seconds_behind":918530.69561900001, + "last_restorable_version":7440569059763, + "mutation_log_bytes_written":760, + "mutation_stream_id":"f938a5d5f598ebccc460eef933be178b", + "range_bytes_written":4324729979, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-02-07":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-02-07-01-10-01.241118", + "current_status":"has been completed", + "last_restorable_seconds_behind":832130.63440400001, + "last_restorable_version":7526969120978, + "mutation_log_bytes_written":760, + "mutation_stream_id":"1bd3ffda185c87580646d76b520c458e", + "range_bytes_written":4313284153, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-02-08":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-02-08-01-10-01.126344", + "current_status":"has been completed", + "last_restorable_seconds_behind":745730.96450700006, + "last_restorable_version":7613368790875, + "mutation_log_bytes_written":760, + "mutation_stream_id":"8f5f2ab1880ddc96cd80d08c981c9107", + "range_bytes_written":4313226971, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-02-09":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-02-09-01-10-01.130475", + "current_status":"has been completed", + "last_restorable_seconds_behind":659331.06280199997, + "last_restorable_version":7699768692580, + "mutation_log_bytes_written":760, + "mutation_stream_id":"87d1afc3b51e680db72bfcd4ebc49bbc", + "range_bytes_written":4319821501, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-02-10":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-02-10-01-10-01.055901", + "current_status":"has been completed", + "last_restorable_seconds_behind":572931.07640999998, + "last_restorable_version":7786168678972, + "mutation_log_bytes_written":760, + "mutation_stream_id":"103643050f0c6a610a13f4c5fb6cc220", + "range_bytes_written":4331977430, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-02-11":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-02-11-01-10-01.285524", + "current_status":"has been completed", + "last_restorable_seconds_behind":486531.04441700003, + "last_restorable_version":7872568710965, + "mutation_log_bytes_written":760, + "mutation_stream_id":"1ef2a2927720dfba043f7997fea82f32", + "range_bytes_written":4335932043, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-02-12":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-02-12-01-10-01.048426", + "current_status":"has been completed", + "last_restorable_seconds_behind":400131.22744599998, + "last_restorable_version":7958968527936, + "mutation_log_bytes_written":760, + "mutation_stream_id":"befc09adb6f864c77dfba330f573a982", + "range_bytes_written":4335815540, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-02-13":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-02-13-01-10-01.251467", + "current_status":"has been completed", + "last_restorable_seconds_behind":313730.69430899998, + "last_restorable_version":8045369061073, + "mutation_log_bytes_written":760, + "mutation_stream_id":"c512e8d62e5a1f4cf70df68ed1ed0205", + "range_bytes_written":4335676611, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-02-14":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-02-14-01-10-01.164890", + "current_status":"has been completed", + "last_restorable_seconds_behind":227331.34368300001, + "last_restorable_version":8131768411699, + "mutation_log_bytes_written":760, + "mutation_stream_id":"70e68e8084e9301e268a1ca675d32829", + "range_bytes_written":4341311073, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-02-15":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-02-15-01-10-01.539850", + "current_status":"has been completed", + "last_restorable_seconds_behind":140930.561984, + "last_restorable_version":8218169193398, + "mutation_log_bytes_written":760, + "mutation_stream_id":"33b9a9914326392f878c041e3bf7d4f6", + "range_bytes_written":4345407197, + "running_backup":false, + "running_backup_is_restorable":false + }, + "2023-02-16":{ + "current_container":"file:///mnt/fdbbackup/backup-2023-02-16-01-10-01.372017", + "current_status":"has been completed", + "last_restorable_seconds_behind":54531.487924000001, + "last_restorable_version":8304568267458, + "mutation_log_bytes_written":760, + "mutation_stream_id":"e8d5dffab42d0757d3bfa186bcd0cf12", + "range_bytes_written":4350263154, + "running_backup":false, + "running_backup_is_restorable":false + }, + "test_1":{ + "current_container":"file:///mnt/fdbbackup/backup-2022-12-13-17-16-20.188059", + "current_status":"has been completed", + "last_restorable_seconds_behind":5613229.0989549998, + "last_restorable_version":2745870656427, + "mutation_log_bytes_written":760, + "mutation_stream_id":"abbc074baa60ba5b4670df39d5ba2ec5", + "range_bytes_written":5055018595, + "running_backup":false, + "running_backup_is_restorable":false + } + }, + "total_workers":30 + } + }, + "logs":[ + { + "begin_version":4152007940720, + "current":true, + "epoch":25, + "log_fault_tolerance":1, + "log_interfaces":[ + { + "address":"10.29.0.115:4504", + "healthy":true, + "id":"989fcf16338447ec" + }, + { + "address":"10.29.0.117:4504", + "healthy":true, + "id":"b648574d30094342" + }, + { + "address":"10.29.0.116:4504", + "healthy":true, + "id":"7e776410c2ceb92f" + } + ], + "log_replication_factor":2, + "log_write_anti_quorum":0, + "possibly_losing_data":false + } + ], + "machines":{ + "a7ebaaa23d084d74b19472bce4f91f70":{ + "address":"10.29.0.115", + "contributing_workers":5, + "cpu":{ + "logical_core_utilization":0.010005 + }, + "excluded":false, + "locality":{ + "machineid":"a7ebaaa23d084d74b19472bce4f91f70", + "processid":"3d961382e4cd167d412d6297b4e8358c", + "zoneid":"a7ebaaa23d084d74b19472bce4f91f70" + }, + "machine_id":"a7ebaaa23d084d74b19472bce4f91f70", + "memory":{ + "committed_bytes":5354520576, + "free_bytes":10866733056, + "total_bytes":16221253632 + }, + "network":{ + "megabits_received":{ + "hz":0.56417799999999996 + }, + "megabits_sent":{ + "hz":1.03609 + }, + "tcp_segments_retransmitted":{ + "hz":1.6000000000000001 + } + } + }, + "c41e87176ba168c067a244c37d97366c":{ + "address":"10.29.0.117", + "contributing_workers":5, + "cpu":{ + "logical_core_utilization":0.0080200400000000008 + }, + "excluded":false, + "locality":{ + "machineid":"c41e87176ba168c067a244c37d97366c", + "processid":"842484bebf46806d7cfbe39d947ff2ff", + "zoneid":"c41e87176ba168c067a244c37d97366c" + }, + "machine_id":"c41e87176ba168c067a244c37d97366c", + "memory":{ + "committed_bytes":5631946752, + "free_bytes":10589306880, + "total_bytes":16221253632 + }, + "network":{ + "megabits_received":{ + "hz":0.434062 + }, + "megabits_sent":{ + "hz":1.0600499999999999 + }, + "tcp_segments_retransmitted":{ + "hz":0 + } + } + }, + "e32b27aba3da1300cb5e56d9644ce9e7":{ + "address":"10.29.0.116", + "contributing_workers":5, + "cpu":{ + "logical_core_utilization":0.0115231 + }, + "excluded":false, + "locality":{ + "machineid":"e32b27aba3da1300cb5e56d9644ce9e7", + "processid":"0a456165f04e1ec1a2ade0ce523d54a8", + "zoneid":"e32b27aba3da1300cb5e56d9644ce9e7" + }, + "machine_id":"e32b27aba3da1300cb5e56d9644ce9e7", + "memory":{ + "committed_bytes":5639344128, + "free_bytes":10581909504, + "total_bytes":16221253632 + }, + "network":{ + "megabits_received":{ + "hz":0.82969899999999996 + }, + "megabits_sent":{ + "hz":1.05907 + }, + "tcp_segments_retransmitted":{ + "hz":0 + } + } + } + }, + "messages":[ + + ], + "page_cache":{ + "log_hit_rate":1, + "storage_hit_rate":1 + }, + "processes":{ + "09ca90b9f3f413e5816b2610ed8b465d":{ + "address":"10.29.0.115:4501", + "class_source":"command_line", + "class_type":"stateless", + "command_line":"/usr/sbin/fdbserver --class=stateless --cluster-file=/etc/foundationdb/fdb.cluster --datadir=/mnt/foundationdb/data/4501 --listen-address=public --logdir=/mnt/foundationdb/log --public-address=auto:4501", + "cpu":{ + "usage_cores":0.0012292 + }, + "disk":{ + "busy":0.0085999800000000001, + "free_bytes":490412584960, + "reads":{ + "counter":854857, + "hz":0, + "sectors":0 + }, + "total_bytes":527295578112, + "writes":{ + "counter":73765457, + "hz":26.1999, + "sectors":1336 + } + }, + "excluded":false, + "fault_domain":"a7ebaaa23d084d74b19472bce4f91f70", + "locality":{ + "machineid":"a7ebaaa23d084d74b19472bce4f91f70", + "processid":"09ca90b9f3f413e5816b2610ed8b465d", + "zoneid":"a7ebaaa23d084d74b19472bce4f91f70" + }, + "machine_id":"a7ebaaa23d084d74b19472bce4f91f70", + "memory":{ + "available_bytes":3065090867, + "limit_bytes":8589934592, + "rss_bytes":46551040, + "unused_allocated_memory":655360, + "used_bytes":122974208 + }, + "messages":[ + + ], + "network":{ + "connection_errors":{ + "hz":0 + }, + "connections_closed":{ + "hz":0 + }, + "connections_established":{ + "hz":0 + }, + "current_connections":8, + "megabits_received":{ + "hz":0.046758299999999996 + }, + "megabits_sent":{ + "hz":0.019513599999999999 + }, + "tls_policy_failures":{ + "hz":0 + } + }, + "roles":[ + { + "id":"1cedbcf5dbe49521", + "role":"resolver" + } + ], + "run_loop_busy":0.00086278800000000007, + "uptime_seconds":4207010, + "version":"7.1.23" + }, + "0a456165f04e1ec1a2ade0ce523d54a8":{ + "address":"10.29.0.116:4500", + "class_source":"command_line", + "class_type":"stateless", + "command_line":"/usr/sbin/fdbserver --class=stateless --cluster-file=/etc/foundationdb/fdb.cluster --datadir=/mnt/foundationdb/data/4500 --listen-address=public --logdir=/mnt/foundationdb/log --public-address=auto:4500", + "cpu":{ + "usage_cores":0.0049765900000000004 + }, + "disk":{ + "busy":0.012200000000000001, + "free_bytes":489160159232, + "reads":{ + "counter":877107, + "hz":0, + "sectors":0 + }, + "total_bytes":527295578112, + "writes":{ + "counter":79316112, + "hz":30.9999, + "sectors":744 + } + }, + "excluded":false, + "fault_domain":"e32b27aba3da1300cb5e56d9644ce9e7", + "locality":{ + "machineid":"e32b27aba3da1300cb5e56d9644ce9e7", + "processid":"0a456165f04e1ec1a2ade0ce523d54a8", + "zoneid":"e32b27aba3da1300cb5e56d9644ce9e7" + }, + "machine_id":"e32b27aba3da1300cb5e56d9644ce9e7", + "memory":{ + "available_bytes":3076787404, + "limit_bytes":8589934592, + "rss_bytes":72359936, + "unused_allocated_memory":393216, + "used_bytes":157978624 + }, + "messages":[ + + ], + "network":{ + "connection_errors":{ + "hz":0 + }, + "connections_closed":{ + "hz":0 + }, + "connections_established":{ + "hz":0 + }, + "current_connections":21, + "megabits_received":{ + "hz":0.20410799999999998 + }, + "megabits_sent":{ + "hz":0.210009 + }, + "tls_policy_failures":{ + "hz":0 + } + }, + "roles":[ + { + "role":"coordinator" + }, + { + "grv_latency_statistics":{ + "batch":{ + "count":0, + "max":0, + "mean":0, + "median":0, + "min":0, + "p25":0, + "p90":0, + "p95":0, + "p99":0, + "p99.9":0 + }, + "default":{ + "count":3968, + "max":0.00029754600000000004, + "mean":0.00011025100000000001, + "median":0.000109196, + "min":0.0000832081, + "p25":0.00010252, + "p90":0.00012421600000000002, + "p95":0.00013232200000000001, + "p99":0.00014853500000000001, + "p99.9":0.00020289400000000001 + } + }, + "id":"35568292c7c8efcb", + "role":"grv_proxy" + } + ], + "run_loop_busy":0.0038921300000000002, + "uptime_seconds":4211540, + "version":"7.1.23" + }, + "20979548fdd7941987b43fd72d2884cd":{ + "address":"10.29.0.117:4501", + "class_source":"command_line", + "class_type":"stateless", + "command_line":"/usr/sbin/fdbserver --class=stateless --cluster-file=/etc/foundationdb/fdb.cluster --datadir=/mnt/foundationdb/data/4501 --listen-address=public --logdir=/mnt/foundationdb/log --public-address=auto:4501", + "cpu":{ + "usage_cores":0.0016627999999999999 + }, + "disk":{ + "busy":0.0040000000000000001, + "free_bytes":489862533120, + "reads":{ + "counter":862623, + "hz":0, + "sectors":0 + }, + "total_bytes":527295578112, + "writes":{ + "counter":89087049, + "hz":17.399999999999999, + "sectors":600 + } + }, + "excluded":false, + "fault_domain":"c41e87176ba168c067a244c37d97366c", + "locality":{ + "machineid":"c41e87176ba168c067a244c37d97366c", + "processid":"20979548fdd7941987b43fd72d2884cd", + "zoneid":"c41e87176ba168c067a244c37d97366c" + }, + "machine_id":"c41e87176ba168c067a244c37d97366c", + "memory":{ + "available_bytes":3083005132, + "limit_bytes":8589934592, + "rss_bytes":65449984, + "unused_allocated_memory":262144, + "used_bytes":130707456 + }, + "messages":[ + + ], + "network":{ + "connection_errors":{ + "hz":0 + }, + "connections_closed":{ + "hz":0 + }, + "connections_established":{ + "hz":0 + }, + "current_connections":15, + "megabits_received":{ + "hz":0.025433599999999997 + }, + "megabits_sent":{ + "hz":0.12354499999999999 + }, + "tls_policy_failures":{ + "hz":0 + } + }, + "roles":[ + { + "id":"d64ff6aec77b0901", + "role":"cluster_controller" + } + ], + "run_loop_busy":0.00115986, + "uptime_seconds":4228940, + "version":"7.1.23" + }, + "22e1a804c8f3284e03e4da595ab51c45":{ + "address":"10.29.0.115:4503", + "class_source":"command_line", + "class_type":"storage", + "command_line":"/usr/sbin/fdbserver --class=storage --cluster-file=/etc/foundationdb/fdb.cluster --datadir=/mnt/foundationdb/data/4503 --listen-address=public --logdir=/mnt/foundationdb/log --public-address=auto:4503", + "cpu":{ + "usage_cores":0.0022559900000000003 + }, + "disk":{ + "busy":0.0095999599999999994, + "free_bytes":490412613632, + "reads":{ + "counter":854857, + "hz":0, + "sectors":0 + }, + "total_bytes":527295578112, + "writes":{ + "counter":73765445, + "hz":28.399899999999999, + "sectors":1384 + } + }, + "excluded":false, + "fault_domain":"a7ebaaa23d084d74b19472bce4f91f70", + "locality":{ + "machineid":"a7ebaaa23d084d74b19472bce4f91f70", + "processid":"22e1a804c8f3284e03e4da595ab51c45", + "zoneid":"a7ebaaa23d084d74b19472bce4f91f70" + }, + "machine_id":"a7ebaaa23d084d74b19472bce4f91f70", + "memory":{ + "available_bytes":3064911462, + "limit_bytes":8589934592, + "rss_bytes":2090733568, + "unused_allocated_memory":131072, + "used_bytes":2391834624 + }, + "messages":[ + + ], + "network":{ + "connection_errors":{ + "hz":0 + }, + "connections_closed":{ + "hz":0 + }, + "connections_established":{ + "hz":0 + }, + "current_connections":13, + "megabits_received":{ + "hz":0.034143899999999998 + }, + "megabits_sent":{ + "hz":0.048409500000000001 + }, + "tls_policy_failures":{ + "hz":0 + } + }, + "roles":[ + { + "bytes_queried":{ + "counter":31650625271, + "hz":0, + "roughness":-1 + }, + "data_lag":{ + "seconds":1.67435, + "versions":1674352 + }, + "data_version":8359117049604, + "durability_lag":{ + "seconds":5, + "versions":5000000 + }, + "durable_bytes":{ + "counter":136752526, + "hz":0, + "roughness":-1 + }, + "durable_version":8359112049604, + "fetched_versions":{ + "counter":4207377935583, + "hz":1059280, + "roughness":1499910 + }, + "fetches_from_logs":{ + "counter":5529509, + "hz":1.6000000000000001, + "roughness":1.26556 + }, + "finished_queries":{ + "counter":1574437, + "hz":0, + "roughness":-1 + }, + "id":"b9ac20f6099187bb", + "input_bytes":{ + "counter":136752526, + "hz":0, + "roughness":-1 + }, + "keys_queried":{ + "counter":38049663, + "hz":0, + "roughness":-1 + }, + "kvstore_available_bytes":490412580864, + "kvstore_free_bytes":490412580864, + "kvstore_inline_keys":0, + "kvstore_total_bytes":527295578112, + "kvstore_total_nodes":0, + "kvstore_total_size":0, + "kvstore_used_bytes":1826340864, + "local_rate":100, + "low_priority_queries":{ + "counter":65, + "hz":0, + "roughness":-1 + }, + "mutation_bytes":{ + "counter":37080948, + "hz":0, + "roughness":-1 + }, + "mutations":{ + "counter":69950, + "hz":0, + "roughness":-1 + }, + "query_queue_max":0, + "read_latency_statistics":{ + "count":0, + "max":0, + "mean":0, + "median":0, + "min":0, + "p25":0, + "p90":0, + "p95":0, + "p99":0, + "p99.9":0 + }, + "role":"storage", + "storage_metadata":{ + "created_time_datetime":"2022-11-11 14:37:57.000 +0000", + "created_time_timestamp":1668180000 + }, + "stored_bytes":1407755562, + "total_queries":{ + "counter":1574437, + "hz":0, + "roughness":-1 + } + } + ], + "run_loop_busy":0.0016757, + "uptime_seconds":4207010, + "version":"7.1.23" + }, + "336ac60dc5fae6d644820659354c3027":{ + "address":"10.29.0.116:4502", + "class_source":"command_line", + "class_type":"storage", + "command_line":"/usr/sbin/fdbserver --class=storage --cluster-file=/etc/foundationdb/fdb.cluster --datadir=/mnt/foundationdb/data/4502 --listen-address=public --logdir=/mnt/foundationdb/log --public-address=auto:4502", + "cpu":{ + "usage_cores":0.0024319999999999997 + }, + "disk":{ + "busy":0.011600000000000001, + "free_bytes":489160142848, + "reads":{ + "counter":877107, + "hz":0, + "sectors":0 + }, + "total_bytes":527295578112, + "writes":{ + "counter":79316112, + "hz":27.599900000000002, + "sectors":664 + } + }, + "excluded":false, + "fault_domain":"e32b27aba3da1300cb5e56d9644ce9e7", + "locality":{ + "machineid":"e32b27aba3da1300cb5e56d9644ce9e7", + "processid":"336ac60dc5fae6d644820659354c3027", + "zoneid":"e32b27aba3da1300cb5e56d9644ce9e7" + }, + "machine_id":"e32b27aba3da1300cb5e56d9644ce9e7", + "memory":{ + "available_bytes":3076784128, + "limit_bytes":8589934592, + "rss_bytes":2242613248, + "unused_allocated_memory":7339680, + "used_bytes":2404286464 + }, + "messages":[ + + ], + "network":{ + "connection_errors":{ + "hz":0 + }, + "connections_closed":{ + "hz":0 + }, + "connections_established":{ + "hz":0 + }, + "current_connections":17, + "megabits_received":{ + "hz":0.041075199999999999 + }, + "megabits_sent":{ + "hz":0.052710299999999995 + }, + "tls_policy_failures":{ + "hz":0 + } + }, + "roles":[ + { + "bytes_queried":{ + "counter":47159882015, + "hz":582.19899999999996, + "roughness":1021.6900000000001 + }, + "data_lag":{ + "seconds":0.29163, + "versions":291630 + }, + "data_version":8359117341234, + "durability_lag":{ + "seconds":5.2916299999999996, + "versions":5291630 + }, + "durable_bytes":{ + "counter":1282267102, + "hz":814.79899999999998, + "roughness":4073 + }, + "durable_version":8359112049604, + "fetched_versions":{ + "counter":4212080954766, + "hz":1117600, + "roughness":1436800 + }, + "fetches_from_logs":{ + "counter":5539359, + "hz":1.8, + "roughness":1.31409 + }, + "finished_queries":{ + "counter":49976345, + "hz":0.79999900000000002, + "roughness":0.405283 + }, + "id":"81eaea1051bc3534", + "input_bytes":{ + "counter":1282269716, + "hz":522.79899999999998, + "roughness":2133.0100000000002 + }, + "keys_queried":{ + "counter":86011047, + "hz":0.79999900000000002, + "roughness":0.405283 + }, + "kvstore_available_bytes":489160085504, + "kvstore_free_bytes":489160085504, + "kvstore_inline_keys":0, + "kvstore_total_bytes":527295578112, + "kvstore_total_nodes":0, + "kvstore_total_size":0, + "kvstore_used_bytes":1831297024, + "local_rate":100, + "low_priority_queries":{ + "counter":18, + "hz":0, + "roughness":-1 + }, + "mutation_bytes":{ + "counter":388680471, + "hz":53.399900000000002, + "roughness":216.97300000000001 + }, + "mutations":{ + "counter":541578, + "hz":0.39999899999999999, + "roughness":0.63275399999999993 + }, + "query_queue_max":1, + "read_latency_statistics":{ + "count":52, + "max":0.00012517, + "mean":0.000042933700000000005, + "median":0.000039100600000000003, + "min":0.0000064373000000000002, + "p25":0.000013828300000000001, + "p90":0.000076770800000000014, + "p95":0.000093936899999999998, + "p99":0.00012517, + "p99.9":0.00012517 + }, + "role":"storage", + "storage_metadata":{ + "created_time_datetime":"2022-11-11 14:40:05.000 +0000", + "created_time_timestamp":1668180000 + }, + "stored_bytes":1418589622, + "total_queries":{ + "counter":49976345, + "hz":0.79999900000000002, + "roughness":0.405283 + } + } + ], + "run_loop_busy":0.0018324300000000001, + "uptime_seconds":4211540, + "version":"7.1.23" + }, + "3bb939e1333a0cafdb52e7b7e4af4eed":{ + "address":"10.29.0.115:4502", + "class_source":"command_line", + "class_type":"storage", + "command_line":"/usr/sbin/fdbserver --class=storage --cluster-file=/etc/foundationdb/fdb.cluster --datadir=/mnt/foundationdb/data/4502 --listen-address=public --logdir=/mnt/foundationdb/log --public-address=auto:4502", + "cpu":{ + "usage_cores":0.0030271899999999999 + }, + "disk":{ + "busy":0.009599980000000001, + "free_bytes":490412613632, + "reads":{ + "counter":854857, + "hz":0, + "sectors":0 + }, + "total_bytes":527295578112, + "writes":{ + "counter":73765445, + "hz":28.399899999999999, + "sectors":1384 + } + }, + "excluded":false, + "fault_domain":"a7ebaaa23d084d74b19472bce4f91f70", + "locality":{ + "machineid":"a7ebaaa23d084d74b19472bce4f91f70", + "processid":"3bb939e1333a0cafdb52e7b7e4af4eed", + "zoneid":"a7ebaaa23d084d74b19472bce4f91f70" + }, + "machine_id":"a7ebaaa23d084d74b19472bce4f91f70", + "memory":{ + "available_bytes":3064898355, + "limit_bytes":8589934592, + "rss_bytes":2170167296, + "unused_allocated_memory":2096928, + "used_bytes":2396160000 + }, + "messages":[ + + ], + "network":{ + "connection_errors":{ + "hz":0 + }, + "connections_closed":{ + "hz":0 + }, + "connections_established":{ + "hz":0 + }, + "current_connections":16, + "megabits_received":{ + "hz":0.106893 + }, + "megabits_sent":{ + "hz":0.10492799999999999 + }, + "tls_policy_failures":{ + "hz":0 + } + }, + "roles":[ + { + "bytes_queried":{ + "counter":31850408674, + "hz":1905.2, + "roughness":3347.0700000000002 + }, + "data_lag":{ + "seconds":0.212421, + "versions":212421 + }, + "data_version":8359115375252, + "durability_lag":{ + "seconds":5, + "versions":5000000 + }, + "durable_bytes":{ + "counter":2788625472, + "hz":0, + "roughness":-1 + }, + "durable_version":8359110375252, + "fetched_versions":{ + "counter":4207376261231, + "hz":724406, + "roughness":1419160 + }, + "fetches_from_logs":{ + "counter":5534025, + "hz":1.3999999999999999, + "roughness":1.7427000000000001 + }, + "finished_queries":{ + "counter":18989130, + "hz":1.6000000000000001, + "roughness":1.8113600000000001 + }, + "id":"47ec3d598172d36d", + "input_bytes":{ + "counter":2788756238, + "hz":4313.6000000000004, + "roughness":17589.299999999999 + }, + "keys_queried":{ + "counter":55062615, + "hz":2, + "roughness":2.5146600000000001 + }, + "kvstore_available_bytes":490412580864, + "kvstore_free_bytes":490412580864, + "kvstore_inline_keys":0, + "kvstore_total_bytes":527295578112, + "kvstore_total_nodes":0, + "kvstore_total_size":0, + "kvstore_used_bytes":1828196352, + "local_rate":100, + "low_priority_queries":{ + "counter":15, + "hz":0, + "roughness":-1 + }, + "mutation_bytes":{ + "counter":683871093, + "hz":793.99900000000002, + "roughness":3236.8299999999999 + }, + "mutations":{ + "counter":1609500, + "hz":3, + "roughness":11.233599999999999 + }, + "query_queue_max":1, + "read_latency_statistics":{ + "count":162, + "max":0.00012064, + "mean":0.000024068500000000002, + "median":0.0000152588, + "min":0.00000333786, + "p25":0.0000071525599999999997, + "p90":0.000065088300000000005, + "p95":0.000073194499999999999, + "p99":0.000099182100000000007, + "p99.9":0.00012064 + }, + "role":"storage", + "storage_metadata":{ + "created_time_datetime":"2022-11-11 14:37:56.000 +0000", + "created_time_timestamp":1668180000 + }, + "stored_bytes":1437039573, + "total_queries":{ + "counter":18989130, + "hz":1.6000000000000001, + "roughness":1.8113600000000001 + } + } + ], + "run_loop_busy":0.0023324999999999999, + "uptime_seconds":4207010, + "version":"7.1.23" + }, + "3d4bf6805fc8ab8830a3c401dbafa937":{ + "address":"10.29.0.117:4502", + "class_source":"command_line", + "class_type":"storage", + "command_line":"/usr/sbin/fdbserver --class=storage --cluster-file=/etc/foundationdb/fdb.cluster --datadir=/mnt/foundationdb/data/4502 --listen-address=public --logdir=/mnt/foundationdb/log --public-address=auto:4502", + "cpu":{ + "usage_cores":0.0030431899999999999 + }, + "disk":{ + "busy":0.0037999900000000001, + "free_bytes":489862565888, + "reads":{ + "counter":862623, + "hz":0, + "sectors":0 + }, + "total_bytes":527295578112, + "writes":{ + "counter":89087039, + "hz":15.4, + "sectors":552 + } + }, + "excluded":false, + "fault_domain":"c41e87176ba168c067a244c37d97366c", + "locality":{ + "machineid":"c41e87176ba168c067a244c37d97366c", + "processid":"3d4bf6805fc8ab8830a3c401dbafa937", + "zoneid":"c41e87176ba168c067a244c37d97366c" + }, + "machine_id":"c41e87176ba168c067a244c37d97366c", + "memory":{ + "available_bytes":3083005132, + "limit_bytes":8589934592, + "rss_bytes":2306105344, + "unused_allocated_memory":8388224, + "used_bytes":2814017536 + }, + "messages":[ + + ], + "network":{ + "connection_errors":{ + "hz":0 + }, + "connections_closed":{ + "hz":0 + }, + "connections_established":{ + "hz":0 + }, + "current_connections":21, + "megabits_received":{ + "hz":0.079244700000000001 + }, + "megabits_sent":{ + "hz":0.16444799999999998 + }, + "tls_policy_failures":{ + "hz":0 + } + }, + "roles":[ + { + "bytes_queried":{ + "counter":84968184874, + "hz":10954.200000000001, + "roughness":7166.5799999999999 + }, + "data_lag":{ + "seconds":0.16997099999999998, + "versions":169971 + }, + "data_version":8359113267260, + "durability_lag":{ + "seconds":5, + "versions":5000000 + }, + "durable_bytes":{ + "counter":14824278172, + "hz":295.60000000000002, + "roughness":1477 + }, + "durable_version":8359108267260, + "fetched_versions":{ + "counter":4229692901927, + "hz":867050, + "roughness":956425 + }, + "fetches_from_logs":{ + "counter":5560953, + "hz":7.1999899999999997, + "roughness":6.9421800000000005 + }, + "finished_queries":{ + "counter":155153277, + "hz":15.800000000000001, + "roughness":6.2782200000000001 + }, + "id":"5790fd044e73cb53", + "input_bytes":{ + "counter":14824305594, + "hz":5188.79, + "roughness":7180.5699999999997 + }, + "keys_queried":{ + "counter":270815691, + "hz":40.799999999999997, + "roughness":25.696400000000001 + }, + "kvstore_available_bytes":489849221120, + "kvstore_free_bytes":489849221120, + "kvstore_inline_keys":0, + "kvstore_total_bytes":527295578112, + "kvstore_total_nodes":0, + "kvstore_total_size":0, + "kvstore_used_bytes":1934802944, + "local_rate":100, + "low_priority_queries":{ + "counter":168, + "hz":0, + "roughness":-1 + }, + "mutation_bytes":{ + "counter":6061387038, + "hz":585.99900000000002, + "roughness":810.05399999999997 + }, + "mutations":{ + "counter":2740347, + "hz":4.7999900000000002, + "roughness":5.6434499999999996 + }, + "query_queue_max":3, + "read_latency_statistics":{ + "count":3193, + "max":0.0024123199999999999, + "mean":0.00018043400000000001, + "median":0.0000083446499999999998, + "min":0.0000038147000000000001, + "p25":0.0000069141399999999998, + "p90":0.00070571900000000005, + "p95":0.0016820400000000001, + "p99":0.0022904900000000001, + "p99.9":0.0024001600000000001 + }, + "role":"storage", + "storage_metadata":{ + "created_time_datetime":"2022-11-11 14:40:23.000 +0000", + "created_time_timestamp":1668180000 + }, + "stored_bytes":1479207326, + "total_queries":{ + "counter":155153277, + "hz":15.800000000000001, + "roughness":6.2782200000000001 + } + } + ], + "run_loop_busy":0.0020840200000000002, + "uptime_seconds":4228940, + "version":"7.1.23" + }, + "3d961382e4cd167d412d6297b4e8358c":{ + "address":"10.29.0.115:4500", + "class_source":"command_line", + "class_type":"stateless", + "command_line":"/usr/sbin/fdbserver --class=stateless --cluster-file=/etc/foundationdb/fdb.cluster --datadir=/mnt/foundationdb/data/4500 --listen-address=public --logdir=/mnt/foundationdb/log --public-address=auto:4500", + "cpu":{ + "usage_cores":0.0026771899999999999 + }, + "disk":{ + "busy":0.009599980000000001, + "free_bytes":490412625920, + "reads":{ + "counter":854857, + "hz":0, + "sectors":0 + }, + "total_bytes":527295578112, + "writes":{ + "counter":73765443, + "hz":29.9999, + "sectors":1424 + } + }, + "excluded":false, + "fault_domain":"a7ebaaa23d084d74b19472bce4f91f70", + "locality":{ + "machineid":"a7ebaaa23d084d74b19472bce4f91f70", + "processid":"3d961382e4cd167d412d6297b4e8358c", + "zoneid":"a7ebaaa23d084d74b19472bce4f91f70" + }, + "machine_id":"a7ebaaa23d084d74b19472bce4f91f70", + "memory":{ + "available_bytes":3064872140, + "limit_bytes":8589934592, + "rss_bytes":67686400, + "unused_allocated_memory":393216, + "used_bytes":155619328 + }, + "messages":[ + + ], + "network":{ + "connection_errors":{ + "hz":0 + }, + "connections_closed":{ + "hz":0 + }, + "connections_established":{ + "hz":0 + }, + "current_connections":21, + "megabits_received":{ + "hz":0.112077 + }, + "megabits_sent":{ + "hz":0.10767299999999999 + }, + "tls_policy_failures":{ + "hz":0 + } + }, + "roles":[ + { + "role":"coordinator" + }, + { + "commit_batching_window_size":{ + "count":99, + "max":0.0017811700000000001, + "mean":0.00102479, + "median":0.001, + "min":0.001, + "p25":0.001, + "p90":0.001, + "p95":0.00119593, + "p99":0.0017811700000000001, + "p99.9":0.0017811700000000001 + }, + "commit_latency_statistics":{ + "count":77, + "max":0.0043594799999999998, + "mean":0.0013010599999999999, + "median":0.00116515, + "min":0.00089025500000000006, + "p25":0.0011117500000000001, + "p90":0.00159526, + "p95":0.0018477400000000001, + "p99":0.0043594799999999998, + "p99.9":0.0043594799999999998 + }, + "id":"b3d9feac5884f537", + "role":"commit_proxy" + } + ], + "run_loop_busy":0.0019783000000000001, + "uptime_seconds":4207010, + "version":"7.1.23" + }, + "40b44135d6a3de79d7ce791d602dda35":{ + "address":"10.29.0.117:4504", + "class_source":"command_line", + "class_type":"log", + "command_line":"/usr/sbin/fdbserver --class=log --cluster-file=/etc/foundationdb/fdb.cluster --datadir=/mnt/foundationdb/data/4504 --listen-address=public --logdir=/mnt/foundationdb/log --public-address=auto:4504", + "cpu":{ + "usage_cores":0.00254499 + }, + "disk":{ + "busy":0.0037999900000000001, + "free_bytes":489862565888, + "reads":{ + "counter":862623, + "hz":0, + "sectors":0 + }, + "total_bytes":527295578112, + "writes":{ + "counter":89087039, + "hz":15.4, + "sectors":552 + } + }, + "excluded":false, + "fault_domain":"c41e87176ba168c067a244c37d97366c", + "locality":{ + "machineid":"c41e87176ba168c067a244c37d97366c", + "processid":"40b44135d6a3de79d7ce791d602dda35", + "zoneid":"c41e87176ba168c067a244c37d97366c" + }, + "machine_id":"c41e87176ba168c067a244c37d97366c", + "memory":{ + "available_bytes":3083005132, + "limit_bytes":8589934592, + "rss_bytes":86855680, + "unused_allocated_memory":786432, + "used_bytes":217092096 + }, + "messages":[ + + ], + "network":{ + "connection_errors":{ + "hz":0 + }, + "connections_closed":{ + "hz":0 + }, + "connections_established":{ + "hz":0 + }, + "current_connections":13, + "megabits_received":{ + "hz":0.061004699999999995 + }, + "megabits_sent":{ + "hz":0.056972699999999994 + }, + "tls_policy_failures":{ + "hz":0 + } + }, + "roles":[ + { + "data_version":8359117341234, + "durable_bytes":{ + "counter":451066741, + "hz":0, + "roughness":-1 + }, + "id":"b648574d30094342", + "input_bytes":{ + "counter":451080187, + "hz":131.80000000000001, + "roughness":658 + }, + "kvstore_available_bytes":489862561792, + "kvstore_free_bytes":489862561792, + "kvstore_total_bytes":527295578112, + "kvstore_used_bytes":104988672, + "queue_disk_available_bytes":489862561792, + "queue_disk_free_bytes":489862561792, + "queue_disk_total_bytes":527295578112, + "queue_disk_used_bytes":190898176, + "role":"log" + } + ], + "run_loop_busy":0.00180211, + "uptime_seconds":4228940, + "version":"7.1.23" + }, + "79ddb666ac6a5b0b8eb2aae9155d91c9":{ + "address":"10.29.0.116:4503", + "class_source":"command_line", + "class_type":"storage", + "command_line":"/usr/sbin/fdbserver --class=storage --cluster-file=/etc/foundationdb/fdb.cluster --datadir=/mnt/foundationdb/data/4503 --listen-address=public --logdir=/mnt/foundationdb/log --public-address=auto:4503", + "cpu":{ + "usage_cores":0.00309939 + }, + "disk":{ + "busy":0.011600000000000001, + "free_bytes":489160142848, + "reads":{ + "counter":877107, + "hz":0, + "sectors":0 + }, + "total_bytes":527295578112, + "writes":{ + "counter":79316112, + "hz":27.599900000000002, + "sectors":664 + } + }, + "excluded":false, + "fault_domain":"e32b27aba3da1300cb5e56d9644ce9e7", + "locality":{ + "machineid":"e32b27aba3da1300cb5e56d9644ce9e7", + "processid":"79ddb666ac6a5b0b8eb2aae9155d91c9", + "zoneid":"e32b27aba3da1300cb5e56d9644ce9e7" + }, + "machine_id":"e32b27aba3da1300cb5e56d9644ce9e7", + "memory":{ + "available_bytes":3076784128, + "limit_bytes":8589934592, + "rss_bytes":2331357184, + "unused_allocated_memory":8519328, + "used_bytes":2806808576 + }, + "messages":[ + + ], + "network":{ + "connection_errors":{ + "hz":0 + }, + "connections_closed":{ + "hz":0 + }, + "connections_established":{ + "hz":0 + }, + "current_connections":21, + "megabits_received":{ + "hz":0.11596099999999999 + }, + "megabits_sent":{ + "hz":0.117267 + }, + "tls_policy_failures":{ + "hz":0 + } + }, + "roles":[ + { + "bytes_queried":{ + "counter":71639343608, + "hz":5026.5900000000001, + "roughness":4796.8100000000004 + }, + "data_lag":{ + "seconds":0.212421, + "versions":212421 + }, + "data_version":8359115375252, + "durability_lag":{ + "seconds":5, + "versions":5000000 + }, + "durable_bytes":{ + "counter":15141755838, + "hz":991.19899999999996, + "roughness":2488.46 + }, + "durable_version":8359110375252, + "fetched_versions":{ + "counter":4212078988784, + "hz":1049730, + "roughness":1289530 + }, + "fetches_from_logs":{ + "counter":5463762, + "hz":4.9999900000000004, + "roughness":5.1421900000000003 + }, + "finished_queries":{ + "counter":83070765, + "hz":6.3999899999999998, + "roughness":3.9809800000000002 + }, + "id":"8bc5e1d932ce3e95", + "input_bytes":{ + "counter":15141903588, + "hz":29550, + "roughness":53589 + }, + "keys_queried":{ + "counter":203406940, + "hz":13, + "roughness":11.408300000000001 + }, + "kvstore_available_bytes":489160097792, + "kvstore_free_bytes":489160097792, + "kvstore_inline_keys":0, + "kvstore_total_bytes":527295578112, + "kvstore_total_nodes":0, + "kvstore_total_size":0, + "kvstore_used_bytes":1929228288, + "local_rate":100, + "low_priority_queries":{ + "counter":104, + "hz":0, + "roughness":-1 + }, + "mutation_bytes":{ + "counter":6103020248, + "hz":6465.9899999999998, + "roughness":11725.299999999999 + }, + "mutations":{ + "counter":3018379, + "hz":19.199999999999999, + "roughness":33.819899999999997 + }, + "query_queue_max":8, + "read_latency_statistics":{ + "count":2777, + "max":0.044108399999999999, + "mean":0.00039714700000000004, + "median":0.000023841900000000003, + "min":0.0000050067899999999994, + "p25":0.000012397800000000002, + "p90":0.0015842900000000001, + "p95":0.0018782600000000001, + "p99":0.00210714, + "p99.9":0.023344500000000001 + }, + "role":"storage", + "storage_metadata":{ + "created_time_datetime":"2022-11-11 14:40:05.000 +0000", + "created_time_timestamp":1668180000 + }, + "stored_bytes":1462314032, + "total_queries":{ + "counter":83070765, + "hz":6.1999899999999997, + "roughness":3.9297900000000001 + } + } + ], + "run_loop_busy":0.00245166, + "uptime_seconds":4211540, + "version":"7.1.23" + }, + "842484bebf46806d7cfbe39d947ff2ff":{ + "address":"10.29.0.117:4500", + "class_source":"command_line", + "class_type":"stateless", + "command_line":"/usr/sbin/fdbserver --class=stateless --cluster-file=/etc/foundationdb/fdb.cluster --datadir=/mnt/foundationdb/data/4500 --listen-address=public --logdir=/mnt/foundationdb/log --public-address=auto:4500", + "cpu":{ + "usage_cores":0.0022472 + }, + "disk":{ + "busy":0.0039999900000000001, + "free_bytes":489862561792, + "reads":{ + "counter":862623, + "hz":0, + "sectors":0 + }, + "total_bytes":527295578112, + "writes":{ + "counter":89087049, + "hz":17.399999999999999, + "sectors":600 + } + }, + "excluded":false, + "fault_domain":"c41e87176ba168c067a244c37d97366c", + "locality":{ + "machineid":"c41e87176ba168c067a244c37d97366c", + "processid":"842484bebf46806d7cfbe39d947ff2ff", + "zoneid":"c41e87176ba168c067a244c37d97366c" + }, + "machine_id":"c41e87176ba168c067a244c37d97366c", + "memory":{ + "available_bytes":3082998579, + "limit_bytes":8589934592, + "rss_bytes":77381632, + "unused_allocated_memory":917504, + "used_bytes":156536832 + }, + "messages":[ + + ], + "network":{ + "connection_errors":{ + "hz":0 + }, + "connections_closed":{ + "hz":0 + }, + "connections_established":{ + "hz":0 + }, + "current_connections":24, + "megabits_received":{ + "hz":0.077939099999999997 + }, + "megabits_sent":{ + "hz":0.055155099999999999 + }, + "tls_policy_failures":{ + "hz":0 + } + }, + "roles":[ + { + "role":"coordinator" + }, + { + "commit_batching_window_size":{ + "count":122, + "max":0.0024159300000000002, + "mean":0.0010496699999999999, + "median":0.001, + "min":0.001, + "p25":0.001, + "p90":0.001, + "p95":0.0013242600000000001, + "p99":0.0021852099999999999, + "p99.9":0.0024159300000000002 + }, + "commit_latency_statistics":{ + "count":105, + "max":0.0023839500000000001, + "mean":0.00122204, + "median":0.0011563300000000001, + "min":0.00074005100000000003, + "p25":0.0011217600000000001, + "p90":0.00146031, + "p95":0.0016674999999999999, + "p99":0.00173616, + "p99.9":0.0023839500000000001 + }, + "id":"3e5c411d00a2c803", + "role":"commit_proxy" + } + ], + "run_loop_busy":0.0016148500000000001, + "uptime_seconds":4228940, + "version":"7.1.23" + }, + "98d518065ffe4bedbe8f25b11257d1cf":{ + "address":"10.29.0.116:4501", + "class_source":"command_line", + "class_type":"stateless", + "command_line":"/usr/sbin/fdbserver --class=stateless --cluster-file=/etc/foundationdb/fdb.cluster --datadir=/mnt/foundationdb/data/4501 --listen-address=public --logdir=/mnt/foundationdb/log --public-address=auto:4501", + "cpu":{ + "usage_cores":0.0073400000000000002 + }, + "disk":{ + "busy":0.012200000000000001, + "free_bytes":489160171520, + "reads":{ + "counter":877107, + "hz":0, + "sectors":0 + }, + "total_bytes":527295578112, + "writes":{ + "counter":79316097, + "hz":31.399999999999999, + "sectors":752 + } + }, + "excluded":false, + "fault_domain":"e32b27aba3da1300cb5e56d9644ce9e7", + "locality":{ + "machineid":"e32b27aba3da1300cb5e56d9644ce9e7", + "processid":"98d518065ffe4bedbe8f25b11257d1cf", + "zoneid":"e32b27aba3da1300cb5e56d9644ce9e7" + }, + "machine_id":"e32b27aba3da1300cb5e56d9644ce9e7", + "memory":{ + "available_bytes":3076789862, + "limit_bytes":8589934592, + "rss_bytes":65310720, + "unused_allocated_memory":393216, + "used_bytes":127037440 + }, + "messages":[ + + ], + "network":{ + "connection_errors":{ + "hz":0 + }, + "connections_closed":{ + "hz":0 + }, + "connections_established":{ + "hz":0 + }, + "current_connections":13, + "megabits_received":{ + "hz":0.41436799999999996 + }, + "megabits_sent":{ + "hz":0.202234 + }, + "tls_policy_failures":{ + "hz":0 + } + }, + "roles":[ + { + "id":"40a2459acecdea6d", + "role":"master" + }, + { + "id":"7fe3252e801f87e8", + "role":"data_distributor" + }, + { + "id":"1b282ff4ff5d8bea", + "role":"ratekeeper" + } + ], + "run_loop_busy":0.0057107399999999997, + "uptime_seconds":4211540, + "version":"7.1.23" + }, + "9ab24113bcdda17dc8bf5cdb258ddc28":{ + "address":"10.29.0.116:4504", + "class_source":"command_line", + "class_type":"log", + "command_line":"/usr/sbin/fdbserver --class=log --cluster-file=/etc/foundationdb/fdb.cluster --datadir=/mnt/foundationdb/data/4504 --listen-address=public --logdir=/mnt/foundationdb/log --public-address=auto:4504", + "cpu":{ + "usage_cores":0.00355419 + }, + "disk":{ + "busy":0.011600000000000001, + "free_bytes":489160126464, + "reads":{ + "counter":877107, + "hz":0, + "sectors":0 + }, + "total_bytes":527295578112, + "writes":{ + "counter":79316126, + "hz":28.9999, + "sectors":720 + } + }, + "excluded":false, + "fault_domain":"e32b27aba3da1300cb5e56d9644ce9e7", + "locality":{ + "machineid":"e32b27aba3da1300cb5e56d9644ce9e7", + "processid":"9ab24113bcdda17dc8bf5cdb258ddc28", + "zoneid":"e32b27aba3da1300cb5e56d9644ce9e7" + }, + "machine_id":"e32b27aba3da1300cb5e56d9644ce9e7", + "memory":{ + "available_bytes":3076809523, + "limit_bytes":8589934592, + "rss_bytes":90386432, + "unused_allocated_memory":524288, + "used_bytes":217092096 + }, + "messages":[ + + ], + "network":{ + "connection_errors":{ + "hz":0 + }, + "connections_closed":{ + "hz":0 + }, + "connections_established":{ + "hz":0 + }, + "current_connections":13, + "megabits_received":{ + "hz":0.11835499999999999 + }, + "megabits_sent":{ + "hz":0.0811582 + }, + "tls_policy_failures":{ + "hz":0 + } + }, + "roles":[ + { + "data_version":8359117049604, + "durable_bytes":{ + "counter":6768816905, + "hz":0, + "roughness":-1 + }, + "id":"7e776410c2ceb92f", + "input_bytes":{ + "counter":6768842675, + "hz":952.399, + "roughness":3119.8099999999999 + }, + "kvstore_available_bytes":489160085504, + "kvstore_free_bytes":489160085504, + "kvstore_total_bytes":527295578112, + "kvstore_used_bytes":104988672, + "queue_disk_available_bytes":489160085504, + "queue_disk_free_bytes":489160085504, + "queue_disk_total_bytes":527295578112, + "queue_disk_used_bytes":211914752, + "role":"log" + } + ], + "run_loop_busy":0.00261415, + "uptime_seconds":4211540, + "version":"7.1.23" + }, + "abb4acdc3d8e63686a601ec7dac84669":{ + "address":"10.29.0.115:4504", + "class_source":"command_line", + "class_type":"log", + "command_line":"/usr/sbin/fdbserver --class=log --cluster-file=/etc/foundationdb/fdb.cluster --datadir=/mnt/foundationdb/data/4504 --listen-address=public --logdir=/mnt/foundationdb/log --public-address=auto:4504", + "cpu":{ + "usage_cores":0.0034145899999999999 + }, + "disk":{ + "busy":0.009599980000000001, + "free_bytes":490412601344, + "reads":{ + "counter":854857, + "hz":0, + "sectors":0 + }, + "total_bytes":527295578112, + "writes":{ + "counter":73765455, + "hz":29.399899999999999, + "sectors":1408 + } + }, + "excluded":false, + "fault_domain":"a7ebaaa23d084d74b19472bce4f91f70", + "locality":{ + "machineid":"a7ebaaa23d084d74b19472bce4f91f70", + "processid":"abb4acdc3d8e63686a601ec7dac84669", + "zoneid":"a7ebaaa23d084d74b19472bce4f91f70" + }, + "machine_id":"a7ebaaa23d084d74b19472bce4f91f70", + "memory":{ + "available_bytes":3064911462, + "limit_bytes":8589934592, + "rss_bytes":82489344, + "unused_allocated_memory":393216, + "used_bytes":200052736 + }, + "messages":[ + + ], + "network":{ + "connection_errors":{ + "hz":0 + }, + "connections_closed":{ + "hz":0 + }, + "connections_established":{ + "hz":0 + }, + "current_connections":13, + "megabits_received":{ + "hz":0.13439299999999998 + }, + "megabits_sent":{ + "hz":0.174368 + }, + "tls_policy_failures":{ + "hz":0 + } + }, + "roles":[ + { + "data_version":8359115375252, + "durable_bytes":{ + "counter":7012656240, + "hz":0, + "roughness":-1 + }, + "id":"989fcf16338447ec", + "input_bytes":{ + "counter":7012694993, + "hz":7462.79, + "roughness":16713.799999999999 + }, + "kvstore_available_bytes":490412584960, + "kvstore_free_bytes":490412584960, + "kvstore_total_bytes":527295578112, + "kvstore_used_bytes":104988672, + "queue_disk_available_bytes":490412584960, + "queue_disk_free_bytes":490412584960, + "queue_disk_total_bytes":527295578112, + "queue_disk_used_bytes":216051712, + "role":"log" + } + ], + "run_loop_busy":0.0024585200000000001, + "uptime_seconds":4207010, + "version":"7.1.23" + }, + "b5c5a6ed2f15ea4289a408626ba08e06":{ + "address":"10.29.0.117:4503", + "class_source":"command_line", + "class_type":"storage", + "command_line":"/usr/sbin/fdbserver --class=storage --cluster-file=/etc/foundationdb/fdb.cluster --datadir=/mnt/foundationdb/data/4503 --listen-address=public --logdir=/mnt/foundationdb/log --public-address=auto:4503", + "cpu":{ + "usage_cores":0.0020691999999999998 + }, + "disk":{ + "busy":0.0039999900000000001, + "free_bytes":489862561792, + "reads":{ + "counter":862623, + "hz":0, + "sectors":0 + }, + "total_bytes":527295578112, + "writes":{ + "counter":89087049, + "hz":17.399999999999999, + "sectors":600 + } + }, + "excluded":false, + "fault_domain":"c41e87176ba168c067a244c37d97366c", + "locality":{ + "machineid":"c41e87176ba168c067a244c37d97366c", + "processid":"b5c5a6ed2f15ea4289a408626ba08e06", + "zoneid":"c41e87176ba168c067a244c37d97366c" + }, + "machine_id":"c41e87176ba168c067a244c37d97366c", + "memory":{ + "available_bytes":3082998579, + "limit_bytes":8589934592, + "rss_bytes":2289893376, + "unused_allocated_memory":16514752, + "used_bytes":2412675072 + }, + "messages":[ + + ], + "network":{ + "connection_errors":{ + "hz":0 + }, + "connections_closed":{ + "hz":0 + }, + "connections_established":{ + "hz":0 + }, + "current_connections":14, + "megabits_received":{ + "hz":0.029510399999999999 + }, + "megabits_sent":{ + "hz":0.043859099999999998 + }, + "tls_policy_failures":{ + "hz":0 + } + }, + "roles":[ + { + "bytes_queried":{ + "counter":44974577182, + "hz":0, + "roughness":-1 + }, + "data_lag":{ + "seconds":0.16997099999999998, + "versions":169971 + }, + "data_version":8359113267260, + "durability_lag":{ + "seconds":5, + "versions":5000000 + }, + "durable_bytes":{ + "counter":1042491484, + "hz":0, + "roughness":-1 + }, + "durable_version":8359108267260, + "fetched_versions":{ + "counter":4229692901927, + "hz":867050, + "roughness":956417 + }, + "fetches_from_logs":{ + "counter":5555906, + "hz":7.1999899999999997, + "roughness":6.9420999999999999 + }, + "finished_queries":{ + "counter":16113422, + "hz":0, + "roughness":-1 + }, + "id":"5b7a53146d029dce", + "input_bytes":{ + "counter":1042491484, + "hz":0, + "roughness":-1 + }, + "keys_queried":{ + "counter":75512629, + "hz":0, + "roughness":-1 + }, + "kvstore_available_bytes":489862598656, + "kvstore_free_bytes":489862598656, + "kvstore_inline_keys":0, + "kvstore_total_bytes":527295578112, + "kvstore_total_nodes":0, + "kvstore_total_size":0, + "kvstore_used_bytes":1827893248, + "local_rate":100, + "low_priority_queries":{ + "counter":0, + "hz":0, + "roughness":-1 + }, + "mutation_bytes":{ + "counter":205439591, + "hz":0, + "roughness":-1 + }, + "mutations":{ + "counter":710592, + "hz":0, + "roughness":-1 + }, + "query_queue_max":0, + "read_latency_statistics":{ + "count":0, + "max":0, + "mean":0, + "median":0, + "min":0, + "p25":0, + "p90":0, + "p95":0, + "p99":0, + "p99.9":0 + }, + "role":"storage", + "storage_metadata":{ + "created_time_datetime":"2022-11-11 14:40:23.000 +0000", + "created_time_timestamp":1668180000 + }, + "stored_bytes":1428225829, + "total_queries":{ + "counter":16113422, + "hz":0, + "roughness":-1 + } + } + ], + "run_loop_busy":0.0014962199999999999, + "uptime_seconds":4228940, + "version":"7.1.23" + } + }, + "protocol_version":"fdb00b071010000", + "qos":{ + "batch_performance_limited_by":{ + "description":"The database is not being saturated by the workload.", + "name":"workload", + "reason_id":2 + }, + "batch_released_transactions_per_second":0.00000000000025579500000000002, + "batch_transactions_per_second_limit":634521, + "limiting_data_lag_storage_server":{ + "seconds":0, + "versions":0 + }, + "limiting_durability_lag_storage_server":{ + "seconds":5.0170399999999997, + "versions":5017044 + }, + "limiting_queue_bytes_storage_server":130569, + "performance_limited_by":{ + "description":"The database is not being saturated by the workload.", + "name":"workload", + "reason_id":2 + }, + "released_transactions_per_second":6.2172599999999996, + "throttled_tags":{ + "auto":{ + "busy_read":0, + "busy_write":0, + "count":0, + "recommended_only":0 + }, + "manual":{ + "count":0 + } + }, + "transactions_per_second_limit":878567, + "worst_data_lag_storage_server":{ + "seconds":0, + "versions":0 + }, + "worst_durability_lag_storage_server":{ + "seconds":5.0183999999999997, + "versions":5018403 + }, + "worst_queue_bytes_log_server":38766, + "worst_queue_bytes_storage_server":147272 + }, + "recovery_state":{ + "active_generations":1, + "description":"Recovery complete.", + "name":"fully_recovered", + "seconds_since_last_recovered":4207010 + }, + "workload":{ + "bytes":{ + "read":{ + "counter":312243021624, + "hz":18468.200000000001, + "roughness":5933.8500000000004 + }, + "written":{ + "counter":6678975461, + "hz":2042.2, + "roughness":4824.7700000000004 + } + }, + "keys":{ + "read":{ + "counter":728858585, + "hz":56.600000000000001, + "roughness":21.238099999999999 + } + }, + "operations":{ + "location_requests":{ + "counter":107543, + "hz":0, + "roughness":0 + }, + "low_priority_reads":{ + "counter":370, + "hz":0, + "roughness":0 + }, + "memory_errors":{ + "counter":0, + "hz":0, + "roughness":0 + }, + "read_requests":{ + "counter":324877376, + "hz":24.399999999999999, + "roughness":5.1960199999999999 + }, + "reads":{ + "counter":324877376, + "hz":24.600000000000001, + "roughness":5.1990499999999997 + }, + "writes":{ + "counter":4336393, + "hz":7.3999899999999998, + "roughness":17.258600000000001 + } + }, + "transactions":{ + "committed":{ + "counter":2083958, + "hz":3.3999999999999999, + "roughness":6.3902299999999999 + }, + "conflicted":{ + "counter":2970, + "hz":0, + "roughness":0 + }, + "rejected_for_queued_too_long":{ + "counter":0, + "hz":0, + "roughness":0 + }, + "started":{ + "counter":196874980, + "hz":12, + "roughness":3.6349400000000003 + }, + "started_batch_priority":{ + "counter":155, + "hz":0, + "roughness":0 + }, + "started_default_priority":{ + "counter":192248190, + "hz":11, + "roughness":3.2502900000000001 + }, + "started_immediate_priority":{ + "counter":4626635, + "hz":0.99999899999999997, + "roughness":0.000000082931100000000009 + } + } + } + } +} \ No newline at end of file diff --git a/cloud/test/http_encode_key_test.cpp b/cloud/test/http_encode_key_test.cpp new file mode 100644 index 00000000000000..ac5eade9735859 --- /dev/null +++ b/cloud/test/http_encode_key_test.cpp @@ -0,0 +1,537 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include +#include + +#include "common/sync_point.h" +#include "meta-service/mem_txn_kv.h" +#include "meta-service/meta_service_http.h" + +using namespace doris::cloud; + +int main(int argc, char** argv) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + +TEST(HttpEncodeKeyTest, process_http_encode_key_test) { + brpc::URI uri; + HttpResponse http_res; + + // test unsupported key type + uri.SetQuery("key_type", "foobarkey"); + http_res = process_http_encode_key(uri); + EXPECT_EQ(http_res.status_code, 400); + EXPECT_NE(http_res.body.find("key_type not supported"), std::string::npos); + + // test missing argument + uri.SetQuery("key_type", "MetaRowsetKey"); + http_res = process_http_encode_key(uri); + EXPECT_EQ(http_res.status_code, 400); + EXPECT_NE(http_res.body.find("instance_id is not given or empty"), std::string::npos) + << http_res.body; + + // clang-format off + std::string unicode_res = R"(┌─────────────────────────────────────────────────────────────────────────────────────── 0. key space: 1 +│ ┌───────────────────────────────────────────────────────────────────────────────────── 1. meta +│ │ ┌─────────────────────────────────────────────────────────────────────── 2. gavin-instance +│ │ │ ┌───────────────────────────────────── 3. rowset +│ │ │ │ ┌─────────────────── 4. 10086 +│ │ │ │ │ ┌─ 5. 10010 +│ │ │ │ │ │ +▼ ▼ ▼ ▼ ▼ ▼ +01106d657461000110676176696e2d696e7374616e6365000110726f77736574000112000000000000276612000000000000271a +\x01\x10\x6d\x65\x74\x61\x00\x01\x10\x67\x61\x76\x69\x6e\x2d\x69\x6e\x73\x74\x61\x6e\x63\x65\x00\x01\x10\x72\x6f\x77\x73\x65\x74\x00\x01\x12\x00\x00\x00\x00\x00\x00\x27\x66\x12\x00\x00\x00\x00\x00\x00\x27\x1a +)"; + + std::string nonunicode_res = R"(/--------------------------------------------------------------------------------------- 0. key space: 1 +| /------------------------------------------------------------------------------------- 1. meta +| | /----------------------------------------------------------------------- 2. gavin-instance +| | | /------------------------------------- 3. rowset +| | | | /------------------- 4. 10086 +| | | | | /- 5. 10010 +| | | | | | +v v v v v v +01106d657461000110676176696e2d696e7374616e6365000110726f77736574000112000000000000276612000000000000271a +\x01\x10\x6d\x65\x74\x61\x00\x01\x10\x67\x61\x76\x69\x6e\x2d\x69\x6e\x73\x74\x61\x6e\x63\x65\x00\x01\x10\x72\x6f\x77\x73\x65\x74\x00\x01\x12\x00\x00\x00\x00\x00\x00\x27\x66\x12\x00\x00\x00\x00\x00\x00\x27\x1a +)"; + // clang-format on + + // test normal path, with unicode + uri.SetQuery("instance_id", "gavin-instance"); + uri.SetQuery("tablet_id", "10086"); + uri.SetQuery("version", "10010"); + http_res = process_http_encode_key(uri); + EXPECT_EQ(http_res.status_code, 200); + EXPECT_EQ(http_res.body, unicode_res); + + // test normal path, with non-unicode + uri.SetQuery("unicode", "false"); + http_res = process_http_encode_key(uri); + EXPECT_EQ(http_res.status_code, 200); + EXPECT_EQ(http_res.body, nonunicode_res); + + // test empty body branch + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("process_http_encode_key::empty_body", + [](void* p) { ((std::string*)p)->clear(); }); + sp->enable_processing(); + + http_res = process_http_encode_key(uri); + EXPECT_EQ(http_res.status_code, 400); + EXPECT_NE(http_res.body.find("failed to decode encoded key"), std::string::npos); +} + +struct Input { + std::string_view key_type; + std::string_view param; + std::string_view key; + std::function gen_value; + std::string_view value; +}; +// clang-format off +static auto test_inputs = std::array { + Input { + "InstanceKey", + "instance_id=gavin-instance", + "0110696e7374616e6365000110676176696e2d696e7374616e63650001", + []() -> std::string { + InstanceInfoPB pb; + pb.set_instance_id("gavin-instance"); + return pb.SerializeAsString(); + }, + R"({"instance_id":"gavin-instance"})", + }, + Input { + "TxnLabelKey", + "instance_id=gavin-instance&db_id=10086&label=test-label", + "011074786e000110676176696e2d696e7374616e636500011074786e5f6c6162656c000112000000000000276610746573742d6c6162656c0001", + []() -> std::string { + TxnLabelPB pb; + pb.add_txn_ids(123456789); + auto val = pb.SerializeAsString(); + MemTxnKv::gen_version_timestamp(123456790, 0, &val); + return val; + }, + R"({"txn_ids":["123456789"]} +txn_id=126419752960)", + }, + Input { + "TxnInfoKey", + "instance_id=gavin-instance&db_id=10086&txn_id=10010", + "011074786e000110676176696e2d696e7374616e636500011074786e5f696e666f000112000000000000276612000000000000271a", + []() -> std::string { + TxnInfoPB pb; + pb.set_db_id(10086); + pb.set_txn_id(10010); + return pb.SerializeAsString(); + }, + R"({"db_id":"10086","txn_id":"10010"})", + }, + Input { + "TxnIndexKey", + "instance_id=gavin-instance&txn_id=10086", + "011074786e000110676176696e2d696e7374616e636500011074786e5f696e6465780001120000000000002766", + []() -> std::string { + TxnIndexPB pb; + pb.mutable_tablet_index()->set_db_id(10086); + return pb.SerializeAsString(); + }, + R"({"tablet_index":{"db_id":"10086"}})", + }, + Input { + "TxnRunningKey", + "instance_id=gavin-instance&db_id=10086&txn_id=10010", + "011074786e000110676176696e2d696e7374616e636500011074786e5f72756e6e696e67000112000000000000276612000000000000271a", + []() -> std::string { + TxnRunningPB pb; + pb.add_table_ids(10001); + return pb.SerializeAsString(); + }, + R"({"table_ids":["10001"]})", + }, + Input { + "VersionKey", + "instance_id=gavin-instance&db_id=10086&tbl_id=10010&partition_id=10000", + "011076657273696f6e000110676176696e2d696e7374616e6365000110706172746974696f6e000112000000000000276612000000000000271a120000000000002710", + []() -> std::string { + VersionPB pb; + pb.set_version(10); + return pb.SerializeAsString(); + }, + R"({"version":"10"})", + }, + Input { + "MetaRowsetKey", + "instance_id=gavin-instance&tablet_id=10086&version=10010", + "01106d657461000110676176696e2d696e7374616e6365000110726f77736574000112000000000000276612000000000000271a", + []() -> std::string { + doris::RowsetMetaCloudPB pb; + pb.set_rowset_id(0); + pb.set_rowset_id_v2("rowset_id_1"); + pb.set_tablet_id(10086); + pb.set_start_version(10010); + pb.set_end_version(10010); + return pb.SerializeAsString(); + }, + R"({"rowset_id":"0","tablet_id":"10086","start_version":"10010","end_version":"10010","rowset_id_v2":"rowset_id_1"})", + }, + Input { + "MetaRowsetTmpKey", + "instance_id=gavin-instance&txn_id=10086&tablet_id=10010", + "01106d657461000110676176696e2d696e7374616e6365000110726f777365745f746d70000112000000000000276612000000000000271a", + []() -> std::string { + doris::RowsetMetaCloudPB pb; + pb.set_rowset_id(0); + pb.set_rowset_id_v2("rowset_id_1"); + pb.set_tablet_id(10010); + pb.set_txn_id(10086); + pb.set_start_version(2); + pb.set_end_version(2); + return pb.SerializeAsString(); + }, + R"({"rowset_id":"0","tablet_id":"10010","txn_id":"10086","start_version":"2","end_version":"2","rowset_id_v2":"rowset_id_1"})", + }, + Input { + "MetaTabletKey", + "instance_id=gavin-instance&table_id=10086&index_id=100010&part_id=10000&tablet_id=1008601", + "01106d657461000110676176696e2d696e7374616e63650001107461626c657400011200000000000027661200000000000186aa1200000000000027101200000000000f63d9", + []() -> std::string { + doris::TabletMetaCloudPB pb; + pb.set_table_id(10086); + pb.set_index_id(100010); + pb.set_partition_id(10000); + pb.set_tablet_id(1008601); + return pb.SerializeAsString(); + }, + R"({"table_id":"10086","partition_id":"10000","tablet_id":"1008601","index_id":"100010"})", + }, + Input { + "MetaTabletIdxKey", + "instance_id=gavin-instance&tablet_id=10086", + "01106d657461000110676176696e2d696e7374616e63650001107461626c65745f696e6465780001120000000000002766", + []() -> std::string { + TabletIndexPB pb; + pb.set_table_id(10006); + pb.set_index_id(100010); + pb.set_partition_id(10000); + pb.set_tablet_id(10086); + return pb.SerializeAsString(); + }, + R"({"table_id":"10006","index_id":"100010","partition_id":"10000","tablet_id":"10086"})", + }, + Input { + "RecycleIndexKey", + "instance_id=gavin-instance&index_id=10086", + "011072656379636c65000110676176696e2d696e7374616e6365000110696e6465780001120000000000002766", + []() -> std::string { + RecycleIndexPB pb; + pb.set_creation_time(12345); + pb.set_table_id(10000); + return pb.SerializeAsString(); + }, + R"({"table_id":"10000","creation_time":"12345"})", + }, + Input { + "RecyclePartKey", + "instance_id=gavin-instance&part_id=10086", + "011072656379636c65000110676176696e2d696e7374616e6365000110706172746974696f6e0001120000000000002766", + []() -> std::string { + RecyclePartitionPB pb; + pb.set_creation_time(12345); + pb.set_table_id(10000); + pb.add_index_id(10001); + return pb.SerializeAsString(); + }, + R"({"table_id":"10000","index_id":["10001"],"creation_time":"12345"})", + }, + Input { + "RecycleRowsetKey", + "instance_id=gavin-instance&tablet_id=10086&rowset_id=10010", + "011072656379636c65000110676176696e2d696e7374616e6365000110726f7773657400011200000000000027661031303031300001", + []() -> std::string { + RecycleRowsetPB pb; + pb.set_creation_time(12345); + auto rs = pb.mutable_rowset_meta(); + rs->set_rowset_id(0); + rs->set_rowset_id_v2("10010"); + rs->set_tablet_id(10086); + return pb.SerializeAsString(); + }, + R"({"creation_time":"12345","rowset_meta":{"rowset_id":"0","tablet_id":"10086","rowset_id_v2":"10010"}})", + }, + Input { + "RecycleTxnKey", + "instance_id=gavin-instance&db_id=10086&txn_id=10010", + "011072656379636c65000110676176696e2d696e7374616e636500011074786e000112000000000000276612000000000000271a", + []() -> std::string { + RecycleTxnPB pb; + pb.set_label("label_1"); + pb.set_creation_time(12345); + return pb.SerializeAsString(); + }, + R"({"creation_time":"12345","label":"label_1"})", + }, + Input { + "StatsTabletKey", + "instance_id=gavin-instance&table_id=10086&index_id=10010&part_id=10000&tablet_id=1008601", + "01107374617473000110676176696e2d696e7374616e63650001107461626c6574000112000000000000276612000000000000271a1200000000000027101200000000000f63d9", + []() -> std::string { + TabletStatsPB pb; + auto idx = pb.mutable_idx(); + idx->set_table_id(10086); + idx->set_index_id(100010); + idx->set_partition_id(10000); + idx->set_tablet_id(1008601); + pb.set_num_rowsets(10); + return pb.SerializeAsString(); + }, + R"({"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008601"},"num_rowsets":"10"})", + }, + Input { + "JobTabletKey", + "instance_id=gavin-instance&table_id=10086&index_id=10010&part_id=10000&tablet_id=1008601", + "01106a6f62000110676176696e2d696e7374616e63650001107461626c6574000112000000000000276612000000000000271a1200000000000027101200000000000f63d9", + []() -> std::string { + TabletJobInfoPB pb; + auto idx = pb.mutable_idx(); + idx->set_table_id(10086); + idx->set_index_id(100010); + idx->set_partition_id(10000); + idx->set_tablet_id(1008601); + auto c = pb.add_compaction(); + c->set_id("compaction_1"); + return pb.SerializeAsString(); + }, + R"({"idx":{"table_id":"10086","index_id":"100010","partition_id":"10000","tablet_id":"1008601"},"compaction":[{"id":"compaction_1"}]})", + }, + Input { + "CopyJobKey", + "instance_id=gavin-instance&stage_id=10086&table_id=10010©_id=10000&group_id=1008601", + "0110636f7079000110676176696e2d696e7374616e63650001106a6f620001103130303836000112000000000000271a10313030303000011200000000000f63d9", + []() -> std::string { + CopyJobPB pb; + pb.set_stage_type(StagePB::EXTERNAL); + pb.set_start_time_ms(12345); + return pb.SerializeAsString(); + }, + R"({"stage_type":"EXTERNAL","start_time_ms":"12345"})", + }, + Input { + "CopyFileKey", + "instance_id=gavin-instance&stage_id=10086&table_id=10010&obj_key=10000&obj_etag=1008601", + "0110636f7079000110676176696e2d696e7374616e63650001106c6f6164696e675f66696c650001103130303836000112000000000000271a103130303030000110313030383630310001", + []() -> std::string { + CopyFilePB pb; + pb.set_copy_id("copy_id_1"); + pb.set_group_id(10000); + return pb.SerializeAsString(); + }, + R"({"copy_id":"copy_id_1","group_id":10000})", + }, + Input { + "RecycleStageKey", + "instance_id=gavin-instance&stage_id=10086", + "011072656379636c65000110676176696e2d696e7374616e6365000110737461676500011031303038360001", + []() -> std::string { + RecycleStagePB pb; + pb.set_instance_id("gavin-instance"); + pb.set_reason("reason"); + return pb.SerializeAsString(); + }, + R"({"instance_id":"gavin-instance","reason":"reason"})", + }, + Input { + "JobRecycleKey", + "instance_id=gavin-instance", + "01106a6f62000110676176696e2d696e7374616e6365000110636865636b0001", + []() -> std::string { + JobRecyclePB pb; + pb.set_instance_id("gavin-instance"); + pb.set_ip_port("host_1"); + return pb.SerializeAsString(); + }, + R"({"instance_id":"gavin-instance","ip_port":"host_1"})", + }, + Input { + "MetaSchemaKey", + "instance_id=gavin-instance&index_id=10086&schema_version=10010", + "01106d657461000110676176696e2d696e7374616e6365000110736368656d61000112000000000000276612000000000000271a", + []() -> std::string { + doris::TabletSchemaCloudPB pb; + pb.set_schema_version(10010); + auto col = pb.add_column(); + col->set_unique_id(6789); + col->set_name("col_1"); + col->set_type("INT"); + return pb.SerializeAsString(); + }, + R"({"column":[{"unique_id":6789,"name":"col_1","type":"INT"}],"schema_version":10010})", + }, + Input { + "MetaDeleteBitmap", + "instance_id=gavin-instance&tablet_id=10086&rowest_id=10010&version=10000&seg_id=1008601", + "01106d657461000110676176696e2d696e7374616e636500011064656c6574655f6269746d6170000112000000000000276610313030313000011200000000000027101200000000000f63d9", + []() -> std::string { + return "abcdefg"; + }, + "61626364656667", + }, + Input { + "MetaDeleteBitmapUpdateLock", + "instance_id=gavin-instance&table_id=10086&partition_id=10010", + "01106d657461000110676176696e2d696e7374616e636500011064656c6574655f6269746d61705f6c6f636b000112000000000000276612000000000000271a", + []() -> std::string { + DeleteBitmapUpdateLockPB pb; + pb.set_lock_id(12345); + pb.add_initiators(114115); + return pb.SerializeAsString(); + }, + R"({"lock_id":"12345","initiators":["114115"]})", + }, + Input { + "MetaPendingDeleteBitmap", + "instance_id=gavin-instance&tablet_id=10086", + "01106d657461000110676176696e2d696e7374616e636500011064656c6574655f6269746d61705f70656e64696e670001120000000000002766", + []() -> std::string { + PendingDeleteBitmapPB pb; + pb.add_delete_bitmap_keys("key_1"); + return pb.SerializeAsString(); + }, + R"({"delete_bitmap_keys":["a2V5XzE="]})", + }, + Input { + "RLJobProgressKey", + "instance_id=gavin-instance&db_id=10086&job_id=10010", + "01106a6f62000110676176696e2d696e7374616e6365000110726f7574696e655f6c6f61645f70726f6772657373000112000000000000276612000000000000271a", + []() -> std::string { + RoutineLoadProgressPB pb; + auto map = pb.mutable_partition_to_offset(); + map->insert({1000, 1234}); + return pb.SerializeAsString(); + }, + R"({"partition_to_offset":{"1000":"1234"}})", + }, + Input { + "MetaServiceRegistryKey", + "", + "021073797374656d0001106d6574612d7365727669636500011072656769737472790001", + []() -> std::string { + ServiceRegistryPB pb; + auto i = pb.add_items(); + i->set_host("host_1"); + i->set_ctime_ms(123456); + return pb.SerializeAsString(); + }, + R"({"items":[{"ctime_ms":"123456","host":"host_1"}]})", + }, + Input { + "MetaServiceArnInfoKey", + "", + "021073797374656d0001106d6574612d7365727669636500011061726e5f696e666f0001", + []() -> std::string { + RamUserPB pb; + pb.set_user_id("user_1"); + pb.set_ak("ak"); + pb.set_sk("sk"); + return pb.SerializeAsString(); + }, + R"({"user_id":"user_1","ak":"ak","sk":"sk"})", + }, + Input { + "MetaServiceEncryptionKey", + "", + "021073797374656d0001106d6574612d73657276696365000110656e6372797074696f6e5f6b65795f696e666f0001", + []() -> std::string { + EncryptionKeyInfoPB pb; + auto i = pb.add_items(); + i->set_key_id(23456); + i->set_key("key_1"); + return pb.SerializeAsString(); + }, + R"({"items":[{"key_id":"23456","key":"key_1"}]})", + }, +}; +// clang-format on + +TEST(HttpEncodeKeyTest, process_http_encode_key_test_cover_all_template) { + brpc::URI uri; + (void)uri.get_query_map(); // initialize query map + for (auto&& input : test_inputs) { + std::stringstream url; + url << "localhost:5000/MetaService/http?key_type=" << input.key_type; + if (!input.param.empty()) { + url << "&" << input.param; + } + // std::cout << url.str() << std::endl; + EXPECT_EQ(uri.SetHttpURL(url.str()), 0); // clear and set query string + auto http_res = process_http_encode_key(uri); + EXPECT_EQ(http_res.status_code, 200); + EXPECT_NE(http_res.body.find(input.key), std::string::npos) + << "real full text: " << http_res.body << "\nexpect contains: " << input.key; + } +} + +TEST(HttpGetValueTest, process_http_get_value_test_cover_all_template) { + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + // Generate kvs + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + for (auto&& input : test_inputs) { + auto key = unhex(input.key); + auto val = input.gen_value(); + txn->put(key, val); + } + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + brpc::URI uri; + (void)uri.get_query_map(); // initialize query map + + auto gen_url = [](const Input& input, bool use_param) { + std::stringstream url; + url << "localhost:5000/MetaService/http?key_type=" << input.key_type; + // Key mode + if (!use_param) { + url << "&key=" << input.key; + } else if (!input.param.empty()) { + url << "&" << input.param; + } + return url.str(); + }; + + for (auto&& input : test_inputs) { + auto url = gen_url(input, true); + // std::cout << url.str() << std::endl; + ASSERT_EQ(uri.SetHttpURL(url), 0); // clear and set query string + auto http_res = process_http_get_value(txn_kv.get(), uri); + EXPECT_EQ(http_res.status_code, 200); + // std::cout << http_res.body << std::endl; + EXPECT_EQ(http_res.body, input.value); + // Key mode + url = gen_url(input, false); + // std::cout << url.str() << std::endl; + ASSERT_EQ(uri.SetHttpURL(url), 0); // clear and set query string + http_res = process_http_get_value(txn_kv.get(), uri); + EXPECT_EQ(http_res.status_code, 200); + // std::cout << http_res.body << std::endl; + EXPECT_EQ(http_res.body, input.value); + } +} diff --git a/cloud/test/keys_test.cpp b/cloud/test/keys_test.cpp new file mode 100644 index 00000000000000..614a17374d0bf5 --- /dev/null +++ b/cloud/test/keys_test.cpp @@ -0,0 +1,988 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "meta-service/keys.h" + +#include +#include +#include + +#include +#include +#include +#include +#include + +#include "common/util.h" + +int main(int argc, char** argv) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + +static void remove_user_space_prefix(std::string_view* key_sv) { + ASSERT_EQ(key_sv->front(), 0x01); + key_sv->remove_prefix(1); +} + +static void remove_system_space_prefix(std::string_view* key_sv) { + ASSERT_EQ(key_sv->front(), 0x02); + key_sv->remove_prefix(1); +} + +// extern +namespace doris::cloud { +void encode_int64(int64_t val, std::string* b); +int decode_int64(std::string_view* in, int64_t* val); +void encode_bytes(std::string_view bytes, std::string* b); +int decode_bytes(std::string_view* in, std::string* out); +} // namespace doris::cloud + +// clang-format off +// Possible key encoding schemas: +// +// 0x01 "instance" ${instance_id} -> InstanceInfoPB +// +// 0x01 "txn" ${instance_id} "txn_label" ${db_id} ${label} -> TxnLabelPB ${version_timestamp} +// 0x01 "txn" ${instance_id} "txn_info" ${db_id} ${version_timestamp} -> TxnInfoPB +// 0x01 "txn" ${instance_id} "txn_index" ${version_timestamp} -> TxnIndexPB +// 0x01 "txn" ${instance_id} "txn_running" ${db_id} ${version_timestamp} -> TxnRunningPB // creaet at begin, delete at commit +// +// 0x01 "version" ${instance_id} "partition" ${db_id} ${tbl_id} ${partition_id} -> VersionPB +// +// 0x01 "meta" ${instance_id} "rowset" ${tablet_id} ${version} ${rowset_id} -> RowsetMetaCloudPB +// 0x01 "meta" ${instance_id} "rowset_tmp" ${txn_id} ${rowset_id} -> RowsetMetaCloudPB +// 0x01 "meta" ${instance_id} "tablet" ${table_id} ${tablet_id} -> TabletMetaCloudPB +// 0x01 "meta" ${instance_id} "tablet_table" ${tablet_id} -> ${table_id} +// 0x01 "meta" ${instance_id} "tablet_tmp" ${table_id} ${tablet_id} -> TabletMetaCloudPB +// +// 0x01 "trash" ${instacne_id} "table" -> TableTrashPB +// +// 0x01 "node_status" ${instance_id} "compute" ${backend_id} -> ComputeNodeStatusPB +// clang-format on + +TEST(KeysTest, InstanceKeyTest) { + using namespace doris::cloud; + + // instance key + // 0x01 "instance" ${instance_id} + std::string instance_id = "instance_id_deadbeef"; + InstanceKeyInfo inst_key {instance_id}; + std::string encoded_instance_key; + instance_key(inst_key, &encoded_instance_key); + + std::string dec_instance_id; + + std::string_view key_sv(encoded_instance_key); + std::string dec_instance_prefix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("instance", dec_instance_prefix); + EXPECT_EQ(instance_id, dec_instance_id); +} + +TEST(KeysTest, MetaKeysTest) { + using namespace doris::cloud; + std::string instance_id = "instance_id_deadbeef"; + + // rowset meta key + // 0x01 "meta" ${instance_id} "rowset" ${tablet_id} ${version} + { + int64_t tablet_id = 10086; + int64_t version = 100; + MetaRowsetKeyInfo rowset_key {instance_id, tablet_id, version}; + std::string encoded_rowset_key0; + meta_rowset_key(rowset_key, &encoded_rowset_key0); + std::cout << hex(encoded_rowset_key0) << std::endl; + + std::string dec_instance_id; + int64_t dec_tablet_id = 0; + int64_t dec_version = 0; + + std::string_view key_sv(encoded_rowset_key0); + std::string dec_meta_prefix; + std::string dec_rowset_prefix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_meta_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_rowset_prefix), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_tablet_id), 0) << hex(key_sv); + ASSERT_EQ(decode_int64(&key_sv, &dec_version), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("meta", dec_meta_prefix); + EXPECT_EQ("rowset", dec_rowset_prefix); + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(tablet_id, dec_tablet_id); + EXPECT_EQ(version, dec_version); + + std::get<2>(rowset_key) = version + 1; + std::string encoded_rowset_key1; + meta_rowset_key(rowset_key, &encoded_rowset_key1); + std::cout << hex(encoded_rowset_key1) << std::endl; + + ASSERT_GT(encoded_rowset_key1, encoded_rowset_key0); + } + + // tmp rowset meta key + // 0x01 "meta" ${instance_id} "rowset_tmp" ${tablet_id} ${version} + { + int64_t tablet_id = 10086; + int64_t version = 100; + MetaRowsetKeyInfo rowset_key {instance_id, tablet_id, version}; + std::string encoded_rowset_key0; + meta_rowset_tmp_key(rowset_key, &encoded_rowset_key0); + std::cout << hex(encoded_rowset_key0) << std::endl; + + std::string dec_instance_id; + int64_t dec_tablet_id = 0; + int64_t dec_version = 0; + + std::string_view key_sv(encoded_rowset_key0); + std::string dec_meta_prefix; + std::string dec_rowset_prefix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_meta_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_rowset_prefix), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_tablet_id), 0) << hex(key_sv); + ASSERT_EQ(decode_int64(&key_sv, &dec_version), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("meta", dec_meta_prefix); + EXPECT_EQ("rowset_tmp", dec_rowset_prefix); + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(tablet_id, dec_tablet_id); + EXPECT_EQ(version, dec_version); + + std::get<2>(rowset_key) = version + 1; + std::string encoded_rowset_key1; + meta_rowset_tmp_key(rowset_key, &encoded_rowset_key1); + std::cout << hex(encoded_rowset_key1) << std::endl; + + ASSERT_GT(encoded_rowset_key1, encoded_rowset_key0); + } + + // tablet meta key + // 0x01 "meta" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} -> TabletMetaCloudPB + { + int64_t table_id = 10010; + int64_t index_id = 10011; + int64_t partition_id = 10012; + int64_t tablet_id = 10086; + MetaTabletKeyInfo tablet_key {instance_id, table_id, index_id, partition_id, tablet_id}; + std::string encoded_rowset_key0; + meta_tablet_key(tablet_key, &encoded_rowset_key0); + std::cout << hex(encoded_rowset_key0) << std::endl; + + std::string dec_instance_id; + int64_t dec_table_id = 0; + int64_t dec_tablet_id = 0; + int64_t dec_index_id = 0; + int64_t dec_partition_id = 0; + + std::string_view key_sv(encoded_rowset_key0); + std::string dec_meta_prefix; + std::string dec_tablet_prefix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_meta_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_tablet_prefix), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_table_id), 0) << hex(key_sv); + ASSERT_EQ(decode_int64(&key_sv, &dec_index_id), 0) << hex(key_sv); + ASSERT_EQ(decode_int64(&key_sv, &dec_partition_id), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_tablet_id), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("meta", dec_meta_prefix); + EXPECT_EQ("tablet", dec_tablet_prefix); + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(table_id, dec_table_id); + EXPECT_EQ(index_id, dec_index_id); + EXPECT_EQ(partition_id, dec_partition_id); + EXPECT_EQ(tablet_id, dec_tablet_id); + + std::get<2>(tablet_key) = tablet_id + 1; + std::string encoded_rowset_key1; + meta_tablet_key(tablet_key, &encoded_rowset_key1); + std::cout << hex(encoded_rowset_key1) << std::endl; + + ASSERT_GT(encoded_rowset_key1, encoded_rowset_key0); + } + + // tablet index key + // 0x01 "meta" ${instance_id} "tablet_index" ${tablet_id} -> ${table_id} + { + int64_t tablet_id = 10086; + MetaTabletIdxKeyInfo tablet_tbl_key {instance_id, tablet_id}; + std::string encoded_rowset_key0; + meta_tablet_idx_key(tablet_tbl_key, &encoded_rowset_key0); + std::cout << hex(encoded_rowset_key0) << std::endl; + + std::string dec_instance_id; + int64_t dec_tablet_id = 0; + + std::string_view key_sv(encoded_rowset_key0); + std::string dec_meta_prefix; + std::string dec_tablet_prefix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_meta_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_tablet_prefix), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_tablet_id), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("meta", dec_meta_prefix); + EXPECT_EQ("tablet_index", dec_tablet_prefix); + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(tablet_id, dec_tablet_id); + + std::get<1>(tablet_tbl_key) = tablet_id + 1; + std::string encoded_rowset_key1; + meta_tablet_idx_key(tablet_tbl_key, &encoded_rowset_key1); + std::cout << hex(encoded_rowset_key1) << std::endl; + + ASSERT_GT(encoded_rowset_key1, encoded_rowset_key0); + } + + // tablet schema key + // 0x01 "meta" ${instance_id} "schema" ${index_id} ${schema_version} -> TabletSchemaCloudPB + { + int64_t index_id = 10000; + int32_t schema_version = 5; + auto key = meta_schema_key({instance_id, index_id, schema_version}); + + std::string dec_instance_id; + int64_t dec_index_id = 0; + int64_t dec_schema_version = 0; + + std::string_view key_sv(key); + std::string dec_schema_prefix; + std::string dec_schema_infix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_schema_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_schema_infix), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_index_id), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_schema_version), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(index_id, dec_index_id); + EXPECT_EQ(schema_version, dec_schema_version); + EXPECT_EQ(dec_schema_prefix, "meta"); + EXPECT_EQ(dec_schema_infix, "schema"); + + // TODO: the order of schema version? + } +} + +TEST(KeysTest, VersionKeyTest) { + using namespace doris::cloud; + std::string instance_id = "instance_id_deadbeef"; + + // 0x01 "version" ${instance_id} "version_id" ${db_id} ${tbl_id} ${partition_id} -> ${version} + { + int64_t db_id = 11111; + int64_t tablet_id = 10086; + int64_t partition_id = 9998; + VersionKeyInfo v_key {instance_id, db_id, tablet_id, partition_id}; + std::string encoded_version_key0; + version_key(v_key, &encoded_version_key0); + std::cout << "version key after encode: " << hex(encoded_version_key0) << std::endl; + + std::string dec_instance_id; + int64_t dec_db_id = 0; + int64_t dec_table_id = 0; + int64_t dec_partition_id = 0; + + std::string_view key_sv(encoded_version_key0); + std::string dec_version_prefix; + std::string dec_version_infix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_version_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_version_infix), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_db_id), 0) << hex(key_sv); + ASSERT_EQ(decode_int64(&key_sv, &dec_table_id), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_partition_id), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("version", dec_version_prefix); + EXPECT_EQ("partition", dec_version_infix); + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(db_id, dec_db_id); + EXPECT_EQ(tablet_id, dec_table_id); + EXPECT_EQ(partition_id, dec_partition_id); + + std::get<3>(v_key) = partition_id + 1; + std::string encoded_version_key1; + version_key(v_key, &encoded_version_key1); + std::cout << "version key after encode: " << hex(encoded_version_key1) << std::endl; + + ASSERT_GT(encoded_version_key1, encoded_version_key0); + } +} + +TEST(KeysTest, TxnKeysTest) { + using namespace doris::cloud; + std::string instance_id = "instance_id_deadbeef"; + + // 0x01 "txn" ${instance_id} "txn_label" ${db_id} ${label} -> set<${version_timestamp}> + { + int64_t db_id = 12345678; + std::string label = "label1xxx"; + TxnLabelKeyInfo index_key {instance_id, db_id, label}; + std::string encoded_txn_index_key0; + txn_label_key(index_key, &encoded_txn_index_key0); + std::cout << hex(encoded_txn_index_key0) << std::endl; + + std::string dec_instance_id; + int64_t dec_db_id = 0; + std::string dec_label; + + std::string_view key_sv(encoded_txn_index_key0); + std::string dec_txn_prefix; + std::string dec_txn_infix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_txn_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_txn_infix), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_db_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_label), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("txn", dec_txn_prefix); + EXPECT_EQ("txn_label", dec_txn_infix); + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(db_id, dec_db_id); + EXPECT_EQ(label, dec_label); + + std::get<1>(index_key) = db_id + 1; + std::string encoded_txn_index_key1; + txn_label_key(index_key, &encoded_txn_index_key1); + std::cout << hex(encoded_txn_index_key1) << std::endl; + + ASSERT_GT(encoded_txn_index_key1, encoded_txn_index_key0); + } + + // 0x01 "txn" ${instance_id} "txn_info" ${db_id} ${version_timestamp} -> TxnInfoPB + { + int64_t db_id = 12345678; + int64_t txn_id = 10086; + TxnInfoKeyInfo info_key {instance_id, db_id, txn_id}; + std::string encoded_txn_info_key0; + txn_info_key(info_key, &encoded_txn_info_key0); + std::cout << hex(encoded_txn_info_key0) << std::endl; + + std::string dec_instance_id; + int64_t dec_db_id = 0; + int64_t dec_txn_id = 0; + + std::string_view key_sv(encoded_txn_info_key0); + std::string dec_txn_prefix; + std::string dec_txn_infix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_txn_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_txn_infix), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_db_id), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_txn_id), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("txn", dec_txn_prefix); + EXPECT_EQ("txn_info", dec_txn_infix); + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(db_id, dec_db_id); + EXPECT_EQ(txn_id, dec_txn_id); + + std::get<2>(info_key) = txn_id + 1; + std::string encoded_txn_info_key1; + txn_info_key(info_key, &encoded_txn_info_key1); + std::cout << hex(encoded_txn_info_key1) << std::endl; + + ASSERT_GT(encoded_txn_info_key1, encoded_txn_info_key0); + + std::get<1>(info_key) = db_id + 1; + std::string encoded_txn_info_key2; + txn_info_key(info_key, &encoded_txn_info_key2); + std::cout << hex(encoded_txn_info_key2) << std::endl; + + ASSERT_GT(encoded_txn_info_key2, encoded_txn_info_key0); + } + + // 0x01 "txn" ${instance_id} "txn_index" ${version_timestamp} -> TxnIndexPB + { + int64_t txn_id = 12343212453; + TxnIndexKeyInfo txn_index_key_ {instance_id, txn_id}; + std::string encoded_txn_index_key0; + txn_index_key(txn_index_key_, &encoded_txn_index_key0); + std::cout << hex(encoded_txn_index_key0) << std::endl; + + std::string dec_instance_id; + int64_t dec_txn_id = 0; + + std::string_view key_sv(encoded_txn_index_key0); + std::string dec_txn_prefix; + std::string dec_txn_infix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_txn_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_txn_infix), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_txn_id), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("txn", dec_txn_prefix); + EXPECT_EQ("txn_index", dec_txn_infix); + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(txn_id, dec_txn_id); + + std::get<1>(txn_index_key_) = txn_id + 1; + std::string encoded_txn_index_key1; + txn_index_key(txn_index_key_, &encoded_txn_index_key1); + std::cout << hex(encoded_txn_index_key1) << std::endl; + + ASSERT_GT(encoded_txn_index_key1, encoded_txn_index_key0); + } + + // 0x01 "txn" ${instance_id} "txn_running" ${db_id} ${version_timestamp} -> ${table_id_list} + { + int64_t db_id = 98712345; + int64_t txn_id = 12343212453; + TxnRunningKeyInfo running_key {instance_id, db_id, txn_id}; + std::string encoded_txn_running_key0; + txn_running_key(running_key, &encoded_txn_running_key0); + std::cout << hex(encoded_txn_running_key0) << std::endl; + + std::string dec_instance_id; + int64_t dec_db_id = 0; + int64_t dec_txn_id = 0; + + std::string_view key_sv(encoded_txn_running_key0); + std::string dec_txn_prefix; + std::string dec_txn_infix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_txn_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_txn_infix), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_db_id), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_txn_id), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("txn", dec_txn_prefix); + EXPECT_EQ("txn_running", dec_txn_infix); + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(db_id, dec_db_id); + EXPECT_EQ(txn_id, dec_txn_id); + + std::get<2>(running_key) = txn_id + 1; + std::string encoded_txn_running_key1; + txn_running_key(running_key, &encoded_txn_running_key1); + std::cout << hex(encoded_txn_running_key1) << std::endl; + + ASSERT_GT(encoded_txn_running_key1, encoded_txn_running_key0); + } +} + +TEST(KeysTest, RecycleKeysTest) { + using namespace doris::cloud; + std::string instance_id = "instance_id_deadbeef"; + + // 0x01 "recycle" ${instance_id} "index" ${index_id} -> RecycleIndexPB + { + int64_t index_id = 1234545; + RecycleIndexKeyInfo recycle_key {instance_id, index_id}; + std::string encoded_recycle_key0; + recycle_index_key(recycle_key, &encoded_recycle_key0); + std::cout << hex(encoded_recycle_key0) << std::endl; + + std::string dec_instance_id; + int64_t dec_index_id = 0; + + std::string_view key_sv(encoded_recycle_key0); + std::string dec_recycle_prefix; + std::string dec_recycle_infix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_recycle_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_recycle_infix), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_index_id), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("recycle", dec_recycle_prefix); + EXPECT_EQ("index", dec_recycle_infix); + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(index_id, dec_index_id); + } + + // 0x01 "recycle" ${instance_id} "partition" ${partition_id} -> RecyclePartitionPB + { + int64_t partition_id = 12345450; + RecyclePartKeyInfo recycle_key {instance_id, partition_id}; + std::string encoded_recycle_key0; + recycle_partition_key(recycle_key, &encoded_recycle_key0); + std::cout << hex(encoded_recycle_key0) << std::endl; + + std::string dec_instance_id; + int64_t dec_partition_id = 0; + + std::string_view key_sv(encoded_recycle_key0); + std::string dec_recycle_prefix; + std::string dec_recycle_infix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_recycle_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_recycle_infix), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_partition_id), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("recycle", dec_recycle_prefix); + EXPECT_EQ("partition", dec_recycle_infix); + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(partition_id, dec_partition_id); + } + + // 0x01 "recycle" ${instance_id} "rowset" ${tablet_id} ${rowset_id} -> RecycleRowsetPB + { + int64_t tablet_id = 100201; + std::string rowset_id = "202201"; + RecycleRowsetKeyInfo recycle_key {instance_id, tablet_id, rowset_id}; + std::string encoded_recycle_key0; + recycle_rowset_key(recycle_key, &encoded_recycle_key0); + std::cout << hex(encoded_recycle_key0) << std::endl; + + std::string dec_instance_id; + int64_t dec_tablet_id = 0; + std::string dec_rowset_id; + + std::string_view key_sv(encoded_recycle_key0); + std::string dec_recycle_prefix; + std::string dec_recycle_infix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_recycle_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_recycle_infix), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_tablet_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_rowset_id), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("recycle", dec_recycle_prefix); + EXPECT_EQ("rowset", dec_recycle_infix); + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(tablet_id, dec_tablet_id); + EXPECT_EQ(rowset_id, dec_rowset_id); + } + + // 0x01 "recycle" ${instance_id} "txn" ${db_id} ${txn_id} -> RecycleTxnKeyInfo + { + int64_t db_id = 98712345; + int64_t txn_id = 12343212453; + RecycleTxnKeyInfo recycle_key {instance_id, db_id, txn_id}; + std::string encoded_recycle_txn_key0; + recycle_txn_key(recycle_key, &encoded_recycle_txn_key0); + std::cout << hex(encoded_recycle_txn_key0) << std::endl; + + std::string dec_instance_id; + int64_t dec_db_id = 0; + int64_t dec_txn_id = 0; + + std::string_view key_sv(encoded_recycle_txn_key0); + std::string dec_txn_prefix; + std::string dec_txn_infix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_txn_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_txn_infix), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_db_id), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_txn_id), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("recycle", dec_txn_prefix); + EXPECT_EQ("txn", dec_txn_infix); + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(db_id, dec_db_id); + EXPECT_EQ(txn_id, dec_txn_id); + } + + // 0x01 "recycle" ${instance_id} "stage" ${stage_id} -> RecycleStagePB + { + std::string stage_id = "100201"; + RecycleStageKeyInfo recycle_key {instance_id, stage_id}; + std::string encoded_recycle_key0; + recycle_stage_key(recycle_key, &encoded_recycle_key0); + std::cout << hex(encoded_recycle_key0) << std::endl; + + std::string dec_instance_id; + std::string dec_stage_id; + + std::string_view key_sv(encoded_recycle_key0); + std::string dec_recycle_prefix; + std::string dec_recycle_infix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_recycle_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_recycle_infix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_stage_id), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("recycle", dec_recycle_prefix); + EXPECT_EQ("stage", dec_recycle_infix); + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(stage_id, dec_stage_id); + } +} + +TEST(KeysTest, StatsKeysTest) { + using namespace doris::cloud; + std::string instance_id = "instance_id_deadbeef"; + + int64_t table_id = 123; + int64_t index_id = 345; + int64_t partition_id = 1231231231; + int64_t tablet_id = 543671234523; + + auto expect_stats_prefix = [&](std::string_view& key_sv) { + std::string dec_instance_id; + int64_t dec_table_id = 0; + int64_t dec_index_id = 0; + int64_t dec_partition_id = 0; + int64_t dec_tablet_id = 0; + + std::string dec_stats_prefix; + std::string dec_stats_infix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_stats_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_stats_infix), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_table_id), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_index_id), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_partition_id), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_tablet_id), 0); + + EXPECT_EQ("stats", dec_stats_prefix); + EXPECT_EQ("tablet", dec_stats_infix); + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(table_id, dec_table_id); + EXPECT_EQ(index_id, dec_index_id); + EXPECT_EQ(partition_id, dec_partition_id); + EXPECT_EQ(tablet_id, dec_tablet_id); + }; + + // 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} -> TabletStatsPB + { + StatsTabletKeyInfo stats_key {instance_id, table_id, index_id, partition_id, tablet_id}; + std::string encoded_stats_key0; + stats_tablet_key(stats_key, &encoded_stats_key0); + std::cout << hex(encoded_stats_key0) << std::endl; + + std::string_view key_sv(encoded_stats_key0); + expect_stats_prefix(key_sv); + ASSERT_TRUE(key_sv.empty()); + } + + // 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "data_size" -> int64 + { + StatsTabletKeyInfo stats_key {instance_id, table_id, index_id, partition_id, tablet_id}; + std::string encoded_stats_key0; + stats_tablet_data_size_key(stats_key, &encoded_stats_key0); + std::cout << hex(encoded_stats_key0) << std::endl; + + std::string dec_stats_suffix; + + std::string_view key_sv(encoded_stats_key0); + expect_stats_prefix(key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_stats_suffix), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("data_size", dec_stats_suffix); + } + + // 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "num_rows" -> int64 + { + StatsTabletKeyInfo stats_key {instance_id, table_id, index_id, partition_id, tablet_id}; + std::string encoded_stats_key0; + stats_tablet_num_rows_key(stats_key, &encoded_stats_key0); + std::cout << hex(encoded_stats_key0) << std::endl; + + std::string dec_stats_suffix; + + std::string_view key_sv(encoded_stats_key0); + expect_stats_prefix(key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_stats_suffix), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("num_rows", dec_stats_suffix); + } + + // 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "num_rowsets" -> int64 + { + StatsTabletKeyInfo stats_key {instance_id, table_id, index_id, partition_id, tablet_id}; + std::string encoded_stats_key0; + stats_tablet_num_rowsets_key(stats_key, &encoded_stats_key0); + std::cout << hex(encoded_stats_key0) << std::endl; + + std::string dec_stats_suffix; + + std::string_view key_sv(encoded_stats_key0); + expect_stats_prefix(key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_stats_suffix), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("num_rowsets", dec_stats_suffix); + } + + // 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "num_segs" -> int64 + { + StatsTabletKeyInfo stats_key {instance_id, table_id, index_id, partition_id, tablet_id}; + std::string encoded_stats_key0; + stats_tablet_num_segs_key(stats_key, &encoded_stats_key0); + std::cout << hex(encoded_stats_key0) << std::endl; + + std::string dec_stats_suffix; + + std::string_view key_sv(encoded_stats_key0); + expect_stats_prefix(key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_stats_suffix), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("num_segs", dec_stats_suffix); + } +} + +TEST(KeysTest, JobKeysTest) { + using namespace doris::cloud; + std::string instance_id = "instance_id_deadbeef"; + + // 0x01 "job" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} -> TabletJobInfoPB + { + int64_t table_id = 123; + int64_t index_id = 345; + int64_t partition_id = 1231231231; + int64_t tablet_id = 543671234523; + JobTabletKeyInfo job_key {instance_id, table_id, index_id, partition_id, tablet_id}; + std::string encoded_job_key0; + job_tablet_key(job_key, &encoded_job_key0); + std::cout << hex(encoded_job_key0) << std::endl; + + std::string dec_instance_id; + int64_t dec_table_id = 0; + int64_t dec_index_id = 0; + int64_t dec_partition_id = 0; + int64_t dec_tablet_id = 0; + + std::string_view key_sv(encoded_job_key0); + std::string dec_job_prefix; + std::string dec_job_infix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_job_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_job_infix), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_table_id), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_index_id), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_partition_id), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_tablet_id), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("job", dec_job_prefix); + EXPECT_EQ("tablet", dec_job_infix); + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(table_id, dec_table_id); + EXPECT_EQ(index_id, dec_index_id); + EXPECT_EQ(partition_id, dec_partition_id); + EXPECT_EQ(tablet_id, dec_tablet_id); + } + + // 0x01 "job" ${instance_id} "recycle" -> JobRecyclePB + { + JobRecycleKeyInfo job_key {instance_id}; + std::string encoded_job_key0; + job_recycle_key(job_key, &encoded_job_key0); + std::cout << hex(encoded_job_key0) << std::endl; + + std::string dec_instance_id; + + std::string_view key_sv(encoded_job_key0); + std::string dec_job_prefix; + std::string dec_job_suffix; + + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_job_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_job_suffix), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("job", dec_job_prefix); + EXPECT_EQ("recycle", dec_job_suffix); + EXPECT_EQ(instance_id, dec_instance_id); + } + + // 0x01 "job" ${instance_id} "check" -> JobRecyclePB + { + JobRecycleKeyInfo job_key {instance_id}; + std::string encoded_job_key0; + job_check_key(job_key, &encoded_job_key0); + std::cout << hex(encoded_job_key0) << std::endl; + + std::string dec_instance_id; + + std::string_view key_sv(encoded_job_key0); + std::string dec_job_prefix; + std::string dec_job_suffix; + + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_job_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_job_suffix), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("job", dec_job_prefix); + EXPECT_EQ("check", dec_job_suffix); + EXPECT_EQ(instance_id, dec_instance_id); + } +} + +TEST(KeysTest, SystemKeysTest) { + using namespace doris::cloud; + std::string instance_id = "instance_id_deadbeef"; + + // 0x02 "system" "meta-service" "registry" -> MetaServiceRegistryPB + // 0x02 "system" "meta-service" "arn_info" -> RamUserPB + // 0x02 "system" "meta-service" "encryption_key_info" -> EncryptionKeyInfoPB + std::vector suffixes {"registry", "arn_info", "encryption_key_info"}; + std::vector> fns { + system_meta_service_registry_key, + system_meta_service_arn_info_key, + system_meta_service_encryption_key_info_key, + }; + size_t num = suffixes.size(); + for (size_t i = 0; i < num; ++i) { + std::string key = fns[i](); + std::cout << hex(key) << std::endl; + + std::string_view key_sv(key); + std::string prefix, infix, suffix; + remove_system_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &infix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &suffix), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("system", prefix); + EXPECT_EQ("meta-service", infix); + EXPECT_EQ(suffixes[i], suffix) << i; + } +} + +TEST(KeysTest, CopyKeysTest) { + using namespace doris::cloud; + std::string instance_id = "instance_id_deadbeef"; + + // 0x01 "copy" ${instance_id} "job" ${stage_id} ${table_id} ${copy_id} ${group_id} -> CopyJobPB + { + int64_t table_id = 3745823784; + int64_t group_id = 1238547388; + std::string stage_id = "9482049283"; + std::string copy_id = "1284758385"; + + CopyJobKeyInfo copy_key {instance_id, stage_id, table_id, copy_id, group_id}; + std::string encoded_copy_job_key0; + copy_job_key(copy_key, &encoded_copy_job_key0); + std::cout << hex(encoded_copy_job_key0) << std::endl; + + std::string dec_instance_id; + std::string dec_stage_id; + std::string dec_copy_id; + int64_t dec_table_id = 0; + int64_t dec_group_id = 0; + + std::string_view key_sv(encoded_copy_job_key0); + std::string dec_copy_job_prefix, dec_copy_job_infix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_copy_job_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_copy_job_infix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_stage_id), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_table_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_copy_id), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_group_id), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("copy", dec_copy_job_prefix); + EXPECT_EQ("job", dec_copy_job_infix); + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(stage_id, dec_stage_id); + EXPECT_EQ(table_id, dec_table_id); + EXPECT_EQ(copy_id, dec_copy_id); + EXPECT_EQ(group_id, dec_group_id); + } + + // 0x01 "copy" ${instance_id} "loading_files" ${stage_id} ${table_id} ${obj_name} ${etag} -> CopyFilePB + { + std::string stage_id = "9482049283"; + int64_t table_id = 3745823784; + std::string obj_name = "test-obj-name"; + std::string etag = "test-etag"; + + CopyFileKeyInfo copy_key {instance_id, stage_id, table_id, obj_name, etag}; + std::string encoded_copy_job_key0; + copy_file_key(copy_key, &encoded_copy_job_key0); + std::cout << hex(encoded_copy_job_key0) << std::endl; + + std::string dec_instance_id; + std::string dec_stage_id; + int64_t dec_table_id = 0; + std::string dec_obj_name; + std::string dec_etag; + + std::string_view key_sv(encoded_copy_job_key0); + std::string dec_copy_prefix, dec_copy_infix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_copy_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_copy_infix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_stage_id), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_table_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_obj_name), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_etag), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("copy", dec_copy_prefix); + EXPECT_EQ("loading_file", dec_copy_infix); + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(stage_id, dec_stage_id); + EXPECT_EQ(table_id, dec_table_id); + EXPECT_EQ(obj_name, dec_obj_name); + EXPECT_EQ(etag, dec_etag); + } +} + +TEST(KeysTest, DecodeKeysTest) { + using namespace doris::cloud; + // clang-format off + std::string key = "011074786e000110696e7374616e63655f69645f646561646265656600011074786e5f696e646578000112000000000000271310696e736572745f336664356164313264303035346139622d386337373664333231386336616462370001"; + // clang-format on + auto pretty_key = prettify_key(key); + ASSERT_TRUE(!pretty_key.empty()) << key; + std::cout << "\n" << pretty_key << std::endl; + + pretty_key = prettify_key(key, true); + ASSERT_TRUE(!pretty_key.empty()) << key; + std::cout << "\n" << pretty_key << std::endl; +} diff --git a/cloud/test/log_test.cpp b/cloud/test/log_test.cpp new file mode 100644 index 00000000000000..aeb8ccd9defc72 --- /dev/null +++ b/cloud/test/log_test.cpp @@ -0,0 +1,110 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include +#include + +#include +#include +#include + +#include "common/logging.h" + +using doris::cloud::AnnotateTag; + +int main(int argc, char** argv) { + if (!doris::cloud::init_glog("log_test")) { + std::cerr << "failed to init glog" << std::endl; + return -1; + } + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + +TEST(LogTest, ConstructionTest) { + // Heap allocation is disabled. + // new AnnotateTag(); + + // Arithmetics + { + char c = 0; + bool b = false; + int8_t i8 = 0; + uint8_t u8 = 0; + int16_t i16 = 0; + uint16_t u16 = 0; + int32_t i32 = 0; + uint32_t u32 = 0; + int64_t i64 = 0; + uint64_t u64 = 0; + + AnnotateTag tag_char("char", c); + AnnotateTag tag_bool("bool", b); + AnnotateTag tag_i8("i8", i8); + AnnotateTag tag_u8("u8", u8); + AnnotateTag tag_i16("i16", i16); + AnnotateTag tag_u16("u16", u16); + AnnotateTag tag_i32("i32", i32); + AnnotateTag tag_u32("u32", u32); + AnnotateTag tag_i64("i64", i64); + AnnotateTag tag_u64("u64", u64); + LOG_INFO("hello"); + } + + // String literals. + { + const char* text = "hello"; + AnnotateTag tag_text("hello", text); + LOG_INFO("hello"); + } + + // String view. + { + std::string test("abc"); + AnnotateTag tag_text("hello", std::string_view(test)); + LOG_INFO("hello"); + } + + // Const string. + { + const std::string test("abc"); + AnnotateTag tag_text("hello", test); + LOG_INFO("hello"); + } +} + +TEST(LogTest, ThreadTest) { + // In pthread. + { + ASSERT_EQ(bthread_self(), 0); + AnnotateTag tag("run_in_bthread", true); + LOG_INFO("thread test"); + } + + // In bthread. + { + auto fn = +[](void*) -> void* { + EXPECT_NE(bthread_self(), 0); + AnnotateTag tag("run_in_bthread", true); + LOG_INFO("thread test"); + return nullptr; + }; + bthread_t tid; + ASSERT_EQ(bthread_start_background(&tid, nullptr, fn, nullptr), 0); + ASSERT_EQ(bthread_join(tid, nullptr), 0); + } +} \ No newline at end of file diff --git a/cloud/test/mem_txn_kv_test.cpp b/cloud/test/mem_txn_kv_test.cpp new file mode 100644 index 00000000000000..9db71ba96c8b5b --- /dev/null +++ b/cloud/test/mem_txn_kv_test.cpp @@ -0,0 +1,940 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "meta-service/mem_txn_kv.h" + +#include +#include + +#include + +#include "common/config.h" +#include "common/util.h" +#include "meta-service/doris_txn.h" +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" + +using namespace doris; + +std::shared_ptr fdb_txn_kv; + +int main(int argc, char** argv) { + cloud::config::init(nullptr, true); + cloud::config::fdb_cluster_file_path = "fdb.cluster"; + fdb_txn_kv = std::dynamic_pointer_cast(std::make_shared()); + if (!fdb_txn_kv.get()) { + std::cout << "exit get FdbTxnKv error" << std::endl; + return -1; + } + if (fdb_txn_kv->init() != 0) { + std::cout << "exit inti FdbTxnKv error" << std::endl; + return -1; + } + + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + +static void put_and_get_test(std::shared_ptr txn_kv) { + using namespace doris::cloud; + std::string txn_kv_class = dynamic_cast(txn_kv.get()) != nullptr ? " memkv" : " fdb"; + std::unique_ptr txn; + std::string key = "testkey1"; + std::string val = "testvalue1"; + { + // put + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK) << txn_kv_class; + int64_t ver1; + ASSERT_EQ(txn->get_committed_version(&ver1), TxnErrorCode::TXN_OK); + + // get + std::string get_val; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(key, &get_val), TxnErrorCode::TXN_OK) << txn_kv_class; + int64_t ver2 = 0; + ASSERT_EQ(txn->get_read_version(&ver2), TxnErrorCode::TXN_OK); + ASSERT_GE(ver2, ver1) << txn_kv_class; + ASSERT_EQ(val, get_val) << txn_kv_class; + std::cout << "val:" << get_val << std::endl; + + // get not exist key + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get("NotExistKey", &get_val), TxnErrorCode::TXN_KEY_NOT_FOUND) + << txn_kv_class; + } +} + +TEST(TxnMemKvTest, PutAndGetTest) { + using namespace doris::cloud; + + auto mem_txn_kv = std::dynamic_pointer_cast(std::make_shared()); + ASSERT_NE(mem_txn_kv.get(), nullptr); + + put_and_get_test(mem_txn_kv); + put_and_get_test(fdb_txn_kv); +} + +static void range_get_test(std::shared_ptr txn_kv) { + using namespace doris::cloud; + std::string txn_kv_class = dynamic_cast(txn_kv.get()) != nullptr ? " memkv" : " fdb"; + std::unique_ptr txn; + std::vector> put_kv = { + std::make_pair("key1", "val1"), std::make_pair("key2", "val2"), + std::make_pair("key3", "val3"), std::make_pair("key4", "val4"), + std::make_pair("key5", "val5"), + }; + + // put some kvs before test + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + for (const auto& [key, val] : put_kv) { + txn->put(key, val); + } + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + // normal range get + { + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::unique_ptr iter; + ASSERT_EQ(txn->get("key1", "key4", &iter), TxnErrorCode::TXN_OK) << txn_kv_class; + ASSERT_EQ(iter->size(), 3) << txn_kv_class; + ASSERT_EQ(iter->more(), false) << txn_kv_class; + int i = 0; + while (iter->has_next()) { + auto [key, val] = iter->next(); + ASSERT_EQ(key, put_kv[i].first) << txn_kv_class; + ASSERT_EQ(val, put_kv[i].second) << txn_kv_class; + ++i; + std::cout << "key:" << key << " val:" << val << std::endl; + } + } + + // range get with not exist end key + { + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::unique_ptr iter; + ASSERT_EQ(txn->get("key2", "key6", &iter), TxnErrorCode::TXN_OK); + ASSERT_EQ(iter->size(), 4) << txn_kv_class; + ASSERT_EQ(iter->more(), false) << txn_kv_class; + int i = 1; + while (iter->has_next()) { + auto [key, val] = iter->next(); + ASSERT_EQ(key, put_kv[i].first) << txn_kv_class; + ASSERT_EQ(val, put_kv[i].second) << txn_kv_class; + ++i; + std::cout << "key:" << key << " val:" << val << std::endl; + } + } + + // range get with limit + { + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::unique_ptr iter; + ASSERT_EQ(txn->get("key1", "key4", &iter, false, 1), TxnErrorCode::TXN_OK); + ASSERT_EQ(iter->size(), 1) << txn_kv_class; + ASSERT_EQ(iter->more(), true) << txn_kv_class; + + auto [key, val] = iter->next(); + ASSERT_EQ(key, put_kv[0].first) << txn_kv_class; + ASSERT_EQ(val, put_kv[0].second) << txn_kv_class; + } + + // range get with begin key larger than end key + { + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::unique_ptr iter; + ASSERT_EQ(txn->get("key4", "key1", &iter), TxnErrorCode::TXN_OK); + ASSERT_EQ(iter->size(), 0) << txn_kv_class; + ASSERT_EQ(txn->get("key1", "key1", &iter), TxnErrorCode::TXN_OK); + ASSERT_EQ(iter->size(), 0) << txn_kv_class; + } +} + +TEST(TxnMemKvTest, RangeGetTest) { + using namespace doris::cloud; + + auto mem_txn_kv = std::dynamic_pointer_cast(std::make_shared()); + ASSERT_NE(mem_txn_kv.get(), nullptr); + + range_get_test(mem_txn_kv); + range_get_test(fdb_txn_kv); +} + +static void remove_test(std::shared_ptr txn_kv) { + using namespace doris::cloud; + std::unique_ptr txn; + std::string txn_kv_class = dynamic_cast(txn_kv.get()) != nullptr ? " memkv" : " fdb"; + std::vector> put_kv = { + std::make_pair("key1", "val1"), std::make_pair("key2", "val2"), + std::make_pair("key3", "val3"), std::make_pair("key4", "val4"), + std::make_pair("key5", "val5"), + }; + + // put some kvs before test + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + for (const auto& [key, val] : put_kv) { + txn->put(key, val); + } + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + // remove single key + { + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->remove("key1"); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string get_val; + ASSERT_EQ(txn->get("key1", &get_val), TxnErrorCode::TXN_KEY_NOT_FOUND) << txn_kv_class; + } + + // range remove with begin key larger than end key + { + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->remove("key5", "key1"); + ASSERT_NE(txn->commit(), TxnErrorCode::TXN_OK); + + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::unique_ptr iter; + ASSERT_EQ(txn->get("key2", "key6", &iter), TxnErrorCode::TXN_OK); + ASSERT_EQ(iter->size(), 4) << txn_kv_class; + } + + // range remove + { + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + + txn->remove("key2", "key6"); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::unique_ptr iter; + ASSERT_EQ(txn->get("key2", "key6", &iter), TxnErrorCode::TXN_OK); + while (iter->has_next()) { + auto [key, value] = iter->next(); + std::cout << "key: " << key << ", value: " << value << std::endl; + } + ASSERT_EQ(iter->size(), 0) << txn_kv_class; + } +} +TEST(TxnMemKvTest, RemoveTest) { + using namespace doris::cloud; + + auto mem_txn_kv = std::dynamic_pointer_cast(std::make_shared()); + ASSERT_NE(mem_txn_kv.get(), nullptr); + remove_test(mem_txn_kv); + remove_test(fdb_txn_kv); +} + +static void atomic_set_ver_value_test(std::shared_ptr txn_kv) { + using namespace doris::cloud; + std::unique_ptr txn; + std::string txn_kv_class = dynamic_cast(txn_kv.get()) != nullptr ? " memkv" : " fdb"; + // txn_kv_test.cpp + { + std::string key; + std::string val; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + key.push_back('\xfe'); + key.append(" unit_test_prefix "); + key.append(" GetVersionTest "); + txn->atomic_set_ver_value(key, ""); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + int64_t ver0 = 0; + ASSERT_EQ(txn->get_committed_version(&ver0), TxnErrorCode::TXN_OK); + ASSERT_GT(ver0, 0) << txn_kv_class; + + TxnErrorCode err = txn_kv->create_txn(&txn); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + err = txn->get(key, &val); + ASSERT_EQ(err, TxnErrorCode::TXN_OK) << txn_kv_class; + int64_t ver1 = 0; + ASSERT_EQ(txn->get_read_version(&ver1), TxnErrorCode::TXN_OK); + ASSERT_GE(ver1, ver0) << txn_kv_class; + + int64_t ver2; + int64_t txn_id; + int ret = get_txn_id_from_fdb_ts(val, &txn_id); + ASSERT_EQ(ret, 0) << txn_kv_class; + ver2 = txn_id >> 10; + std::cout << "ver0=" << ver0 << " ver1=" << ver1 << " ver2=" << ver2 << std::endl; + + // ASSERT_EQ(ver0, ver2); + } +} + +TEST(TxnMemKvTest, AtomicSetVerValueTest) { + using namespace doris::cloud; + auto mem_txn_kv = std::dynamic_pointer_cast(std::make_shared()); + ASSERT_NE(mem_txn_kv.get(), nullptr); + + atomic_set_ver_value_test(mem_txn_kv); + atomic_set_ver_value_test(fdb_txn_kv); +} + +static void atomic_set_ver_key_test(std::shared_ptr txn_kv) { + using namespace doris::cloud; + std::string key_prefix = "key_1"; + + std::string versionstamp_1; + { + // write key_1 + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->atomic_set_ver_key(key_prefix, "1"); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + // read key_1 + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string end_key = key_prefix + "\xFF"; + std::unique_ptr it; + ASSERT_EQ(txn->get(key_prefix, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_TRUE(it->has_next()); + auto&& [key_1, _1] = it->next(); + ASSERT_EQ(key_1.length(), key_prefix.size() + 10); // versionstamp = 10bytes + key_1.remove_prefix(key_prefix.size()); + versionstamp_1 = key_1; + } + + std::string versionstamp_2; + { + // write key_2 + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + key_prefix = "key_2"; + txn->atomic_set_ver_key(key_prefix, "2"); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + // read key_2 + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string end_key = key_prefix + "\xFF"; + std::unique_ptr it; + ASSERT_EQ(txn->get(key_prefix, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_TRUE(it->has_next()); + auto&& [key_2, _2] = it->next(); + ASSERT_EQ(key_2.length(), key_prefix.size() + 10); // versionstamp = 10bytes + key_2.remove_prefix(key_prefix.size()); + versionstamp_2 = key_2; + } + + ASSERT_LT(versionstamp_1, versionstamp_2); +} + +TEST(TxnMemKvTest, AtomicSetVerKeyTest) { + using namespace doris::cloud; + auto mem_txn_kv = std::dynamic_pointer_cast(std::make_shared()); + ASSERT_NE(mem_txn_kv.get(), nullptr); + + atomic_set_ver_key_test(mem_txn_kv); + atomic_set_ver_key_test(fdb_txn_kv); +} + +static void atomic_add_test(std::shared_ptr txn_kv) { + using namespace doris::cloud; + std::unique_ptr txn; + std::string txn_kv_class = dynamic_cast(txn_kv.get()) != nullptr ? " memkv" : " fdb"; + // clear counter + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->remove("counter"); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + // add to uninitialized kv + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->atomic_add("counter", 123); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string val; + ASSERT_EQ(txn->get("counter", &val), TxnErrorCode::TXN_OK); + int64_t val_int = *reinterpret_cast(val.data()); + ASSERT_EQ(val_int, 123) << txn_kv_class; + + txn->put("counter", "1"); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + // add + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->atomic_add("counter", 10); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get("counter", &val), TxnErrorCode::TXN_OK); + val_int = *reinterpret_cast(val.data()); + std::cout << "atomic add: " << val_int << std::endl; + ASSERT_EQ(val_int, 59) << txn_kv_class; // "1" + 10 = ASCII("1") + 10 = 49 + 10 = 59 + + // sub + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->atomic_add("counter", -5); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get("counter", &val), TxnErrorCode::TXN_OK); + val_int = *reinterpret_cast(val.data()); + std::cout << "atomic sub: " << val_int << std::endl; + ASSERT_EQ(val_int, 54) << txn_kv_class; +} + +TEST(TxnMemKvTest, AtomicAddTest) { + using namespace doris::cloud; + auto mem_txn_kv = std::dynamic_pointer_cast(std::make_shared()); + ASSERT_NE(mem_txn_kv.get(), nullptr); + + atomic_add_test(mem_txn_kv); + atomic_add_test(fdb_txn_kv); +} + +// modify identical key in one transcation +static void modify_identical_key_test(std::shared_ptr txn_kv) { + using namespace doris::cloud; + std::unique_ptr txn; + std::string txn_kv_class = dynamic_cast(txn_kv.get()) != nullptr ? " memkv" : " fdb"; + // put after remove + { + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put("test", "1"); + txn->remove("test"); + txn->put("test", "2"); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + std::string get_val; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get("test", &get_val), TxnErrorCode::TXN_OK) << txn_kv_class; + ASSERT_EQ(get_val, "2") << txn_kv_class; + } + + // remove after put + { + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put("test", "1"); + txn->remove("test"); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + std::string get_val; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get("test", &get_val), TxnErrorCode::TXN_KEY_NOT_FOUND) << txn_kv_class; + } +} + +TEST(TxnMemKvTest, ModifyIdenticalKeyTest) { + using namespace doris::cloud; + auto mem_txn_kv = std::dynamic_pointer_cast(std::make_shared()); + ASSERT_NE(mem_txn_kv.get(), nullptr); + + modify_identical_key_test(mem_txn_kv); + modify_identical_key_test(fdb_txn_kv); +} + +static void modify_snapshot_test(std::shared_ptr txn_kv) { + using namespace doris::cloud; + std::unique_ptr txn_1; + std::unique_ptr txn_2; + std::string txn_kv_class = dynamic_cast(txn_kv.get()) != nullptr ? " memkv" : " fdb"; + { + std::string get_val; + // txn_1: put and commit + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->put("test", "version1"); + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + // txn_2: get the snapshot of database, will see + ASSERT_EQ(txn_kv->create_txn(&txn_2), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn_2->get("test", &get_val), TxnErrorCode::TXN_OK); + ASSERT_EQ(get_val, "version1") << txn_kv_class; + + // txn_1: modify to and commit + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->put("test", "version2"); + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + // txn_2: should still see the + ASSERT_EQ(txn_2->get("test", &get_val), TxnErrorCode::TXN_OK); + ASSERT_EQ(get_val, "version1") << txn_kv_class; + + // txn_2: modify to but not commit, + // txn_2 should get + txn_2->put("test", "version3"); + ASSERT_EQ(txn_2->get("test", &get_val), TxnErrorCode::TXN_OK); + ASSERT_EQ(get_val, "version3") << txn_kv_class; + + // txn_2: remove bu not commit, + // txn_2 should not get + txn_2->remove("test"); + ASSERT_EQ(txn_2->get("test", &get_val), TxnErrorCode::TXN_KEY_NOT_FOUND) << txn_kv_class; + + // txn_1: will still see + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn_1->get("test", &get_val), TxnErrorCode::TXN_OK); + ASSERT_EQ(get_val, "version2") << txn_kv_class; + + // txn_2: commit all changes, should conflict + ASSERT_NE(txn_2->commit(), TxnErrorCode::TXN_OK) << txn_kv_class; + + // txn_1: should not get + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn_1->get("test", &get_val), TxnErrorCode::TXN_OK) << txn_kv_class; + ASSERT_EQ(get_val, "version2") << txn_kv_class; + } + + { + std::string get_val; + + // txn_1: put and commit + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->put("test", "version1"); + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + // txn_2: read the key set by atomic_set_xxx before commit + ASSERT_EQ(txn_kv->create_txn(&txn_2), TxnErrorCode::TXN_OK); + txn_2->atomic_set_ver_value("test", ""); + TxnErrorCode err = txn_2->get("test", &get_val); + // can not read the unreadable key + ASSERT_TRUE(err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) + << txn_kv_class; + // after read the unreadable key, can not commit + ASSERT_NE(txn_2->commit(), TxnErrorCode::TXN_OK); + + // txn_1: still see the + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn_1->get("test", &get_val), TxnErrorCode::TXN_OK) << txn_kv_class; + ASSERT_EQ(get_val, "version1") << txn_kv_class; + } +} + +TEST(TxnMemKvTest, ModifySnapshotTest) { + using namespace doris::cloud; + auto mem_txn_kv = std::dynamic_pointer_cast(std::make_shared()); + ASSERT_NE(mem_txn_kv.get(), nullptr); + + modify_snapshot_test(mem_txn_kv); + modify_snapshot_test(fdb_txn_kv); +} + +static void check_conflicts_test(std::shared_ptr txn_kv) { + using namespace doris::cloud; + std::string txn_kv_class = dynamic_cast(txn_kv.get()) != nullptr ? " memkv" : " fdb"; + std::unique_ptr txn_1; + std::unique_ptr txn_2; + + // txn1 change "key" after txn2 get "key", txn2 should conflict when change "key". + { + std::string get_val; + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->put("key", "txn1_1"); + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + ASSERT_EQ(txn_kv->create_txn(&txn_2), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn_2->get("key", &get_val), TxnErrorCode::TXN_OK); + ASSERT_EQ(get_val, "txn1_1"); + + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->put("key", "txn1_2"); + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + txn_2->put("key", "txn2_1"); + ASSERT_EQ(txn_2->get("key", &get_val), TxnErrorCode::TXN_OK); + ASSERT_EQ(get_val, "txn2_1"); + + // conflicts + ASSERT_NE(txn_2->commit(), TxnErrorCode::TXN_OK) << txn_kv_class; + } + + // txn1 add "key" after txn2 get "key", txn2 should conflict when add "key2". + { + std::string get_val; + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->remove("key"); + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + ASSERT_EQ(txn_kv->create_txn(&txn_2), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn_2->get("key", &get_val), TxnErrorCode::TXN_KEY_NOT_FOUND) << txn_kv_class; + + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->put("key", "txn1_1"); + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + txn_2->put("key2", "txn2_1"); + + // conflicts + ASSERT_NE(txn_2->commit(), TxnErrorCode::TXN_OK) << txn_kv_class; + } + + // txn1 change "key" after txn2 get "key", + // txn2 can read "key2" before commit, but commit conflict. + { + std::string get_val; + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->put("key", "txn1_1"); + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + ASSERT_EQ(txn_kv->create_txn(&txn_2), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn_2->get("key", &get_val), TxnErrorCode::TXN_OK); + ASSERT_EQ(get_val, "txn1_1"); + + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->put("key", "txn1_2"); + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + txn_2->put("key2", "txn2_2"); + ASSERT_EQ(txn_2->get("key2", &get_val), TxnErrorCode::TXN_OK) << txn_kv_class; + ASSERT_EQ(get_val, "txn2_2") << txn_kv_class; + + // conflicts + ASSERT_NE(txn_2->commit(), TxnErrorCode::TXN_OK) << txn_kv_class; + } + + // txn1 change "key" after txn2 get "key", txn2 should conflict when atomic_set "key". + { + std::string get_val; + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->put("key", "txn1_1"); + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + ASSERT_EQ(txn_kv->create_txn(&txn_2), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn_2->get("key", &get_val), TxnErrorCode::TXN_OK); + ASSERT_EQ(get_val, "txn1_1"); + + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->put("key", "txn1_2"); + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + txn_2->atomic_set_ver_value("key", "txn2_2"); + + // conflicts + ASSERT_NE(txn_2->commit(), TxnErrorCode::TXN_OK) << txn_kv_class; + } + + // txn1 change "key1" after txn2 range get "key1~key5", txn2 should conflict when change "key2" + { + std::string get_val; + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->put("key1", "v1"); + txn_1->put("key2", "v2"); + txn_1->put("key3", "v3"); + txn_1->put("key4", "v4"); + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + ASSERT_EQ(txn_kv->create_txn(&txn_2), TxnErrorCode::TXN_OK); + std::unique_ptr iter; + ASSERT_EQ(txn_2->get("key1", "key5", &iter), TxnErrorCode::TXN_OK); + ASSERT_EQ(iter->size(), 4) << txn_kv_class; + + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->put("key1", "v11"); + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + txn_2->put("key2", "v22"); + ASSERT_EQ(txn_2->get("key2", &get_val), TxnErrorCode::TXN_OK) << txn_kv_class; + ASSERT_EQ(get_val, "v22") << txn_kv_class; + + // conflicts + ASSERT_NE(txn_2->commit(), TxnErrorCode::TXN_OK) << txn_kv_class; + } + + // txn1 change "key3" after txn2 limit range get "key1~key5", txn2 do not conflict when change "key4" + { + std::string get_val; + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->put("key1", "v1"); + txn_1->put("key2", "v2"); + txn_1->put("key3", "v3"); + txn_1->put("key4", "v4"); + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + ASSERT_EQ(txn_kv->create_txn(&txn_2), TxnErrorCode::TXN_OK); + std::unique_ptr iter; + ASSERT_EQ(txn_2->get("key1", "key5", &iter, false, 1), TxnErrorCode::TXN_OK); + ASSERT_EQ(iter->size(), 1) << txn_kv_class; + + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->put("key3", "v33"); + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + txn_2->put("key4", "v44"); + ASSERT_EQ(txn_2->get("key4", &get_val), TxnErrorCode::TXN_OK) << txn_kv_class; + ASSERT_EQ(get_val, "v44") << txn_kv_class; + + // not conflicts + ASSERT_EQ(txn_2->commit(), TxnErrorCode::TXN_OK) << txn_kv_class; + } + + // txn1 remove "key1" after txn2 get "key1", txn2 should conflict when change "key5". + { + std::string get_val; + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->put("key1", "v1"); + txn_1->put("key2", "v2"); + txn_1->put("key3", "v3"); + txn_1->put("key4", "v4"); + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + ASSERT_EQ(txn_kv->create_txn(&txn_2), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn_2->get("key1", &get_val), TxnErrorCode::TXN_OK) << txn_kv_class; + ASSERT_EQ(get_val, "v1") << txn_kv_class; + + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->remove("key1"); + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + ASSERT_EQ(txn_2->get("key1", &get_val), TxnErrorCode::TXN_OK) << txn_kv_class; + ASSERT_EQ(get_val, "v1") << txn_kv_class; + + txn_2->put("key5", "v5"); + ASSERT_EQ(txn_2->get("key5", &get_val), TxnErrorCode::TXN_OK) << txn_kv_class; + ASSERT_EQ(get_val, "v5") << txn_kv_class; + + // conflicts + ASSERT_NE(txn_2->commit(), TxnErrorCode::TXN_OK) << txn_kv_class; + } + + // txn1 range remove "key1~key4" after txn2 get "key1", txn2 should conflict when change "key1". + { + std::string get_val; + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->put("key1", "v1"); + txn_1->put("key2", "v2"); + txn_1->put("key3", "v3"); + txn_1->put("key4", "v4"); + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + ASSERT_EQ(txn_kv->create_txn(&txn_2), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn_2->get("key1", &get_val), TxnErrorCode::TXN_OK); + ASSERT_EQ(get_val, "v1"); + + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->remove("key1", "key4"); + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + ASSERT_EQ(txn_2->get("key1", &get_val), TxnErrorCode::TXN_OK); + ASSERT_EQ(get_val, "v1"); + txn_2->put("key1", "v11"); + // conflicts + ASSERT_NE(txn_2->commit(), TxnErrorCode::TXN_OK) << txn_kv_class; + } +} + +TEST(TxnMemKvTest, CheckConflictsTest) { + using namespace doris::cloud; + auto mem_txn_kv = std::dynamic_pointer_cast(std::make_shared()); + ASSERT_NE(mem_txn_kv.get(), nullptr); + + check_conflicts_test(mem_txn_kv); + check_conflicts_test(fdb_txn_kv); +} + +// ConflictTest of txn_kv_test.cpp +TEST(TxnMemKvTest, ConflictTest) { + using namespace doris::cloud; + auto txn_kv = std::dynamic_pointer_cast(std::make_shared()); + ASSERT_NE(txn_kv.get(), nullptr); + std::unique_ptr txn, txn1, txn2; + std::string key = "unit_test"; + std::string val, val1, val2; + + // Historical data + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put("unit_test", "xxxxxxxxxxxxx"); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + // txn1 begin + ASSERT_EQ(txn_kv->create_txn(&txn1), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn1->get(key, &val), TxnErrorCode::TXN_OK); + std::cout << "val1=" << val1 << std::endl; + + // txn2 begin + ASSERT_EQ(txn_kv->create_txn(&txn2), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn2->get(key, &val), TxnErrorCode::TXN_OK); + std::cout << "val2=" << val2 << std::endl; + + // txn2 commit + val2 = "zzzzzzzzzzzzzzz"; + txn2->put(key, val2); + ASSERT_EQ(txn2->commit(), TxnErrorCode::TXN_OK); + + // txn1 commit, intend to fail + val1 = "yyyyyyyyyyyyyyy"; + txn1->put(key, val1); + ASSERT_NE(txn1->commit(), TxnErrorCode::TXN_OK); + + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(key, &val), TxnErrorCode::TXN_OK); + ASSERT_EQ(val, val2); // First wins + std::cout << "final val=" << val << std::endl; +} + +static void txn_behavior_test(std::shared_ptr txn_kv) { + using namespace doris::cloud; + std::string txn_kv_class = dynamic_cast(txn_kv.get()) != nullptr ? " memkv" : " fdb"; + std::unique_ptr txn_1; + std::unique_ptr txn_2; + // av: atomic_set_ver_value + // ak: atomic_set_ver_key + // ad: atomic_add + // c : commit + + // txn_1: --- put -- av --------------- c + // txn_2: ------------------------------- ad ----- put --- c + // result: + { + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->put("key", "v1"); + txn_1->atomic_set_ver_value("key", "v1"); + + ASSERT_EQ(txn_kv->create_txn(&txn_2), TxnErrorCode::TXN_OK); + txn_2->atomic_add("key", 1); + + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + txn_2->put("key", "v2"); + ASSERT_EQ(txn_2->commit(), TxnErrorCode::TXN_OK); + + std::string get_val; + ASSERT_EQ(txn_kv->create_txn(&txn_2), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn_2->get("key", &get_val), TxnErrorCode::TXN_OK); + ASSERT_EQ(get_val, "v2"); + } + + // txn_1: --- ad<"key",1> --- av<"key", "v1"> ------ c + // txn_2: ------------------- av<"key", "v2"> ---- c + // result: <"key", "version"+"v1"> + { + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->atomic_add("key", 1); + txn_1->atomic_set_ver_value("key", "v1"); + + ASSERT_EQ(txn_kv->create_txn(&txn_2), TxnErrorCode::TXN_OK); + txn_2->atomic_set_ver_value("key", "v2"); + + ASSERT_EQ(txn_2->commit(), TxnErrorCode::TXN_OK); + + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + std::string get_val; + ASSERT_EQ(txn_kv->create_txn(&txn_2), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn_2->get("key", &get_val), TxnErrorCode::TXN_OK); + std::cout << get_val << std::endl; + } + + // txn_1: --- put<"key", "1"> --- get<"key"> --- c + // result: can get "1" and commit success + { + std::string get_val; + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->put("key", "1"); + ASSERT_EQ(txn_1->get("key", &get_val), TxnErrorCode::TXN_OK) << txn_kv_class; + ASSERT_EQ(get_val, "1") << txn_kv_class; + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + } + + // txn_1: --- ad<"key",1> --- get<"key"> --- c + // result: commit success + { + std::string get_val; + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->atomic_add("key", 1); + ASSERT_EQ(txn_1->get("key", &get_val), TxnErrorCode::TXN_OK) << txn_kv_class; + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK) << txn_kv_class; + } + + // txn_1: --- av<"key", "1"> --- get<"key"> --- c + // result: can not read the unreadable key and commit error + { + std::string get_val; + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->atomic_set_ver_value("key", "1"); + TxnErrorCode err = txn_1->get("key", &get_val); + // can not read the unreadable key + ASSERT_TRUE(err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) + << txn_kv_class; + ASSERT_NE(txn_1->commit(), TxnErrorCode::TXN_OK) << txn_kv_class; + } + + // txn_1: --- get<"keyNotExit"> --- put<"keyNotExit", "1"> --- get<"keyNotExit"> --- c + // txn_2: --- get<"keyNotExit"> --- put<"keyNotExit", "1"> --- get<"keyNotExit"> ------ c + // result: txn_2 commit conflict + { + std::string get_val; + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + txn_1->remove("keyNotExit"); + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK); + + ASSERT_EQ(txn_kv->create_txn(&txn_1), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn_1->get("keyNotExit", &get_val), TxnErrorCode::TXN_KEY_NOT_FOUND); + txn_1->put("keyNotExit", "1"); + ASSERT_EQ(txn_1->get("keyNotExit", &get_val), TxnErrorCode::TXN_OK); + ASSERT_EQ(get_val, "1"); + + ASSERT_EQ(txn_kv->create_txn(&txn_2), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn_2->get("keyNotExit", &get_val), TxnErrorCode::TXN_KEY_NOT_FOUND); + txn_2->put("keyNotExit", "1"); + ASSERT_EQ(txn_2->get("keyNotExit", &get_val), TxnErrorCode::TXN_OK); + ASSERT_EQ(get_val, "1"); + + ASSERT_EQ(txn_1->commit(), TxnErrorCode::TXN_OK) << txn_kv_class; + ASSERT_NE(txn_2->commit(), TxnErrorCode::TXN_OK) << txn_kv_class; + } +} + +TEST(TxnMemKvTest, TxnBehaviorTest) { + using namespace doris::cloud; + auto mem_txn_kv = std::dynamic_pointer_cast(std::make_shared()); + ASSERT_NE(mem_txn_kv.get(), nullptr); + + txn_behavior_test(mem_txn_kv); + txn_behavior_test(fdb_txn_kv); +} + +TEST(TxnMemKvTest, MaybeUnusedFunctionTest) { + using namespace doris::cloud; + auto mem_txn_kv = std::make_shared(); + ASSERT_EQ(mem_txn_kv->init(), 0); + + { + std::unique_ptr txn; + ASSERT_EQ(mem_txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put("key", "v1"); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + ASSERT_EQ(mem_txn_kv->get_last_commited_version(), 1); + ASSERT_EQ(mem_txn_kv->get_last_read_version(), 1); + } + + { + std::unique_ptr txn; + ASSERT_EQ(mem_txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + auto t = dynamic_cast(txn.get()); + ASSERT_EQ(t->init(), 0); + ASSERT_EQ(t->abort(), TxnErrorCode::TXN_OK); + } + + { + auto new_mem_txn_kv = std::make_shared(); + std::unique_ptr txn; + ASSERT_EQ(new_mem_txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->atomic_set_ver_key("", "v2"); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + std::unique_ptr txn2; + ASSERT_EQ(new_mem_txn_kv->create_txn(&txn2), TxnErrorCode::TXN_OK); + for (auto& t : new_mem_txn_kv->mem_kv_) { + int64_t txn_id; + ASSERT_EQ(get_txn_id_from_fdb_ts(t.first, &txn_id), 0); + auto ver = txn_id >> 10; + std::cout << "version: " << ver << std::endl; + ASSERT_EQ(ver, 1); + } + } +} diff --git a/cloud/test/meta_server_test.cpp b/cloud/test/meta_server_test.cpp new file mode 100644 index 00000000000000..3cf6843be19963 --- /dev/null +++ b/cloud/test/meta_server_test.cpp @@ -0,0 +1,219 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "meta-service/meta_server.h" + +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include "common/config.h" +#include "common/logging.h" +#include "common/sync_point.h" +#include "meta-service/keys.h" +#include "meta-service/mem_txn_kv.h" +#include "meta-service/meta_service.h" +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" +#include "mock_resource_manager.h" +#include "rate-limiter/rate_limiter.h" +#include "resource-manager/resource_manager.h" + +using namespace doris; + +int main(int argc, char** argv) { + const std::string conf_file = "cloud.conf"; + if (!cloud::config::init(conf_file.c_str(), true)) { + std::cerr << "failed to init config file, conf=" << conf_file << std::endl; + return -1; + } + + if (!cloud::init_glog("meta_server_test")) { + std::cerr << "failed to init glog" << std::endl; + return -1; + } + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + +namespace doris::cloud { +void notify_refresh_instance(std::shared_ptr txn_kv, const std::string& instance_id); +} // namespace doris::cloud + +TEST(MetaServerTest, FQDNRefreshInstance) { + class MockMetaService : public cloud::MetaServiceImpl { + public: + MockMetaService(std::shared_ptr txn_kv, + std::shared_ptr resource_mgr, + std::shared_ptr rate_controller) + : MetaServiceImpl(txn_kv, resource_mgr, rate_controller) {} + ~MockMetaService() override = default; + + void alter_instance(google::protobuf::RpcController* controller, + const ::doris::cloud::AlterInstanceRequest* request, + ::doris::cloud::AlterInstanceResponse* response, + ::google::protobuf::Closure* done) override { + (void)controller; + response->mutable_status()->set_code(cloud::MetaServiceCode::OK); + LOG(INFO) << "alter instance " << request->DebugString(); + if (request->op() == cloud::AlterInstanceRequest::REFRESH) { + std::unique_lock lock(mu_); + LOG(INFO) << "refresh instance " << request->instance_id(); + refreshed_instances_.insert(request->instance_id()); + } + done->Run(); + } + + bool is_instance_refreshed(std::string instance_id) { + std::unique_lock lock(mu_); + return refreshed_instances_.count(instance_id) > 0; + } + + std::mutex mu_; + std::unordered_set refreshed_instances_; + }; + + std::shared_ptr txn_kv = std::make_shared(); + auto resource_mgr = std::make_shared(txn_kv); + auto rate_limiter = std::make_shared(); + auto mock_service = std::make_unique(txn_kv, resource_mgr, rate_limiter); + MockMetaService* mock_service_ptr = mock_service.get(); + MetaServiceProxy meta_service(std::move(mock_service)); + + brpc::ServerOptions options; + options.num_threads = 1; + brpc::Server server; + ASSERT_EQ(server.AddService(&meta_service, brpc::ServiceOwnership::SERVER_DOESNT_OWN_SERVICE), + 0); + ASSERT_EQ(server.Start(0, &options), 0); + auto addr = server.listen_address(); + + config::hostname = butil::my_hostname(); + config::brpc_listen_port = addr.port; + config::meta_server_register_interval_ms = 1; + + // Register meta service. + cloud::MetaServerRegister server_register(txn_kv); + ASSERT_EQ(server_register.start(), 0); + + while (true) { + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + auto system_key = cloud::system_meta_service_registry_key(); + std::string value; + if (txn->get(system_key, &value) == TxnErrorCode::TXN_OK) { + break; + } + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + } + + server_register.stop(); + + // Refresh instance with FQDN endpoint. + config::hostname = ""; + notify_refresh_instance(txn_kv, "fqdn_instance_id"); + + bool refreshed = false; + for (size_t i = 0; i < 100; ++i) { + if (mock_service_ptr->is_instance_refreshed("fqdn_instance_id")) { + refreshed = true; + break; + } + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + } + + EXPECT_TRUE(refreshed); + + server.Stop(1); + server.Join(); +} + +TEST(MetaServerTest, StartAndStop) { + std::shared_ptr txn_kv = std::make_shared(); + auto server = std::make_unique(txn_kv); + auto resource_mgr = std::make_shared(txn_kv); + auto rate_limiter = std::make_shared(); + + brpc::ServerOptions options; + options.num_threads = 1; + brpc::Server brpc_server; + + auto sp = cloud::SyncPoint::get_instance(); + + std::array sps {"MetaServer::start:1", "MetaServer::start:2", + "MetaServer::start:3"}; + // use structured binding for point alias (avoid multi lines of declaration) + auto [meta_server_start_1, meta_server_start_2, meta_server_start_3] = sps; + sp->enable_processing(); + std::unique_ptr> defer((int*)0x01, [&](...) { + for (auto& i : sps) { + sp->clear_call_back(i); + } // redundant + sp->disable_processing(); + }); + + auto foo = [](void* ret) { *((int*)ret) = 1; }; + + // failed to init resource mgr + sp->set_call_back(meta_server_start_1, foo); + ASSERT_EQ(server->start(&brpc_server), 1); + sp->clear_call_back(meta_server_start_1); + + // failed to start registry + sp->set_call_back(meta_server_start_2, foo); + ASSERT_EQ(server->start(&brpc_server), -1); + sp->clear_call_back(meta_server_start_2); + + // failed to start fdb metrics exporter + sp->set_call_back(meta_server_start_3, foo); + ASSERT_EQ(server->start(&brpc_server), -2); + sp->clear_call_back(meta_server_start_3); + + ASSERT_EQ(server->start(&brpc_server), 0); + ASSERT_EQ(brpc_server.Start(0, &options), 0); + auto addr = brpc_server.listen_address(); + + config::hostname = butil::my_hostname(); + config::brpc_listen_port = addr.port; + config::meta_server_register_interval_ms = 1; + + while (true) { + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + auto system_key = cloud::system_meta_service_registry_key(); + std::string value; + if (txn->get(system_key, &value) == TxnErrorCode::TXN_OK) { + break; + } + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + } + + server->stop(); + brpc_server.Stop(1); + brpc_server.Join(); +} diff --git a/cloud/test/meta_service_http_test.cpp b/cloud/test/meta_service_http_test.cpp new file mode 100644 index 00000000000000..9f5f2274c7dd2e --- /dev/null +++ b/cloud/test/meta_service_http_test.cpp @@ -0,0 +1,1347 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include "common/config.h" +#include "common/logging.h" +#include "common/sync_point.h" +#include "common/util.h" +#include "meta-service/keys.h" +#include "meta-service/mem_txn_kv.h" +#include "meta-service/meta_service.h" +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" +#include "mock_resource_manager.h" +#include "resource-manager/resource_manager.h" + +namespace doris::cloud { + +extern std::unique_ptr get_meta_service(bool mock_resource_mgr); + +template +using MetaServiceMethod = void (MetaService::*)(google::protobuf::RpcController*, const Request*, + Response*, google::protobuf::Closure*); + +template +struct JsonTemplate { + MetaServiceResponseStatus status; + std::optional result; + + static JsonTemplate parse(const std::string& json) { + static_assert(std::is_base_of_v<::google::protobuf::Message, Result>); + + MetaServiceResponseStatus status; + google::protobuf::util::JsonParseOptions options; + options.ignore_unknown_fields = true; + auto ss = google::protobuf::util::JsonStringToMessage(json, &status, options); + EXPECT_TRUE(ss.ok()) << "JSON Parse result: " << ss.ToString() << ", body: " << json; + + rapidjson::Document d; + rapidjson::ParseResult ps = d.Parse(json.c_str()); + EXPECT_TRUE(ps) << __PRETTY_FUNCTION__ + << " parse failed: " << rapidjson::GetParseError_En(ps.Code()) + << ", body: " << json; + + if (!ps.IsError() && d.HasMember("result")) { + rapidjson::StringBuffer sb; + rapidjson::PrettyWriter writer(sb); + d["result"].Accept(writer); + std::string content = sb.GetString(); + Result result; + auto s = google::protobuf::util::JsonStringToMessage(content, &result); + EXPECT_TRUE(s.ok()) << "JSON Parse result: " << s.ToString() + << ", content: " << content; + return {std::move(status), std::move(result)}; + } + return {std::move(status), {}}; + } +}; + +class HttpContext { +public: + HttpContext(bool mock_resource_mgr = false) + : meta_service_(get_meta_service(mock_resource_mgr)) { + auto sp = SyncPoint::get_instance(); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_id", + [](void* p) { *reinterpret_cast(p) = 1; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->enable_processing(); + + brpc::ServerOptions options; + server.AddService(meta_service_.get(), brpc::ServiceOwnership::SERVER_DOESNT_OWN_SERVICE); + if (server.Start("0.0.0.0:0", &options) == -1) { + perror("Start brpc server"); + } + } + + ~HttpContext() { + server.Stop(0); + server.Join(); + + auto sp = SyncPoint::get_instance(); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); + } + + template + std::tuple query(std::string_view resource, std::string_view params, + std::optional body = {}) { + butil::EndPoint endpoint = server.listen_address(); + + brpc::Channel channel; + brpc::ChannelOptions options; + options.protocol = brpc::PROTOCOL_HTTP; + EXPECT_EQ(channel.Init(endpoint, &options), 0) << "Fail to initialize channel"; + + brpc::Controller ctrl; + if (params.find("token=") != std::string_view::npos) { + ctrl.http_request().uri() = fmt::format("0.0.0.0:{}/MetaService/http/{}?{}", + endpoint.port, resource, params); + } else { + ctrl.http_request().uri() = + fmt::format("0.0.0.0:{}/MetaService/http/{}?token={}&{}", endpoint.port, + resource, config::http_token, params); + } + if (body.has_value()) { + ctrl.http_request().set_method(brpc::HTTP_METHOD_POST); + ctrl.request_attachment().append(body->data(), body->size()); + } + channel.CallMethod(nullptr, &ctrl, nullptr, nullptr, nullptr); + int status_code = ctrl.http_response().status_code(); + + std::string response_body = ctrl.response_attachment().to_string(); + if constexpr (std::is_base_of_v<::google::protobuf::Message, Response>) { + Response resp; + auto s = google::protobuf::util::JsonStringToMessage(response_body, &resp); + static_assert(std::is_base_of_v<::google::protobuf::Message, Response>); + EXPECT_TRUE(s.ok()) << __PRETTY_FUNCTION__ << " Parse JSON: " << s.ToString(); + return {status_code, std::move(resp)}; + } else if constexpr (std::is_same_v) { + return {status_code, std::move(response_body)}; + } else { + return {status_code, {}}; + } + } + + template + std::tuple> query_with_result(std::string_view resource, + std::string_view param) { + auto [status_code, body] = query(resource, param); + LOG_INFO(__PRETTY_FUNCTION__).tag("body", body); + return {status_code, JsonTemplate::parse(body)}; + } + + template + std::tuple forward(std::string_view query, const Request& req) { + static_assert(std::is_base_of_v<::google::protobuf::Message, Request>); + + butil::EndPoint endpoint = server.listen_address(); + + brpc::Channel channel; + brpc::ChannelOptions options; + options.protocol = brpc::PROTOCOL_HTTP; + EXPECT_EQ(channel.Init(endpoint, &options), 0) << "Fail to initialize channel"; + + brpc::Controller ctrl; + ctrl.http_request().set_method(brpc::HTTP_METHOD_POST); + ctrl.http_request().uri() = fmt::format( + "0.0.0.0:{}/MetaService/http/{}{}token={}", endpoint.port, query, + (query.find('?') != std::string_view::npos) ? "&" : "?", config::http_token); + ctrl.request_attachment().append(proto_to_json(req)); + LOG_INFO("request attachment").tag("msg", ctrl.request_attachment().to_string()); + channel.CallMethod(nullptr, &ctrl, nullptr, nullptr, nullptr); + int status_code = ctrl.http_response().status_code(); + + std::string response_body = ctrl.response_attachment().to_string(); + if constexpr (std::is_base_of_v<::google::protobuf::Message, Response>) { + Response resp; + auto s = google::protobuf::util::JsonStringToMessage(response_body, &resp); + EXPECT_TRUE(s.ok()) << __PRETTY_FUNCTION__ << " Parse JSON: " << s.ToString() + << ", body: " << response_body << ", query: " << query; + return {status_code, std::move(resp)}; + } else if (std::is_same_v) { + return {status_code, std::move(response_body)}; + } else { + return {status_code, {}}; + } + } + template + std::tuple> forward_with_result(std::string_view query, + const Request& req) { + auto [status_code, body] = forward(query, req); + LOG_INFO(__PRETTY_FUNCTION__).tag("body", body); + return {status_code, JsonTemplate::parse(body)}; + } + + InstanceInfoPB get_instance_info(std::string_view instance_id) { + InstanceKeyInfo key_info {instance_id}; + std::string key; + std::string val; + instance_key(key_info, &key); + std::unique_ptr txn; + EXPECT_EQ(meta_service_->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + EXPECT_EQ(txn->get(key, &val), TxnErrorCode::TXN_OK); + InstanceInfoPB instance; + instance.ParseFromString(val); + return instance; + } + +private: + std::unique_ptr meta_service_; + brpc::Server server; +}; + +static std::string next_rowset_id() { + static int cnt = 0; + return std::to_string(++cnt); +} + +static void add_tablet(CreateTabletsRequest& req, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id) { + auto tablet = req.add_tablet_metas(); + tablet->set_table_id(table_id); + tablet->set_index_id(index_id); + tablet->set_partition_id(partition_id); + tablet->set_tablet_id(tablet_id); + auto schema = tablet->mutable_schema(); + schema->set_schema_version(0); + auto first_rowset = tablet->add_rs_metas(); + first_rowset->set_rowset_id(0); // required + first_rowset->set_rowset_id_v2(next_rowset_id()); + first_rowset->set_start_version(0); + first_rowset->set_end_version(1); + first_rowset->mutable_tablet_schema()->CopyFrom(*schema); +} + +static void create_tablet(MetaService* meta_service, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id) { + brpc::Controller cntl; + CreateTabletsRequest req; + CreateTabletsResponse res; + add_tablet(req, table_id, index_id, partition_id, tablet_id); + meta_service->create_tablets(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << tablet_id; +} + +static void get_tablet_stats(MetaService* meta_service, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id, GetTabletStatsResponse& res) { + brpc::Controller cntl; + GetTabletStatsRequest req; + auto idx = req.add_tablet_idx(); + idx->set_table_id(table_id); + idx->set_index_id(index_id); + idx->set_partition_id(partition_id); + idx->set_tablet_id(tablet_id); + meta_service->get_tablet_stats(&cntl, &req, &res, nullptr); +} + +static void begin_txn(MetaService* meta_service, int64_t db_id, const std::string& label, + int64_t table_id, int64_t& txn_id) { + brpc::Controller cntl; + BeginTxnRequest req; + BeginTxnResponse res; + auto txn_info = req.mutable_txn_info(); + txn_info->set_db_id(db_id); + txn_info->set_label(label); + txn_info->add_table_ids(table_id); + txn_info->set_timeout_ms(36000); + meta_service->begin_txn(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + ASSERT_TRUE(res.has_txn_id()) << label; + txn_id = res.txn_id(); +} + +static void commit_txn(MetaService* meta_service, int64_t db_id, int64_t txn_id, + const std::string& label) { + brpc::Controller cntl; + CommitTxnRequest req; + CommitTxnResponse res; + req.set_db_id(db_id); + req.set_txn_id(txn_id); + meta_service->commit_txn(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; +} + +static doris::RowsetMetaCloudPB create_rowset(int64_t txn_id, int64_t tablet_id, + int64_t version = -1, int num_rows = 100) { + doris::RowsetMetaCloudPB rowset; + rowset.set_rowset_id(0); // required + rowset.set_rowset_id_v2(next_rowset_id()); + rowset.set_tablet_id(tablet_id); + rowset.set_txn_id(txn_id); + if (version > 0) { + rowset.set_start_version(version); + rowset.set_end_version(version); + } + rowset.set_num_segments(1); + rowset.set_num_rows(num_rows); + rowset.set_data_disk_size(num_rows * 100); + rowset.mutable_tablet_schema()->set_schema_version(0); + rowset.set_txn_expiration(::time(nullptr)); // Required by DCHECK + return rowset; +} + +static void prepare_rowset(MetaService* meta_service, const doris::RowsetMetaCloudPB& rowset, + CreateRowsetResponse& res) { + brpc::Controller cntl; + auto arena = res.GetArena(); + auto req = google::protobuf::Arena::CreateMessage(arena); + req->set_temporary(true); + req->mutable_rowset_meta()->CopyFrom(rowset); + meta_service->prepare_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; +} + +static void commit_rowset(MetaService* meta_service, const doris::RowsetMetaCloudPB& rowset, + CreateRowsetResponse& res) { + brpc::Controller cntl; + auto arena = res.GetArena(); + auto req = google::protobuf::Arena::CreateMessage(arena); + req->set_temporary(true); + req->mutable_rowset_meta()->CopyFrom(rowset); + meta_service->commit_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; +} + +static void insert_rowset(MetaService* meta_service, int64_t db_id, const std::string& label, + int64_t table_id, int64_t tablet_id) { + int64_t txn_id = 0; + ASSERT_NO_FATAL_FAILURE(begin_txn(meta_service, db_id, label, table_id, txn_id)); + CreateRowsetResponse res; + auto rowset = create_rowset(txn_id, tablet_id); + prepare_rowset(meta_service, rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + res.Clear(); + ASSERT_NO_FATAL_FAILURE(commit_rowset(meta_service, rowset, res)); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + commit_txn(meta_service, db_id, txn_id, label); +} + +/// NOTICE: Not ALL `code`, returned by http server, are supported by `MetaServiceCode`. + +TEST(MetaServiceHttpTest, InstanceTest) { + HttpContext ctx; + + // case: normal create instance + { + CreateInstanceRequest req; + req.set_instance_id("test_instance"); + req.set_user_id("test_user"); + req.set_name("test_name"); + ObjectStoreInfoPB obj; + obj.set_ak("123"); + obj.set_sk("321"); + obj.set_bucket("456"); + obj.set_prefix("654"); + obj.set_endpoint("789"); + obj.set_region("987"); + obj.set_external_endpoint("888"); + obj.set_provider(ObjectStoreInfoPB::BOS); + req.mutable_obj_info()->CopyFrom(obj); + + auto [status_code, resp] = ctx.forward("create_instance", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + } + + // case: request has invalid argument + { + CreateInstanceRequest req; + auto [status_code, resp] = ctx.forward("create_instance", req); + ASSERT_EQ(status_code, 400); + ASSERT_EQ(resp.code(), MetaServiceCode::INVALID_ARGUMENT); + } + + // case: rename instance + { + AlterInstanceRequest req; + req.set_instance_id("test_instance"); + req.set_name("new_name"); + auto [status_code, resp] = ctx.forward("rename_instance", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + + InstanceInfoPB instance = ctx.get_instance_info("test_instance"); + ASSERT_EQ(instance.name(), "new_name"); + } + + // The default instance sse is disabled, to execute enable first. + // case: enable instance sse + { + AlterInstanceRequest req; + req.set_instance_id("test_instance"); + auto [status_code, resp] = + ctx.forward("enable_instance_sse", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + + InstanceInfoPB instance = ctx.get_instance_info("test_instance"); + ASSERT_TRUE(instance.sse_enabled()); + } + + // case: disable instance sse + { + AlterInstanceRequest req; + req.set_instance_id("test_instance"); + auto [status_code, resp] = + ctx.forward("disable_instance_sse", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + + InstanceInfoPB instance = ctx.get_instance_info("test_instance"); + ASSERT_FALSE(instance.sse_enabled()); + } + + // case: get instance + { + auto [status_code, resp] = + ctx.query_with_result("get_instance", "instance_id=test_instance"); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.status.code(), MetaServiceCode::OK); + ASSERT_TRUE(resp.result.has_value()); + InstanceInfoPB instance = resp.result.value(); + ASSERT_EQ(instance.instance_id(), "test_instance"); + ASSERT_EQ(instance.status(), InstanceInfoPB::NORMAL); + } + + // case: set over_due instance + { + AlterInstanceRequest req; + req.set_op(AlterInstanceRequest::SET_OVERDUE); + req.set_instance_id("test_instance"); + auto [status_code, resp] = + ctx.forward("set_instance_status", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + InstanceInfoPB instance = ctx.get_instance_info("test_instance"); + ASSERT_EQ(instance.status(), InstanceInfoPB::OVERDUE); + } + + // case: set_normal instance + { + AlterInstanceRequest req; + req.set_op(AlterInstanceRequest::SET_NORMAL); + req.set_instance_id("test_instance"); + auto [status_code, resp] = + ctx.forward("set_instance_status", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + + InstanceInfoPB instance = ctx.get_instance_info("test_instance"); + ASSERT_EQ(instance.status(), InstanceInfoPB::NORMAL); + } + + // case: get instance by cloud_unique_id + { + auto [status_code, resp] = ctx.query_with_result( + "get_instance", "cloud_unique_id=1:test_instance:1"); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.status.code(), MetaServiceCode::OK); + ASSERT_TRUE(resp.result.has_value()); + InstanceInfoPB instance = resp.result.value(); + ASSERT_EQ(instance.instance_id(), "test_instance"); + ASSERT_EQ(instance.status(), InstanceInfoPB::NORMAL); + } + + // case: normal drop instance + { + AlterInstanceRequest req; + req.set_instance_id("test_instance"); + auto [status_code, resp] = ctx.forward("drop_instance", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + + InstanceInfoPB instance = ctx.get_instance_info("test_instance"); + ASSERT_EQ(instance.status(), InstanceInfoPB::DELETED); + } +} + +TEST(MetaServiceHttpTest, InstanceTestWithVersion) { + HttpContext ctx; + + // case: normal create instance + { + CreateInstanceRequest req; + req.set_instance_id("test_instance"); + req.set_user_id("test_user"); + req.set_name("test_name"); + ObjectStoreInfoPB obj; + obj.set_ak("123"); + obj.set_sk("321"); + obj.set_bucket("456"); + obj.set_prefix("654"); + obj.set_endpoint("789"); + obj.set_region("987"); + obj.set_external_endpoint("888"); + obj.set_provider(ObjectStoreInfoPB::BOS); + req.mutable_obj_info()->CopyFrom(obj); + + auto [status_code, resp] = + ctx.forward("v1/create_instance", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + } + + // case: request has invalid argument + { + CreateInstanceRequest req; + auto [status_code, resp] = + ctx.forward("v1/create_instance", req); + ASSERT_EQ(status_code, 400); + ASSERT_EQ(resp.code(), MetaServiceCode::INVALID_ARGUMENT); + } + + // case: rename instance + { + AlterInstanceRequest req; + req.set_instance_id("test_instance"); + req.set_name("new_name"); + auto [status_code, resp] = + ctx.forward("v1/rename_instance", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + + InstanceInfoPB instance = ctx.get_instance_info("test_instance"); + ASSERT_EQ(instance.name(), "new_name"); + } + + // The default instance sse is disabled, to execute enable first. + // case: enable instance sse + { + AlterInstanceRequest req; + req.set_instance_id("test_instance"); + auto [status_code, resp] = + ctx.forward("v1/enable_instance_sse", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + + InstanceInfoPB instance = ctx.get_instance_info("test_instance"); + ASSERT_TRUE(instance.sse_enabled()); + } + + // case: disable instance sse + { + AlterInstanceRequest req; + req.set_instance_id("test_instance"); + auto [status_code, resp] = + ctx.forward("v1/disable_instance_sse", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + + InstanceInfoPB instance = ctx.get_instance_info("test_instance"); + ASSERT_FALSE(instance.sse_enabled()); + } + + // case: get instance + { + auto [status_code, resp] = ctx.query_with_result( + "v1/get_instance", "instance_id=test_instance"); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.status.code(), MetaServiceCode::OK); + ASSERT_TRUE(resp.result.has_value()); + InstanceInfoPB instance = resp.result.value(); + ASSERT_EQ(instance.instance_id(), "test_instance"); + ASSERT_EQ(instance.status(), InstanceInfoPB::NORMAL); + } + + // case: normal drop instance + { + AlterInstanceRequest req; + req.set_instance_id("test_instance"); + auto [status_code, resp] = ctx.forward("v1/drop_instance", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + + InstanceInfoPB instance = ctx.get_instance_info("test_instance"); + ASSERT_EQ(instance.status(), InstanceInfoPB::DELETED); + } +} + +TEST(MetaServiceHttpTest, AlterClusterTest) { + HttpContext ctx; + { + CreateInstanceRequest req; + req.set_instance_id(mock_instance); + req.set_user_id("test_user"); + req.set_name("test_name"); + ObjectStoreInfoPB obj; + obj.set_ak("123"); + obj.set_sk("321"); + obj.set_bucket("456"); + obj.set_prefix("654"); + obj.set_endpoint("789"); + obj.set_region("987"); + obj.set_external_endpoint("888"); + obj.set_provider(ObjectStoreInfoPB::BOS); + req.mutable_obj_info()->CopyFrom(obj); + + auto [status_code, resp] = ctx.forward("create_instance", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + } + + // case: normal add cluster + { + AlterClusterRequest req; + req.set_instance_id(mock_instance); + req.mutable_cluster()->set_cluster_name(mock_cluster_name); + req.mutable_cluster()->set_type(ClusterPB::COMPUTE); + req.mutable_cluster()->set_cluster_id(mock_cluster_id); + auto [status_code, resp] = ctx.forward("add_cluster", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + } + + // case: request has invalid argument + { + AlterClusterRequest req; + auto [status_code, resp] = ctx.forward("drop_cluster", req); + ASSERT_EQ(status_code, 400); + ASSERT_EQ(resp.code(), MetaServiceCode::INVALID_ARGUMENT); + } + + // add node + { + AlterClusterRequest req; + req.set_instance_id(mock_instance); + req.mutable_cluster()->set_cluster_name(mock_cluster_name); + req.mutable_cluster()->set_cluster_id(mock_cluster_id); + req.mutable_cluster()->set_type(ClusterPB::COMPUTE); + auto node = req.mutable_cluster()->add_nodes(); + node->set_ip("127.0.0.1"); + node->set_heartbeat_port(9999); + auto [status_code, resp] = ctx.forward("add_node", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + } + + // drop node + { + AlterClusterRequest req; + req.set_instance_id(mock_instance); + req.mutable_cluster()->set_cluster_name(mock_cluster_name); + req.mutable_cluster()->set_cluster_id(mock_cluster_id); + req.mutable_cluster()->set_type(ClusterPB::COMPUTE); + auto node = req.mutable_cluster()->add_nodes(); + node->set_ip("127.0.0.1"); + node->set_heartbeat_port(9999); + auto [status_code, resp] = ctx.forward("drop_node", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + } + + // rename cluster + { + AlterClusterRequest req; + req.set_instance_id(mock_instance); + req.mutable_cluster()->set_cluster_id(mock_cluster_id); + req.mutable_cluster()->set_cluster_name("rename_cluster_name"); + auto [status_code, resp] = ctx.forward("rename_cluster", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + } + + // alter cluster status + { + AlterClusterRequest req; + req.set_instance_id(mock_instance); + req.mutable_cluster()->set_cluster_id(mock_cluster_id); + req.mutable_cluster()->set_cluster_status(ClusterStatus::SUSPENDED); + req.set_op(AlterClusterRequest::SET_CLUSTER_STATUS); + auto [status_code, resp] = + ctx.forward("set_cluster_status", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + } + + // update cluster mysql user name + { + AlterClusterRequest req; + req.mutable_cluster()->add_mysql_user_name("test_user"); + req.set_instance_id(mock_instance); + req.mutable_cluster()->set_cluster_id(mock_cluster_id); + auto [status_code, resp] = + ctx.forward("update_cluster_mysql_user_name", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + } + + // decommission_node + { + AlterClusterRequest req; + req.set_instance_id(mock_instance); + req.mutable_cluster()->set_cluster_name(mock_cluster_name); + req.mutable_cluster()->set_cluster_id(mock_cluster_id); + req.mutable_cluster()->set_type(ClusterPB::COMPUTE); + auto node = req.mutable_cluster()->add_nodes(); + node->set_ip("127.0.0.1"); + node->set_heartbeat_port(9999); + node->set_cloud_unique_id("cloud_unique_id"); + auto& meta_service = ctx.meta_service_; + NodeInfoPB npb; + npb.set_heartbeat_port(9999); + npb.set_ip("127.0.0.1"); + npb.set_cloud_unique_id("cloud_unique_id"); + meta_service->resource_mgr()->node_info_.insert( + {"cloud_unique_id", NodeInfo {Role::COMPUTE_NODE, mock_instance, + "rename_cluster_name", mock_cluster_id, npb}}); + auto [status_code, resp] = ctx.forward("decommission_node", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + } + + // notify_decommissioned + { + AlterClusterRequest req; + req.set_instance_id(mock_instance); + req.mutable_cluster()->set_cluster_name(mock_cluster_name); + req.mutable_cluster()->set_cluster_id(mock_cluster_id); + req.mutable_cluster()->set_type(ClusterPB::COMPUTE); + auto node = req.mutable_cluster()->add_nodes(); + node->set_ip("127.0.0.1"); + node->set_heartbeat_port(9999); + node->set_cloud_unique_id("cloud_unique_id"); + auto& meta_service = ctx.meta_service_; + NodeInfoPB npb; + npb.set_heartbeat_port(9999); + npb.set_ip("127.0.0.1"); + npb.set_cloud_unique_id("cloud_unique_id"); + meta_service->resource_mgr()->node_info_.insert( + {"cloud_unique_id", NodeInfo {Role::COMPUTE_NODE, mock_instance, + "rename_cluster_name", mock_cluster_id, npb}}); + auto [status_code, resp] = + ctx.forward("notify_decommissioned", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + } + + // update_cluster_endpoint + { + AlterClusterRequest req; + req.mutable_cluster()->add_mysql_user_name("test_user"); + req.set_instance_id(mock_instance); + req.mutable_cluster()->set_cluster_id(mock_cluster_id); + req.mutable_cluster()->set_public_endpoint("127.0.0.2"); + req.mutable_cluster()->set_private_endpoint("127.0.0.3"); + auto [status_code, resp] = + ctx.forward("update_cluster_endpoint", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + } +} + +TEST(MetaServiceHttpTest, GetClusterTest) { + HttpContext ctx(true); + + // add cluster first + InstanceKeyInfo key_info {mock_instance}; + std::string key; + std::string val; + instance_key(key_info, &key); + + InstanceInfoPB instance; + instance.set_instance_id(mock_instance); + ClusterPB c1; + c1.set_cluster_name(mock_cluster_name); + c1.set_cluster_id(mock_cluster_id); + c1.add_mysql_user_name()->append("m1"); + instance.add_clusters()->CopyFrom(c1); + ClusterPB c2; + c2.set_cluster_name(mock_cluster_name + "2"); + c2.set_cluster_id(mock_cluster_id + "2"); + c2.add_mysql_user_name()->append("m2"); + instance.add_clusters()->CopyFrom(c2); + val = instance.SerializeAsString(); + + std::unique_ptr txn; + std::string get_val; + ASSERT_EQ(ctx.meta_service_->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + // case: normal get + { + GetClusterRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_cluster_id(mock_cluster_id); + req.set_cluster_name(mock_cluster_name); + auto [status_code, resp] = ctx.forward_with_result("get_cluster", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.status.code(), MetaServiceCode::OK); + ASSERT_TRUE(resp.result.has_value()); + ASSERT_EQ(resp.result->cluster_id(), mock_cluster_id); + } + + // case: not found + { + GetClusterRequest req; + req.set_cloud_unique_id("unknown_id"); + req.set_cluster_id("unknown_cluster_id"); + req.set_cluster_name("unknown_cluster_name"); + auto [status_code, resp] = ctx.forward_with_result("get_cluster", req); + ASSERT_EQ(status_code, 404); + } + + // case: get all clusters + { + GetClusterRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + auto [status_code, resp] = ctx.forward_with_result("get_cluster", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.status.code(), MetaServiceCode::OK); + ASSERT_TRUE(resp.result.has_value()); + ASSERT_EQ(resp.result->cluster_size(), 2); + } +} + +TEST(MetaServiceHttpTest, AbortTxnTest) { + HttpContext ctx(true); + + // case: abort txn by txn_id + { + int64_t db_id = 666; + int64_t table_id = 12345; + std::string label = "abort_txn_by_txn_id"; + std::string cloud_unique_id = "test_cloud_unique_id"; + int64_t txn_id = -1; + // begin txn + { + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id(cloud_unique_id); + TxnInfoPB txn_info_pb; + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label(label); + txn_info_pb.add_table_ids(table_id); + txn_info_pb.set_timeout_ms(36000); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + ctx.meta_service_->begin_txn( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, + nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + } + + // abort txn by txn_id + { + AbortTxnRequest req; + req.set_cloud_unique_id(cloud_unique_id); + req.set_txn_id(txn_id); + req.set_reason("test"); + auto [status_code, resp] = ctx.forward("abort_txn", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + } + } +} + +TEST(MetaServiceHttpTest, AlterIamTest) { + HttpContext ctx; + + brpc::Controller cntl; + auto cloud_unique_id = "test_cloud_unique_id"; + std::string instance_id = "alter_iam_test_instance_id"; + [[maybe_unused]] auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_id", + [](void* p) { *reinterpret_cast(p) = 1; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->enable_processing(); + + config::arn_id = "iam_arn"; + config::arn_ak = "iam_ak"; + config::arn_sk = "iam_sk"; + + ObjectStoreInfoPB obj; + obj.set_ak("123"); + obj.set_sk("321"); + obj.set_bucket("456"); + obj.set_prefix("654"); + obj.set_endpoint("789"); + obj.set_region("987"); + obj.set_external_endpoint("888"); + obj.set_provider(ObjectStoreInfoPB::BOS); + + // create instance without ram user + CreateInstanceRequest create_instance_req; + create_instance_req.set_instance_id(instance_id); + create_instance_req.set_user_id("test_user"); + create_instance_req.set_name("test_name"); + create_instance_req.mutable_obj_info()->CopyFrom(obj); + CreateInstanceResponse create_instance_res; + ctx.meta_service_->create_instance(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &create_instance_req, &create_instance_res, nullptr); + ASSERT_EQ(create_instance_res.status().code(), MetaServiceCode::OK); + + // get iam and ram user + GetIamRequest request; + request.set_cloud_unique_id(cloud_unique_id); + GetIamResponse response; + ctx.meta_service_->get_iam(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &request, &response, nullptr); + ASSERT_EQ(response.status().code(), MetaServiceCode::OK); + ASSERT_EQ(response.has_ram_user(), false); + ASSERT_EQ(response.iam_user().user_id(), "iam_arn"); + ASSERT_EQ(response.iam_user().ak(), "iam_ak"); + ASSERT_EQ(response.iam_user().sk(), "iam_sk"); + + // alter ram user + RamUserPB ram_user; + ram_user.set_user_id("test_user_id"); + ram_user.set_ak("test_ak"); + ram_user.set_sk("test_sk"); + AlterRamUserRequest alter_ram_user_request; + alter_ram_user_request.set_instance_id(instance_id); + alter_ram_user_request.mutable_ram_user()->CopyFrom(ram_user); + auto [status_code, resp] = + ctx.forward("alter_ram_user", alter_ram_user_request); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + + // alter iam + { + AlterIamRequest alter_iam_request; + alter_iam_request.set_ak("new_ak"); + alter_iam_request.set_sk("new_sk"); + alter_iam_request.set_account_id("account_id"); + auto [status_code, resp] = + ctx.forward("alter_iam", alter_iam_request); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + } + + // get iam and ram user + ctx.meta_service_->get_iam(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &request, &response, nullptr); + ASSERT_EQ(response.status().code(), MetaServiceCode::OK); + ASSERT_EQ(response.has_ram_user(), true); + ASSERT_EQ(response.ram_user().user_id(), "test_user_id"); + ASSERT_EQ(response.ram_user().ak(), "test_ak"); + ASSERT_EQ(response.ram_user().sk(), "test_sk"); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); +} + +TEST(MetaServiceHttpTest, AlterObjStoreInfoTest) { + HttpContext ctx(true); + + { + // Prepare instance info. + CreateInstanceRequest req; + req.set_instance_id(mock_instance); + req.set_user_id("test_user"); + req.set_name("test_name"); + ObjectStoreInfoPB obj; + obj.set_ak("123"); + obj.set_sk("321"); + obj.set_bucket("456"); + obj.set_prefix("654"); + obj.set_endpoint("789"); + obj.set_region("987"); + obj.set_external_endpoint("888"); + obj.set_provider(ObjectStoreInfoPB::BOS); + req.mutable_obj_info()->CopyFrom(obj); + + auto [status_code, resp] = ctx.forward("create_instance", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + } + + // case: add new obj store info. + { + AlterObjStoreInfoRequest req; + req.set_cloud_unique_id("cloud_unique_id"); + auto* obj = req.mutable_obj(); + obj->set_ak("123_1"); + obj->set_sk("321_2"); + obj->set_bucket("456_3"); + obj->set_prefix("654_4"); + obj->set_endpoint("789_5"); + obj->set_region("987_5"); + obj->set_external_endpoint("888_"); + obj->set_provider(ObjectStoreInfoPB::BOS); + auto [status_code, resp] = ctx.forward("add_obj_info", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + + InstanceInfoPB instance = ctx.get_instance_info(mock_instance); + ASSERT_EQ(instance.obj_info().size(), 2); + } +} + +TEST(MetaServiceHttpTest, GetObjStoreInfoTest) { + HttpContext ctx(true); + + { + // Prepare instance info. + CreateInstanceRequest req; + req.set_instance_id(mock_instance); + req.set_user_id("test_user"); + req.set_name("test_name"); + ObjectStoreInfoPB obj; + obj.set_ak("123"); + obj.set_sk("321"); + obj.set_bucket("456"); + obj.set_prefix("654"); + obj.set_endpoint("789"); + obj.set_region("987"); + obj.set_external_endpoint("888"); + obj.set_provider(ObjectStoreInfoPB::BOS); + req.mutable_obj_info()->CopyFrom(obj); + + auto [status_code, resp] = ctx.forward("create_instance", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + } + + GetObjStoreInfoRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + auto [status_code, resp] = + ctx.forward_with_result("get_obj_store_info", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.status.code(), MetaServiceCode::OK); + ASSERT_TRUE(resp.result.has_value()); + ASSERT_EQ(resp.result->obj_info_size(), 1); + ObjectStoreInfoPB info = resp.result.value().obj_info().at(0); + ASSERT_EQ(info.ak(), "123"); + ASSERT_EQ(info.sk(), "321"); +} + +TEST(MetaServiceHttpTest, UpdateAkSkTest) { + HttpContext ctx(true); + + // Prepare instance info. + { + CreateInstanceRequest req; + req.set_instance_id(mock_instance); + req.set_user_id("test_user"); + req.set_name("test_name"); + ObjectStoreInfoPB obj; + obj.set_ak("123"); + obj.set_sk("321"); + obj.set_bucket("456"); + obj.set_prefix("654"); + obj.set_endpoint("789"); + obj.set_region("987"); + obj.set_external_endpoint("888"); + obj.set_provider(ObjectStoreInfoPB::BOS); + req.mutable_obj_info()->CopyFrom(obj); + auto* user = req.mutable_ram_user(); + user->set_user_id("user_id"); + user->set_ak("old_ak"); + user->set_sk("old_sk"); + + auto [status_code, resp] = ctx.forward("create_instance", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + } + + // Case update user ak,sk + { + UpdateAkSkRequest req; + req.set_instance_id(mock_instance); + auto* user = req.mutable_ram_user(); + user->set_ak("ak"); + user->set_user_id("user_id"); + user->set_sk("sk"); + + auto [status_code, resp] = ctx.forward("update_ak_sk", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + } +} + +TEST(MetaServiceHttpTest, GetStageTest) { + HttpContext ctx(true); + + // Prepare instance info. + { + CreateInstanceRequest req; + req.set_instance_id(mock_instance); + req.set_user_id("test_user"); + req.set_name("test_name"); + ObjectStoreInfoPB obj; + obj.set_ak("123"); + obj.set_sk("321"); + obj.set_bucket("456"); + obj.set_prefix("654"); + obj.set_endpoint("789"); + obj.set_region("987"); + obj.set_external_endpoint("888"); + obj.set_provider(ObjectStoreInfoPB::BOS); + req.mutable_obj_info()->CopyFrom(obj); + auto [status_code, resp] = ctx.forward("create_instance", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + } + + // Create a stage + { + CreateStageRequest req; + req.set_cloud_unique_id("test"); + auto* stage = req.mutable_stage(); + stage->set_stage_id("stage_id"); + stage->set_arn("arn"); + stage->set_comment("comment"); + stage->set_name("stage_name"); + stage->add_mysql_user_name("mysql_user_name"); + stage->add_mysql_user_id("mysql_user_id"); + stage->set_type(StagePB::INTERNAL); + + brpc::Controller ctrl; + CreateStageResponse resp; + ctx.meta_service_->create_stage(&ctrl, &req, &resp, nullptr); + ASSERT_EQ(resp.status().code(), MetaServiceCode::OK); + } + + // Get stage + { + GetStageRequest req; + req.set_stage_name("stage_name"); + req.set_type(StagePB::INTERNAL); + req.set_cloud_unique_id("test"); + req.set_mysql_user_id("mysql_user_id"); + req.set_mysql_user_name("mysql_user_name"); + auto [status_code, resp] = ctx.forward_with_result("get_stage", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.status.code(), MetaServiceCode::OK); + ASSERT_TRUE(resp.result.has_value()); + ASSERT_EQ(resp.result->stage_size(), 1); + auto& stage = resp.result->stage(0); + ASSERT_EQ(stage.stage_id(), "stage_id"); + } +} + +TEST(MetaServiceHttpTest, GetTabletStatsTest) { + HttpContext ctx(true); + auto& meta_service = ctx.meta_service_; + + constexpr auto table_id = 10001, index_id = 10002, partition_id = 10003, tablet_id = 10004; + ASSERT_NO_FATAL_FAILURE( + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id)); + GetTabletStatsResponse res; + get_tablet_stats(meta_service.get(), table_id, index_id, partition_id, tablet_id, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.tablet_stats_size(), 1); + EXPECT_EQ(res.tablet_stats(0).data_size(), 0); + EXPECT_EQ(res.tablet_stats(0).num_rows(), 0); + EXPECT_EQ(res.tablet_stats(0).num_rowsets(), 1); + EXPECT_EQ(res.tablet_stats(0).num_segments(), 0); + { + GetTabletStatsRequest req; + auto idx = req.add_tablet_idx(); + idx->set_table_id(table_id); + idx->set_index_id(index_id); + idx->set_partition_id(partition_id); + idx->set_tablet_id(tablet_id); + auto [status_code, content] = ctx.forward("get_tablet_stats", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(content, res.DebugString() + "\n"); + } + + // Insert rowset + config::split_tablet_stats = false; + ASSERT_NO_FATAL_FAILURE( + insert_rowset(meta_service.get(), 10000, "label1", table_id, tablet_id)); + ASSERT_NO_FATAL_FAILURE( + insert_rowset(meta_service.get(), 10000, "label2", table_id, tablet_id)); + config::split_tablet_stats = true; + ASSERT_NO_FATAL_FAILURE( + insert_rowset(meta_service.get(), 10000, "label3", table_id, tablet_id)); + ASSERT_NO_FATAL_FAILURE( + insert_rowset(meta_service.get(), 10000, "label4", table_id, tablet_id)); + // Check tablet stats kv + std::unique_ptr txn; + ASSERT_EQ(ctx.meta_service_->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string data_size_key, data_size_val; + stats_tablet_data_size_key({mock_instance, table_id, index_id, partition_id, tablet_id}, + &data_size_key); + ASSERT_EQ(txn->get(data_size_key, &data_size_val), TxnErrorCode::TXN_OK); + EXPECT_EQ(*(int64_t*)data_size_val.data(), 20000); + std::string num_rows_key, num_rows_val; + stats_tablet_num_rows_key({mock_instance, table_id, index_id, partition_id, tablet_id}, + &num_rows_key); + ASSERT_EQ(txn->get(num_rows_key, &num_rows_val), TxnErrorCode::TXN_OK); + EXPECT_EQ(*(int64_t*)num_rows_val.data(), 200); + std::string num_rowsets_key, num_rowsets_val; + stats_tablet_num_rowsets_key({mock_instance, table_id, index_id, partition_id, tablet_id}, + &num_rowsets_key); + ASSERT_EQ(txn->get(num_rowsets_key, &num_rowsets_val), TxnErrorCode::TXN_OK); + EXPECT_EQ(*(int64_t*)num_rowsets_val.data(), 2); + std::string num_segs_key, num_segs_val; + stats_tablet_num_segs_key({mock_instance, table_id, index_id, partition_id, tablet_id}, + &num_segs_key); + ASSERT_EQ(txn->get(num_segs_key, &num_segs_val), TxnErrorCode::TXN_OK); + EXPECT_EQ(*(int64_t*)num_segs_val.data(), 2); + // Get tablet stats + res.Clear(); + get_tablet_stats(meta_service.get(), table_id, index_id, partition_id, tablet_id, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.tablet_stats_size(), 1); + EXPECT_EQ(res.tablet_stats(0).data_size(), 40000); + EXPECT_EQ(res.tablet_stats(0).num_rows(), 400); + EXPECT_EQ(res.tablet_stats(0).num_rowsets(), 5); + EXPECT_EQ(res.tablet_stats(0).num_segments(), 4); + { + GetTabletStatsRequest req; + auto idx = req.add_tablet_idx(); + idx->set_table_id(table_id); + idx->set_index_id(index_id); + idx->set_partition_id(partition_id); + idx->set_tablet_id(tablet_id); + auto [status_code, content] = ctx.forward("get_tablet_stats", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(content, res.DebugString() + "\n"); + } +} + +TEST(MetaServiceHttpTest, ToUnknownUrlTest) { + HttpContext ctx; + auto [status_code, content] = ctx.query("unkown_resource_xxxxxx", ""); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(content, "{\n \"code\": \"OK\",\n \"msg\": \"\"\n}\n"); +} + +TEST(MetaServiceHttpTest, UnknownFields) { + // LOG: + // parse http request 'get_tablet_stats': INVALID_ARGUMENT:an_unknown_field: Cannot find field. body="{"table_id": 1, "an_unknown_field": "xxxx"}" + HttpContext ctx; + std::string body = + "{\"table_id\": 1, \"an_unknown_field\": \"xxxx\", \"cloud_unique_id\": " + "\"1:test_instance:1\"}"; + auto [status_code, content] = ctx.query("get_tablet_stats", "", body); + ASSERT_EQ(status_code, 200); +} + +TEST(MetaServiceHttpTest, EncodeAndDecodeKey) { + HttpContext ctx; + { + auto [status_code, content] = + ctx.query("encode_key", "key_type=InstanceKey&instance_id=test", ""); + ASSERT_EQ(status_code, 200); + const char* encode_key_output = R"( +┌───────────────────────── 0. key space: 1 +│ ┌─────────────────────── 1. instance +│ │ ┌─ 2. test +│ │ │ +▼ ▼ ▼ +0110696e7374616e6365000110746573740001 +\x01\x10\x69\x6e\x73\x74\x61\x6e\x63\x65\x00\x01\x10\x74\x65\x73\x74\x00\x01 + +)"; + content.insert(0, 1, '\n'); + ASSERT_EQ(content, encode_key_output); + } + + { + auto [status_code, content] = ctx.query( + "decode_key", "key=0110696e7374616e6365000110746573740001", ""); + ASSERT_EQ(status_code, 200); + const char* decode_key_output = R"( +┌───────────────────────── 0. key space: 1 +│ ┌─────────────────────── 1. instance +│ │ ┌─ 2. test +│ │ │ +▼ ▼ ▼ +0110696e7374616e6365000110746573740001 + +)"; + content.insert(0, 1, '\n'); + ASSERT_EQ(content, decode_key_output); + } +} + +TEST(MetaServiceHttpTest, GetValue) { + HttpContext ctx(true); + + // Prepare instance info. + { + CreateInstanceRequest req; + req.set_instance_id("get_value_instance_id"); + req.set_user_id("test_user"); + req.set_name("test_name"); + ObjectStoreInfoPB obj; + obj.set_ak("123"); + obj.set_sk("321"); + obj.set_bucket("456"); + obj.set_prefix("654"); + obj.set_endpoint("789"); + obj.set_region("987"); + obj.set_external_endpoint("888"); + obj.set_provider(ObjectStoreInfoPB::BOS); + req.mutable_obj_info()->CopyFrom(obj); + auto [status_code, resp] = ctx.forward("create_instance", req); + ASSERT_EQ(status_code, 200); + ASSERT_EQ(resp.code(), MetaServiceCode::OK); + } + + auto param = "key_type=InstanceKey&instance_id=get_value_instance_id"; + auto [status_code, content] = ctx.query("get_value", param, ""); + ASSERT_EQ(status_code, 200); + auto instance_info = ctx.get_instance_info("get_value_instance_id"); + auto get_value_output = proto_to_json(instance_info); + get_value_output.push_back('\n'); + ASSERT_EQ(content, get_value_output); +} + +TEST(MetaServiceHttpTest, InvalidToken) { + HttpContext ctx(true); + auto [status_code, content] = ctx.query("get_value", "token=invalid_token", ""); + ASSERT_EQ(status_code, 403); + const char* invalid_token_output = "incorrect token, token=invalid_token\n"; + ASSERT_EQ(content, invalid_token_output); +} + +} // namespace doris::cloud diff --git a/cloud/test/meta_service_job_test.cpp b/cloud/test/meta_service_job_test.cpp new file mode 100644 index 00000000000000..2d8f973eaff47f --- /dev/null +++ b/cloud/test/meta_service_job_test.cpp @@ -0,0 +1,2191 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +#include "common/sync_point.h" +#include "common/util.h" +#include "meta-service/keys.h" +#include "meta-service/meta_service.h" +#include "meta-service/txn_kv_error.h" + +namespace doris::cloud { +extern std::unique_ptr get_meta_service(); + +namespace { +std::string instance_id = "MetaServiceJobTest"; + +void start_compaction_job(MetaService* meta_service, int64_t tablet_id, const std::string& job_id, + const std::string& initiator, int base_compaction_cnt, + int cumu_compaction_cnt, TabletCompactionJobPB::CompactionType type, + StartTabletJobResponse& res, + std::pair input_version = {0, 0}) { + brpc::Controller cntl; + StartTabletJobRequest req; + req.mutable_job()->mutable_idx()->set_tablet_id(tablet_id); + auto compaction = req.mutable_job()->add_compaction(); + compaction->set_id(job_id); + compaction->set_initiator(initiator); + compaction->set_base_compaction_cnt(base_compaction_cnt); + compaction->set_cumulative_compaction_cnt(cumu_compaction_cnt); + compaction->set_type(type); + long now = time(nullptr); + compaction->set_expiration(now + 12); + compaction->set_lease(now + 3); + if (input_version.first > 0 && input_version.second > 0) { + compaction->add_input_versions(input_version.first); + compaction->add_input_versions(input_version.second); + } + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); +}; + +void get_tablet_stats(MetaService* meta_service, int64_t tablet_id, TabletStatsPB& stats) { + brpc::Controller cntl; + GetTabletStatsRequest req; + GetTabletStatsResponse res; + auto idx = req.add_tablet_idx(); + idx->set_tablet_id(tablet_id); + meta_service->get_tablet_stats(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << tablet_id; + stats = res.tablet_stats(0); +} + +std::string next_rowset_id() { + static int cnt = 0; + return fmt::format("{:04}", ++cnt); +} + +doris::RowsetMetaCloudPB create_rowset(int64_t tablet_id, int64_t start_version, + int64_t end_version, int num_rows = 100) { + doris::RowsetMetaCloudPB rowset; + rowset.set_rowset_id(0); // required + rowset.set_rowset_id_v2(next_rowset_id()); + rowset.set_tablet_id(tablet_id); + rowset.set_txn_id(start_version << 32 | end_version); + rowset.set_start_version(start_version); + rowset.set_end_version(end_version); + rowset.set_num_segments(1); + rowset.set_num_rows(num_rows); + rowset.set_data_disk_size(num_rows * 100); + rowset.mutable_tablet_schema()->set_schema_version(0); + rowset.set_txn_expiration(::time(nullptr)); // Required by DCHECK + return rowset; +} + +void commit_rowset(MetaService* meta_service, const doris::RowsetMetaCloudPB& rowset, + CreateRowsetResponse& res) { + brpc::Controller cntl; + CreateRowsetRequest req; + req.set_temporary(true); + req.mutable_rowset_meta()->CopyFrom(rowset); + meta_service->commit_rowset(&cntl, &req, &res, nullptr); +} + +void insert_rowsets(TxnKv* txn_kv, int64_t table_id, int64_t index_id, int64_t partition_id, + int64_t tablet_id, const std::vector& rowsets) { + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK) << tablet_id; + std::deque buffer; + int data_size = 0, num_rows = 0, num_seg = 0; + for (auto& rowset : rowsets) { + data_size += rowset.data_disk_size(); + num_rows += rowset.num_rows(); + num_seg += rowset.num_segments(); + auto& key = buffer.emplace_back(); + auto& val = buffer.emplace_back(); + meta_rowset_key({instance_id, tablet_id, rowset.end_version()}, &key); + ASSERT_TRUE(rowset.SerializeToString(&val)) << tablet_id; + txn->put(key, val); + } + StatsTabletKeyInfo info {instance_id, table_id, index_id, partition_id, tablet_id}; + std::string data_size_key; + stats_tablet_data_size_key(info, &data_size_key); + txn->atomic_add(data_size_key, data_size); + std::string num_rows_key; + stats_tablet_num_rows_key(info, &num_rows_key); + txn->atomic_add(num_rows_key, num_rows); + std::string num_rowsets_key; + stats_tablet_num_rowsets_key(info, &num_rowsets_key); + txn->atomic_add(num_rowsets_key, rowsets.size()); + std::string num_segs_key; + stats_tablet_num_segs_key(info, &num_segs_key); + txn->atomic_add(num_segs_key, num_seg); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK) << tablet_id; +} + +MetaServiceCode get_delete_bitmap_lock(MetaServiceProxy* meta_service, int64_t table_id, + int64_t lock_id, int64_t initor) { + brpc::Controller cntl; + GetDeleteBitmapUpdateLockRequest req; + GetDeleteBitmapUpdateLockResponse res; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_table_id(table_id); + req.set_expiration(5); + req.set_lock_id(lock_id); + req.set_initiator(initor); + meta_service->get_delete_bitmap_update_lock( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, nullptr); + return res.status().code(); +} + +void remove_delete_bitmap_lock(MetaServiceProxy* meta_service, int64_t table_id) { + std::string lock_key = meta_delete_bitmap_update_lock_key({instance_id, table_id, -1}); + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->remove(lock_key); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); +} + +void create_tablet(MetaService* meta_service, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id, bool enable_mow, + bool not_ready = false) { + brpc::Controller cntl; + CreateTabletsRequest req; + CreateTabletsResponse res; + auto tablet = req.add_tablet_metas(); + tablet->set_tablet_state(not_ready ? doris::TabletStatePB::PB_NOTREADY + : doris::TabletStatePB::PB_RUNNING); + tablet->set_table_id(table_id); + tablet->set_index_id(index_id); + tablet->set_partition_id(partition_id); + tablet->set_tablet_id(tablet_id); + tablet->set_enable_unique_key_merge_on_write(enable_mow); + auto schema = tablet->mutable_schema(); + schema->set_schema_version(0); + auto first_rowset = tablet->add_rs_metas(); + first_rowset->set_rowset_id(0); // required + first_rowset->set_rowset_id_v2(next_rowset_id()); + first_rowset->set_start_version(0); + first_rowset->set_end_version(1); + first_rowset->mutable_tablet_schema()->CopyFrom(*schema); + meta_service->create_tablets(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << tablet_id; +} + +void start_schema_change_job(MetaServiceProxy* meta_service, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id, int64_t new_tablet_id, + const std::string& job_id, const std::string& initiator) { + brpc::Controller cntl; + StartTabletJobRequest req; + StartTabletJobResponse res; + req.mutable_job()->mutable_idx()->set_tablet_id(tablet_id); + auto sc = req.mutable_job()->mutable_schema_change(); + sc->set_id(job_id); + sc->set_initiator(initiator); + sc->mutable_new_tablet_idx()->set_tablet_id(new_tablet_id); + long now = time(nullptr); + sc->set_expiration(now + 12); + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) + << job_id << ' ' << initiator << ' ' << res.status().msg(); + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK) + << job_id << ' ' << initiator; + auto job_key = job_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + std::string job_val; + ASSERT_EQ(txn->get(job_key, &job_val), TxnErrorCode::TXN_OK) << job_id << ' ' << initiator; + TabletJobInfoPB job_pb; + ASSERT_TRUE(job_pb.ParseFromString(job_val)) << job_id << ' ' << initiator; + ASSERT_TRUE(job_pb.has_schema_change()) << job_id << ' ' << initiator; + EXPECT_EQ(job_pb.schema_change().id(), job_id) << ' ' << initiator; +}; + +void finish_schema_change_job(MetaService* meta_service, int64_t tablet_id, int64_t new_tablet_id, + const std::string& job_id, const std::string& initiator, + const std::vector& output_rowsets, + FinishTabletJobResponse& res) { + brpc::Controller cntl; + FinishTabletJobRequest req; + req.set_action(FinishTabletJobRequest::COMMIT); + req.mutable_job()->mutable_idx()->set_tablet_id(tablet_id); + auto sc = req.mutable_job()->mutable_schema_change(); + sc->mutable_new_tablet_idx()->set_tablet_id(new_tablet_id); + if (output_rowsets.empty()) { + sc->set_alter_version(0); + } else { + sc->set_alter_version(output_rowsets.back().end_version()); + for (auto& rowset : output_rowsets) { + sc->add_txn_ids(rowset.txn_id()); + sc->add_output_versions(rowset.end_version()); + sc->set_num_output_rows(sc->num_output_rows() + rowset.num_rows()); + sc->set_num_output_segments(sc->num_output_segments() + rowset.num_segments()); + sc->set_size_output_rowsets(sc->size_output_rowsets() + rowset.data_disk_size()); + } + sc->set_num_output_rowsets(output_rowsets.size()); + } + sc->set_id(job_id); + sc->set_initiator(initiator); + sc->set_delete_bitmap_lock_initiator(12345); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); +} +} // namespace + +TEST(MetaServiceJobTest, StartCompactionArguments) { + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + auto meta_service = get_meta_service(); + brpc::Controller cntl; + StartTabletJobRequest req; + StartTabletJobResponse res; + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("no valid job"), std::string::npos) << res.status().msg(); + + auto* job = req.mutable_job(); + auto* compaction = job->add_compaction(); + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("no valid tablet_id"), std::string::npos) + << res.status().msg(); + + auto* idx = job->mutable_idx(); + constexpr int64_t table_id = 10001; + constexpr int64_t index_id = 10002; + constexpr int64_t partition_id = 10003; + constexpr int64_t tablet_id = 10004; + idx->set_tablet_id(tablet_id); + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::TABLET_NOT_FOUND) << res.status().msg(); + + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id, false); + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("no job id"), std::string::npos) << res.status().msg(); + + compaction->set_id("compaction1"); + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("no valid compaction_cnt"), std::string::npos) + << res.status().msg(); + + compaction->set_base_compaction_cnt(0); + compaction->set_cumulative_compaction_cnt(0); + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("no valid expiration"), std::string::npos) + << res.status().msg(); + + compaction->set_expiration(114115); + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("no valid lease"), std::string::npos) << res.status().msg(); + + compaction->set_lease(114115); + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << res.status().msg(); +} + +TEST(MetaServiceJobTest, StartFullCompaction) { + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + auto meta_service = get_meta_service(); + constexpr int64_t table_id = 10001; + constexpr int64_t index_id = 10002; + constexpr int64_t partition_id = 10003; + constexpr int64_t tablet_id = 10004; + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id, false); + + StartTabletJobResponse res; + { + start_compaction_job(meta_service.get(), tablet_id, "compaction1", "ip:port", 0, 0, + TabletCompactionJobPB::BASE, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + start_compaction_job(meta_service.get(), tablet_id, "compaction2", "ip:port", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + start_compaction_job(meta_service.get(), tablet_id, "compaction3", "ip:port", 0, 0, + TabletCompactionJobPB::BASE, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_TABLET_BUSY); + + start_compaction_job(meta_service.get(), tablet_id, "compaction4", "ip:port", 0, 0, + TabletCompactionJobPB::FULL, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + start_compaction_job(meta_service.get(), tablet_id, "compaction5", "ip:port", 0, 0, + TabletCompactionJobPB::BASE, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_TABLET_BUSY); + } + { + start_compaction_job(meta_service.get(), tablet_id, "compaction6", "ip:port", 0, 0, + TabletCompactionJobPB::FULL, res, {1, 20}); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + start_compaction_job(meta_service.get(), tablet_id, "compaction7", "ip:port", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, {18, 22}); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_TABLET_BUSY); + + start_compaction_job(meta_service.get(), tablet_id, "compaction8", "ip:port", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, {21, 26}); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } +} + +TEST(MetaServiceJobTest, StartSchemaChangeArguments) { + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + auto meta_service = get_meta_service(); + brpc::Controller cntl; + StartTabletJobRequest req; + StartTabletJobResponse res; + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("no valid job"), std::string::npos) << res.status().msg(); + + auto* job = req.mutable_job(); + auto* sc = job->mutable_schema_change(); + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("no valid tablet_id"), std::string::npos) + << res.status().msg(); + + auto* idx = job->mutable_idx(); + constexpr int64_t table_id = 10001; + constexpr int64_t index_id = 10002; + constexpr int64_t partition_id = 10003; + constexpr int64_t tablet_id = 10004; + idx->set_tablet_id(tablet_id); + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::TABLET_NOT_FOUND) << res.status().msg(); + + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id, false); + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("no job id"), std::string::npos) << res.status().msg(); + + sc->set_id("sc1"); + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("no initiator"), std::string::npos) << res.status().msg(); + + sc->set_initiator("BE1"); + // meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + // ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + // EXPECT_NE(res.status().msg().find("no valid expiration"), std::string::npos) + // << res.status().msg(); + + sc->set_expiration(114115); + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("no valid new_tablet_id"), std::string::npos) + << res.status().msg(); + + auto* new_idx = sc->mutable_new_tablet_idx(); + new_idx->set_tablet_id(tablet_id); + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("new_tablet_id same with base_tablet_id"), std::string::npos) + << res.status().msg(); + + constexpr int64_t new_index_id = 10005; + constexpr int64_t new_tablet_id = 10006; + new_idx->set_tablet_id(new_tablet_id); + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::TABLET_NOT_FOUND) << res.status().msg(); + + create_tablet(meta_service.get(), table_id, new_index_id, partition_id, new_tablet_id, false); + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_ALREADY_SUCCESS) << res.status().msg(); + + // Reset tablet state + auto tablet_key = + meta_tablet_key({instance_id, table_id, new_index_id, partition_id, new_tablet_id}); + std::string tablet_val; + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tablet_key, &tablet_val), TxnErrorCode::TXN_OK); + doris::TabletMetaCloudPB tablet_meta; + ASSERT_TRUE(tablet_meta.ParseFromString(tablet_val)); + tablet_meta.clear_tablet_state(); + tablet_val = tablet_meta.SerializeAsString(); + txn->put(tablet_key, tablet_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("invalid new tablet state"), std::string::npos) + << res.status().msg(); + + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tablet_key, &tablet_val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(tablet_meta.ParseFromString(tablet_val)); + tablet_meta.set_tablet_state(doris::TabletStatePB::PB_NOTREADY); + tablet_val = tablet_meta.SerializeAsString(); + txn->put(tablet_key, tablet_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + meta_service->start_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << res.status().msg(); +} + +TEST(MetaServiceJobTest, ProcessCompactionArguments) { + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + auto meta_service = get_meta_service(); + brpc::Controller cntl; + FinishTabletJobRequest req; + FinishTabletJobResponse res; + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("no valid job"), std::string::npos) << res.status().msg(); + + auto* job = req.mutable_job(); + auto* compaction = job->add_compaction(); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("no valid tablet_id"), std::string::npos) + << res.status().msg(); + + auto* idx = job->mutable_idx(); + constexpr int64_t table_id = 10001; + constexpr int64_t index_id = 10002; + constexpr int64_t partition_id = 10003; + constexpr int64_t tablet_id = 10004; + idx->set_tablet_id(tablet_id); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::TABLET_NOT_FOUND) << res.status().msg(); + + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id, false); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("job not found"), std::string::npos) << res.status().msg(); + + auto job_key = job_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + TabletJobInfoPB recorded_job; + auto job_val = recorded_job.SerializeAsString(); + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(job_key, job_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("no running compaction"), std::string::npos) + << res.status().msg(); + + auto* recorded_compaction = recorded_job.add_compaction(); + recorded_compaction->set_id("compaction1"); + recorded_compaction->set_expiration(114115); + job_val = recorded_job.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(job_key, job_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("unmatched job id"), std::string::npos) << res.status().msg(); + + compaction->set_id("compaction1"); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_EXPIRED) << res.status().msg(); + + // Prepare job kv + recorded_compaction->set_expiration(::time(nullptr) + 10); + job_val = recorded_job.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(job_key, job_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("unsupported action"), std::string::npos) + << res.status().msg(); + + req.set_action(FinishTabletJobRequest::LEASE); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("invalid lease"), std::string::npos) << res.status().msg(); + + compaction->set_lease(::time(nullptr) + 5); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << res.status().msg(); + + req.set_action(FinishTabletJobRequest::COMMIT); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("invalid compaction type"), std::string::npos) + << res.status().msg(); + + compaction->set_type(TabletCompactionJobPB::EMPTY_CUMULATIVE); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << res.status().msg(); +} + +TEST(MetaServiceJobTest, ProcessSchemaChangeArguments) { + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + auto meta_service = get_meta_service(); + brpc::Controller cntl; + FinishTabletJobRequest req; + FinishTabletJobResponse res; + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("no valid job"), std::string::npos) << res.status().msg(); + + auto* job = req.mutable_job(); + auto* sc = job->mutable_schema_change(); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("no valid tablet_id"), std::string::npos) + << res.status().msg(); + + auto* idx = job->mutable_idx(); + constexpr int64_t table_id = 10001; + constexpr int64_t index_id = 10002; + constexpr int64_t partition_id = 10003; + constexpr int64_t tablet_id = 10004; + idx->set_tablet_id(tablet_id); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::TABLET_NOT_FOUND) << res.status().msg(); + + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id, false); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("job not found"), std::string::npos) << res.status().msg(); + + auto job_key = job_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + TabletJobInfoPB recorded_job; + auto job_val = recorded_job.SerializeAsString(); + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(job_key, job_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("no valid new_tablet_id"), std::string::npos) + << res.status().msg(); + + auto* new_idx = sc->mutable_new_tablet_idx(); + new_idx->set_tablet_id(tablet_id); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("new_tablet_id same with base_tablet_id"), std::string::npos) + << res.status().msg(); + + constexpr int64_t new_index_id = 10005; + constexpr int64_t new_tablet_id = 10006; + new_idx->set_tablet_id(new_tablet_id); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::TABLET_NOT_FOUND) << res.status().msg(); + + create_tablet(meta_service.get(), table_id, new_index_id, partition_id, new_tablet_id, false); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_ALREADY_SUCCESS) << res.status().msg(); + + // Reset tablet state + auto tablet_key = + meta_tablet_key({instance_id, table_id, new_index_id, partition_id, new_tablet_id}); + std::string tablet_val; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tablet_key, &tablet_val), TxnErrorCode::TXN_OK); + doris::TabletMetaCloudPB tablet_meta; + ASSERT_TRUE(tablet_meta.ParseFromString(tablet_val)); + tablet_meta.clear_tablet_state(); + tablet_val = tablet_meta.SerializeAsString(); + txn->put(tablet_key, tablet_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("invalid new tablet state"), std::string::npos) + << res.status().msg(); + + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(tablet_key, &tablet_val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(tablet_meta.ParseFromString(tablet_val)); + tablet_meta.set_tablet_state(doris::TabletStatePB::PB_NOTREADY); + tablet_val = tablet_meta.SerializeAsString(); + txn->put(tablet_key, tablet_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("no running schema_change"), std::string::npos) + << res.status().msg(); + + auto* recorded_sc = recorded_job.mutable_schema_change(); + recorded_sc->set_expiration(114115); + job_val = recorded_job.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(job_key, job_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_EXPIRED) << res.status().msg(); + + recorded_sc->set_expiration(::time(nullptr) + 10); + recorded_sc->set_id("sc1"); + recorded_sc->set_initiator("BE1"); + job_val = recorded_job.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(job_key, job_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("unmatched job id or initiator"), std::string::npos) + << res.status().msg(); + + sc->set_id("sc1"); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("unmatched job id or initiator"), std::string::npos) + << res.status().msg(); + + sc->set_initiator("BE1"); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); + EXPECT_NE(res.status().msg().find("unsupported action"), std::string::npos) + << res.status().msg(); + + req.set_action(FinishTabletJobRequest::ABORT); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << res.status().msg(); +} + +TEST(MetaServiceJobTest, CompactionJobTest) { + auto meta_service = get_meta_service(); + // meta_service->resource_mgr().reset(); // Do not use resource manager + + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + brpc::Controller cntl; + + // Start compaction job + auto test_start_compaction_job = [&](int64_t table_id, int64_t index_id, int64_t partition_id, + int64_t tablet_id, + TabletCompactionJobPB::CompactionType type) { + StartTabletJobResponse res; + std::string job_id = "job_id123"; + + start_compaction_job(meta_service.get(), 0, job_id, "ip:port", 0, 0, type, res); + ASSERT_NE(res.status().msg().find("no valid tablet_id given"), std::string::npos); + + start_compaction_job(meta_service.get(), tablet_id, job_id, "ip:port", 0, 0, type, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::TABLET_NOT_FOUND); + + auto index_key = meta_tablet_idx_key({instance_id, tablet_id}); + TabletIndexPB idx_pb; + idx_pb.set_table_id(1); + idx_pb.set_index_id(2); + idx_pb.set_partition_id(3); + idx_pb.set_tablet_id(tablet_id + 1); // error, tablet_id not match + std::string idx_val = idx_pb.SerializeAsString(); + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(index_key, idx_val); + std::string stats_key = + stats_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + TabletStatsPB stats; + stats.set_base_compaction_cnt(9); + stats.set_cumulative_compaction_cnt(19); + txn->put(stats_key, stats.SerializeAsString()); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + start_compaction_job(meta_service.get(), tablet_id, job_id, "ip:port", 0, 0, type, res); + ASSERT_NE(res.status().msg().find("internal error"), std::string::npos); + idx_pb.set_tablet_id(tablet_id); // Correct tablet_id + idx_val = idx_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(index_key, idx_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + start_compaction_job(meta_service.get(), tablet_id, job_id, "ip:port", 9, 18, type, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::STALE_TABLET_CACHE); + start_compaction_job(meta_service.get(), tablet_id, job_id, "ip:port", 9, 19, type, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + auto job_key = job_tablet_key({instance_id, idx_pb.table_id(), idx_pb.index_id(), + idx_pb.partition_id(), idx_pb.tablet_id()}); + std::string job_val; + ASSERT_EQ(txn->get(job_key, &job_val), TxnErrorCode::TXN_OK); + TabletJobInfoPB job_pb; + ASSERT_TRUE(job_pb.ParseFromString(job_val)); + start_compaction_job(meta_service.get(), tablet_id, job_id, "ip:port", 9, 19, type, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); // Same job_id, return OK + }; + + auto test_commit_compaction_job = [&](int64_t table_id, int64_t index_id, int64_t partition_id, + int64_t tablet_id, + TabletCompactionJobPB::CompactionType type) { + FinishTabletJobRequest req; + FinishTabletJobResponse res; + std::string job_id = "job_id123"; + + auto compaction = req.mutable_job()->add_compaction(); + compaction->set_id(job_id); + compaction->set_initiator("ip:port"); + meta_service->finish_tablet_job(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_NE(res.status().msg().find("no valid tablet_id given"), std::string::npos); + + req.mutable_job()->mutable_idx()->set_table_id(table_id); + req.mutable_job()->mutable_idx()->set_index_id(index_id); + req.mutable_job()->mutable_idx()->set_partition_id(partition_id); + req.mutable_job()->mutable_idx()->set_tablet_id(tablet_id); + compaction->set_base_compaction_cnt(10); + compaction->set_cumulative_compaction_cnt(20); + // Action is not set + meta_service->finish_tablet_job(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_NE(res.status().msg().find("unsupported action"), std::string::npos); + + //====================================================================== + // Test commit + //====================================================================== + req.set_action(FinishTabletJobRequest::COMMIT); + + auto tablet_meta_key = + meta_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + doris::TabletMetaCloudPB tablet_meta_pb; + tablet_meta_pb.set_table_id(table_id); + tablet_meta_pb.set_index_id(index_id); + tablet_meta_pb.set_partition_id(partition_id); + tablet_meta_pb.set_tablet_id(tablet_id); + tablet_meta_pb.set_cumulative_layer_point(50); + std::string tablet_meta_val = tablet_meta_pb.SerializeAsString(); + ASSERT_FALSE(tablet_meta_val.empty()); + txn->put(tablet_meta_key, tablet_meta_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + meta_service->finish_tablet_job(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + + // Create create tablet stats, compation job will will update stats + auto tablet_stats_key = + stats_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + TabletStatsPB tablet_stats_pb; + tablet_stats_pb.mutable_idx()->set_table_id(table_id); + tablet_stats_pb.mutable_idx()->set_index_id(index_id); + tablet_stats_pb.mutable_idx()->set_partition_id(partition_id); + tablet_stats_pb.mutable_idx()->set_tablet_id(tablet_id); + + std::mt19937 rng(std::chrono::system_clock::now().time_since_epoch().count()); + std::uniform_int_distribution dist(1, 10000); // Positive numbers + + compaction->set_output_cumulative_point(tablet_stats_pb.cumulative_point() + dist(rng)); + compaction->set_num_output_rows(dist(rng)); + compaction->set_num_output_rowsets(dist(rng)); + compaction->set_num_output_segments(dist(rng)); + compaction->set_num_input_rows(dist(rng)); + compaction->set_num_input_rowsets(dist(rng)); + compaction->set_num_input_segments(dist(rng)); + compaction->set_size_input_rowsets(dist(rng)); + compaction->set_size_output_rowsets(dist(rng)); + compaction->set_type(type); + + tablet_stats_pb.set_cumulative_compaction_cnt(dist(rng)); + tablet_stats_pb.set_base_compaction_cnt(dist(rng)); + tablet_stats_pb.set_cumulative_point(tablet_meta_pb.cumulative_layer_point()); + // MUST let data stats be larger than input data size + tablet_stats_pb.set_num_rows(dist(rng) + compaction->num_input_rows()); + tablet_stats_pb.set_data_size(dist(rng) + compaction->size_input_rowsets()); + tablet_stats_pb.set_num_rowsets(dist(rng) + compaction->num_input_rowsets()); + tablet_stats_pb.set_num_segments(dist(rng) + compaction->num_input_segments()); + + std::string tablet_stats_val = tablet_stats_pb.SerializeAsString(); + ASSERT_FALSE(tablet_stats_val.empty()); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(tablet_stats_key, tablet_stats_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + // Input rowset not valid + meta_service->finish_tablet_job(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_NE(res.status().msg().find("invalid input"), std::string::npos); + + // Provide input and output rowset info + int64_t input_version_start = dist(rng); + int64_t input_version_end = input_version_start + 100; + compaction->add_input_versions(input_version_start); + compaction->add_input_versions(input_version_end); + compaction->add_output_versions(input_version_end); + compaction->add_output_rowset_ids("output rowset id"); + + // Input rowsets must exist, and more than 0 + // Check number input rowsets + sp->set_call_back("process_compaction_job::loop_input_done", [](void* c) { + int& num_input_rowsets = *(int*)c; + ASSERT_EQ(num_input_rowsets, 0); // zero existed rowsets + }); + sp->set_call_back("process_compaction_job::too_few_rowsets", [](void* c) { + auto& need_commit = *(bool*)c; + ASSERT_EQ(need_commit, true); + need_commit = false; // Donot remove tablet job in order to continue test + }); + + meta_service->finish_tablet_job(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_NE(res.status().msg().find("too few input rowsets"), std::string::npos); + + // Provide input rowset KVs, boundary test, 5 input rowsets + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + // clang-format off + std::vector input_rowset_keys = { + meta_rowset_key({instance_id, tablet_id, input_version_start - 1}), + meta_rowset_key({instance_id, tablet_id, input_version_start}), + meta_rowset_key({instance_id, tablet_id, input_version_start + 1}), + meta_rowset_key({instance_id, tablet_id, (input_version_start + input_version_end) / 2}), + meta_rowset_key({instance_id, tablet_id, input_version_end - 1}), + meta_rowset_key({instance_id, tablet_id, input_version_end}), + meta_rowset_key({instance_id, tablet_id, input_version_end + 1}), + }; + // clang-format on + std::vector> input_rowset_vals; + for (auto& i : input_rowset_keys) { + doris::RowsetMetaCloudPB rs_pb; + rs_pb.set_rowset_id(0); + rs_pb.set_rowset_id_v2(hex(i)); + input_rowset_vals.emplace_back(new std::string(rs_pb.SerializeAsString())); + txn->put(i, *input_rowset_vals.back()); + } + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + // Check number input rowsets + sp->set_call_back("process_compaction_job::loop_input_done", [](void* c) { + int& num_input_rowsets = *(int*)c; + ASSERT_EQ(num_input_rowsets, 5); + }); + // No tmp rowset key (output rowset) + meta_service->finish_tablet_job(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_NE(res.status().msg().find("invalid txn_id"), std::string::npos); + + int64_t txn_id = dist(rng); + compaction->add_txn_id(txn_id); + + meta_service->finish_tablet_job(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_NE(res.status().msg().find("failed to get tmp rowset key"), std::string::npos); + + // Provide invalid output rowset meta + auto tmp_rowset_key = meta_rowset_tmp_key({instance_id, txn_id, tablet_id}); + doris::RowsetMetaCloudPB tmp_rs_pb; + tmp_rs_pb.set_rowset_id(0); + auto tmp_rowset_val = tmp_rs_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(tmp_rowset_key, tmp_rowset_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + meta_service->finish_tablet_job(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_NE(res.status().msg().find("invalid txn_id in output tmp rowset meta"), + std::string::npos); + + // Provide txn_id in output rowset meta + tmp_rs_pb.set_txn_id(10086); + tmp_rowset_val = tmp_rs_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(tmp_rowset_key, tmp_rowset_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + meta_service->finish_tablet_job(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + //===================================================================== + // All branch tests done, we are done commit a compaction job + //===================================================================== + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + tablet_stats_val.clear(); + ASSERT_EQ(txn->get(tablet_stats_key, &tablet_stats_val), TxnErrorCode::TXN_OK); + TabletStatsPB stats; + ASSERT_TRUE(stats.ParseFromString(tablet_stats_val)); + + // clang-format off + EXPECT_EQ(stats.base_compaction_cnt() , tablet_stats_pb.base_compaction_cnt() + (req.job().compaction(0).type() == TabletCompactionJobPB::BASE)); + EXPECT_EQ(stats.cumulative_compaction_cnt(), tablet_stats_pb.cumulative_compaction_cnt() + (req.job().compaction(0).type() == TabletCompactionJobPB::CUMULATIVE)); + EXPECT_EQ(stats.cumulative_point() , type == TabletCompactionJobPB::BASE ? 50 : req.job().compaction(0).output_cumulative_point()); + EXPECT_EQ(stats.num_rows() , tablet_stats_pb.num_rows() + (req.job().compaction(0).num_output_rows() - req.job().compaction(0).num_input_rows())); + EXPECT_EQ(stats.data_size() , tablet_stats_pb.data_size() + (req.job().compaction(0).size_output_rowsets() - req.job().compaction(0).size_input_rowsets())); + EXPECT_EQ(stats.num_rowsets() , tablet_stats_pb.num_rowsets() + (req.job().compaction(0).num_output_rowsets() - req.job().compaction(0).num_input_rowsets())); + EXPECT_EQ(stats.num_segments() , tablet_stats_pb.num_segments() + (req.job().compaction(0).num_output_segments() - req.job().compaction(0).num_input_segments())); + // clang-format on + + // Check job removed, tablet meta updated + auto job_key = job_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + std::string job_val; + ASSERT_EQ(txn->get(job_key, &job_val), TxnErrorCode::TXN_OK); + TabletJobInfoPB job_pb; + ASSERT_TRUE(job_pb.ParseFromString(job_val)); + ASSERT_TRUE(job_pb.compaction().empty()); + tablet_meta_val.clear(); + + // Check tmp rowset removed + ASSERT_EQ(txn->get(tmp_rowset_key, &tmp_rowset_val), TxnErrorCode::TXN_KEY_NOT_FOUND); + // Check input rowsets removed, the largest version remains + for (int i = 1; i < input_rowset_keys.size() - 2; ++i) { + std::string val; + EXPECT_EQ(txn->get(input_rowset_keys[i], &val), TxnErrorCode::TXN_KEY_NOT_FOUND) + << hex(input_rowset_keys[i]); + } + // Check recycle rowsets added + for (int i = 1; i < input_rowset_vals.size() - 1; ++i) { + doris::RowsetMetaCloudPB rs; + ASSERT_TRUE(rs.ParseFromString(*input_rowset_vals[i])); + auto key = recycle_rowset_key({instance_id, tablet_id, rs.rowset_id_v2()}); + std::string val; + EXPECT_EQ(txn->get(key, &val), TxnErrorCode::TXN_OK) << hex(key); + } + // Check output rowset added + auto rowset_key = meta_rowset_key({instance_id, tablet_id, input_version_end}); + std::string rowset_val; + EXPECT_EQ(txn->get(rowset_key, &rowset_val), TxnErrorCode::TXN_OK) << hex(rowset_key); + }; + + auto test_abort_compaction_job = [&](int64_t table_id, int64_t index_id, int64_t partition_id, + int64_t tablet_id) { + FinishTabletJobRequest req; + FinishTabletJobResponse res; + std::string job_id = "job_id123"; + + auto compaction = req.mutable_job()->add_compaction(); + compaction->set_id(job_id); + compaction->set_initiator("ip:port"); + req.mutable_job()->mutable_idx()->set_tablet_id(tablet_id); + req.set_action(FinishTabletJobRequest::ABORT); + meta_service->finish_tablet_job(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + auto job_key = job_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + std::string job_val; + ASSERT_EQ(txn->get(job_key, &job_val), TxnErrorCode::TXN_OK); + TabletJobInfoPB job_pb; + ASSERT_TRUE(job_pb.ParseFromString(job_val)); + ASSERT_TRUE(job_pb.compaction().empty()); + }; + ASSERT_NO_FATAL_FAILURE( + test_start_compaction_job(1, 2, 3, 4, TabletCompactionJobPB::CUMULATIVE)); + ASSERT_NO_FATAL_FAILURE( + test_commit_compaction_job(1, 2, 3, 4, TabletCompactionJobPB::CUMULATIVE)); + ASSERT_NO_FATAL_FAILURE( + test_start_compaction_job(1, 2, 3, 5, TabletCompactionJobPB::CUMULATIVE)); + ASSERT_NO_FATAL_FAILURE(test_abort_compaction_job(1, 2, 3, 5)); + ASSERT_NO_FATAL_FAILURE(test_start_compaction_job(1, 2, 3, 6, TabletCompactionJobPB::BASE)); + ASSERT_NO_FATAL_FAILURE(test_commit_compaction_job(1, 2, 3, 6, TabletCompactionJobPB::BASE)); + ASSERT_NO_FATAL_FAILURE(test_start_compaction_job(1, 2, 3, 7, TabletCompactionJobPB::BASE)); + ASSERT_NO_FATAL_FAILURE(test_abort_compaction_job(1, 2, 3, 7)); +} + +TEST(MetaServiceJobTest, CompactionJobWithMoWTest) { + auto meta_service = get_meta_service(); + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + brpc::Controller cntl; + + // Start compaction job + auto test_start_compaction_job = [&](int64_t table_id, int64_t index_id, int64_t partition_id, + int64_t tablet_id, + TabletCompactionJobPB::CompactionType type) { + StartTabletJobResponse res; + std::string job_id = "job_id123"; + + auto index_key = meta_tablet_idx_key({instance_id, tablet_id}); + TabletIndexPB idx_pb; + idx_pb.set_table_id(table_id); + idx_pb.set_index_id(index_id); + idx_pb.set_partition_id(partition_id); + idx_pb.set_tablet_id(tablet_id); + std::string idx_val = idx_pb.SerializeAsString(); + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(index_key, idx_val); + std::string stats_key = + stats_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + TabletStatsPB stats; + stats.set_base_compaction_cnt(9); + stats.set_cumulative_compaction_cnt(19); + txn->put(stats_key, stats.SerializeAsString()); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + start_compaction_job(meta_service.get(), tablet_id, job_id, "ip:port", 9, 19, type, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + }; + + FinishTabletJobResponse res; + auto test_commit_compaction_job = [&](int64_t table_id, int64_t index_id, int64_t partition_id, + int64_t tablet_id, + TabletCompactionJobPB::CompactionType type) { + FinishTabletJobRequest req; + std::string job_id = "job_id123"; + + auto compaction = req.mutable_job()->add_compaction(); + compaction->set_id(job_id); + compaction->set_initiator("ip:port"); + compaction->set_base_compaction_cnt(10); + compaction->set_cumulative_compaction_cnt(20); + compaction->set_delete_bitmap_lock_initiator(12345); + req.mutable_job()->mutable_idx()->set_table_id(table_id); + req.mutable_job()->mutable_idx()->set_index_id(index_id); + req.mutable_job()->mutable_idx()->set_partition_id(partition_id); + req.mutable_job()->mutable_idx()->set_tablet_id(tablet_id); + req.set_action(FinishTabletJobRequest::COMMIT); + + auto tablet_meta_key = + meta_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + doris::TabletMetaCloudPB tablet_meta_pb; + tablet_meta_pb.set_table_id(table_id); + tablet_meta_pb.set_index_id(index_id); + tablet_meta_pb.set_partition_id(partition_id); + tablet_meta_pb.set_tablet_id(tablet_id); + tablet_meta_pb.set_cumulative_layer_point(50); + std::string tablet_meta_val = tablet_meta_pb.SerializeAsString(); + ASSERT_FALSE(tablet_meta_val.empty()); + txn->put(tablet_meta_key, tablet_meta_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + // Create create tablet stats, compation job will will update stats + auto tablet_stats_key = + stats_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + TabletStatsPB tablet_stats_pb; + tablet_stats_pb.mutable_idx()->set_table_id(table_id); + tablet_stats_pb.mutable_idx()->set_index_id(index_id); + tablet_stats_pb.mutable_idx()->set_partition_id(partition_id); + tablet_stats_pb.mutable_idx()->set_tablet_id(tablet_id); + + std::mt19937 rng(std::chrono::system_clock::now().time_since_epoch().count()); + std::uniform_int_distribution dist(1, 10000); // Positive numbers + + compaction->set_output_cumulative_point(tablet_stats_pb.cumulative_point() + dist(rng)); + compaction->set_num_output_rows(dist(rng)); + compaction->set_num_output_rowsets(dist(rng)); + compaction->set_num_output_segments(dist(rng)); + compaction->set_num_input_rows(dist(rng)); + compaction->set_num_input_rowsets(dist(rng)); + compaction->set_num_input_segments(dist(rng)); + compaction->set_size_input_rowsets(dist(rng)); + compaction->set_size_output_rowsets(dist(rng)); + compaction->set_type(type); + + tablet_stats_pb.set_cumulative_compaction_cnt(dist(rng)); + tablet_stats_pb.set_base_compaction_cnt(dist(rng)); + tablet_stats_pb.set_cumulative_point(tablet_meta_pb.cumulative_layer_point()); + // MUST let data stats be larger than input data size + tablet_stats_pb.set_num_rows(dist(rng) + compaction->num_input_rows()); + tablet_stats_pb.set_data_size(dist(rng) + compaction->size_input_rowsets()); + tablet_stats_pb.set_num_rowsets(dist(rng) + compaction->num_input_rowsets()); + tablet_stats_pb.set_num_segments(dist(rng) + compaction->num_input_segments()); + + std::string tablet_stats_val = tablet_stats_pb.SerializeAsString(); + ASSERT_FALSE(tablet_stats_val.empty()); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(tablet_stats_key, tablet_stats_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + // Provide input and output rowset info + int64_t input_version_start = dist(rng); + int64_t input_version_end = input_version_start + 100; + compaction->add_input_versions(input_version_start); + compaction->add_input_versions(input_version_end); + compaction->add_output_versions(input_version_end); + compaction->add_output_rowset_ids("output rowset id"); + + // Input rowsets must exist, and more than 0 + // Check number input rowsets + sp->set_call_back("process_compaction_job::loop_input_done", [](void* c) { + int& num_input_rowsets = *(int*)c; + ASSERT_EQ(num_input_rowsets, 0); // zero existed rowsets + }); + sp->set_call_back("process_compaction_job::too_few_rowsets", [](void* c) { + auto& need_commit = *(bool*)c; + ASSERT_EQ(need_commit, true); + need_commit = false; // Donot remove tablet job in order to continue test + }); + + // Provide input rowset KVs, boundary test, 5 input rowsets + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + // clang-format off + std::vector input_rowset_keys = { + meta_rowset_key({instance_id, tablet_id, input_version_start - 1}), + meta_rowset_key({instance_id, tablet_id, input_version_start}), + meta_rowset_key({instance_id, tablet_id, input_version_start + 1}), + meta_rowset_key({instance_id, tablet_id, (input_version_start + input_version_end) / 2}), + meta_rowset_key({instance_id, tablet_id, input_version_end - 1}), + meta_rowset_key({instance_id, tablet_id, input_version_end}), + meta_rowset_key({instance_id, tablet_id, input_version_end + 1}), + }; + // clang-format on + std::vector> input_rowset_vals; + for (auto& i : input_rowset_keys) { + doris::RowsetMetaCloudPB rs_pb; + rs_pb.set_rowset_id(0); + rs_pb.set_rowset_id_v2(hex(i)); + input_rowset_vals.emplace_back(new std::string(rs_pb.SerializeAsString())); + txn->put(i, *input_rowset_vals.back()); + } + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + // Check number input rowsets + sp->set_call_back("process_compaction_job::loop_input_done", [](void* c) { + int& num_input_rowsets = *(int*)c; + ASSERT_EQ(num_input_rowsets, 5); + }); + int64_t txn_id = dist(rng); + compaction->add_txn_id(txn_id); + + // Provide output rowset meta + auto tmp_rowset_key = meta_rowset_tmp_key({instance_id, txn_id, tablet_id}); + doris::RowsetMetaCloudPB tmp_rs_pb; + tmp_rs_pb.set_rowset_id(0); + tmp_rs_pb.set_txn_id(10086); + auto tmp_rowset_val = tmp_rs_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(tmp_rowset_key, tmp_rowset_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + meta_service->finish_tablet_job(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + }; + + auto test_abort_compaction_job = [&](int64_t table_id, int64_t index_id, int64_t partition_id, + int64_t tablet_id) { + FinishTabletJobRequest req; + FinishTabletJobResponse res; + std::string job_id = "job_id123"; + + auto compaction = req.mutable_job()->add_compaction(); + compaction->set_id(job_id); + compaction->set_initiator("ip:port"); + compaction->set_delete_bitmap_lock_initiator(12345); + req.mutable_job()->mutable_idx()->set_tablet_id(tablet_id); + req.set_action(FinishTabletJobRequest::ABORT); + meta_service->finish_tablet_job(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + auto job_key = job_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + std::string job_val; + ASSERT_EQ(txn->get(job_key, &job_val), TxnErrorCode::TXN_OK); + TabletJobInfoPB job_pb; + ASSERT_TRUE(job_pb.ParseFromString(job_val)); + ASSERT_TRUE(job_pb.compaction().empty()); + }; + + auto clear_rowsets = [&](int64_t tablet_id) { + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string key1 = meta_rowset_key({instance_id, tablet_id, 1}); + std::string key2 = meta_rowset_key({instance_id, tablet_id, 10001}); + txn->remove(key1, key2); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + }; + + test_start_compaction_job(1, 2, 3, 4, TabletCompactionJobPB::CUMULATIVE); + test_commit_compaction_job(1, 2, 3, 4, TabletCompactionJobPB::CUMULATIVE); + ASSERT_EQ(res.status().code(), MetaServiceCode::KV_TXN_GET_ERR); + clear_rowsets(4); + + auto res_code = get_delete_bitmap_lock(meta_service.get(), 1, 1, 1); + ASSERT_EQ(res_code, MetaServiceCode::OK); + test_commit_compaction_job(1, 2, 3, 4, TabletCompactionJobPB::CUMULATIVE); + ASSERT_EQ(res.status().code(), MetaServiceCode::LOCK_EXPIRED); + remove_delete_bitmap_lock(meta_service.get(), 1); + clear_rowsets(4); + + res_code = get_delete_bitmap_lock(meta_service.get(), 1, -1, 1); + ASSERT_EQ(res_code, MetaServiceCode::OK); + test_commit_compaction_job(1, 2, 3, 4, TabletCompactionJobPB::CUMULATIVE); + ASSERT_EQ(res.status().code(), MetaServiceCode::LOCK_EXPIRED); + remove_delete_bitmap_lock(meta_service.get(), 1); + clear_rowsets(4); + + res_code = get_delete_bitmap_lock(meta_service.get(), 1, -1, 12345); + ASSERT_EQ(res_code, MetaServiceCode::OK); + test_commit_compaction_job(1, 2, 3, 4, TabletCompactionJobPB::CUMULATIVE); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + remove_delete_bitmap_lock(meta_service.get(), 1); + clear_rowsets(4); + + test_start_compaction_job(2, 2, 3, 5, TabletCompactionJobPB::BASE); + res_code = get_delete_bitmap_lock(meta_service.get(), 2, -1, 12345); + ASSERT_EQ(res_code, MetaServiceCode::OK); + res_code = get_delete_bitmap_lock(meta_service.get(), 2, -1, 2345); + ASSERT_EQ(res_code, MetaServiceCode::OK); + test_commit_compaction_job(2, 2, 3, 5, TabletCompactionJobPB::BASE); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + remove_delete_bitmap_lock(meta_service.get(), 2); + clear_rowsets(5); + + test_start_compaction_job(2, 2, 3, 6, TabletCompactionJobPB::BASE); + res_code = get_delete_bitmap_lock(meta_service.get(), 2, -1, 12345); + ASSERT_EQ(res_code, MetaServiceCode::OK); + res_code = get_delete_bitmap_lock(meta_service.get(), 2, 123, -1); + ASSERT_EQ(res_code, MetaServiceCode::LOCK_CONFLICT); + test_abort_compaction_job(2, 2, 3, 6); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + res_code = get_delete_bitmap_lock(meta_service.get(), 2, 123, -1); + ASSERT_EQ(res_code, MetaServiceCode::OK); + remove_delete_bitmap_lock(meta_service.get(), 2); + clear_rowsets(6); +} + +TEST(MetaServiceJobTest, SchemaChangeJobTest) { + auto meta_service = get_meta_service(); + // meta_service->resource_mgr().reset(); // Do not use resource manager + + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + brpc::Controller cntl; + + int64_t table_id = 1; + int64_t index_id = 2; + int64_t partition_id = 3; + int64_t tablet_id = 4; + ASSERT_NO_FATAL_FAILURE( + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id, false)); + + // commit schema_change job with alter_version == 1 + { + int64_t new_tablet_id = 14; + ASSERT_NO_FATAL_FAILURE(create_tablet(meta_service.get(), table_id, index_id, partition_id, + new_tablet_id, false, true)); + ASSERT_NO_FATAL_FAILURE(start_schema_change_job(meta_service.get(), table_id, index_id, + partition_id, tablet_id, new_tablet_id, + "job1", "be1")); + FinishTabletJobResponse res; + finish_schema_change_job(meta_service.get(), tablet_id, new_tablet_id, "job2", "be1", {}, + res); + ASSERT_NE(res.status().code(), MetaServiceCode::OK); + ASSERT_NE(res.status().msg().find("unmatched job id or initiator"), std::string::npos); + res.Clear(); + finish_schema_change_job(meta_service.get(), tablet_id, new_tablet_id, "job1", "be2", {}, + res); + ASSERT_NE(res.status().code(), MetaServiceCode::OK); + ASSERT_NE(res.status().msg().find("unmatched job id or initiator"), std::string::npos); + res.Clear(); + finish_schema_change_job(meta_service.get(), tablet_id, new_tablet_id, "job1", "be1", {}, + res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + res.Clear(); + finish_schema_change_job(meta_service.get(), tablet_id, new_tablet_id, "job1", "be1", {}, + res); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_ALREADY_SUCCESS); + } + + // commit schema_change job with txn_ids + { + int64_t new_tablet_id = 24; + ASSERT_NO_FATAL_FAILURE(create_tablet(meta_service.get(), table_id, index_id, partition_id, + new_tablet_id, false, true)); + ASSERT_NO_FATAL_FAILURE(start_schema_change_job(meta_service.get(), table_id, index_id, + partition_id, tablet_id, new_tablet_id, + "job2", "be1")); + + std::vector output_rowsets; + for (int64_t i = 0; i < 5; ++i) { + output_rowsets.push_back(create_rowset(new_tablet_id, i + 2, i + 2)); + CreateRowsetResponse res; + commit_rowset(meta_service.get(), output_rowsets.back(), res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << i; + } + + FinishTabletJobResponse res; + finish_schema_change_job(meta_service.get(), tablet_id, new_tablet_id, "job2", "be1", + output_rowsets, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_TRUE(res.has_stats()); + EXPECT_EQ(res.stats().num_rows(), 500); + EXPECT_EQ(res.stats().num_rowsets(), 6); + EXPECT_EQ(res.stats().num_segments(), 5); + EXPECT_EQ(res.stats().data_size(), 50000); + TabletStatsPB tablet_stats; + get_tablet_stats(meta_service.get(), new_tablet_id, tablet_stats); + EXPECT_EQ(tablet_stats.num_rows(), 500); + EXPECT_EQ(tablet_stats.num_rowsets(), 6); + EXPECT_EQ(tablet_stats.num_segments(), 5); + EXPECT_EQ(tablet_stats.data_size(), 50000); + + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + // check tablet state + auto tablet_key = + meta_tablet_key({instance_id, table_id, index_id, partition_id, new_tablet_id}); + std::string tablet_val; + ASSERT_EQ(txn->get(tablet_key, &tablet_val), TxnErrorCode::TXN_OK); + doris::TabletMetaCloudPB tablet_pb; + ASSERT_TRUE(tablet_pb.ParseFromString(tablet_val)); + ASSERT_EQ(tablet_pb.tablet_state(), doris::TabletStatePB::PB_RUNNING); + // check visible rowsets + std::unique_ptr it; + auto rs_start = meta_rowset_key({instance_id, new_tablet_id, 0}); + auto rs_end = meta_rowset_key({instance_id, new_tablet_id, 100}); + ASSERT_EQ(txn->get(rs_start, rs_end, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 6); + auto [k, v] = it->next(); + doris::RowsetMetaCloudPB saved_rowset; + ASSERT_TRUE(saved_rowset.ParseFromArray(v.data(), v.size())); + EXPECT_EQ(saved_rowset.start_version(), 0); + EXPECT_EQ(saved_rowset.end_version(), 1); + for (auto& rs : output_rowsets) { + auto [k, v] = it->next(); + ASSERT_TRUE(saved_rowset.ParseFromArray(v.data(), v.size())); + EXPECT_EQ(saved_rowset.rowset_id_v2(), rs.rowset_id_v2()); + EXPECT_EQ(saved_rowset.start_version(), rs.start_version()); + EXPECT_EQ(saved_rowset.end_version(), rs.end_version()); + } + } + + // commit schema_change job with rowsets which overlapped with visible rowsets + { + int64_t new_tablet_id = 34; + ASSERT_NO_FATAL_FAILURE(create_tablet(meta_service.get(), table_id, index_id, partition_id, + new_tablet_id, false, true)); + ASSERT_NO_FATAL_FAILURE(start_schema_change_job(meta_service.get(), table_id, index_id, + partition_id, tablet_id, new_tablet_id, + "job3", "be1")); + // provide existed rowsets + std::vector existed_rowsets; + for (int i = 0; i < 5; ++i) { + existed_rowsets.push_back(create_rowset(new_tablet_id, i + 11, i + 11)); + } + ASSERT_NO_FATAL_FAILURE(insert_rowsets(meta_service->txn_kv().get(), table_id, index_id, + partition_id, new_tablet_id, existed_rowsets)); + + std::vector output_rowsets; + output_rowsets.push_back(create_rowset(new_tablet_id, 2, 8)); + output_rowsets.push_back(create_rowset(new_tablet_id, 9, 12)); + output_rowsets.push_back(create_rowset(new_tablet_id, 13, 13)); + for (auto& rs : output_rowsets) { + CreateRowsetResponse res; + commit_rowset(meta_service.get(), rs, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << rs.end_version(); + } + + FinishTabletJobResponse res; + finish_schema_change_job(meta_service.get(), tablet_id, new_tablet_id, "job3", "be1", + output_rowsets, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_TRUE(res.has_stats()); + // overwrite [11-13], retain [14-15] + EXPECT_EQ(res.stats().num_rows(), 500); + EXPECT_EQ(res.stats().num_rowsets(), 6); + EXPECT_EQ(res.stats().num_segments(), 5); + EXPECT_EQ(res.stats().data_size(), 50000); + TabletStatsPB tablet_stats; + get_tablet_stats(meta_service.get(), new_tablet_id, tablet_stats); + EXPECT_EQ(tablet_stats.num_rows(), 500); + EXPECT_EQ(tablet_stats.num_rowsets(), 6); + EXPECT_EQ(tablet_stats.num_segments(), 5); + EXPECT_EQ(tablet_stats.data_size(), 50000); + + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + // check tablet state + auto tablet_key = + meta_tablet_key({instance_id, table_id, index_id, partition_id, new_tablet_id}); + std::string tablet_val; + ASSERT_EQ(txn->get(tablet_key, &tablet_val), TxnErrorCode::TXN_OK); + doris::TabletMetaCloudPB tablet_pb; + ASSERT_TRUE(tablet_pb.ParseFromString(tablet_val)); + ASSERT_EQ(tablet_pb.tablet_state(), doris::TabletStatePB::PB_RUNNING); + + // check visible rowsets + std::unique_ptr it; + auto rs_start = meta_rowset_key({instance_id, new_tablet_id, 0}); + auto rs_end = meta_rowset_key({instance_id, new_tablet_id, 100}); + ASSERT_EQ(txn->get(rs_start, rs_end, &it), TxnErrorCode::TXN_OK); + // [0-1][2-8][9-12][13-13][14-14][15-15] + EXPECT_EQ(it->size(), 6); + auto [k, v] = it->next(); + doris::RowsetMetaCloudPB saved_rowset; + ASSERT_TRUE(saved_rowset.ParseFromArray(v.data(), v.size())); + EXPECT_EQ(saved_rowset.start_version(), 0); + EXPECT_EQ(saved_rowset.end_version(), 1); + for (auto& rs : output_rowsets) { + auto [k, v] = it->next(); + ASSERT_TRUE(saved_rowset.ParseFromArray(v.data(), v.size())); + EXPECT_EQ(saved_rowset.start_version(), rs.start_version()); + EXPECT_EQ(saved_rowset.end_version(), rs.end_version()); + EXPECT_EQ(saved_rowset.rowset_id_v2(), rs.rowset_id_v2()); + } + for (int i = 3; i < 5; ++i) { // [14-14][15-15] + auto [k, v] = it->next(); + ASSERT_TRUE(saved_rowset.ParseFromArray(v.data(), v.size())); + EXPECT_EQ(saved_rowset.start_version(), existed_rowsets[i].start_version()); + EXPECT_EQ(saved_rowset.end_version(), existed_rowsets[i].end_version()); + EXPECT_EQ(saved_rowset.rowset_id_v2(), existed_rowsets[i].rowset_id_v2()); + } + + // check recycled rowsets + auto recycl_rs_start = recycle_rowset_key({instance_id, new_tablet_id, ""}); + auto recycl_rs_end = recycle_rowset_key({instance_id, new_tablet_id, "\xff"}); + ASSERT_EQ(txn->get(recycl_rs_start, recycl_rs_end, &it), TxnErrorCode::TXN_OK); + // [11-11], [12-12], old[13-13] + ASSERT_EQ(it->size(), 3); + for (int i = 0; i < 3; ++i) { + auto [k, v] = it->next(); + k.remove_prefix(1); + std::vector, int, int>> out; + decode_key(&k, &out); + // 0x01 "recycle" ${instance_id} "rowset" ${tablet_id} ${rowset_id} -> RecycleRowsetPB + const auto& rowset_id = std::get(std::get<0>(out[4])); + EXPECT_EQ(rowset_id, existed_rowsets[i].rowset_id_v2()); + } + } +} + +TEST(MetaServiceJobTest, RetrySchemaChangeJobTest) { + auto meta_service = get_meta_service(); + // meta_service->resource_mgr().reset(); // Do not use resource manager + + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + brpc::Controller cntl; + + int64_t table_id = 1; + int64_t index_id = 2; + int64_t partition_id = 3; + int64_t tablet_id = 4; + ASSERT_NO_FATAL_FAILURE( + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id, false)); + int64_t new_tablet_id = 14; + // start "job1" on BE1 + ASSERT_NO_FATAL_FAILURE(create_tablet(meta_service.get(), table_id, index_id, partition_id, + new_tablet_id, false, true)); + ASSERT_NO_FATAL_FAILURE(start_schema_change_job(meta_service.get(), table_id, index_id, + partition_id, tablet_id, new_tablet_id, "job1", + "be1")); + // provide existed rowsets + std::vector existed_rowsets; + for (int i = 0; i < 5; ++i) { + existed_rowsets.push_back(create_rowset(new_tablet_id, i + 11, i + 11)); + } + ASSERT_NO_FATAL_FAILURE(insert_rowsets(meta_service->txn_kv().get(), table_id, index_id, + partition_id, new_tablet_id, existed_rowsets)); + + // FE canceled "job1" and starts "job2" on BE1, should preempt previous "job1" + ASSERT_NO_FATAL_FAILURE(start_schema_change_job(meta_service.get(), table_id, index_id, + partition_id, tablet_id, new_tablet_id, "job2", + "be1")); + // retry "job2" on BE1 + ASSERT_NO_FATAL_FAILURE(start_schema_change_job(meta_service.get(), table_id, index_id, + partition_id, tablet_id, new_tablet_id, "job2", + "be1")); + // BE1 output_versions=[2-8][9-9][10-10][11-11] + std::vector be1_output_rowsets; + be1_output_rowsets.push_back(create_rowset(new_tablet_id, 2, 8)); + be1_output_rowsets.push_back(create_rowset(new_tablet_id, 9, 9)); + be1_output_rowsets.push_back(create_rowset(new_tablet_id, 10, 10)); + be1_output_rowsets.push_back(create_rowset(new_tablet_id, 11, 11)); + for (auto& rs : be1_output_rowsets) { + CreateRowsetResponse res; + commit_rowset(meta_service.get(), rs, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << rs.end_version(); + } + + // FE thinks BE1 is not alive and retries "job2" on BE2, should preempt "job2" created by BE1 + ASSERT_NO_FATAL_FAILURE(start_schema_change_job(meta_service.get(), table_id, index_id, + partition_id, tablet_id, new_tablet_id, "job2", + "be2")); + // BE2 output_versions=[2-8][9-12][13-13] + std::vector be2_output_rowsets; + { + CreateRowsetResponse res; + // [2-8] has committed by BE1 + commit_rowset(meta_service.get(), create_rowset(new_tablet_id, 2, 8), res); + ASSERT_EQ(res.status().code(), MetaServiceCode::ALREADY_EXISTED); + ASSERT_TRUE(res.has_existed_rowset_meta()); + ASSERT_EQ(res.existed_rowset_meta().rowset_id_v2(), be1_output_rowsets[0].rowset_id_v2()); + be2_output_rowsets.push_back(res.existed_rowset_meta()); + res.Clear(); + be2_output_rowsets.push_back(create_rowset(new_tablet_id, 9, 12)); + commit_rowset(meta_service.get(), be2_output_rowsets.back(), res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + res.Clear(); + be2_output_rowsets.push_back(create_rowset(new_tablet_id, 13, 13)); + commit_rowset(meta_service.get(), be2_output_rowsets.back(), res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // BE1 commit job, but check initiator failed + FinishTabletJobResponse res; + finish_schema_change_job(meta_service.get(), tablet_id, new_tablet_id, "job2", "be1", + be1_output_rowsets, res); + ASSERT_NE(res.status().code(), MetaServiceCode::OK); + ASSERT_NE(res.status().msg().find("unmatched job id or initiator"), std::string::npos); + // BE2 commit job + res.Clear(); + finish_schema_change_job(meta_service.get(), tablet_id, new_tablet_id, "job2", "be2", + be2_output_rowsets, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_TRUE(res.has_stats()); + // [0-1][2-8][9-12][13-13][14-14][15-15] + EXPECT_EQ(res.stats().num_rows(), 500); + EXPECT_EQ(res.stats().num_rowsets(), 6); + EXPECT_EQ(res.stats().num_segments(), 5); + EXPECT_EQ(res.stats().data_size(), 50000); + TabletStatsPB tablet_stats; + get_tablet_stats(meta_service.get(), new_tablet_id, tablet_stats); + EXPECT_EQ(tablet_stats.num_rows(), 500); + EXPECT_EQ(tablet_stats.num_rowsets(), 6); + EXPECT_EQ(tablet_stats.num_segments(), 5); + EXPECT_EQ(tablet_stats.data_size(), 50000); + + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + // check tablet state + auto tablet_key = + meta_tablet_key({instance_id, table_id, index_id, partition_id, new_tablet_id}); + std::string tablet_val; + ASSERT_EQ(txn->get(tablet_key, &tablet_val), TxnErrorCode::TXN_OK); + doris::TabletMetaCloudPB tablet_pb; + ASSERT_TRUE(tablet_pb.ParseFromString(tablet_val)); + ASSERT_EQ(tablet_pb.tablet_state(), doris::TabletStatePB::PB_RUNNING); + + // check visible rowsets + std::unique_ptr it; + auto rs_start = meta_rowset_key({instance_id, new_tablet_id, 0}); + auto rs_end = meta_rowset_key({instance_id, new_tablet_id, 100}); + ASSERT_EQ(txn->get(rs_start, rs_end, &it), TxnErrorCode::TXN_OK); + EXPECT_EQ(it->size(), 6); + auto [k, v] = it->next(); + doris::RowsetMetaCloudPB saved_rowset; + ASSERT_TRUE(saved_rowset.ParseFromArray(v.data(), v.size())); + EXPECT_EQ(saved_rowset.start_version(), 0); + EXPECT_EQ(saved_rowset.end_version(), 1); + for (auto& rs : be2_output_rowsets) { + auto [k, v] = it->next(); + ASSERT_TRUE(saved_rowset.ParseFromArray(v.data(), v.size())); + EXPECT_EQ(saved_rowset.start_version(), rs.start_version()); + EXPECT_EQ(saved_rowset.end_version(), rs.end_version()); + EXPECT_EQ(saved_rowset.rowset_id_v2(), rs.rowset_id_v2()); + } + for (int i = 3; i < 5; ++i) { // [14-14][15-15] + auto [k, v] = it->next(); + ASSERT_TRUE(saved_rowset.ParseFromArray(v.data(), v.size())); + EXPECT_EQ(saved_rowset.start_version(), existed_rowsets[i].start_version()); + EXPECT_EQ(saved_rowset.end_version(), existed_rowsets[i].end_version()); + EXPECT_EQ(saved_rowset.rowset_id_v2(), existed_rowsets[i].rowset_id_v2()); + } + + // check recycled rowsets + auto recycl_rs_start = recycle_rowset_key({instance_id, new_tablet_id, ""}); + auto recycl_rs_end = recycle_rowset_key({instance_id, new_tablet_id, "\xff"}); + ASSERT_EQ(txn->get(recycl_rs_start, recycl_rs_end, &it), TxnErrorCode::TXN_OK); + // [11-11], [12-12], old[13-13] + ASSERT_EQ(it->size(), 3); + for (int i = 0; i < 3; ++i) { + auto [k, v] = it->next(); + k.remove_prefix(1); + std::vector, int, int>> out; + decode_key(&k, &out); + // 0x01 "recycle" ${instance_id} "rowset" ${tablet_id} ${rowset_id} -> RecycleRowsetPB + const auto& rowset_id = std::get(std::get<0>(out[4])); + EXPECT_EQ(rowset_id, existed_rowsets[i].rowset_id_v2()); + } +} + +TEST(MetaServiceJobTest, SchemaChangeJobWithMoWTest) { + auto meta_service = get_meta_service(); + // meta_service->resource_mgr().reset(); // Do not use resource manager + + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + brpc::Controller cntl; + + int64_t table_id = 1; + int64_t index_id = 2; + int64_t partition_id = 3; + int64_t tablet_id = 4; + ASSERT_NO_FATAL_FAILURE( + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id, true)); + + { + int64_t new_tablet_id = 14; + ASSERT_NO_FATAL_FAILURE(create_tablet(meta_service.get(), table_id, index_id, partition_id, + new_tablet_id, true, true)); + ASSERT_NO_FATAL_FAILURE(start_schema_change_job(meta_service.get(), table_id, index_id, + partition_id, tablet_id, new_tablet_id, + "job1", "be1")); + std::vector output_rowsets; + for (int64_t i = 0; i < 5; ++i) { + output_rowsets.push_back(create_rowset(new_tablet_id, i + 2, i + 2)); + CreateRowsetResponse res; + commit_rowset(meta_service.get(), output_rowsets.back(), res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << i; + } + FinishTabletJobResponse res; + finish_schema_change_job(meta_service.get(), tablet_id, new_tablet_id, "job1", "be1", + output_rowsets, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::KV_TXN_GET_ERR); + res.Clear(); + + auto res_code = get_delete_bitmap_lock(meta_service.get(), table_id, -1, 2345); + ASSERT_EQ(res_code, MetaServiceCode::OK); + finish_schema_change_job(meta_service.get(), tablet_id, new_tablet_id, "job1", "be1", + output_rowsets, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::LOCK_EXPIRED); + ASSERT_NE(res.status().msg().find("lock id not match"), std::string::npos); + remove_delete_bitmap_lock(meta_service.get(), table_id); + res.Clear(); + + res_code = get_delete_bitmap_lock(meta_service.get(), table_id, -2, 2345); + ASSERT_EQ(res_code, MetaServiceCode::OK); + finish_schema_change_job(meta_service.get(), tablet_id, new_tablet_id, "job1", "be1", + output_rowsets, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::LOCK_EXPIRED); + ASSERT_NE(res.status().msg().find("lock initiator not exist"), std::string::npos); + remove_delete_bitmap_lock(meta_service.get(), table_id); + res.Clear(); + + res_code = get_delete_bitmap_lock(meta_service.get(), table_id, -2, 12345); + ASSERT_EQ(res_code, MetaServiceCode::OK); + finish_schema_change_job(meta_service.get(), tablet_id, new_tablet_id, "job1", "be1", + output_rowsets, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + remove_delete_bitmap_lock(meta_service.get(), table_id); + res.Clear(); + } + + { + int64_t new_tablet_id = 15; + ASSERT_NO_FATAL_FAILURE(create_tablet(meta_service.get(), table_id, index_id, partition_id, + new_tablet_id, true, true)); + ASSERT_NO_FATAL_FAILURE(start_schema_change_job(meta_service.get(), table_id, index_id, + partition_id, tablet_id, new_tablet_id, + "job2", "be1")); + std::vector output_rowsets; + for (int64_t i = 0; i < 5; ++i) { + output_rowsets.push_back(create_rowset(new_tablet_id, i + 2, i + 2)); + CreateRowsetResponse res; + commit_rowset(meta_service.get(), output_rowsets.back(), res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << i; + } + auto res_code = get_delete_bitmap_lock(meta_service.get(), table_id, -2, 12345); + ASSERT_EQ(res_code, MetaServiceCode::OK); + res_code = get_delete_bitmap_lock(meta_service.get(), table_id, -2, 12346); + ASSERT_EQ(res_code, MetaServiceCode::OK); + FinishTabletJobResponse res; + finish_schema_change_job(meta_service.get(), tablet_id, new_tablet_id, "job2", "be1", + output_rowsets, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + remove_delete_bitmap_lock(meta_service.get(), table_id); + res.Clear(); + } +} + +TEST(MetaServiceJobTest, ConcurrentCompactionTest) { + auto meta_service = get_meta_service(); + // meta_service->resource_mgr().reset(); // Do not use resource manager + + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + brpc::Controller cntl; + + int64_t table_id = 1; + int64_t index_id = 2; + int64_t partition_id = 3; + int64_t tablet_id = 4; + ASSERT_NO_FATAL_FAILURE( + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id, false)); + + StartTabletJobResponse res; + start_compaction_job(meta_service.get(), tablet_id, "job1", "BE1", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + res.Clear(); + start_compaction_job(meta_service.get(), tablet_id, "job1", "BE1", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); // Same job id, return OK + res.Clear(); + start_compaction_job(meta_service.get(), tablet_id, "job2", "BE1", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_TABLET_BUSY); + ::sleep(5); + res.Clear(); + start_compaction_job(meta_service.get(), tablet_id, "job3", "BE2", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + res.Clear(); + start_compaction_job(meta_service.get(), tablet_id, "job4", "BE1", 0, 0, + TabletCompactionJobPB::BASE, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + res.Clear(); + start_compaction_job(meta_service.get(), tablet_id, "job4", "BE1", 0, 0, + TabletCompactionJobPB::BASE, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); // Same job id, return OK + res.Clear(); + start_compaction_job(meta_service.get(), tablet_id, "job5", "BE1", 0, 0, + TabletCompactionJobPB::BASE, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_TABLET_BUSY); + + // check job kv + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string job_key = + job_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + std::string job_val; + TabletJobInfoPB job_pb; + ASSERT_EQ(txn->get(job_key, &job_val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(job_pb.ParseFromString(job_val)); + ASSERT_EQ(job_pb.compaction_size(), 2); + ASSERT_EQ(job_pb.compaction(0).id(), "job3"); + ASSERT_EQ(job_pb.compaction(0).initiator(), "BE2"); + ASSERT_EQ(job_pb.compaction(1).id(), "job4"); + ASSERT_EQ(job_pb.compaction(1).initiator(), "BE1"); + + // BE2 abort job3 + { + FinishTabletJobRequest req; + FinishTabletJobResponse res; + req.set_action(FinishTabletJobRequest::ABORT); + auto compaction = req.mutable_job()->add_compaction(); + compaction->set_id("job3"); + compaction->set_initiator("BE2"); + compaction->set_type(TabletCompactionJobPB::CUMULATIVE); + req.mutable_job()->mutable_idx()->set_tablet_id(tablet_id); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + ASSERT_NE(res.status().msg().find("unmatched job id"), std::string::npos); + } + + // BE1 lease job4 + long now = time(nullptr); + { + FinishTabletJobRequest req; + FinishTabletJobResponse res; + req.set_action(FinishTabletJobRequest::LEASE); + auto compaction = req.mutable_job()->add_compaction(); + compaction->set_id("job4"); + compaction->set_initiator("BE1"); + compaction->set_lease(now + 10); + req.mutable_job()->mutable_idx()->set_tablet_id(tablet_id); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(job_key, &job_val), TxnErrorCode::TXN_OK); + job_pb.Clear(); + ASSERT_TRUE(job_pb.ParseFromString(job_val)); + ASSERT_EQ(job_pb.compaction_size(), 1); + ASSERT_EQ(job_pb.compaction(0).id(), "job4"); + ASSERT_EQ(job_pb.compaction(0).initiator(), "BE1"); + ASSERT_EQ(job_pb.compaction(0).lease(), now + 10); + + res.Clear(); + start_compaction_job(meta_service.get(), tablet_id, "job5", "BE2", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(job_key, &job_val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(job_pb.ParseFromString(job_val)); + ASSERT_EQ(job_pb.compaction_size(), 2); + ASSERT_EQ(job_pb.compaction(0).id(), "job4"); + ASSERT_EQ(job_pb.compaction(0).initiator(), "BE1"); + ASSERT_EQ(job_pb.compaction(1).id(), "job5"); + ASSERT_EQ(job_pb.compaction(1).initiator(), "BE2"); + + // Provide existed rowsets + std::vector existed_rowsets; + for (int i = 0; i < 10; ++i) { // [2-11] + existed_rowsets.push_back(create_rowset(tablet_id, i + 2, i + 2)); + } + insert_rowsets(meta_service->txn_kv().get(), table_id, index_id, partition_id, tablet_id, + existed_rowsets); + + // BE2 commit job5 + { + // Provide output rowset + auto output_rowset = create_rowset(tablet_id, 5, 10); + CreateRowsetResponse rowset_res; + commit_rowset(meta_service.get(), output_rowset, rowset_res); + ASSERT_EQ(rowset_res.status().code(), MetaServiceCode::OK); + + FinishTabletJobRequest req; + FinishTabletJobResponse res; + req.set_action(FinishTabletJobRequest::COMMIT); + req.mutable_job()->mutable_idx()->set_tablet_id(tablet_id); + auto compaction = req.mutable_job()->add_compaction(); + compaction->set_id("job5"); + compaction->set_initiator("BE2"); + compaction->set_type(TabletCompactionJobPB::CUMULATIVE); + compaction->add_input_versions(5); + compaction->add_input_versions(10); + compaction->add_txn_id(output_rowset.txn_id()); + compaction->add_output_versions(10); + compaction->add_output_rowset_ids(output_rowset.rowset_id_v2()); + compaction->set_output_cumulative_point(11); + compaction->set_size_input_rowsets(60000); + compaction->set_num_input_rows(600); + compaction->set_num_input_rowsets(6); + compaction->set_num_input_segments(6); + compaction->set_size_output_rowsets(10000); + compaction->set_num_output_rows(100); + compaction->set_num_output_rowsets(1); + compaction->set_num_output_segments(1); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_TRUE(res.has_stats()); + EXPECT_EQ(res.stats().cumulative_point(), 11); + // [0-1][2][3][4][5-10][11] + EXPECT_EQ(res.stats().num_rows(), 500); + EXPECT_EQ(res.stats().num_rowsets(), 6); + EXPECT_EQ(res.stats().num_segments(), 5); + EXPECT_EQ(res.stats().data_size(), 50000); + TabletStatsPB tablet_stats; + get_tablet_stats(meta_service.get(), tablet_id, tablet_stats); + EXPECT_EQ(tablet_stats.num_rows(), 500); + EXPECT_EQ(tablet_stats.num_rowsets(), 6); + EXPECT_EQ(tablet_stats.num_segments(), 5); + EXPECT_EQ(tablet_stats.data_size(), 50000); + + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + // Check tmp rowsets + std::string tmp_rs_key, tmp_rs_val; + meta_rowset_tmp_key({instance_id, output_rowset.txn_id(), tablet_id}, &tmp_rs_val); + ASSERT_EQ(txn->get(tmp_rs_key, &tmp_rs_val), TxnErrorCode::TXN_KEY_NOT_FOUND); + // Check visible rowsets + std::unique_ptr it; + auto rs_start = meta_rowset_key({instance_id, table_id, 0}); + auto rs_end = meta_rowset_key({instance_id, tablet_id, 100}); + ASSERT_EQ(txn->get(rs_start, rs_end, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 6); + auto [k, v] = it->next(); // [0-1] + doris::RowsetMetaCloudPB visible_rowset; + ASSERT_TRUE(visible_rowset.ParseFromArray(v.data(), v.size())); + EXPECT_EQ(visible_rowset.start_version(), 0); + EXPECT_EQ(visible_rowset.end_version(), 1); + for (int i = 0; i < 3; ++i) { // [2][3][4] + std::tie(k, v) = it->next(); + ASSERT_TRUE(visible_rowset.ParseFromArray(v.data(), v.size())); + EXPECT_EQ(visible_rowset.start_version(), i + 2); + EXPECT_EQ(visible_rowset.end_version(), i + 2); + EXPECT_EQ(visible_rowset.rowset_id_v2(), existed_rowsets[i].rowset_id_v2()); + } + std::tie(k, v) = it->next(); // [5-10] + ASSERT_TRUE(visible_rowset.ParseFromArray(v.data(), v.size())); + EXPECT_EQ(visible_rowset.start_version(), 5); + EXPECT_EQ(visible_rowset.end_version(), 10); + EXPECT_EQ(visible_rowset.rowset_id_v2(), output_rowset.rowset_id_v2()); + std::tie(k, v) = it->next(); // [11] + ASSERT_TRUE(visible_rowset.ParseFromArray(v.data(), v.size())); + EXPECT_EQ(visible_rowset.start_version(), 11); + EXPECT_EQ(visible_rowset.end_version(), 11); + EXPECT_EQ(visible_rowset.rowset_id_v2(), existed_rowsets[9].rowset_id_v2()); + + // Check recycle rowsets + auto recycl_rs_start = recycle_rowset_key({instance_id, tablet_id, ""}); + auto recycl_rs_end = recycle_rowset_key({instance_id, tablet_id, "\xff"}); + ASSERT_EQ(txn->get(recycl_rs_start, recycl_rs_end, &it), TxnErrorCode::TXN_OK); + // [5][6][7][8][9][10] + ASSERT_EQ(it->size(), 6); + for (int i = 0; i < 6; ++i) { + auto [k, v] = it->next(); + k.remove_prefix(1); + std::vector, int, int>> out; + decode_key(&k, &out); + // 0x01 "recycle" ${instance_id} "rowset" ${tablet_id} ${rowset_id} -> RecycleRowsetPB + const auto& rowset_id = std::get(std::get<0>(out[4])); + EXPECT_EQ(rowset_id, existed_rowsets[i + 3].rowset_id_v2()); + } + } + + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(job_key, &job_val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(job_pb.ParseFromString(job_val)); + ASSERT_EQ(job_pb.compaction_size(), 1); + ASSERT_EQ(job_pb.compaction(0).id(), "job4"); + ASSERT_EQ(job_pb.compaction(0).initiator(), "BE1"); + + // BE1 commit job4 + { + // Provide output rowset + auto output_rowset = create_rowset(tablet_id, 2, 4); + CreateRowsetResponse rowset_res; + commit_rowset(meta_service.get(), output_rowset, rowset_res); + ASSERT_EQ(rowset_res.status().code(), MetaServiceCode::OK); + + FinishTabletJobRequest req; + FinishTabletJobResponse res; + req.set_action(FinishTabletJobRequest::COMMIT); + req.mutable_job()->mutable_idx()->set_tablet_id(tablet_id); + auto compaction = req.mutable_job()->add_compaction(); + compaction->set_id("job4"); + compaction->set_initiator("BE1"); + compaction->set_type(TabletCompactionJobPB::BASE); + compaction->add_input_versions(2); + compaction->add_input_versions(4); + compaction->add_txn_id(output_rowset.txn_id()); + compaction->add_output_versions(4); + compaction->add_output_rowset_ids(output_rowset.rowset_id_v2()); + compaction->set_output_cumulative_point(5); + compaction->set_size_input_rowsets(30000); + compaction->set_num_input_rows(300); + compaction->set_num_input_rowsets(3); + compaction->set_num_input_segments(3); + compaction->set_size_output_rowsets(10000); + compaction->set_num_output_rows(100); + compaction->set_num_output_rowsets(1); + compaction->set_num_output_segments(1); + meta_service->finish_tablet_job(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_TRUE(res.has_stats()); + EXPECT_EQ(res.stats().cumulative_point(), 11); + // [0-1][2-4][5-10][11] + EXPECT_EQ(res.stats().num_rows(), 300); + EXPECT_EQ(res.stats().num_rowsets(), 4); + EXPECT_EQ(res.stats().num_segments(), 3); + EXPECT_EQ(res.stats().data_size(), 30000); + TabletStatsPB tablet_stats; + get_tablet_stats(meta_service.get(), tablet_id, tablet_stats); + EXPECT_EQ(tablet_stats.num_rows(), 300); + EXPECT_EQ(tablet_stats.num_rowsets(), 4); + EXPECT_EQ(tablet_stats.num_segments(), 3); + EXPECT_EQ(tablet_stats.data_size(), 30000); + + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + // Check tmp rowsets + std::string tmp_rs_key, tmp_rs_val; + meta_rowset_tmp_key({instance_id, output_rowset.txn_id(), tablet_id}, &tmp_rs_val); + ASSERT_EQ(txn->get(tmp_rs_key, &tmp_rs_val), TxnErrorCode::TXN_KEY_NOT_FOUND); + // Check visible rowsets + std::unique_ptr it; + auto rs_start = meta_rowset_key({instance_id, table_id, 0}); + auto rs_end = meta_rowset_key({instance_id, tablet_id, 100}); + ASSERT_EQ(txn->get(rs_start, rs_end, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 4); + auto [k, v] = it->next(); + doris::RowsetMetaCloudPB visible_rowset; + ASSERT_TRUE(visible_rowset.ParseFromArray(v.data(), v.size())); + EXPECT_EQ(visible_rowset.start_version(), 0); + EXPECT_EQ(visible_rowset.end_version(), 1); + std::tie(k, v) = it->next(); + ASSERT_TRUE(visible_rowset.ParseFromArray(v.data(), v.size())); + EXPECT_EQ(visible_rowset.start_version(), 2); + EXPECT_EQ(visible_rowset.end_version(), 4); + EXPECT_EQ(visible_rowset.rowset_id_v2(), output_rowset.rowset_id_v2()); + std::tie(k, v) = it->next(); + ASSERT_TRUE(visible_rowset.ParseFromArray(v.data(), v.size())); + EXPECT_EQ(visible_rowset.start_version(), 5); + EXPECT_EQ(visible_rowset.end_version(), 10); + std::tie(k, v) = it->next(); + ASSERT_TRUE(visible_rowset.ParseFromArray(v.data(), v.size())); + EXPECT_EQ(visible_rowset.start_version(), 11); + EXPECT_EQ(visible_rowset.end_version(), 11); + EXPECT_EQ(visible_rowset.rowset_id_v2(), existed_rowsets[9].rowset_id_v2()); + + // Check recycle rowsets + auto recycl_rs_start = recycle_rowset_key({instance_id, tablet_id, ""}); + auto recycl_rs_end = recycle_rowset_key({instance_id, tablet_id, "\xff"}); + ASSERT_EQ(txn->get(recycl_rs_start, recycl_rs_end, &it), TxnErrorCode::TXN_OK); + // [2][3][4][5][6][7][8][9][10] + ASSERT_EQ(it->size(), 9); + for (int i = 0; i < 9; ++i) { + auto [k, v] = it->next(); + k.remove_prefix(1); + std::vector, int, int>> out; + decode_key(&k, &out); + // 0x01 "recycle" ${instance_id} "rowset" ${tablet_id} ${rowset_id} -> RecycleRowsetPB + const auto& rowset_id = std::get(std::get<0>(out[4])); + EXPECT_EQ(rowset_id, existed_rowsets[i].rowset_id_v2()); + } + } + + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(job_key, &job_val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(job_pb.ParseFromString(job_val)); + ASSERT_EQ(job_pb.compaction_size(), 0); +} + +TEST(MetaServiceJobTest, ParallelCumuCompactionTest) { + auto meta_service = get_meta_service(); + // meta_service->resource_mgr().reset(); // Do not use resource manager + + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + brpc::Controller cntl; + + int64_t table_id = 1; + int64_t index_id = 2; + int64_t partition_id = 3; + int64_t tablet_id = 4; + ASSERT_NO_FATAL_FAILURE( + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id, false)); + + StartTabletJobResponse res; + start_compaction_job(meta_service.get(), tablet_id, "job1", "BE1", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + res.Clear(); + start_compaction_job(meta_service.get(), tablet_id, "job2", "BE1", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_TABLET_BUSY); + // Don't return `version_in_compaction` to disable parallel cumu compaction + EXPECT_EQ(res.version_in_compaction_size(), 0); + res.Clear(); + start_compaction_job(meta_service.get(), tablet_id, "job3", "BE2", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, {5, 10}); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_TABLET_BUSY); + // Don't return `version_in_compaction` to disable parallel cumu compaction + EXPECT_EQ(res.version_in_compaction_size(), 0); + + ::sleep(5); // Wait for job1 expired + + res.Clear(); + start_compaction_job(meta_service.get(), tablet_id, "job4", "BE1", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, {5, 10}); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + res.Clear(); + start_compaction_job(meta_service.get(), tablet_id, "job5", "BE1", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, {12, 15}); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + res.Clear(); + start_compaction_job(meta_service.get(), tablet_id, "job6", "BE1", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, {6, 9}); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_TABLET_BUSY); + ASSERT_EQ(res.version_in_compaction_size(), 4); // [5-10][11-15] + EXPECT_EQ(res.version_in_compaction(0), 5); + EXPECT_EQ(res.version_in_compaction(1), 10); + EXPECT_EQ(res.version_in_compaction(2), 12); + EXPECT_EQ(res.version_in_compaction(3), 15); + res.Clear(); + start_compaction_job(meta_service.get(), tablet_id, "job7", "BE2", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, {5, 9}); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_TABLET_BUSY); + ASSERT_EQ(res.version_in_compaction_size(), 4); // [5-10][11-15] + EXPECT_EQ(res.version_in_compaction(0), 5); + EXPECT_EQ(res.version_in_compaction(1), 10); + EXPECT_EQ(res.version_in_compaction(2), 12); + EXPECT_EQ(res.version_in_compaction(3), 15); + res.Clear(); + start_compaction_job(meta_service.get(), tablet_id, "job8", "BE2", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, {6, 10}); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_TABLET_BUSY); + ASSERT_EQ(res.version_in_compaction_size(), 4); // [5-10][11-15] + EXPECT_EQ(res.version_in_compaction(0), 5); + EXPECT_EQ(res.version_in_compaction(1), 10); + EXPECT_EQ(res.version_in_compaction(2), 12); + EXPECT_EQ(res.version_in_compaction(3), 15); + res.Clear(); + start_compaction_job(meta_service.get(), tablet_id, "job9", "BE2", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, {4, 11}); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_TABLET_BUSY); + ASSERT_EQ(res.version_in_compaction_size(), 4); // [5-10][11-15] + EXPECT_EQ(res.version_in_compaction(0), 5); + EXPECT_EQ(res.version_in_compaction(1), 10); + EXPECT_EQ(res.version_in_compaction(2), 12); + EXPECT_EQ(res.version_in_compaction(3), 15); + res.Clear(); + start_compaction_job(meta_service.get(), tablet_id, "job10", "BE2", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, {3, 5}); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_TABLET_BUSY); + ASSERT_EQ(res.version_in_compaction_size(), 4); // [5-10][11-15] + EXPECT_EQ(res.version_in_compaction(0), 5); + EXPECT_EQ(res.version_in_compaction(1), 10); + EXPECT_EQ(res.version_in_compaction(2), 12); + EXPECT_EQ(res.version_in_compaction(3), 15); + res.Clear(); + start_compaction_job(meta_service.get(), tablet_id, "job11", "BE2", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, {10, 11}); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_TABLET_BUSY); + ASSERT_EQ(res.version_in_compaction_size(), 4); // [5-10][11-15] + EXPECT_EQ(res.version_in_compaction(0), 5); + EXPECT_EQ(res.version_in_compaction(1), 10); + EXPECT_EQ(res.version_in_compaction(2), 12); + EXPECT_EQ(res.version_in_compaction(3), 15); + res.Clear(); + start_compaction_job(meta_service.get(), tablet_id, "job12", "BE1", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, {2, 4}); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + res.Clear(); + start_compaction_job(meta_service.get(), tablet_id, "job13", "BE1", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::JOB_TABLET_BUSY); + // Don't return `version_in_compaction` to disable parallel cumu compaction + EXPECT_EQ(res.version_in_compaction_size(), 0); + res.Clear(); + start_compaction_job(meta_service.get(), tablet_id, "job14", "BE1", 0, 0, + TabletCompactionJobPB::CUMULATIVE, res, {11, 11}); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); +} + +} // namespace doris::cloud diff --git a/cloud/test/meta_service_test.cpp b/cloud/test/meta_service_test.cpp new file mode 100644 index 00000000000000..9a9dc3e72ec67d --- /dev/null +++ b/cloud/test/meta_service_test.cpp @@ -0,0 +1,5213 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "meta-service/meta_service.h" + +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +#include "common/config.h" +#include "common/logging.h" +#include "common/sync_point.h" +#include "common/util.h" +#include "meta-service/keys.h" +#include "meta-service/mem_txn_kv.h" +#include "meta-service/meta_service_helper.h" +#include "meta-service/txn_kv_error.h" +#include "mock_resource_manager.h" +#include "rate-limiter/rate_limiter.h" +#include "resource-manager/resource_manager.h" + +int main(int argc, char** argv) { + const std::string conf_file = "doris_cloud.conf"; + if (!doris::cloud::config::init(conf_file.c_str(), true)) { + std::cerr << "failed to init config file, conf=" << conf_file << std::endl; + return -1; + } + + if (!doris::cloud::init_glog("meta_service_test")) { + std::cerr << "failed to init glog" << std::endl; + return -1; + } + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + +namespace doris::cloud { + +std::unique_ptr get_meta_service(bool mock_resource_mgr) { + int ret = 0; + // MemKv + auto txn_kv = std::dynamic_pointer_cast(std::make_shared()); + if (txn_kv != nullptr) { + ret = txn_kv->init(); + [&] { ASSERT_EQ(ret, 0); }(); + } + [&] { ASSERT_NE(txn_kv.get(), nullptr); }(); + + // FdbKv + // config::fdb_cluster_file_path = "fdb.cluster"; + // static auto txn_kv = std::dynamic_pointer_cast(std::make_shared()); + // static std::atomic init {false}; + // bool tmp = false; + // if (init.compare_exchange_strong(tmp, true)) { + // int ret = txn_kv->init(); + // [&] { ASSERT_EQ(ret, 0); ASSERT_NE(txn_kv.get(), nullptr); }(); + // } + + std::unique_ptr txn; + EXPECT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->remove("\x00", "\xfe"); // This is dangerous if the fdb is not correctly set + EXPECT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + auto rs = mock_resource_mgr ? std::make_shared(txn_kv) + : std::make_shared(txn_kv); + auto rl = std::make_shared(); + auto meta_service = std::make_unique(txn_kv, rs, rl); + return std::make_unique(std::move(meta_service)); +} + +std::unique_ptr get_meta_service() { + return get_meta_service(true); +} + +static std::string next_rowset_id() { + static int cnt = 0; + return std::to_string(++cnt); +} + +static void add_tablet(CreateTabletsRequest& req, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id) { + auto tablet = req.add_tablet_metas(); + tablet->set_table_id(table_id); + tablet->set_index_id(index_id); + tablet->set_partition_id(partition_id); + tablet->set_tablet_id(tablet_id); + auto schema = tablet->mutable_schema(); + schema->set_schema_version(0); + auto first_rowset = tablet->add_rs_metas(); + first_rowset->set_rowset_id(0); // required + first_rowset->set_rowset_id_v2(next_rowset_id()); + first_rowset->set_start_version(0); + first_rowset->set_end_version(1); + first_rowset->mutable_tablet_schema()->CopyFrom(*schema); +} + +static void create_tablet(MetaServiceProxy* meta_service, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id) { + brpc::Controller cntl; + CreateTabletsRequest req; + CreateTabletsResponse res; + add_tablet(req, table_id, index_id, partition_id, tablet_id); + meta_service->create_tablets(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << tablet_id; +} + +static void begin_txn(MetaServiceProxy* meta_service, int64_t db_id, const std::string& label, + int64_t table_id, int64_t& txn_id) { + brpc::Controller cntl; + BeginTxnRequest req; + BeginTxnResponse res; + auto txn_info = req.mutable_txn_info(); + txn_info->set_db_id(db_id); + txn_info->set_label(label); + txn_info->add_table_ids(table_id); + txn_info->set_timeout_ms(36000); + meta_service->begin_txn(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + ASSERT_TRUE(res.has_txn_id()) << label; + txn_id = res.txn_id(); +} + +static void commit_txn(MetaServiceProxy* meta_service, int64_t db_id, int64_t txn_id, + const std::string& label) { + brpc::Controller cntl; + CommitTxnRequest req; + CommitTxnResponse res; + req.set_db_id(db_id); + req.set_txn_id(txn_id); + meta_service->commit_txn(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; +} + +static doris::RowsetMetaCloudPB create_rowset(int64_t txn_id, int64_t tablet_id, + int partition_id = 0, int64_t version = -1, + int num_rows = 100) { + doris::RowsetMetaCloudPB rowset; + rowset.set_rowset_id(0); // required + rowset.set_rowset_id_v2(next_rowset_id()); + rowset.set_tablet_id(tablet_id); + rowset.set_partition_id(partition_id); + rowset.set_txn_id(txn_id); + if (version > 0) { + rowset.set_start_version(version); + rowset.set_end_version(version); + } + rowset.set_num_segments(1); + rowset.set_num_rows(num_rows); + rowset.set_data_disk_size(num_rows * 100); + rowset.mutable_tablet_schema()->set_schema_version(0); + rowset.set_txn_expiration(::time(nullptr)); // Required by DCHECK + return rowset; +} + +static void prepare_rowset(MetaServiceProxy* meta_service, const doris::RowsetMetaCloudPB& rowset, + CreateRowsetResponse& res) { + brpc::Controller cntl; + auto arena = res.GetArena(); + auto req = google::protobuf::Arena::CreateMessage(arena); + req->set_temporary(true); + req->mutable_rowset_meta()->CopyFrom(rowset); + meta_service->prepare_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; +} + +static void commit_rowset(MetaServiceProxy* meta_service, const doris::RowsetMetaCloudPB& rowset, + CreateRowsetResponse& res) { + brpc::Controller cntl; + auto arena = res.GetArena(); + auto req = google::protobuf::Arena::CreateMessage(arena); + req->set_temporary(true); + req->mutable_rowset_meta()->CopyFrom(rowset); + meta_service->commit_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; +} + +static void update_tmp_rowset(MetaServiceProxy* meta_service, + const doris::RowsetMetaCloudPB& rowset, CreateRowsetResponse& res) { + brpc::Controller cntl; + auto arena = res.GetArena(); + auto req = google::protobuf::Arena::CreateMessage(arena); + req->mutable_rowset_meta()->CopyFrom(rowset); + meta_service->update_tmp_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; +} + +static void insert_rowset(MetaServiceProxy* meta_service, int64_t db_id, const std::string& label, + int64_t table_id, int64_t partition_id, int64_t tablet_id) { + int64_t txn_id = 0; + ASSERT_NO_FATAL_FAILURE(begin_txn(meta_service, db_id, label, table_id, txn_id)); + CreateRowsetResponse res; + auto rowset = create_rowset(txn_id, tablet_id, partition_id); + prepare_rowset(meta_service, rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + res.Clear(); + ASSERT_NO_FATAL_FAILURE(commit_rowset(meta_service, rowset, res)); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + commit_txn(meta_service, db_id, txn_id, label); +} + +TEST(MetaServiceTest, GetInstanceIdTest) { + extern std::string get_instance_id(const std::shared_ptr& rc_mgr, + const std::string& cloud_unique_id); + auto meta_service = get_meta_service(); + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id_err", [&](void* args) { + std::string* err = reinterpret_cast(args); + *err = "can't find node from cache"; + }); + sp->enable_processing(); + + auto instance_id = + get_instance_id(meta_service->resource_mgr(), "1:ALBJLH4Q:m-n3qdpyal27rh8iprxx"); + ASSERT_EQ(instance_id, "ALBJLH4Q"); + + // version not support + instance_id = get_instance_id(meta_service->resource_mgr(), "2:ALBJLH4Q:m-n3qdpyal27rh8iprxx"); + ASSERT_EQ(instance_id, ""); + + // degraded format err + instance_id = get_instance_id(meta_service->resource_mgr(), "1:ALBJLH4Q"); + ASSERT_EQ(instance_id, ""); + + // std::invalid_argument + instance_id = get_instance_id(meta_service->resource_mgr(), + "invalid_version:ALBJLH4Q:m-n3qdpyal27rh8iprxx"); + ASSERT_EQ(instance_id, ""); + + // std::out_of_range + instance_id = get_instance_id(meta_service->resource_mgr(), + "12345678901:ALBJLH4Q:m-n3qdpyal27rh8iprxx"); + ASSERT_EQ(instance_id, ""); + + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); +} + +TEST(MetaServiceTest, CreateInstanceTest) { + auto meta_service = get_meta_service(); + + // case: normal create instance + { + brpc::Controller cntl; + CreateInstanceRequest req; + req.set_instance_id("test_instance"); + req.set_user_id("test_user"); + req.set_name("test_name"); + ObjectStoreInfoPB obj; + obj.set_ak("123"); + obj.set_sk("321"); + obj.set_bucket("456"); + obj.set_prefix("654"); + obj.set_endpoint("789"); + obj.set_region("987"); + obj.set_external_endpoint("888"); + obj.set_provider(ObjectStoreInfoPB::BOS); + req.mutable_obj_info()->CopyFrom(obj); + + auto sp = SyncPoint::get_instance(); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_id", + [](void* p) { *reinterpret_cast(p) = 1; }); + sp->enable_processing(); + CreateInstanceResponse res; + meta_service->create_instance(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); + } + + // case: request has invalid argument + { + brpc::Controller cntl; + CreateInstanceRequest req; + CreateInstanceResponse res; + meta_service->create_instance(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + } + + // case: normal drop instance + { + brpc::Controller cntl; + AlterInstanceRequest req; + AlterInstanceResponse res; + req.set_op(AlterInstanceRequest::DROP); + req.set_instance_id("test_instance"); + meta_service->alter_instance(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + InstanceKeyInfo key_info {"test_instance"}; + std::string key; + std::string val; + instance_key(key_info, &key); + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(key, &val), TxnErrorCode::TXN_OK); + InstanceInfoPB instance; + instance.ParseFromString(val); + ASSERT_EQ(instance.status(), InstanceInfoPB::DELETED); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // case: normal refresh instance + { + brpc::Controller cntl; + AlterInstanceRequest req; + AlterInstanceResponse res; + req.set_op(AlterInstanceRequest::REFRESH); + req.set_instance_id("test_instance"); + meta_service->alter_instance(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // case: rpc get instance + { + brpc::Controller cntl; + GetInstanceRequest req; + GetInstanceResponse res; + meta_service->get_instance(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + req.set_cloud_unique_id("1:test_instance:m-n3qdpyal27rh8iprxx"); + meta_service->get_instance(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } +} + +TEST(MetaServiceTest, AlterClusterTest) { + auto meta_service = get_meta_service(); + ASSERT_NE(meta_service, nullptr); + + // case: normal add cluster + { + brpc::Controller cntl; + AlterClusterRequest req; + req.set_instance_id(mock_instance); + req.mutable_cluster()->set_cluster_name(mock_cluster_name); + req.set_op(AlterClusterRequest::ADD_CLUSTER); + AlterClusterResponse res; + meta_service->alter_cluster(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // case: request has invalid argument + { + brpc::Controller cntl; + AlterClusterRequest req; + req.set_op(AlterClusterRequest::DROP_CLUSTER); + AlterClusterResponse res; + meta_service->alter_cluster(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + } + + // add node + { + brpc::Controller cntl; + AlterClusterRequest req; + req.set_instance_id(mock_instance); + req.set_op(AlterClusterRequest::ADD_NODE); + req.mutable_cluster()->set_cluster_name(mock_cluster_name); + req.mutable_cluster()->set_cluster_id(mock_cluster_id); + req.mutable_cluster()->set_type(ClusterPB::COMPUTE); + auto node = req.mutable_cluster()->add_nodes(); + node->set_ip("127.0.0.1"); + node->set_heartbeat_port(9999); + AlterClusterResponse res; + meta_service->alter_cluster(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // drop node + { + brpc::Controller cntl; + AlterClusterRequest req; + req.set_instance_id(mock_instance); + req.set_op(AlterClusterRequest::DROP_NODE); + req.mutable_cluster()->set_cluster_name(mock_cluster_name); + req.mutable_cluster()->set_cluster_id(mock_cluster_id); + req.mutable_cluster()->set_type(ClusterPB::COMPUTE); + auto node = req.mutable_cluster()->add_nodes(); + node->set_ip("127.0.0.1"); + node->set_heartbeat_port(9999); + AlterClusterResponse res; + meta_service->alter_cluster(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // rename cluster + { + brpc::Controller cntl; + AlterClusterRequest req; + req.set_instance_id(mock_instance); + req.mutable_cluster()->set_cluster_id(mock_cluster_id); + req.mutable_cluster()->set_cluster_name("rename_cluster_name"); + req.set_op(AlterClusterRequest::RENAME_CLUSTER); + AlterClusterResponse res; + meta_service->alter_cluster(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // set cluster status + { + brpc::Controller cntl; + AlterClusterRequest req; + req.set_instance_id(mock_instance); + req.mutable_cluster()->set_cluster_id(mock_cluster_id); + req.mutable_cluster()->set_cluster_status(ClusterStatus::SUSPENDED); + req.set_op(AlterClusterRequest::SET_CLUSTER_STATUS); + AlterClusterResponse res; + meta_service->alter_cluster(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // set UPDATE_CLUSTER_MYSQL_USER_NAME + { + brpc::Controller cntl; + AlterClusterRequest req; + req.set_instance_id(mock_instance); + req.mutable_cluster()->set_cluster_id(mock_cluster_id); + req.mutable_cluster()->add_mysql_user_name("test_user"); + req.set_op(AlterClusterRequest::UPDATE_CLUSTER_MYSQL_USER_NAME); + AlterClusterResponse res; + meta_service->alter_cluster(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } +} + +TEST(MetaServiceTest, GetClusterTest) { + auto meta_service = get_meta_service(); + + // add cluster first + InstanceKeyInfo key_info {mock_instance}; + std::string key; + std::string val; + instance_key(key_info, &key); + + InstanceInfoPB instance; + instance.set_instance_id(mock_instance); + ClusterPB c1; + c1.set_cluster_name(mock_cluster_name); + c1.set_cluster_id(mock_cluster_id); + c1.add_mysql_user_name()->append("m1"); + instance.add_clusters()->CopyFrom(c1); + val = instance.SerializeAsString(); + + std::unique_ptr txn; + std::string get_val; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + // case: normal get + { + brpc::Controller cntl; + GetClusterRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_cluster_id(mock_cluster_id); + req.set_cluster_name("test_cluster"); + GetClusterResponse res; + meta_service->get_cluster(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } +} + +TEST(MetaServiceTest, BeginTxnTest) { + auto meta_service = get_meta_service(); + int64_t db_id = 666; + int64_t table_id = 123; + const std::string& label = "test_label"; + int64_t timeout_ms = 60 * 1000; + + // test invalid argument + { + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + } + + { + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + + TxnInfoPB txn_info; + txn_info.set_db_id(db_id); + txn_info.add_table_ids(table_id); + txn_info.set_timeout_ms(timeout_ms); + req.mutable_txn_info()->CopyFrom(txn_info); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + } + + { + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + + TxnInfoPB txn_info; + txn_info.set_db_id(db_id); + txn_info.set_label(label); + txn_info.add_table_ids(table_id); + txn_info.set_timeout_ms(timeout_ms); + req.mutable_txn_info()->CopyFrom(txn_info); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // case: label already used + { + brpc::Controller cntl; + BeginTxnRequest req; + auto label_already_in_use = "test_label_already_in_use"; + + req.set_cloud_unique_id("test_cloud_unique_id"); + TxnInfoPB txn_info; + txn_info.set_db_id(888); + txn_info.set_label(label_already_in_use); + txn_info.add_table_ids(456); + txn_info.set_timeout_ms(36000); + req.mutable_txn_info()->CopyFrom(txn_info); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::TXN_LABEL_ALREADY_USED); + auto found = res.status().msg().find(fmt::format( + "Label [{}] has already been used, relate to txn", label_already_in_use)); + ASSERT_NE(found, std::string::npos); + } + + // case: dup begin txn request + { + brpc::Controller cntl; + BeginTxnRequest req; + + req.set_cloud_unique_id("test_cloud_unique_id"); + TxnInfoPB txn_info; + txn_info.set_db_id(999); + txn_info.set_label("test_label_dup_request"); + txn_info.add_table_ids(789); + UniqueIdPB unique_id_pb; + unique_id_pb.set_hi(100); + unique_id_pb.set_lo(10); + txn_info.mutable_request_id()->CopyFrom(unique_id_pb); + txn_info.set_timeout_ms(36000); + req.mutable_txn_info()->CopyFrom(txn_info); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::TXN_DUPLICATED_REQ); + } + + { + // =========================================================================== + // threads concurrent execution with sequence in begin_txn with same label: + // + // thread1 thread2 + // | | + // | commit_txn1 + // | | + // | | + // | | + // commit_txn2 | + // | | + // v v + // + + std::mutex go_mutex; + std::condition_variable go_cv; + bool go = false; + auto sp = cloud::SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + + std::atomic count_txn1 = {0}; + std::atomic count_txn2 = {0}; + std::atomic count_txn3 = {0}; + + int64_t db_id = 1928354123; + int64_t table_id = 12131231231; + std::string test_label = "test_race_with_same_label"; + + std::atomic success_txn = {0}; + + sp->set_call_back("begin_txn:before:commit_txn:1", [&](void* args) { + std::string label = *reinterpret_cast(args); + std::unique_lock _lock(go_mutex); + count_txn1++; + LOG(INFO) << "count_txn1:" << count_txn1 << " label=" << label; + if (count_txn1 == 1) { + { + LOG(INFO) << "count_txn1:" << count_txn1 << " label=" << label << " go=" << go; + go_cv.wait(_lock); + } + } + + if (count_txn1 == 2) { + { + LOG(INFO) << "count_txn1:" << count_txn1 << " label=" << label << " go=" << go; + go_cv.notify_all(); + } + } + }); + + sp->set_call_back("begin_txn:after:commit_txn:1", [&](void* args) { + std::string label = *reinterpret_cast(args); + std::unique_lock _lock(go_mutex); + count_txn2++; + LOG(INFO) << "count_txn2:" << count_txn2 << " label=" << label; + if (count_txn2 == 1) { + { + LOG(INFO) << "count_txn2:" << count_txn2 << " label=" << label << " go=" << go; + go_cv.wait(_lock); + } + } + + if (count_txn2 == 2) { + { + LOG(INFO) << "count_txn2:" << count_txn2 << " label=" << label << " go=" << go; + go_cv.notify_all(); + } + } + }); + + sp->set_call_back("begin_txn:after:commit_txn:2", [&](void* args) { + int64_t txn_id = *reinterpret_cast(args); + count_txn3++; + LOG(INFO) << "count_txn3:" << count_txn3 << " txn_id=" << txn_id; + }); + + sp->enable_processing(); + + std::thread thread1([&] { + { + std::unique_lock _lock(go_mutex); + go_cv.wait(_lock, [&] { return go; }); + } + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + TxnInfoPB txn_info; + txn_info.set_db_id(db_id); + txn_info.set_label(test_label); + txn_info.add_table_ids(table_id); + UniqueIdPB unique_id_pb; + unique_id_pb.set_hi(1001); + unique_id_pb.set_lo(11); + txn_info.mutable_request_id()->CopyFrom(unique_id_pb); + txn_info.set_timeout_ms(36000); + req.mutable_txn_info()->CopyFrom(txn_info); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + if (res.status().code() == MetaServiceCode::OK) { + success_txn++; + } else { + ASSERT_EQ(res.status().code(), MetaServiceCode::KV_TXN_CONFLICT); + } + }); + + std::thread thread2([&] { + { + std::unique_lock _lock(go_mutex); + go_cv.wait(_lock, [&] { return go; }); + } + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + TxnInfoPB txn_info; + txn_info.set_db_id(db_id); + txn_info.set_label(test_label); + txn_info.add_table_ids(table_id); + UniqueIdPB unique_id_pb; + unique_id_pb.set_hi(100); + unique_id_pb.set_lo(10); + txn_info.mutable_request_id()->CopyFrom(unique_id_pb); + txn_info.set_timeout_ms(36000); + req.mutable_txn_info()->CopyFrom(txn_info); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + if (res.status().code() == MetaServiceCode::OK) { + success_txn++; + } else { + ASSERT_EQ(res.status().code(), MetaServiceCode::KV_TXN_CONFLICT); + } + }); + + std::unique_lock go_lock(go_mutex); + go = true; + go_lock.unlock(); + go_cv.notify_all(); + + thread1.join(); + thread2.join(); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); + ASSERT_EQ(success_txn.load(), 1); + } + { + // =========================================================================== + // threads concurrent execution with sequence in begin_txn with different label: + // + // thread1 thread2 + // | | + // | commit_txn1 + // | | + // | | + // | | + // commit_txn2 | + // | | + // v v + + std::mutex go_mutex; + std::condition_variable go_cv; + bool go = false; + auto sp = cloud::SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + + std::atomic count_txn1 = {0}; + std::atomic count_txn2 = {0}; + std::mutex flow_mutex_1; + std::condition_variable flow_cv_1; + + int64_t db_id = 19541231112; + int64_t table_id = 312312321211; + std::string test_label1 = "test_race_with_diff_label1"; + std::string test_label2 = "test_race_with_diff_label2"; + + std::atomic success_txn = {0}; + + sp->set_call_back("begin_txn:before:commit_txn:1", [&](void* args) { + std::string label = *reinterpret_cast(args); + if (count_txn1.load() == 1) { + std::unique_lock flow_lock_1(flow_mutex_1); + flow_cv_1.wait(flow_lock_1); + } + count_txn1++; + LOG(INFO) << "count_txn1:" << count_txn1 << " label=" << label; + }); + + sp->set_call_back("begin_txn:after:commit_txn:2", [&](void* args) { + int64_t txn_id = *reinterpret_cast(args); + while (count_txn2.load() == 0 && count_txn1.load() == 1) { + sleep(1); + flow_cv_1.notify_all(); + } + count_txn2++; + LOG(INFO) << "count_txn2:" << count_txn2 << " txn_id=" << txn_id; + }); + sp->enable_processing(); + + std::thread thread1([&] { + { + std::unique_lock _lock(go_mutex); + go_cv.wait(_lock, [&] { return go; }); + } + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + TxnInfoPB txn_info; + txn_info.set_db_id(db_id); + txn_info.set_label(test_label1); + txn_info.add_table_ids(table_id); + UniqueIdPB unique_id_pb; + unique_id_pb.set_hi(1001); + unique_id_pb.set_lo(11); + txn_info.mutable_request_id()->CopyFrom(unique_id_pb); + txn_info.set_timeout_ms(36000); + req.mutable_txn_info()->CopyFrom(txn_info); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + if (res.status().code() == MetaServiceCode::OK) { + success_txn++; + } else { + ASSERT_EQ(res.status().code(), MetaServiceCode::TXN_LABEL_ALREADY_USED); + } + }); + + std::thread thread2([&] { + { + std::unique_lock _lock(go_mutex); + go_cv.wait(_lock, [&] { return go; }); + } + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + TxnInfoPB txn_info; + txn_info.set_db_id(db_id); + txn_info.set_label(test_label2); + txn_info.add_table_ids(table_id); + txn_info.set_timeout_ms(36000); + UniqueIdPB unique_id_pb; + unique_id_pb.set_hi(100); + unique_id_pb.set_lo(10); + txn_info.mutable_request_id()->CopyFrom(unique_id_pb); + req.mutable_txn_info()->CopyFrom(txn_info); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + if (res.status().code() == MetaServiceCode::OK) { + success_txn++; + } else { + ASSERT_EQ(res.status().code(), MetaServiceCode::TXN_LABEL_ALREADY_USED); + } + }); + + std::unique_lock go_lock(go_mutex); + go = true; + go_lock.unlock(); + go_cv.notify_all(); + + thread1.join(); + thread2.join(); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); + ASSERT_EQ(success_txn.load(), 2); + } + { + // test reuse label + // 1. beigin_txn + // 2. abort_txn + // 3. begin_txn again can successfully + + std::string cloud_unique_id = "test_cloud_unique_id"; + int64_t db_id = 124343989; + int64_t table_id = 1231311; + int64_t txn_id = -1; + std::string label = "test_reuse_label"; + { + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id(cloud_unique_id); + TxnInfoPB txn_info; + txn_info.set_db_id(db_id); + txn_info.set_label(label); + txn_info.add_table_ids(table_id); + txn_info.set_timeout_ms(36000); + UniqueIdPB unique_id_pb; + unique_id_pb.set_hi(100); + unique_id_pb.set_lo(10); + txn_info.mutable_request_id()->CopyFrom(unique_id_pb); + req.mutable_txn_info()->CopyFrom(txn_info); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + } + // abort txn + { + brpc::Controller cntl; + AbortTxnRequest req; + req.set_cloud_unique_id(cloud_unique_id); + ASSERT_GT(txn_id, 0); + req.set_txn_id(txn_id); + req.set_reason("test"); + AbortTxnResponse res; + meta_service->abort_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.txn_info().status(), TxnStatusPB::TXN_STATUS_ABORTED); + } + { + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id(cloud_unique_id); + TxnInfoPB txn_info; + txn_info.set_db_id(db_id); + txn_info.set_label(label); + txn_info.add_table_ids(table_id); + UniqueIdPB unique_id_pb; + unique_id_pb.set_hi(100); + unique_id_pb.set_lo(10); + txn_info.mutable_request_id()->CopyFrom(unique_id_pb); + txn_info.set_timeout_ms(36000); + req.mutable_txn_info()->CopyFrom(txn_info); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_GT(res.txn_id(), txn_id); + } + } +} + +TEST(MetaServiceTest, PrecommitTest1) { + // PrecommitTestCase1: only use db_id for precommit_txn + auto meta_service = get_meta_service(); + const int64_t db_id = 563413; + const int64_t table_id = 417417878; + const std::string& label = "label_123dae121das"; + int64_t txn_id = -1; + { + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + TxnInfoPB txn_info; + txn_info.set_db_id(db_id); + txn_info.set_label(label); + txn_info.add_table_ids(table_id); + txn_info.set_timeout_ms(36000); + req.mutable_txn_info()->CopyFrom(txn_info); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + ASSERT_GT(txn_id, -1); + } + + { + brpc::Controller cntl; + PrecommitTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_precommit_timeout_ms(36000); + PrecommitTxnResponse res; + meta_service->precommit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + } + + { + std::unique_ptr txn; + TxnErrorCode err = meta_service->txn_kv()->create_txn(&txn); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + + const std::string info_key = txn_info_key({mock_instance, db_id, txn_id}); + std::string info_val; + ASSERT_EQ(txn->get(info_key, &info_val), TxnErrorCode::TXN_OK); + TxnInfoPB txn_info; + txn_info.ParseFromString(info_val); + ASSERT_EQ(txn_info.status(), TxnStatusPB::TXN_STATUS_PREPARED); + + brpc::Controller cntl; + PrecommitTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_txn_id(txn_id); + req.set_precommit_timeout_ms(36000); + PrecommitTxnResponse res; + meta_service->precommit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + err = meta_service->txn_kv()->create_txn(&txn); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(info_key, &info_val), TxnErrorCode::TXN_OK); + txn_info.ParseFromString(info_val); + ASSERT_EQ(txn_info.status(), TxnStatusPB::TXN_STATUS_PRECOMMITTED); + } +} + +TEST(MetaServiceTest, PrecommitTxnTest2) { + auto meta_service = get_meta_service(); + const int64_t db_id = 563413; + const int64_t table_id = 417417878; + const std::string& label = "label_123dae121das"; + int64_t txn_id = -1; + // begin txn first + { + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + TxnInfoPB txn_info; + txn_info.set_db_id(db_id); + txn_info.set_label(label); + txn_info.add_table_ids(table_id); + txn_info.set_timeout_ms(36000); + req.mutable_txn_info()->CopyFrom(txn_info); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + ASSERT_GT(txn_id, -1); + } + + // case: txn's status should be TXN_STATUS_PRECOMMITTED + { + std::unique_ptr txn; + TxnErrorCode err = meta_service->txn_kv()->create_txn(&txn); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + + const std::string info_key = txn_info_key({mock_instance, db_id, txn_id}); + std::string info_val; + ASSERT_EQ(txn->get(info_key, &info_val), TxnErrorCode::TXN_OK); + TxnInfoPB txn_info; + txn_info.ParseFromString(info_val); + // before call precommit_txn, txn's status is TXN_STATUS_PREPARED + ASSERT_EQ(txn_info.status(), TxnStatusPB::TXN_STATUS_PREPARED); + + brpc::Controller cntl; + PrecommitTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(db_id); + req.set_txn_id(txn_id); + req.set_precommit_timeout_ms(36000); + PrecommitTxnResponse res; + meta_service->precommit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + err = meta_service->txn_kv()->create_txn(&txn); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(info_key, &info_val), TxnErrorCode::TXN_OK); + txn_info.ParseFromString(info_val); + // after call precommit_txn, txn's status is TXN_STATUS_PRECOMMITTED + ASSERT_EQ(txn_info.status(), TxnStatusPB::TXN_STATUS_PRECOMMITTED); + } + + // case: when txn's status is TXN_STATUS_ABORTED/TXN_STATUS_VISIBLE/TXN_STATUS_PRECOMMITTED + { + // TXN_STATUS_ABORTED + std::unique_ptr txn; + TxnErrorCode err = meta_service->txn_kv()->create_txn(&txn); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + + const std::string info_key = txn_info_key({mock_instance, db_id, txn_id}); + std::string info_val; + ASSERT_EQ(txn->get(info_key, &info_val), TxnErrorCode::TXN_OK); + TxnInfoPB txn_info; + txn_info.ParseFromString(info_val); + txn_info.set_status(TxnStatusPB::TXN_STATUS_ABORTED); + info_val.clear(); + txn_info.SerializeToString(&info_val); + txn->put(info_key, info_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + brpc::Controller cntl; + PrecommitTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(db_id); + req.set_txn_id(txn_id); + req.set_precommit_timeout_ms(36000); + PrecommitTxnResponse res; + meta_service->precommit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::TXN_ALREADY_ABORTED); + + // TXN_STATUS_VISIBLE + txn_info.set_status(TxnStatusPB::TXN_STATUS_VISIBLE); + info_val.clear(); + txn_info.SerializeToString(&info_val); + txn->put(info_key, info_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + meta_service->precommit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::TXN_ALREADY_VISIBLE); + + // TXN_STATUS_PRECOMMITTED + txn_info.set_status(TxnStatusPB::TXN_STATUS_PRECOMMITTED); + info_val.clear(); + txn_info.SerializeToString(&info_val); + txn->put(info_key, info_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + meta_service->precommit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::TXN_ALREADY_PRECOMMITED); + } +} + +TEST(MetaServiceTest, CommitTxnTest) { + auto meta_service = get_meta_service(); + // case: first version of rowset + { + int64_t txn_id = -1; + // begin txn + { + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + TxnInfoPB txn_info_pb; + txn_info_pb.set_db_id(666); + txn_info_pb.set_label("test_label"); + txn_info_pb.add_table_ids(1234); + txn_info_pb.set_timeout_ms(36000); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + } + + // mock rowset and tablet + int64_t tablet_id_base = 1103; + for (int i = 0; i < 5; ++i) { + create_tablet(meta_service.get(), 1234, 1235, 1236, tablet_id_base + i); + auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i); + CreateRowsetResponse res; + commit_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // precommit txn + { + brpc::Controller cntl; + PrecommitTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(666); + req.set_txn_id(txn_id); + req.set_precommit_timeout_ms(36000); + PrecommitTxnResponse res; + meta_service->precommit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // commit txn + { + brpc::Controller cntl; + CommitTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(666); + req.set_txn_id(txn_id); + CommitTxnResponse res; + meta_service->commit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // doubly commit txn + { + brpc::Controller cntl; + CommitTxnRequest req; + auto db_id = 666; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(db_id); + req.set_txn_id(txn_id); + CommitTxnResponse res; + meta_service->commit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::TXN_ALREADY_VISIBLE); + auto found = res.status().msg().find(fmt::format( + "transaction is already visible: db_id={} txn_id={}", db_id, txn_id)); + ASSERT_TRUE(found != std::string::npos); + } + + // doubly commit txn(2pc) + { + brpc::Controller cntl; + CommitTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(666); + req.set_txn_id(txn_id); + req.set_is_2pc(true); + CommitTxnResponse res; + meta_service->commit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::TXN_ALREADY_VISIBLE); + auto found = res.status().msg().find( + fmt::format("transaction [{}] is already visible, not pre-committed.", txn_id)); + ASSERT_TRUE(found != std::string::npos); + } + } +} + +TEST(MetaServiceTest, CommitTxnExpiredTest) { + auto meta_service = get_meta_service(); + + // case: first version of rowset + { + int64_t txn_id = -1; + int64_t db_id = 713232132; + // begin txn + { + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + TxnInfoPB txn_info_pb; + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label("test_commit_txn_expired"); + txn_info_pb.add_table_ids(1234789234); + txn_info_pb.set_timeout_ms(1); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + } + + // mock rowset and tablet + int64_t tablet_id_base = 1103; + for (int i = 0; i < 5; ++i) { + create_tablet(meta_service.get(), 1234789234, 1235, 1236, tablet_id_base + i); + auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i); + CreateRowsetResponse res; + commit_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + // sleep 1 second for txn timeout + sleep(1); + // commit txn + { + brpc::Controller cntl; + CommitTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(db_id); + req.set_txn_id(txn_id); + CommitTxnResponse res; + meta_service->commit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::UNDEFINED_ERR); + ASSERT_TRUE(res.status().msg().find("txn is expired, not allow to commit txn_id=") != + std::string::npos); + } + } +} + +TEST(MetaServiceTest, AbortTxnTest) { + auto meta_service = get_meta_service(); + + // case: abort txn by txn_id + { + int64_t db_id = 666; + int64_t table_id = 12345; + std::string label = "abort_txn_by_txn_id"; + std::string cloud_unique_id = "test_cloud_unique_id"; + int64_t tablet_id_base = 1104; + int64_t txn_id = -1; + // begin txn + { + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id(cloud_unique_id); + TxnInfoPB txn_info_pb; + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label(label); + txn_info_pb.add_table_ids(table_id); + txn_info_pb.set_timeout_ms(36000); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + } + + // mock rowset and tablet + for (int i = 0; i < 5; ++i) { + create_tablet(meta_service.get(), 12345, 1235, 1236, tablet_id_base + i); + auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i); + CreateRowsetResponse res; + commit_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // abort txn by txn_id + { + brpc::Controller cntl; + AbortTxnRequest req; + req.set_cloud_unique_id(cloud_unique_id); + req.set_txn_id(txn_id); + req.set_reason("test"); + AbortTxnResponse res; + meta_service->abort_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.txn_info().status(), TxnStatusPB::TXN_STATUS_ABORTED); + } + } + + // case: abort txn by db_id + label + { + int64_t db_id = 66631313131; + int64_t table_id = 12345; + std::string label = "abort_txn_by_db_id_and_label"; + std::string cloud_unique_id = "test_cloud_unique_id"; + int64_t tablet_id_base = 1104; + int64_t txn_id = -1; + // begin txn + { + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id(cloud_unique_id); + TxnInfoPB txn_info_pb; + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label(label); + txn_info_pb.add_table_ids(table_id); + txn_info_pb.set_timeout_ms(36000); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + } + + // mock rowset and tablet + for (int i = 0; i < 5; ++i) { + create_tablet(meta_service.get(), table_id, 1235, 1236, tablet_id_base + i); + auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i); + CreateRowsetResponse res; + commit_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // abort txn by db_id and label + { + brpc::Controller cntl; + AbortTxnRequest req; + req.set_cloud_unique_id(cloud_unique_id); + req.set_db_id(db_id); + req.set_label(label); + req.set_reason("test"); + AbortTxnResponse res; + meta_service->abort_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.txn_info().status(), TxnStatusPB::TXN_STATUS_ABORTED); + + std::string recycle_txn_key_; + std::string recycle_txn_val; + RecycleTxnKeyInfo recycle_txn_key_info {mock_instance, db_id, txn_id}; + recycle_txn_key(recycle_txn_key_info, &recycle_txn_key_); + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(recycle_txn_key_, &recycle_txn_val), TxnErrorCode::TXN_OK); + ASSERT_NE(txn_id, -1); + } + } +} + +TEST(MetaServiceTest, GetCurrentMaxTxnIdTest) { + auto meta_service = get_meta_service(); + + const int64_t db_id = 123; + const std::string label = "test_label123"; + const std::string cloud_unique_id = "test_cloud_unique_id"; + + brpc::Controller begin_txn_cntl; + BeginTxnRequest begin_txn_req; + BeginTxnResponse begin_txn_res; + TxnInfoPB txn_info_pb; + + begin_txn_req.set_cloud_unique_id(cloud_unique_id); + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label(label); + txn_info_pb.add_table_ids(12345); + txn_info_pb.set_timeout_ms(36000); + begin_txn_req.mutable_txn_info()->CopyFrom(txn_info_pb); + + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&begin_txn_cntl), + &begin_txn_req, &begin_txn_res, nullptr); + ASSERT_EQ(begin_txn_res.status().code(), MetaServiceCode::OK); + + brpc::Controller max_txn_id_cntl; + GetCurrentMaxTxnRequest max_txn_id_req; + GetCurrentMaxTxnResponse max_txn_id_res; + + max_txn_id_req.set_cloud_unique_id(cloud_unique_id); + + meta_service->get_current_max_txn_id( + reinterpret_cast<::google::protobuf::RpcController*>(&max_txn_id_cntl), &max_txn_id_req, + &max_txn_id_res, nullptr); + + ASSERT_EQ(max_txn_id_res.status().code(), MetaServiceCode::OK); + ASSERT_GE(max_txn_id_res.current_max_txn_id(), begin_txn_res.txn_id()); +} + +TEST(MetaServiceTest, CheckTxnConflictTest) { + auto meta_service = get_meta_service(); + + const int64_t db_id = 666; + const int64_t table_id = 777; + const std::string label = "test_label"; + const std::string cloud_unique_id = "test_cloud_unique_id"; + int64_t txn_id = -1; + + brpc::Controller begin_txn_cntl; + BeginTxnRequest begin_txn_req; + BeginTxnResponse begin_txn_res; + TxnInfoPB txn_info_pb; + + begin_txn_req.set_cloud_unique_id(cloud_unique_id); + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label(label); + txn_info_pb.add_table_ids(table_id); + txn_info_pb.set_timeout_ms(36000); + begin_txn_req.mutable_txn_info()->CopyFrom(txn_info_pb); + + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&begin_txn_cntl), + &begin_txn_req, &begin_txn_res, nullptr); + ASSERT_EQ(begin_txn_res.status().code(), MetaServiceCode::OK); + txn_id = begin_txn_res.txn_id(); + ASSERT_GT(txn_id, -1); + + brpc::Controller check_txn_conflict_cntl; + CheckTxnConflictRequest check_txn_conflict_req; + CheckTxnConflictResponse check_txn_conflict_res; + + check_txn_conflict_req.set_cloud_unique_id(cloud_unique_id); + check_txn_conflict_req.set_db_id(db_id); + check_txn_conflict_req.set_end_txn_id(txn_id + 1); + check_txn_conflict_req.add_table_ids(table_id); + + // first time to check txn conflict + meta_service->check_txn_conflict( + reinterpret_cast<::google::protobuf::RpcController*>(&begin_txn_cntl), + &check_txn_conflict_req, &check_txn_conflict_res, nullptr); + + ASSERT_EQ(check_txn_conflict_res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(check_txn_conflict_res.finished(), false); + + // mock rowset and tablet + int64_t tablet_id_base = 123456; + for (int i = 0; i < 5; ++i) { + create_tablet(meta_service.get(), table_id, 1235, 1236, tablet_id_base + i); + auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i); + CreateRowsetResponse res; + commit_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + brpc::Controller commit_txn_cntl; + CommitTxnRequest commit_txn_req; + commit_txn_req.set_cloud_unique_id(cloud_unique_id); + commit_txn_req.set_db_id(db_id); + commit_txn_req.set_txn_id(txn_id); + CommitTxnResponse commit_txn_res; + meta_service->commit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&commit_txn_cntl), + &commit_txn_req, &commit_txn_res, nullptr); + ASSERT_EQ(commit_txn_res.status().code(), MetaServiceCode::OK); + + // second time to check txn conflict + meta_service->check_txn_conflict( + reinterpret_cast<::google::protobuf::RpcController*>(&check_txn_conflict_cntl), + &check_txn_conflict_req, &check_txn_conflict_res, nullptr); + + ASSERT_EQ(check_txn_conflict_res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(check_txn_conflict_res.finished(), true); + + { + std::string running_key = txn_running_key({mock_instance, db_id, txn_id}); + std::string running_value; + std::unique_ptr txn; + TxnErrorCode err = meta_service->txn_kv()->create_txn(&txn); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(running_key, &running_value), TxnErrorCode::TXN_KEY_NOT_FOUND); + } +} + +TEST(MetaServiceTest, CheckNotTimeoutTxnConflictTest) { + auto meta_service = get_meta_service(); + + const int64_t db_id = 666; + const int64_t table_id = 777; + const std::string label = "test_label"; + const std::string cloud_unique_id = "test_cloud_unique_id"; + int64_t txn_id = -1; + + brpc::Controller begin_txn_cntl; + BeginTxnRequest begin_txn_req; + BeginTxnResponse begin_txn_res; + TxnInfoPB txn_info_pb; + + begin_txn_req.set_cloud_unique_id(cloud_unique_id); + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label(label); + txn_info_pb.add_table_ids(table_id); + txn_info_pb.set_timeout_ms(3); + begin_txn_req.mutable_txn_info()->CopyFrom(txn_info_pb); + + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&begin_txn_cntl), + &begin_txn_req, &begin_txn_res, nullptr); + ASSERT_EQ(begin_txn_res.status().code(), MetaServiceCode::OK); + txn_id = begin_txn_res.txn_id(); + ASSERT_GT(txn_id, -1); + + brpc::Controller check_txn_conflict_cntl; + CheckTxnConflictRequest check_txn_conflict_req; + CheckTxnConflictResponse check_txn_conflict_res; + + check_txn_conflict_req.set_cloud_unique_id(cloud_unique_id); + check_txn_conflict_req.set_db_id(db_id); + check_txn_conflict_req.set_end_txn_id(txn_id + 1); + check_txn_conflict_req.add_table_ids(table_id); + + // wait txn timeout + sleep(5); + // first time to check txn conflict + meta_service->check_txn_conflict( + reinterpret_cast<::google::protobuf::RpcController*>(&begin_txn_cntl), + &check_txn_conflict_req, &check_txn_conflict_res, nullptr); + + ASSERT_EQ(check_txn_conflict_res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(check_txn_conflict_res.finished(), true); +} + +TEST(MetaServiceTest, CheckTxnConflictWithAbortLabelTest) { + int ret = 0; + + auto txn_kv = std::dynamic_pointer_cast(std::make_shared()); + if (txn_kv != nullptr) { + ret = txn_kv->init(); + [&] { ASSERT_EQ(ret, 0); }(); + } + [&] { ASSERT_NE(txn_kv.get(), nullptr); }(); + + { + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->remove("\x00", "\xfe"); // This is dangerous if the fdb is not correctly set + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + } + + auto rs = std::make_shared(txn_kv); + auto rl = std::make_shared(); + auto meta_service = + std::make_unique(std::make_unique(txn_kv, rs, rl)); + + const int64_t db_id = 666; + const int64_t table_id = 777; + const std::string label = "test_label"; + const std::string cloud_unique_id = "test_cloud_unique_id"; + int64_t txn_id = -1; + + brpc::Controller begin_txn_cntl; + BeginTxnRequest begin_txn_req; + BeginTxnResponse begin_txn_res; + TxnInfoPB txn_info_pb; + + begin_txn_req.set_cloud_unique_id(cloud_unique_id); + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label(label); + txn_info_pb.add_table_ids(table_id); + txn_info_pb.set_timeout_ms(36000); + begin_txn_req.mutable_txn_info()->CopyFrom(txn_info_pb); + + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&begin_txn_cntl), + &begin_txn_req, &begin_txn_res, nullptr); + ASSERT_EQ(begin_txn_res.status().code(), MetaServiceCode::OK); + txn_id = begin_txn_res.txn_id(); + ASSERT_GT(txn_id, -1); + + brpc::Controller check_txn_conflict_cntl; + CheckTxnConflictRequest check_txn_conflict_req; + CheckTxnConflictResponse check_txn_conflict_res; + + check_txn_conflict_req.set_cloud_unique_id(cloud_unique_id); + check_txn_conflict_req.set_db_id(db_id); + check_txn_conflict_req.set_end_txn_id(txn_id + 1); + check_txn_conflict_req.add_table_ids(table_id); + + // first time to check txn conflict + meta_service->check_txn_conflict( + reinterpret_cast<::google::protobuf::RpcController*>(&begin_txn_cntl), + &check_txn_conflict_req, &check_txn_conflict_res, nullptr); + + ASSERT_EQ(check_txn_conflict_res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(check_txn_conflict_res.finished(), false); + + std::string running_key; + std::string running_val; + txn_running_key({mock_instance, db_id, txn_id}, &running_key); + { + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(running_key, &running_val), TxnErrorCode::TXN_OK); + } + + brpc::Controller abort_txn_cntl; + AbortTxnRequest abort_txn_req; + abort_txn_req.set_cloud_unique_id(cloud_unique_id); + abort_txn_req.set_db_id(db_id); + abort_txn_req.set_label(label); + AbortTxnResponse abort_txn_res; + meta_service->abort_txn(reinterpret_cast<::google::protobuf::RpcController*>(&abort_txn_cntl), + &abort_txn_req, &abort_txn_res, nullptr); + ASSERT_EQ(abort_txn_res.status().code(), MetaServiceCode::OK); + + // second time to check txn conflict + meta_service->check_txn_conflict( + reinterpret_cast<::google::protobuf::RpcController*>(&check_txn_conflict_cntl), + &check_txn_conflict_req, &check_txn_conflict_res, nullptr); + + ASSERT_EQ(check_txn_conflict_res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(check_txn_conflict_res.finished(), true); + + { + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(running_key, &running_val), TxnErrorCode::TXN_KEY_NOT_FOUND); + } +} + +TEST(MetaServiceTest, CleanTxnLabelTest) { + int ret = 0; + auto txn_kv = std::dynamic_pointer_cast(std::make_shared()); + if (txn_kv != nullptr) { + ret = txn_kv->init(); + [&] { ASSERT_EQ(ret, 0); }(); + } + [&] { ASSERT_NE(txn_kv.get(), nullptr); }(); + + { + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->remove("\x00", "\xfe"); // This is dangerous if the fdb is not correctly set + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + } + + auto rs = std::make_shared(txn_kv); + auto rl = std::make_shared(); + auto meta_service = + std::make_unique(std::make_unique(txn_kv, rs, rl)); + + // clean txn label by db_id and label + { + int64_t txn_id = -1; + int64_t db_id = 1987211; + const std::string& label = "test_clean_label"; + + { + brpc::Controller cntl; + CleanTxnLabelRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(db_id); + req.add_labels(label); + CleanTxnLabelResponse res; + meta_service->clean_txn_label( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, + nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // begin txn + { + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + TxnInfoPB txn_info_pb; + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label(label); + txn_info_pb.add_table_ids(1234); + txn_info_pb.set_timeout_ms(36000); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + } + + const std::string info_key = txn_info_key({mock_instance, db_id, txn_id}); + std::string info_val; + + const std::string label_key = txn_label_key({mock_instance, db_id, label}); + std::string label_val; + + const std::string index_key = txn_index_key({mock_instance, txn_id}); + std::string index_val; + + const std::string running_key = txn_running_key({mock_instance, db_id, txn_id}); + std::string running_val; + + const std::string recycle_key = recycle_txn_key({mock_instance, db_id, txn_id}); + std::string recycle_val; + + { + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + TxnErrorCode err = txn->get(info_key, &info_val); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + err = txn->get(label_key, &label_val); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + err = txn->get(index_key, &index_val); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + err = txn->get(running_key, &running_val); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + err = txn->get(recycle_key, &recycle_val); + ASSERT_EQ(err, TxnErrorCode::TXN_KEY_NOT_FOUND); + } + + // mock rowset and tablet + int64_t tablet_id_base = 110313131; + for (int i = 0; i < 2; ++i) { + create_tablet(meta_service.get(), 1234, 1235, 1236, tablet_id_base + i); + auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i); + CreateRowsetResponse res; + commit_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // commit txn + { + brpc::Controller cntl; + CommitTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(db_id); + req.set_txn_id(txn_id); + CommitTxnResponse res; + meta_service->commit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // begin txn + { + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + TxnInfoPB txn_info_pb; + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label(label); + txn_info_pb.add_table_ids(1234); + txn_info_pb.set_timeout_ms(36000); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::TXN_LABEL_ALREADY_USED); + } + + // clean txn label + { + brpc::Controller cntl; + CleanTxnLabelRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + CleanTxnLabelResponse res; + meta_service->clean_txn_label( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, + nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + } + + // clean txn label + { + brpc::Controller cntl; + CleanTxnLabelRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(db_id); + req.add_labels(label); + CleanTxnLabelResponse res; + meta_service->clean_txn_label( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, + nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + { + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + TxnErrorCode err = txn->get(info_key, &info_val); + ASSERT_EQ(err, TxnErrorCode::TXN_KEY_NOT_FOUND); + err = txn->get(label_key, &label_val); + ASSERT_EQ(err, TxnErrorCode::TXN_KEY_NOT_FOUND); + err = txn->get(index_key, &index_val); + ASSERT_EQ(err, TxnErrorCode::TXN_KEY_NOT_FOUND); + err = txn->get(running_key, &running_val); + ASSERT_EQ(err, TxnErrorCode::TXN_KEY_NOT_FOUND); + err = txn->get(recycle_key, &recycle_val); + ASSERT_EQ(err, TxnErrorCode::TXN_KEY_NOT_FOUND); + } + + { + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + TxnInfoPB txn_info_pb; + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label(label); + txn_info_pb.add_table_ids(1234); + txn_info_pb.set_timeout_ms(36000); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + } + + // abort txn + { + brpc::Controller cntl; + AbortTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + ASSERT_GT(txn_id, 0); + req.set_txn_id(txn_id); + req.set_reason("test"); + AbortTxnResponse res; + meta_service->abort_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.txn_info().status(), TxnStatusPB::TXN_STATUS_ABORTED); + } + + { + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + TxnInfoPB txn_info_pb; + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label(label); + txn_info_pb.add_table_ids(1234); + txn_info_pb.set_timeout_ms(36000); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + } + + // clean txn label + { + brpc::Controller cntl; + CleanTxnLabelRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(db_id); + req.add_labels(label); + CleanTxnLabelResponse res; + meta_service->clean_txn_label( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, + nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // abort txn + { + brpc::Controller cntl; + AbortTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + ASSERT_GT(txn_id, 0); + req.set_txn_id(txn_id); + req.set_reason("test"); + AbortTxnResponse res; + meta_service->abort_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.txn_info().status(), TxnStatusPB::TXN_STATUS_ABORTED); + } + + // clean txn label + { + brpc::Controller cntl; + CleanTxnLabelRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(db_id); + req.add_labels(label); + CleanTxnLabelResponse res; + meta_service->clean_txn_label( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, + nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + { + const std::string info_key = txn_info_key({mock_instance, db_id, txn_id}); + std::string info_val; + + const std::string label_key = txn_label_key({mock_instance, db_id, label}); + std::string label_val; + + const std::string index_key = txn_index_key({mock_instance, txn_id}); + std::string index_val; + + const std::string running_key = txn_running_key({mock_instance, db_id, txn_id}); + std::string running_val; + + const std::string recycle_key = recycle_txn_key({mock_instance, db_id, txn_id}); + std::string recycle_val; + + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + TxnErrorCode ret = txn->get(info_key, &info_val); + ASSERT_EQ(ret, TxnErrorCode::TXN_KEY_NOT_FOUND); + ret = txn->get(label_key, &label_val); + ASSERT_EQ(ret, TxnErrorCode::TXN_KEY_NOT_FOUND); + ret = txn->get(index_key, &index_val); + ASSERT_EQ(ret, TxnErrorCode::TXN_KEY_NOT_FOUND); + ret = txn->get(running_key, &running_val); + ASSERT_EQ(ret, TxnErrorCode::TXN_KEY_NOT_FOUND); + ret = txn->get(recycle_key, &recycle_val); + ASSERT_EQ(ret, TxnErrorCode::TXN_KEY_NOT_FOUND); + } + } + // clean txn label only by db_id + { + int64_t db_id = 1987211123; + const std::string& label = "test_clean_label"; + + TxnInfoPB txn_info_pb; + txn_info_pb.set_db_id(db_id); + txn_info_pb.add_table_ids(1234); + txn_info_pb.set_timeout_ms(36000); + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + + //clean not exist label + { + brpc::Controller cntl; + CleanTxnLabelRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(db_id); + CleanTxnLabelResponse res; + meta_service->clean_txn_label( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, + nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // inject internal_clean_label err = TXN_CONFLICT + { + auto sp = cloud::SyncPoint::get_instance(); + sp->set_call_back("internal_clean_label:err", [&](void* args) { + *reinterpret_cast(args) = TxnErrorCode::TXN_CONFLICT; + }); + sp->enable_processing(); + int64_t txn_id = -1; + for (int i = 100; i < 101; i++) { + { + std::stringstream label_ss; + label_ss << label << i; + brpc::Controller cntl; + txn_info_pb.set_label(label_ss.str()); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + meta_service->begin_txn( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, + nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + } + + { + // mock rowset and tablet + int64_t tablet_id_base = 110313131; + for (int i = 0; i < 1; ++i) { + create_tablet(meta_service.get(), 1234, 1235, 1236, tablet_id_base + i); + auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i); + CreateRowsetResponse res; + commit_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + } + + // commit txn + { + brpc::Controller cntl; + CommitTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(db_id); + req.set_txn_id(txn_id); + CommitTxnResponse res; + meta_service->commit_txn( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, + nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + } + + { + brpc::Controller cntl; + CleanTxnLabelRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(db_id); + CleanTxnLabelResponse res; + meta_service->clean_txn_label( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, + nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::KV_TXN_CONFLICT); + } + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); + } + + // create 12 committed txns and clean label by id + { + auto sp = cloud::SyncPoint::get_instance(); + sp->set_call_back("clean_txn_label:limit", [&](void* args) { + int* limit = reinterpret_cast(args); + *limit = 5; + }); + sp->enable_processing(); + + int64_t txn_id = -1; + for (int i = 0; i < 12; i++) { + { + std::stringstream label_ss; + label_ss << label << i; + brpc::Controller cntl; + txn_info_pb.set_label(label_ss.str()); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + meta_service->begin_txn( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, + nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + } + + { + // mock rowset and tablet + int64_t tablet_id_base = 110313131; + for (int i = 0; i < 1; ++i) { + create_tablet(meta_service.get(), 1234, 1235, 1236, tablet_id_base + i); + auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i); + CreateRowsetResponse res; + commit_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + } + + // commit txn + { + brpc::Controller cntl; + CommitTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(db_id); + req.set_txn_id(txn_id); + CommitTxnResponse res; + meta_service->commit_txn( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, + nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + } + + { + brpc::Controller cntl; + CleanTxnLabelRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(db_id); + CleanTxnLabelResponse res; + meta_service->clean_txn_label( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, + nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); + } + } +} + +TEST(MetaServiceTest, GetTxnTest) { + int ret = 0; + auto txn_kv = std::dynamic_pointer_cast(std::make_shared()); + if (txn_kv != nullptr) { + ret = txn_kv->init(); + [&] { ASSERT_EQ(ret, 0); }(); + } + [&] { ASSERT_NE(txn_kv.get(), nullptr); }(); + + { + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->remove("\x00", "\xfe"); // This is dangerous if the fdb is not correctly set + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + } + + auto rs = std::make_shared(txn_kv); + auto rl = std::make_shared(); + auto meta_service = + std::make_unique(std::make_unique(txn_kv, rs, rl)); + + { + int64_t txn_id = -1; + int64_t db_id = 34521431231; + const std::string& label = "test_get_txn"; + + // begin txn + { + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + TxnInfoPB txn_info_pb; + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label(label); + txn_info_pb.add_table_ids(1234); + txn_info_pb.set_timeout_ms(36000); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + } + + { + brpc::Controller cntl; + GetTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(db_id); + req.set_txn_id(-1); + GetTxnResponse res; + meta_service->get_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + } + + { + brpc::Controller cntl; + GetTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(db_id); + req.set_txn_id(txn_id); + GetTxnResponse res; + meta_service->get_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + { + brpc::Controller cntl; + GetTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_txn_id(txn_id); + GetTxnResponse res; + meta_service->get_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + } +} +// + +TEST(MetaServiceTest, CopyJobTest) { + auto meta_service = get_meta_service(); + brpc::Controller cntl; + auto cloud_unique_id = "test_cloud_unique_id"; + auto stage_id = "test_stage_id"; + int64_t table_id = 100; + std::string instance_id = "copy_job_test_instance_id"; + + [[maybe_unused]] auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + // generate a begin copy request + BeginCopyRequest begin_copy_request; + begin_copy_request.set_cloud_unique_id(cloud_unique_id); + begin_copy_request.set_stage_id(stage_id); + begin_copy_request.set_stage_type(StagePB::EXTERNAL); + begin_copy_request.set_table_id(table_id); + begin_copy_request.set_copy_id("test_copy_id"); + begin_copy_request.set_group_id(0); + begin_copy_request.set_start_time_ms(200); + begin_copy_request.set_timeout_time_ms(300); + for (int i = 0; i < 20; ++i) { + ObjectFilePB object_file_pb; + object_file_pb.set_relative_path("obj_" + std::to_string(i)); + object_file_pb.set_etag("obj_" + std::to_string(i) + "_etag"); + begin_copy_request.add_object_files()->CopyFrom(object_file_pb); + } + + // generate a finish copy request + FinishCopyRequest finish_copy_request; + finish_copy_request.set_cloud_unique_id(cloud_unique_id); + finish_copy_request.set_stage_id(stage_id); + finish_copy_request.set_stage_type(StagePB::EXTERNAL); + finish_copy_request.set_table_id(table_id); + finish_copy_request.set_copy_id("test_copy_id"); + finish_copy_request.set_group_id(0); + finish_copy_request.set_action(FinishCopyRequest::COMMIT); + + // generate a get copy files request + GetCopyFilesRequest get_copy_file_req; + get_copy_file_req.set_cloud_unique_id(cloud_unique_id); + get_copy_file_req.set_stage_id(stage_id); + get_copy_file_req.set_table_id(table_id); + + // generate a get copy job request + GetCopyJobRequest get_copy_job_request; + get_copy_job_request.set_cloud_unique_id(cloud_unique_id); + get_copy_job_request.set_stage_id(stage_id); + get_copy_job_request.set_table_id(table_id); + get_copy_job_request.set_copy_id("test_copy_id"); + get_copy_job_request.set_group_id(0); + + // get copy job + { + GetCopyJobResponse res; + meta_service->get_copy_job(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_copy_job_request, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.has_copy_job(), false); + } + // begin copy + { + BeginCopyResponse res; + meta_service->begin_copy(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &begin_copy_request, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.filtered_object_files_size(), 20); + } + // get copy files + { + GetCopyFilesResponse res; + meta_service->get_copy_files(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_copy_file_req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.object_files_size(), 20); + } + // get copy job + { + GetCopyJobResponse res; + meta_service->get_copy_job(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_copy_job_request, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.copy_job().object_files().size(), 20); + } + // begin copy with duplicate files + { + begin_copy_request.set_copy_id("test_copy_id_1"); + begin_copy_request.clear_object_files(); + for (int i = 15; i < 30; ++i) { + ObjectFilePB object_file_pb; + object_file_pb.set_relative_path("obj_" + std::to_string(i)); + object_file_pb.set_etag("obj_" + std::to_string(i) + "_etag"); + begin_copy_request.add_object_files()->CopyFrom(object_file_pb); + } + + BeginCopyResponse res; + meta_service->begin_copy(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &begin_copy_request, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.filtered_object_files_size(), 10); + } + // get copy files + { + GetCopyFilesResponse res; + meta_service->get_copy_files(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_copy_file_req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.object_files_size(), 30); + } + // finish the first copy job + { + FinishCopyResponse res; + meta_service->finish_copy(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &finish_copy_request, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + // get copy files + { + GetCopyFilesResponse res; + meta_service->get_copy_files(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_copy_file_req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.object_files_size(), 30); + } + // abort the second copy job + { + finish_copy_request.set_copy_id("test_copy_id_1"); + finish_copy_request.set_action(FinishCopyRequest::ABORT); + + FinishCopyResponse res; + meta_service->finish_copy(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &finish_copy_request, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + // get copy files + { + GetCopyFilesResponse res; + meta_service->get_copy_files(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_copy_file_req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.object_files_size(), 20); + } + { + // begin a copy job whose files are all loaded, the copy job key should not be created + begin_copy_request.set_copy_id("tmp_id"); + begin_copy_request.clear_object_files(); + for (int i = 0; i < 20; ++i) { + ObjectFilePB object_file_pb; + object_file_pb.set_relative_path("obj_" + std::to_string(i)); + object_file_pb.set_etag("obj_" + std::to_string(i) + "_etag"); + begin_copy_request.add_object_files()->CopyFrom(object_file_pb); + } + BeginCopyResponse res; + meta_service->begin_copy(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &begin_copy_request, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.filtered_object_files_size(), 0); + // get copy job + get_copy_job_request.set_copy_id("tmp_id"); + GetCopyJobResponse res2; + meta_service->get_copy_job(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_copy_job_request, &res2, nullptr); + ASSERT_EQ(res2.status().code(), MetaServiceCode::OK); + ASSERT_FALSE(res2.has_copy_job()); + } + // scan fdb + { + std::unique_ptr txn; + std::string get_val; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + // 20 copy files + { + CopyFileKeyInfo key_info0 {instance_id, stage_id, table_id, "", ""}; + CopyFileKeyInfo key_info1 {instance_id, stage_id, table_id + 1, "", ""}; + std::string key0; + std::string key1; + copy_file_key(key_info0, &key0); + copy_file_key(key_info1, &key1); + std::unique_ptr it; + ASSERT_EQ(txn->get(key0, key1, &it), TxnErrorCode::TXN_OK); + int file_cnt = 0; + do { + ASSERT_EQ(txn->get(key0, key1, &it), TxnErrorCode::TXN_OK); + while (it->has_next()) { + auto [k, v] = it->next(); + CopyFilePB copy_file; + ASSERT_TRUE(copy_file.ParseFromArray(v.data(), v.size())); + ASSERT_EQ(copy_file.copy_id(), "test_copy_id"); + ++file_cnt; + if (!it->has_next()) { + key0 = k; + } + } + key0.push_back('\x00'); + } while (it->more()); + ASSERT_EQ(file_cnt, 20); + } + // 1 copy job with finish status + { + CopyJobKeyInfo key_info0 {instance_id, stage_id, table_id, "", 0}; + CopyJobKeyInfo key_info1 {instance_id, stage_id, table_id + 1, "", 0}; + std::string key0; + std::string key1; + copy_job_key(key_info0, &key0); + copy_job_key(key_info1, &key1); + std::unique_ptr it; + int job_cnt = 0; + do { + ASSERT_EQ(txn->get(key0, key1, &it), TxnErrorCode::TXN_OK); + while (it->has_next()) { + auto [k, v] = it->next(); + CopyJobPB copy_job; + ASSERT_EQ(copy_job.ParseFromArray(v.data(), v.size()), true); + ASSERT_EQ(copy_job.object_files_size(), 20); + ASSERT_EQ(copy_job.job_status(), CopyJobPB::FINISH); + ++job_cnt; + if (!it->has_next()) { + key0 = k; + } + } + key0.push_back('\x00'); + } while (it->more()); + ASSERT_EQ(job_cnt, 1); + } + } +} + +TEST(MetaServiceTest, FilterCopyFilesTest) { + auto meta_service = get_meta_service(); + brpc::Controller cntl; + auto cloud_unique_id = "test_cloud_unique_id"; + std::string instance_id = "stage_test_instance_id"; + auto stage_id = "test_stage_id"; + int64_t table_id = 100; + [[maybe_unused]] auto sp = SyncPoint::get_instance(); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_id", + [](void* p) { *reinterpret_cast(p) = 1; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->enable_processing(); + + FilterCopyFilesRequest request; + request.set_cloud_unique_id(cloud_unique_id); + request.set_stage_id(stage_id); + request.set_table_id(table_id); + for (int i = 0; i < 10; ++i) { + ObjectFilePB object_file; + object_file.set_relative_path("file" + std::to_string(i)); + object_file.set_etag("etag" + std::to_string(i)); + request.add_object_files()->CopyFrom(object_file); + } + + // all files are not loaded + { + FilterCopyFilesResponse res; + meta_service->filter_copy_files(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &request, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.object_files().size(), 10); + } + + // some files are loaded + { + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + for (int i = 0; i < 4; ++i) { + CopyFileKeyInfo key_info {instance_id, stage_id, table_id, "file" + std::to_string(i), + "etag" + std::to_string(i)}; + std::string key; + copy_file_key(key_info, &key); + CopyFilePB copy_file; + copy_file.set_copy_id("test_copy_id"); + std::string val; + copy_file.SerializeToString(&val); + txn->put(key, val); + } + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + FilterCopyFilesResponse res; + meta_service->filter_copy_files(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &request, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.object_files().size(), 6); + ASSERT_EQ(res.object_files().at(0).relative_path(), "file4"); + } + + // all files are loaded + { + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + for (int i = 4; i < 10; ++i) { + CopyFileKeyInfo key_info {instance_id, stage_id, table_id, "file" + std::to_string(i), + "etag" + std::to_string(i)}; + std::string key; + copy_file_key(key_info, &key); + CopyFilePB copy_file; + copy_file.set_copy_id("test_copy_id"); + std::string val; + copy_file.SerializeToString(&val); + txn->put(key, val); + } + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + FilterCopyFilesResponse res; + meta_service->filter_copy_files(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &request, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.object_files().size(), 0); + } +} + +extern std::vector> calc_sync_versions( + int64_t req_bc_cnt, int64_t bc_cnt, int64_t req_cc_cnt, int64_t cc_cnt, int64_t req_cp, + int64_t cp, int64_t req_start, int64_t req_end); + +TEST(MetaServiceTest, CalcSyncVersionsTest) { + using Versions = std::vector>; + // * no compaction happened + // req_cc_cnt == ms_cc_cnt && req_bc_cnt == ms_bc_cnt && req_cp == ms_cp + // BE [=][=][=][=][=====][=][=]<.......> + // ^~~~~ req_cp + // BE [=][=][=][=][=====][=][=][=][=][=][=] + // ^~~~~ ms_cp + // ^_____^ versions_return: [req_start, req_end] + { + auto [req_bc_cnt, bc_cnt] = std::tuple {0, 0}; + auto [req_cc_cnt, cc_cnt] = std::tuple {1, 1}; + auto [req_cp, cp] = std::tuple {5, 5}; + auto [req_start, req_end] = std::tuple {8, 12}; + auto versions = calc_sync_versions(req_bc_cnt, bc_cnt, req_cc_cnt, cc_cnt, req_cp, cp, + req_start, req_end); + ASSERT_EQ(versions, (Versions {{8, 12}})); + } + // * only one CC happened and CP changed + // req_cc_cnt == ms_cc_cnt - 1 && req_bc_cnt == ms_bc_cnt && req_cp < ms_cp + // BE [=][=][=][=][=====][=][=]<.......> + // ^~~~~ req_cp + // MS [=][=][=][=][xxxxxxxxxxxxxx][=======][=][=] + // ^~~~~~~ ms_cp + // ^__________________^ versions_return: [req_cp, ms_cp - 1] v [req_start, req_end] + { + auto [req_bc_cnt, bc_cnt] = std::tuple {0, 0}; + auto [req_cc_cnt, cc_cnt] = std::tuple {1, 2}; + auto [req_cp, cp] = std::tuple {5, 10}; + auto [req_start, req_end] = std::tuple {8, 12}; + auto versions = calc_sync_versions(req_bc_cnt, bc_cnt, req_cc_cnt, cc_cnt, req_cp, cp, + req_start, req_end); + ASSERT_EQ(versions, (Versions {{5, 12}})); // [5, 9] v [8, 12] + } + { + auto [req_bc_cnt, bc_cnt] = std::tuple {0, 0}; + auto [req_cc_cnt, cc_cnt] = std::tuple {1, 2}; + auto [req_cp, cp] = std::tuple {5, 15}; + auto [req_start, req_end] = std::tuple {8, 12}; + auto versions = calc_sync_versions(req_bc_cnt, bc_cnt, req_cc_cnt, cc_cnt, req_cp, cp, + req_start, req_end); + ASSERT_EQ(versions, (Versions {{5, 14}})); // [5, 14] v [8, 12] + } + // * only one CC happened and CP remain unchanged + // req_cc_cnt == ms_cc_cnt - 1 && req_bc_cnt == ms_bc_cnt && req_cp == ms_cp + // BE [=][=][=][=][=====][=][=]<.......> + // ^~~~~ req_cp + // MS [=][=][=][=][xxxxxxxxxxxxxx][=][=][=][=][=] + // ^~~~~~~~~~~~~~ ms_cp + // ^__________________^ versions_return: [req_cp, max] v [req_start, req_end] + // + { + auto [req_bc_cnt, bc_cnt] = std::tuple {0, 0}; + auto [req_cc_cnt, cc_cnt] = std::tuple {1, 2}; + auto [req_cp, cp] = std::tuple {5, 5}; + auto [req_start, req_end] = std::tuple {8, 12}; + auto versions = calc_sync_versions(req_bc_cnt, bc_cnt, req_cc_cnt, cc_cnt, req_cp, cp, + req_start, req_end); + ASSERT_EQ(versions, (Versions {{5, INT64_MAX - 1}})); // [5, max] v [8, 12] + } + // * more than one CC happened and CP remain unchanged + // req_cc_cnt < ms_cc_cnt - 1 && req_bc_cnt == ms_bc_cnt && req_cp == ms_cp + // BE [=][=][=][=][=====][=][=]<.......> + // ^~~~~ req_cp + // MS [=][=][=][=][xxxxxxxxxxxxxx][xxxxxxx][=][=] + // ^~~~~~~~~~~~~~ ms_cp + // ^_____________________^ versions_return: [req_cp, max] v [req_start, req_end] + { + auto [req_bc_cnt, bc_cnt] = std::tuple {0, 0}; + auto [req_cc_cnt, cc_cnt] = std::tuple {1, 3}; + auto [req_cp, cp] = std::tuple {5, 5}; + auto [req_start, req_end] = std::tuple {8, 12}; + auto versions = calc_sync_versions(req_bc_cnt, bc_cnt, req_cc_cnt, cc_cnt, req_cp, cp, + req_start, req_end); + ASSERT_EQ(versions, (Versions {{5, INT64_MAX - 1}})); // [5, max] v [8, 12] + } + // * more than one CC happened and CP changed + // BE [=][=][=][=][=====][=][=] + // ^~~~~ req_cp + // MS [=][=][=][=][xxxxxxxxxxxxxx][xxxxxxx][=][=] + // ^~~~~~~ ms_cp + // ^_____________________^ related_versions: [req_cp, max] v [req_start, req_end] + { + auto [req_bc_cnt, bc_cnt] = std::tuple {0, 0}; + auto [req_cc_cnt, cc_cnt] = std::tuple {1, 3}; + auto [req_cp, cp] = std::tuple {5, 15}; + auto [req_start, req_end] = std::tuple {8, 12}; + auto versions = calc_sync_versions(req_bc_cnt, bc_cnt, req_cc_cnt, cc_cnt, req_cp, cp, + req_start, req_end); + ASSERT_EQ(versions, (Versions {{5, INT64_MAX - 1}})); // [5, max] v [8, 12] + } + // * for any BC happended + // req_bc_cnt < ms_bc_cnt + // BE [=][=][=][=][=====][=][=]<.......> + // ^~~~~ req_cp + // MS [xxxxxxxxxx][xxxxxxxxxxxxxx][=======][=][=] + // ^~~~~~~ ms_cp + // ^_________________________^ versions_return: [0, ms_cp - 1] v versions_return_in_above_case + { + auto [req_bc_cnt, bc_cnt] = std::tuple {0, 1}; + auto [req_cc_cnt, cc_cnt] = std::tuple {1, 1}; + auto [req_cp, cp] = std::tuple {5, 5}; + auto [req_start, req_end] = std::tuple {8, 12}; + auto versions = calc_sync_versions(req_bc_cnt, bc_cnt, req_cc_cnt, cc_cnt, req_cp, cp, + req_start, req_end); + ASSERT_EQ(versions, (Versions {{0, 4}, {8, 12}})); + } + { + auto [req_bc_cnt, bc_cnt] = std::tuple {0, 1}; + auto [req_cc_cnt, cc_cnt] = std::tuple {1, 1}; + auto [req_cp, cp] = std::tuple {8, 8}; + auto [req_start, req_end] = std::tuple {8, 12}; + auto versions = calc_sync_versions(req_bc_cnt, bc_cnt, req_cc_cnt, cc_cnt, req_cp, cp, + req_start, req_end); + ASSERT_EQ(versions, (Versions {{0, 12}})); // [0, 7] v [8, 12] + } + { + auto [req_bc_cnt, bc_cnt] = std::tuple {0, 1}; + auto [req_cc_cnt, cc_cnt] = std::tuple {1, 2}; + auto [req_cp, cp] = std::tuple {5, 10}; + auto [req_start, req_end] = std::tuple {8, 12}; + auto versions = calc_sync_versions(req_bc_cnt, bc_cnt, req_cc_cnt, cc_cnt, req_cp, cp, + req_start, req_end); + ASSERT_EQ(versions, (Versions {{0, 12}})); // [0, 4] v [5, 9] v [8, 12] + } + { + auto [req_bc_cnt, bc_cnt] = std::tuple {0, 1}; + auto [req_cc_cnt, cc_cnt] = std::tuple {1, 2}; + auto [req_cp, cp] = std::tuple {5, 15}; + auto [req_start, req_end] = std::tuple {8, 12}; + auto versions = calc_sync_versions(req_bc_cnt, bc_cnt, req_cc_cnt, cc_cnt, req_cp, cp, + req_start, req_end); + ASSERT_EQ(versions, (Versions {{0, 14}})); // [0, 4] v [5, 14] v [8, 12] + } + { + auto [req_bc_cnt, bc_cnt] = std::tuple {0, 1}; + auto [req_cc_cnt, cc_cnt] = std::tuple {1, 2}; + auto [req_cp, cp] = std::tuple {5, 5}; + auto [req_start, req_end] = std::tuple {8, 12}; + auto versions = calc_sync_versions(req_bc_cnt, bc_cnt, req_cc_cnt, cc_cnt, req_cp, cp, + req_start, req_end); + // [0, 4] v [5, max] v [8, 12] + ASSERT_EQ(versions, (Versions {{0, INT64_MAX - 1}})); + } +} + +TEST(MetaServiceTest, StageTest) { + auto meta_service = get_meta_service(); + brpc::Controller cntl; + auto cloud_unique_id = "test_cloud_unique_id"; + std::string instance_id = "stage_test_instance_id"; + [[maybe_unused]] auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_id", + [](void* p) { *reinterpret_cast(p) = 1; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->enable_processing(); + + ObjectStoreInfoPB obj; + obj.set_ak("123"); + obj.set_sk("321"); + obj.set_bucket("456"); + obj.set_prefix("654"); + obj.set_endpoint("789"); + obj.set_region("987"); + obj.set_external_endpoint("888"); + obj.set_provider(ObjectStoreInfoPB::BOS); + + RamUserPB ram_user; + ram_user.set_user_id("test_user_id"); + ram_user.set_ak("test_ak"); + ram_user.set_sk("test_sk"); + EncryptionInfoPB encry_info; + encry_info.set_encryption_method("encry_method_test"); + encry_info.set_key_id(1111); + ram_user.mutable_encryption_info()->CopyFrom(encry_info); + + // create instance + { + CreateInstanceRequest req; + req.set_instance_id(instance_id); + req.set_user_id("test_user"); + req.set_name("test_name"); + req.mutable_ram_user()->CopyFrom(ram_user); + req.mutable_obj_info()->CopyFrom(obj); + + CreateInstanceResponse res; + meta_service->create_instance(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // test create and get internal stage + { + // get a non-existent internal stage + GetStageRequest get_stage_req; + GetStageResponse res; + // no cloud_unique_id + meta_service->get_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_stage_req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + + get_stage_req.set_cloud_unique_id(cloud_unique_id); + // no instance_id + sp->clear_call_back("get_instance_id"); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = ""; }); + meta_service->get_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_stage_req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + sp->clear_call_back("get_instance_id"); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + + // no stage type + meta_service->get_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_stage_req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + get_stage_req.set_type(StagePB::INTERNAL); + + // no internal stage user name + meta_service->get_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_stage_req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + get_stage_req.set_mysql_user_name("root"); + + // no internal stage user id + meta_service->get_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_stage_req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + get_stage_req.set_mysql_user_id("root_id"); + meta_service->get_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_stage_req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::STAGE_NOT_FOUND); + + // create an internal stage + CreateStageRequest create_stage_request; + StagePB stage; + stage.set_type(StagePB::INTERNAL); + stage.add_mysql_user_name("root"); + stage.add_mysql_user_id("root_id"); + stage.set_stage_id("internal_stage_id"); + create_stage_request.set_cloud_unique_id(cloud_unique_id); + create_stage_request.mutable_stage()->CopyFrom(stage); + CreateStageResponse create_stage_response; + meta_service->create_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &create_stage_request, &create_stage_response, nullptr); + ASSERT_EQ(create_stage_response.status().code(), MetaServiceCode::OK); + + // get existent internal stage + GetStageResponse res2; + meta_service->get_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_stage_req, &res2, nullptr); + ASSERT_EQ(res2.status().code(), MetaServiceCode::OK); + ASSERT_EQ(1, res2.stage().size()); + + // can't find user id's stage + GetStageResponse res3; + get_stage_req.set_mysql_user_id("not_root_id_exist"); + meta_service->get_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_stage_req, &res3, nullptr); + ASSERT_EQ(res3.status().code(), MetaServiceCode::STATE_ALREADY_EXISTED_FOR_USER); + ASSERT_EQ(1, res3.stage().size()); + + // drop internal stage + DropStageRequest drop_stage_request; + drop_stage_request.set_cloud_unique_id(cloud_unique_id); + drop_stage_request.set_type(StagePB::INTERNAL); + drop_stage_request.set_mysql_user_id("root_id"); + drop_stage_request.set_reason("Drop"); + DropStageResponse drop_stage_response; + meta_service->drop_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &drop_stage_request, &drop_stage_response, nullptr); + ASSERT_EQ(drop_stage_response.status().code(), MetaServiceCode::OK); + // scan fdb has recycle_stage key + { + RecycleStageKeyInfo key_info0 {instance_id, ""}; + RecycleStageKeyInfo key_info1 {instance_id, "{"}; + std::string key0; + std::string key1; + recycle_stage_key(key_info0, &key0); + recycle_stage_key(key_info1, &key1); + std::unique_ptr txn; + std::string get_val; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + std::unique_ptr it; + ASSERT_EQ(txn->get(key0, key1, &it), TxnErrorCode::TXN_OK); + int stage_cnt = 0; + do { + ASSERT_EQ(txn->get(key0, key1, &it), TxnErrorCode::TXN_OK); + while (it->has_next()) { + auto [k, v] = it->next(); + ++stage_cnt; + if (!it->has_next()) { + key0 = k; + } + } + key0.push_back('\x00'); + } while (it->more()); + ASSERT_EQ(stage_cnt, 1); + } + + // get internal stage + meta_service->get_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_stage_req, &res2, nullptr); + ASSERT_EQ(res2.status().code(), MetaServiceCode::STAGE_NOT_FOUND); + + // drop a non-exist internal stage + drop_stage_request.set_mysql_user_id("root_id2"); + meta_service->drop_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &drop_stage_request, &drop_stage_response, nullptr); + ASSERT_EQ(drop_stage_response.status().code(), MetaServiceCode::STAGE_NOT_FOUND); + } + + // test create and get external stage + { + // get an external stage with name + GetStageRequest get_stage_req; + get_stage_req.set_cloud_unique_id(cloud_unique_id); + get_stage_req.set_type(StagePB::EXTERNAL); + get_stage_req.set_stage_name("ex_name_1"); + + { + GetStageResponse res; + meta_service->get_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_stage_req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::STAGE_NOT_FOUND); + } + + // create 4 stages + for (auto i = 0; i < 4; ++i) { + StagePB stage; + stage.set_type(StagePB::EXTERNAL); + stage.set_stage_id("ex_id_" + std::to_string(i)); + stage.set_name("ex_name_" + std::to_string(i)); + if (i == 2) { + stage.set_access_type(StagePB::BUCKET_ACL); + } else if (i == 3) { + stage.set_access_type(StagePB::IAM); + } + stage.mutable_obj_info()->CopyFrom(obj); + + CreateStageRequest create_stage_req; + create_stage_req.set_cloud_unique_id(cloud_unique_id); + create_stage_req.mutable_stage()->CopyFrom(stage); + + CreateStageResponse create_stage_res; + meta_service->create_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &create_stage_req, &create_stage_res, nullptr); + ASSERT_EQ(create_stage_res.status().code(), MetaServiceCode::OK); + } + + // stages number bigger than config + { + config::max_num_stages = 4; + StagePB stage; + stage.set_type(StagePB::INTERNAL); + stage.add_mysql_user_name("root1"); + stage.add_mysql_user_id("root_id1"); + stage.set_stage_id("internal_stage_id1"); + CreateStageRequest create_stage_req; + create_stage_req.set_cloud_unique_id(cloud_unique_id); + create_stage_req.mutable_stage()->CopyFrom(stage); + + CreateStageResponse create_stage_res; + meta_service->create_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &create_stage_req, &create_stage_res, nullptr); + ASSERT_EQ(create_stage_res.status().code(), MetaServiceCode::UNDEFINED_ERR); + } + + // get an external stage with name + { + GetStageResponse res; + meta_service->get_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_stage_req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(1, res.stage().size()); + ASSERT_EQ("ex_id_1", res.stage().at(0).stage_id()); + } + + // get an external stage with name, type StagePB::BUCKET_ACL + { + get_stage_req.set_stage_name("ex_name_2"); + GetStageResponse res; + meta_service->get_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_stage_req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(1, res.stage().size()); + ASSERT_EQ("ex_id_2", res.stage().at(0).stage_id()); + } + + // get an external stage with name, type StagePB::IAM + { + GetStageResponse res; + get_stage_req.set_stage_name("ex_name_3"); + meta_service->get_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_stage_req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(1, res.stage().size()); + ASSERT_EQ("ex_id_3", res.stage().at(0).stage_id()); + + GetStageResponse res1; + std::unique_ptr txn; + TxnErrorCode err = meta_service->txn_kv()->create_txn(&txn); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + + RamUserPB iam_user; + txn->put(system_meta_service_arn_info_key(), iam_user.SerializeAsString()); + err = txn->commit(); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + LOG_INFO("err=", err); + meta_service->get_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &get_stage_req, &res1, nullptr); + ASSERT_EQ(res1.status().code(), MetaServiceCode::OK); + ASSERT_EQ(1, res1.stage().size()); + ASSERT_EQ("ex_id_3", res1.stage().at(0).stage_id()); + } + + GetStageRequest req; + req.set_cloud_unique_id(cloud_unique_id); + req.set_type(StagePB::EXTERNAL); + // get all stages + { + GetStageResponse res; + meta_service->get_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(4, res.stage().size()); + ASSERT_EQ("ex_id_0", res.stage().at(0).stage_id()); + ASSERT_EQ("ex_id_1", res.stage().at(1).stage_id()); + } + + // drop one stage + { + DropStageRequest drop_stage_req; + drop_stage_req.set_cloud_unique_id(cloud_unique_id); + drop_stage_req.set_type(StagePB::EXTERNAL); + drop_stage_req.set_stage_name("tmp"); + DropStageResponse res; + meta_service->drop_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &drop_stage_req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::STAGE_NOT_FOUND); + + drop_stage_req.set_stage_name("ex_name_1"); + meta_service->drop_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &drop_stage_req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + // get all stage + GetStageResponse get_stage_res; + meta_service->get_stage(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &get_stage_res, nullptr); + ASSERT_EQ(get_stage_res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(3, get_stage_res.stage().size()); + ASSERT_EQ("ex_name_0", get_stage_res.stage().at(0).name()); + } + } + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); +} + +TEST(MetaServiceTest, GetIamTest) { + auto meta_service = get_meta_service(); + brpc::Controller cntl; + auto cloud_unique_id = "test_cloud_unique_id"; + std::string instance_id = "get_iam_test_instance_id"; + [[maybe_unused]] auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_id", + [](void* p) { *reinterpret_cast(p) = 1; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->enable_processing(); + + config::arn_id = "iam_arn"; + config::arn_ak = "iam_ak"; + config::arn_sk = "iam_sk"; + + // create instance + { + ObjectStoreInfoPB obj; + obj.set_ak("123"); + obj.set_sk("321"); + obj.set_bucket("456"); + obj.set_prefix("654"); + obj.set_endpoint("789"); + obj.set_region("987"); + obj.set_external_endpoint("888"); + obj.set_provider(ObjectStoreInfoPB::BOS); + + RamUserPB ram_user; + ram_user.set_user_id("test_user_id"); + ram_user.set_ak("test_ak"); + ram_user.set_sk("test_sk"); + + CreateInstanceRequest req; + req.set_instance_id(instance_id); + req.set_user_id("test_user"); + req.set_name("test_name"); + req.mutable_ram_user()->CopyFrom(ram_user); + req.mutable_obj_info()->CopyFrom(obj); + + CreateInstanceResponse res; + meta_service->create_instance(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + GetIamRequest request; + request.set_cloud_unique_id(cloud_unique_id); + GetIamResponse response; + meta_service->get_iam(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &request, + &response, nullptr); + ASSERT_EQ(response.status().code(), MetaServiceCode::OK); + ASSERT_EQ(response.ram_user().user_id(), "test_user_id"); + ASSERT_EQ(response.ram_user().ak(), "test_ak"); + ASSERT_EQ(response.ram_user().sk(), "test_sk"); + ASSERT_TRUE(response.ram_user().external_id().empty()); + + ASSERT_EQ(response.iam_user().user_id(), "iam_arn"); + ASSERT_EQ(response.iam_user().external_id(), instance_id); + ASSERT_EQ(response.iam_user().ak(), "iam_ak"); + ASSERT_EQ(response.iam_user().sk(), "iam_sk"); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); +} + +TEST(MetaServiceTest, AlterRamTest) { + auto meta_service = get_meta_service(); + brpc::Controller cntl; + auto cloud_unique_id = "test_cloud_unique_id"; + std::string instance_id = "alter_iam_test_instance_id"; + [[maybe_unused]] auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_id", + [](void* p) { *reinterpret_cast(p) = 1; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->enable_processing(); + + config::arn_id = "iam_arn"; + config::arn_ak = "iam_ak"; + config::arn_sk = "iam_sk"; + + ObjectStoreInfoPB obj; + obj.set_ak("123"); + obj.set_sk("321"); + obj.set_bucket("456"); + obj.set_prefix("654"); + obj.set_endpoint("789"); + obj.set_region("987"); + obj.set_external_endpoint("888"); + obj.set_provider(ObjectStoreInfoPB::BOS); + + // create instance without ram user + CreateInstanceRequest create_instance_req; + create_instance_req.set_instance_id(instance_id); + create_instance_req.set_user_id("test_user"); + create_instance_req.set_name("test_name"); + create_instance_req.mutable_obj_info()->CopyFrom(obj); + CreateInstanceResponse create_instance_res; + meta_service->create_instance(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &create_instance_req, &create_instance_res, nullptr); + ASSERT_EQ(create_instance_res.status().code(), MetaServiceCode::OK); + + // get iam and ram user + GetIamRequest request; + request.set_cloud_unique_id(cloud_unique_id); + GetIamResponse response; + meta_service->get_iam(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &request, + &response, nullptr); + ASSERT_EQ(response.status().code(), MetaServiceCode::OK); + ASSERT_EQ(response.has_ram_user(), false); + ASSERT_EQ(response.iam_user().user_id(), "iam_arn"); + ASSERT_EQ(response.iam_user().ak(), "iam_ak"); + ASSERT_EQ(response.iam_user().sk(), "iam_sk"); + + // alter ram user + RamUserPB ram_user; + ram_user.set_user_id("test_user_id"); + ram_user.set_ak("test_ak"); + ram_user.set_sk("test_sk"); + AlterRamUserRequest alter_ram_user_request; + alter_ram_user_request.set_instance_id(instance_id); + alter_ram_user_request.mutable_ram_user()->CopyFrom(ram_user); + AlterRamUserResponse alter_ram_user_response; + meta_service->alter_ram_user(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &alter_ram_user_request, &alter_ram_user_response, nullptr); + + // get iam and ram user + meta_service->get_iam(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &request, + &response, nullptr); + ASSERT_EQ(response.status().code(), MetaServiceCode::OK); + ASSERT_EQ(response.has_ram_user(), true); + ASSERT_EQ(response.ram_user().user_id(), "test_user_id"); + ASSERT_EQ(response.ram_user().ak(), "test_ak"); + ASSERT_EQ(response.ram_user().sk(), "test_sk"); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); +} + +std::string to_raw_string(std::string_view v) { + std::string ret; + ret.reserve(v.size() / 1.5); + while (!v.empty()) { + if (v[0] == '\\') { + if (v[1] == 'x') { + ret.push_back(unhex(std::string_view {v.data() + 2, 2})[0]); + v.remove_prefix(4); + } else if (v[1] == '\\') { + ret.push_back('\\'); + v.remove_prefix(2); + } else { + std::abort(); + } + continue; + } + ret.push_back(v[0]); + v.remove_prefix(1); + } + return ret; +} + +TEST(MetaServiceTest, DecodeTest) { + // 504 + std::string v1 = + R"(\x08\x00\x10\xa0[\x18\xb3[ \xde\xc5\xa4\x8e\xbd\xf0\x97\xc62(\xf4\x96\xe6\xb0\x070\x018\x02@\x02H\x0bX\x05`\xa0\x07h\xa0\x07p\xa0\x01\x88\x01\x00\xa0\x01\x86\x8b\x9a\x9b\x06\xaa\x01\x16\x08\xe6\x9e\x91\xa3\xfb\xbe\xf5\xf0\xc4\x01\x10\xfe\x8b\x90\xa7\xb5\xec\xd5\xc8\xbf\x01\xb0\x01\x01\xba\x0100200000000000071fb4aabb58c570cbcadb10857d3131b97\xc2\x01\x011\xc8\x01\x84\x8b\x9a\x9b\x06\xd0\x01\x85\x8b\x9a\x9b\x06\xda\x01\x04\x0a\x00\x12\x00\xe2\x01\xcd\x02\x08\x02\x121\x08\x00\x12\x06datek1\x1a\x04DATE \x01*\x04NONE0\x01:\x0a2022-01-01@\x00H\x00P\x03X\x03\x80\x01\x01\x12>\x08\x01\x12\x06datek2\x1a\x08DATETIME \x01*\x04NONE0\x01:\x132022-01-01 11:11:11@\x00H\x00P\x08X\x08\x80\x01\x01\x123\x08\x04\x12\x06datev3\x1a\x06DATEV2 \x01*\x04NONE0\x01:\x0a2022-01-01@\x00H\x00P\x04X\x04\x80\x01\x01\x120\x08\x02\x12\x06datev1\x1a\x04DATE \x00*\x03MAX0\x01:\x0a2022-01-01@\x00H\x00P\x03X\x03\x80\x01\x01\x12=\x08\x03\x12\x06datev2\x1a\x08DATETIME \x00*\x03MAX0\x01:\x132022-01-01 11:11:11@\x00H\x00P\x08X\x08\x80\x01\x01\x18\x03 \x80\x08(\x021\x00\x00\x00\x00\x00\x00\x00\x008\x00@\xff\xff\xff\xff\xff\xff\xff\xff\xff\x01H\xff\xff\xff\xff\xff\xff\xff\xff\xff\x01P\x00X\x02`\x05h\x00p\x00\xe8\x01\x85\xae\x9f\x9b\x06\x98\x03\x02)"; + std::string val1 = to_raw_string(v1); + std::cout << "val1 size " << val1.size() << std::endl; + + // 525 + std::string v2 = + R"(\x08\x00\x10\xa0[\x18\xb3[ \x80\xb0\x85\xe3\xda\xcc\x8c\x0f(\xf4\x96\xe6\xb0\x070\x018\x01@\x0cH\x0cX\x00`\x00h\x00p\x00\x82\x01\x1e\x08\xff\xff\xff\xff\xff\xff\xff\xff\xff\x01\x12\x11datev3=2022-01-01\x88\x01\x01\x92\x01\x04\x08\x00\x10\x00\xa0\x01\x87\x8b\x9a\x9b\x06\xaa\x01\x16\x08\xe6\x9e\x91\xa3\xfb\xbe\xf5\xf0\xc4\x01\x10\xfe\x8b\x90\xa7\xb5\xec\xd5\xc8\xbf\x01\xb0\x01\x00\xba\x0100200000000000072fb4aabb58c570cbcadb10857d3131b97\xc8\x01\x87\x8b\x9a\x9b\x06\xd0\x01\x87\x8b\x9a\x9b\x06\xe2\x01\xcd\x02\x08\x02\x121\x08\x00\x12\x06datek1\x1a\x04DATE \x01*\x04NONE0\x01:\x0a2022-01-01@\x00H\x00P\x03X\x03\x80\x01\x01\x12>\x08\x01\x12\x06datek2\x1a\x08DATETIME \x01*\x04NONE0\x01:\x132022-01-01 11:11:11@\x00H\x00P\x08X\x08\x80\x01\x01\x123\x08\x04\x12\x06datev3\x1a\x06DATEV2 \x01*\x04NONE0\x01:\x0a2022-01-01@\x00H\x00P\x04X\x04\x80\x01\x01\x120\x08\x02\x12\x06datev1\x1a\x04DATE \x00*\x03MAX0\x01:\x0a2022-01-01@\x00H\x00P\x03X\x03\x80\x01\x01\x12=\x08\x03\x12\x06datev2\x1a\x08DATETIME \x00*\x03MAX0\x01:\x132022-01-01 11:11:11@\x00H\x00P\x08X\x08\x80\x01\x01\x18\x03 \x80\x08(\x021\x00\x00\x00\x00\x00\x00\x00\x008\x00@\xff\xff\xff\xff\xff\xff\xff\xff\xff\x01H\xff\xff\xff\xff\xff\xff\xff\xff\xff\x01P\x00X\x02`\x05h\x00p\x00\xe8\x01\x00\x98\x03\x02)"; + std::string val2 = to_raw_string(v2); + std::cout << "val2 size " << val2.size() << std::endl; + + [[maybe_unused]] std::string key1( + "\x01\x10meta\x00\x01\x10selectdb-cloud-" + "dev\x00\x01\x10rowset\x00\x01\x12\x00\x00\x00\x00\x00\x00-" + "\xb3\x12\x00\x00\x00\x00\x00\x00\x00\x0b", + 56); + [[maybe_unused]] std::string key2( + "\x01\x10meta\x00\x01\x10selectdb-cloud-" + "dev\x00\x01\x10rowset\x00\x01\x12\x00\x00\x00\x00\x00\x00-" + "\xb3\x12\x00\x00\x00\x00\x00\x00\x00\x0c", + 56); + std::cout << "key1 " << key1.size() << " " << hex(key1) << std::endl; + std::cout << "key2 " << key2.size() << " " << hex(key2) << std::endl; + + doris::RowsetMetaCloudPB rowset1; + doris::RowsetMetaCloudPB rowset2; + + rowset1.ParseFromString(val1); + rowset2.ParseFromString(val2); + std::cout << "rowset1=" << proto_to_json(rowset1) << std::endl; + std::cout << "rowset2=" << proto_to_json(rowset2) << std::endl; +} + +static void get_tablet_stats(MetaServiceProxy* meta_service, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id, GetTabletStatsResponse& res) { + brpc::Controller cntl; + GetTabletStatsRequest req; + auto idx = req.add_tablet_idx(); + idx->set_table_id(table_id); + idx->set_index_id(index_id); + idx->set_partition_id(partition_id); + idx->set_tablet_id(tablet_id); + meta_service->get_tablet_stats(&cntl, &req, &res, nullptr); +} + +TEST(MetaServiceTest, UpdateTablet) { + auto meta_service = get_meta_service(); + std::string cloud_unique_id = "test_cloud_unique_id"; + constexpr auto table_id = 11231, index_id = 11232, partition_id = 11233, tablet_id1 = 11234, + tablet_id2 = 21234; + ASSERT_NO_FATAL_FAILURE( + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id1)); + ASSERT_NO_FATAL_FAILURE( + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id2)); + auto get_and_check_tablet_meta = [&](int tablet_id, int64_t ttl_seconds, bool in_memory, + bool is_persistent) { + brpc::Controller cntl; + GetTabletRequest req; + req.set_cloud_unique_id(cloud_unique_id); + req.set_tablet_id(tablet_id); + GetTabletResponse resp; + meta_service->get_tablet(&cntl, &req, &resp, nullptr); + ASSERT_EQ(resp.status().code(), MetaServiceCode::OK) << tablet_id; + EXPECT_EQ(resp.tablet_meta().ttl_seconds(), ttl_seconds); + EXPECT_EQ(resp.tablet_meta().is_in_memory(), in_memory); + EXPECT_EQ(resp.tablet_meta().is_persistent(), is_persistent); + }; + get_and_check_tablet_meta(tablet_id1, 0, false, false); + get_and_check_tablet_meta(tablet_id2, 0, false, false); + { + brpc::Controller cntl; + UpdateTabletRequest req; + UpdateTabletResponse resp; + req.set_cloud_unique_id(cloud_unique_id); + TabletMetaInfoPB* tablet_meta_info = req.add_tablet_meta_infos(); + tablet_meta_info->set_tablet_id(tablet_id1); + tablet_meta_info->set_ttl_seconds(300); + tablet_meta_info = req.add_tablet_meta_infos(); + tablet_meta_info->set_tablet_id(tablet_id2); + tablet_meta_info->set_ttl_seconds(3000); + meta_service->update_tablet(&cntl, &req, &resp, nullptr); + ASSERT_EQ(resp.status().code(), MetaServiceCode::OK); + } + get_and_check_tablet_meta(tablet_id1, 300, false, false); + get_and_check_tablet_meta(tablet_id2, 3000, false, false); + { + brpc::Controller cntl; + UpdateTabletRequest req; + UpdateTabletResponse resp; + req.set_cloud_unique_id(cloud_unique_id); + TabletMetaInfoPB* tablet_meta_info = req.add_tablet_meta_infos(); + tablet_meta_info->set_tablet_id(tablet_id1); + tablet_meta_info->set_is_in_memory(true); + meta_service->update_tablet(&cntl, &req, &resp, nullptr); + ASSERT_EQ(resp.status().code(), MetaServiceCode::OK); + } + { + brpc::Controller cntl; + UpdateTabletRequest req; + UpdateTabletResponse resp; + req.set_cloud_unique_id(cloud_unique_id); + TabletMetaInfoPB* tablet_meta_info = req.add_tablet_meta_infos(); + tablet_meta_info->set_tablet_id(tablet_id1); + tablet_meta_info->set_is_persistent(true); + meta_service->update_tablet(&cntl, &req, &resp, nullptr); + ASSERT_EQ(resp.status().code(), MetaServiceCode::OK); + } + get_and_check_tablet_meta(tablet_id1, 300, true, true); +} + +TEST(MetaServiceTest, GetTabletStatsTest) { + auto meta_service = get_meta_service(); + + constexpr auto table_id = 10001, index_id = 10002, partition_id = 10003, tablet_id = 10004; + ASSERT_NO_FATAL_FAILURE( + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id)); + GetTabletStatsResponse res; + get_tablet_stats(meta_service.get(), table_id, index_id, partition_id, tablet_id, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.tablet_stats_size(), 1); + EXPECT_EQ(res.tablet_stats(0).data_size(), 0); + EXPECT_EQ(res.tablet_stats(0).num_rows(), 0); + EXPECT_EQ(res.tablet_stats(0).num_rowsets(), 1); + EXPECT_EQ(res.tablet_stats(0).num_segments(), 0); + // Insert rowset + config::split_tablet_stats = false; + ASSERT_NO_FATAL_FAILURE( + insert_rowset(meta_service.get(), 10000, "label1", table_id, partition_id, tablet_id)); + ASSERT_NO_FATAL_FAILURE( + insert_rowset(meta_service.get(), 10000, "label2", table_id, partition_id, tablet_id)); + config::split_tablet_stats = true; + ASSERT_NO_FATAL_FAILURE( + insert_rowset(meta_service.get(), 10000, "label3", table_id, partition_id, tablet_id)); + ASSERT_NO_FATAL_FAILURE( + insert_rowset(meta_service.get(), 10000, "label4", table_id, partition_id, tablet_id)); + // Check tablet stats kv + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string data_size_key, data_size_val; + stats_tablet_data_size_key({mock_instance, table_id, index_id, partition_id, tablet_id}, + &data_size_key); + ASSERT_EQ(txn->get(data_size_key, &data_size_val), TxnErrorCode::TXN_OK); + EXPECT_EQ(*(int64_t*)data_size_val.data(), 20000); + std::string num_rows_key, num_rows_val; + stats_tablet_num_rows_key({mock_instance, table_id, index_id, partition_id, tablet_id}, + &num_rows_key); + ASSERT_EQ(txn->get(num_rows_key, &num_rows_val), TxnErrorCode::TXN_OK); + EXPECT_EQ(*(int64_t*)num_rows_val.data(), 200); + std::string num_rowsets_key, num_rowsets_val; + stats_tablet_num_rowsets_key({mock_instance, table_id, index_id, partition_id, tablet_id}, + &num_rowsets_key); + ASSERT_EQ(txn->get(num_rowsets_key, &num_rowsets_val), TxnErrorCode::TXN_OK); + EXPECT_EQ(*(int64_t*)num_rowsets_val.data(), 2); + std::string num_segs_key, num_segs_val; + stats_tablet_num_segs_key({mock_instance, table_id, index_id, partition_id, tablet_id}, + &num_segs_key); + ASSERT_EQ(txn->get(num_segs_key, &num_segs_val), TxnErrorCode::TXN_OK); + EXPECT_EQ(*(int64_t*)num_segs_val.data(), 2); + // Get tablet stats + res.Clear(); + get_tablet_stats(meta_service.get(), table_id, index_id, partition_id, tablet_id, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.tablet_stats_size(), 1); + EXPECT_EQ(res.tablet_stats(0).data_size(), 40000); + EXPECT_EQ(res.tablet_stats(0).num_rows(), 400); + EXPECT_EQ(res.tablet_stats(0).num_rowsets(), 5); + EXPECT_EQ(res.tablet_stats(0).num_segments(), 4); +} + +TEST(MetaServiceTest, GetDeleteBitmapUpdateLock) { + auto meta_service = get_meta_service(); + + brpc::Controller cntl; + GetDeleteBitmapUpdateLockRequest req; + GetDeleteBitmapUpdateLockResponse res; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_table_id(111); + req.add_partition_ids(123); + req.set_expiration(5); + req.set_lock_id(888); + req.set_initiator(-1); + meta_service->get_delete_bitmap_update_lock( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + // same lock_id + meta_service->get_delete_bitmap_update_lock( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + + // different lock_id + req.set_lock_id(999); + meta_service->get_delete_bitmap_update_lock( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::LOCK_CONFLICT); + + // lock expired + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_table_id(222); + req.set_expiration(0); + req.set_lock_id(666); + meta_service->get_delete_bitmap_update_lock( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + sleep(1); + req.set_lock_id(667); + meta_service->get_delete_bitmap_update_lock( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); +} + +static std::string generate_random_string(int length) { + std::string char_set = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789"; + std::random_device rd; + std::mt19937 generator(rd()); + std::uniform_int_distribution distribution(0, char_set.length() - 1); + + std::string randomString; + for (int i = 0; i < length; ++i) { + randomString += char_set[distribution(generator)]; + } + return randomString; +} + +TEST(MetaServiceTest, UpdateDeleteBitmap) { + auto meta_service = get_meta_service(); + + // get delete bitmap update lock + brpc::Controller cntl; + GetDeleteBitmapUpdateLockRequest get_lock_req; + GetDeleteBitmapUpdateLockResponse get_lock_res; + get_lock_req.set_cloud_unique_id("test_cloud_unique_id"); + get_lock_req.set_table_id(112); + get_lock_req.add_partition_ids(123); + get_lock_req.set_expiration(5); + get_lock_req.set_lock_id(888); + get_lock_req.set_initiator(-1); + meta_service->get_delete_bitmap_update_lock( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &get_lock_req, + &get_lock_res, nullptr); + ASSERT_EQ(get_lock_res.status().code(), MetaServiceCode::OK); + + // first update delete bitmap + { + UpdateDeleteBitmapRequest update_delete_bitmap_req; + UpdateDeleteBitmapResponse update_delete_bitmap_res; + update_delete_bitmap_req.set_cloud_unique_id("test_cloud_unique_id"); + update_delete_bitmap_req.set_table_id(112); + update_delete_bitmap_req.set_partition_id(123); + update_delete_bitmap_req.set_lock_id(888); + update_delete_bitmap_req.set_initiator(-1); + update_delete_bitmap_req.set_tablet_id(333); + + update_delete_bitmap_req.add_rowset_ids("123"); + update_delete_bitmap_req.add_segment_ids(1); + update_delete_bitmap_req.add_versions(2); + update_delete_bitmap_req.add_segment_delete_bitmaps("abc0"); + + update_delete_bitmap_req.add_rowset_ids("123"); + update_delete_bitmap_req.add_segment_ids(0); + update_delete_bitmap_req.add_versions(3); + update_delete_bitmap_req.add_segment_delete_bitmaps("abc1"); + + update_delete_bitmap_req.add_rowset_ids("123"); + update_delete_bitmap_req.add_segment_ids(1); + update_delete_bitmap_req.add_versions(3); + update_delete_bitmap_req.add_segment_delete_bitmaps("abc2"); + + update_delete_bitmap_req.add_rowset_ids("124"); + update_delete_bitmap_req.add_segment_ids(0); + update_delete_bitmap_req.add_versions(2); + update_delete_bitmap_req.add_segment_delete_bitmaps("abc3"); + + std::string large_value = generate_random_string(300 * 1000); + update_delete_bitmap_req.add_rowset_ids("124"); + update_delete_bitmap_req.add_segment_ids(1); + update_delete_bitmap_req.add_versions(2); + update_delete_bitmap_req.add_segment_delete_bitmaps(large_value); + + update_delete_bitmap_req.add_rowset_ids("124"); + update_delete_bitmap_req.add_segment_ids(0); + update_delete_bitmap_req.add_versions(3); + update_delete_bitmap_req.add_segment_delete_bitmaps("abc4"); + + meta_service->update_delete_bitmap( + reinterpret_cast(&cntl), + &update_delete_bitmap_req, &update_delete_bitmap_res, nullptr); + ASSERT_EQ(update_delete_bitmap_res.status().code(), MetaServiceCode::OK); + + // first get delete bitmap + GetDeleteBitmapRequest get_delete_bitmap_req; + GetDeleteBitmapResponse get_delete_bitmap_res; + get_delete_bitmap_req.set_cloud_unique_id("test_cloud_unique_id"); + get_delete_bitmap_req.set_tablet_id(333); + + get_delete_bitmap_req.add_rowset_ids("123"); + get_delete_bitmap_req.add_begin_versions(3); + get_delete_bitmap_req.add_end_versions(3); + + get_delete_bitmap_req.add_rowset_ids("124"); + get_delete_bitmap_req.add_begin_versions(0); + get_delete_bitmap_req.add_end_versions(3); + + meta_service->get_delete_bitmap(reinterpret_cast(&cntl), + &get_delete_bitmap_req, &get_delete_bitmap_res, nullptr); + ASSERT_EQ(get_delete_bitmap_res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(get_delete_bitmap_res.rowset_ids_size(), 5); + ASSERT_EQ(get_delete_bitmap_res.segment_delete_bitmaps_size(), 5); + ASSERT_EQ(get_delete_bitmap_res.versions_size(), 5); + ASSERT_EQ(get_delete_bitmap_res.segment_delete_bitmaps_size(), 5); + + ASSERT_EQ(get_delete_bitmap_res.rowset_ids(0), "123"); + ASSERT_EQ(get_delete_bitmap_res.segment_ids(0), 0); + ASSERT_EQ(get_delete_bitmap_res.versions(0), 3); + ASSERT_EQ(get_delete_bitmap_res.segment_delete_bitmaps(0), "abc1"); + + ASSERT_EQ(get_delete_bitmap_res.rowset_ids(1), "123"); + ASSERT_EQ(get_delete_bitmap_res.segment_ids(1), 1); + ASSERT_EQ(get_delete_bitmap_res.versions(1), 3); + ASSERT_EQ(get_delete_bitmap_res.segment_delete_bitmaps(1), "abc2"); + + ASSERT_EQ(get_delete_bitmap_res.rowset_ids(2), "124"); + ASSERT_EQ(get_delete_bitmap_res.segment_ids(2), 0); + ASSERT_EQ(get_delete_bitmap_res.versions(2), 2); + ASSERT_EQ(get_delete_bitmap_res.segment_delete_bitmaps(2), "abc3"); + + ASSERT_EQ(get_delete_bitmap_res.rowset_ids(3), "124"); + ASSERT_EQ(get_delete_bitmap_res.segment_ids(3), 1); + ASSERT_EQ(get_delete_bitmap_res.versions(3), 2); + ASSERT_EQ(get_delete_bitmap_res.segment_delete_bitmaps(3), large_value); + + ASSERT_EQ(get_delete_bitmap_res.rowset_ids(4), "124"); + ASSERT_EQ(get_delete_bitmap_res.segment_ids(4), 0); + ASSERT_EQ(get_delete_bitmap_res.versions(4), 3); + ASSERT_EQ(get_delete_bitmap_res.segment_delete_bitmaps(4), "abc4"); + } + + // second update delete bitmap + { + UpdateDeleteBitmapRequest update_delete_bitmap_req; + UpdateDeleteBitmapResponse update_delete_bitmap_res; + update_delete_bitmap_req.set_cloud_unique_id("test_cloud_unique_id"); + update_delete_bitmap_req.set_table_id(112); + update_delete_bitmap_req.set_partition_id(123); + update_delete_bitmap_req.set_lock_id(888); + update_delete_bitmap_req.set_initiator(-1); + update_delete_bitmap_req.set_tablet_id(333); + + std::string large_value = generate_random_string(200 * 1000); + update_delete_bitmap_req.add_rowset_ids("123"); + update_delete_bitmap_req.add_segment_ids(0); + update_delete_bitmap_req.add_versions(2); + update_delete_bitmap_req.add_segment_delete_bitmaps(large_value); + + update_delete_bitmap_req.add_rowset_ids("123"); + update_delete_bitmap_req.add_segment_ids(1); + update_delete_bitmap_req.add_versions(2); + update_delete_bitmap_req.add_segment_delete_bitmaps("bbb0"); + + update_delete_bitmap_req.add_rowset_ids("123"); + update_delete_bitmap_req.add_segment_ids(1); + update_delete_bitmap_req.add_versions(3); + update_delete_bitmap_req.add_segment_delete_bitmaps("bbb1"); + + update_delete_bitmap_req.add_rowset_ids("124"); + update_delete_bitmap_req.add_segment_ids(1); + update_delete_bitmap_req.add_versions(3); + update_delete_bitmap_req.add_segment_delete_bitmaps("bbb2"); + + meta_service->update_delete_bitmap( + reinterpret_cast(&cntl), + &update_delete_bitmap_req, &update_delete_bitmap_res, nullptr); + ASSERT_EQ(update_delete_bitmap_res.status().code(), MetaServiceCode::OK); + + // second get delete bitmap + GetDeleteBitmapRequest get_delete_bitmap_req; + GetDeleteBitmapResponse get_delete_bitmap_res; + get_delete_bitmap_req.set_cloud_unique_id("test_cloud_unique_id"); + get_delete_bitmap_req.set_tablet_id(333); + + get_delete_bitmap_req.add_rowset_ids("123"); + get_delete_bitmap_req.add_begin_versions(0); + get_delete_bitmap_req.add_end_versions(3); + + get_delete_bitmap_req.add_rowset_ids("124"); + get_delete_bitmap_req.add_begin_versions(0); + get_delete_bitmap_req.add_end_versions(3); + + meta_service->get_delete_bitmap(reinterpret_cast(&cntl), + &get_delete_bitmap_req, &get_delete_bitmap_res, nullptr); + ASSERT_EQ(get_delete_bitmap_res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(get_delete_bitmap_res.rowset_ids_size(), 4); + ASSERT_EQ(get_delete_bitmap_res.segment_delete_bitmaps_size(), 4); + ASSERT_EQ(get_delete_bitmap_res.versions_size(), 4); + ASSERT_EQ(get_delete_bitmap_res.segment_delete_bitmaps_size(), 4); + + ASSERT_EQ(get_delete_bitmap_res.rowset_ids(0), "123"); + ASSERT_EQ(get_delete_bitmap_res.segment_ids(0), 0); + ASSERT_EQ(get_delete_bitmap_res.versions(0), 2); + ASSERT_EQ(get_delete_bitmap_res.segment_delete_bitmaps(0), large_value); + + ASSERT_EQ(get_delete_bitmap_res.rowset_ids(1), "123"); + ASSERT_EQ(get_delete_bitmap_res.segment_ids(1), 1); + ASSERT_EQ(get_delete_bitmap_res.versions(1), 2); + ASSERT_EQ(get_delete_bitmap_res.segment_delete_bitmaps(1), "bbb0"); + + ASSERT_EQ(get_delete_bitmap_res.rowset_ids(2), "123"); + ASSERT_EQ(get_delete_bitmap_res.segment_ids(2), 1); + ASSERT_EQ(get_delete_bitmap_res.versions(2), 3); + ASSERT_EQ(get_delete_bitmap_res.segment_delete_bitmaps(2), "bbb1"); + + ASSERT_EQ(get_delete_bitmap_res.rowset_ids(3), "124"); + ASSERT_EQ(get_delete_bitmap_res.segment_ids(3), 1); + ASSERT_EQ(get_delete_bitmap_res.versions(3), 3); + ASSERT_EQ(get_delete_bitmap_res.segment_delete_bitmaps(3), "bbb2"); + } + + // large size txn + { + UpdateDeleteBitmapRequest update_delete_bitmap_req; + UpdateDeleteBitmapResponse update_delete_bitmap_res; + update_delete_bitmap_req.set_cloud_unique_id("test_cloud_unique_id"); + update_delete_bitmap_req.set_table_id(112); + update_delete_bitmap_req.set_partition_id(123); + update_delete_bitmap_req.set_lock_id(888); + update_delete_bitmap_req.set_initiator(-1); + update_delete_bitmap_req.set_tablet_id(333); + + std::string large_value = generate_random_string(300 * 1000); + for (size_t i = 0; i < 100; ++i) { + update_delete_bitmap_req.add_rowset_ids("123"); + update_delete_bitmap_req.add_segment_ids(1); + update_delete_bitmap_req.add_versions(i); + update_delete_bitmap_req.add_segment_delete_bitmaps(large_value); + } + + update_delete_bitmap_req.add_rowset_ids("124"); + update_delete_bitmap_req.add_segment_ids(0); + update_delete_bitmap_req.add_versions(3); + update_delete_bitmap_req.add_segment_delete_bitmaps("abcd4"); + + meta_service->update_delete_bitmap( + reinterpret_cast(&cntl), + &update_delete_bitmap_req, &update_delete_bitmap_res, nullptr); + ASSERT_EQ(update_delete_bitmap_res.status().code(), MetaServiceCode::OK); + + GetDeleteBitmapRequest get_delete_bitmap_req; + GetDeleteBitmapResponse get_delete_bitmap_res; + get_delete_bitmap_req.set_cloud_unique_id("test_cloud_unique_id"); + get_delete_bitmap_req.set_tablet_id(333); + + get_delete_bitmap_req.add_rowset_ids("123"); + get_delete_bitmap_req.add_begin_versions(0); + get_delete_bitmap_req.add_end_versions(101); + + get_delete_bitmap_req.add_rowset_ids("124"); + get_delete_bitmap_req.add_begin_versions(0); + get_delete_bitmap_req.add_end_versions(3); + + meta_service->get_delete_bitmap(reinterpret_cast(&cntl), + &get_delete_bitmap_req, &get_delete_bitmap_res, nullptr); + ASSERT_EQ(get_delete_bitmap_res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(get_delete_bitmap_res.rowset_ids_size(), 101); + ASSERT_EQ(get_delete_bitmap_res.segment_delete_bitmaps_size(), 101); + ASSERT_EQ(get_delete_bitmap_res.versions_size(), 101); + ASSERT_EQ(get_delete_bitmap_res.segment_delete_bitmaps_size(), 101); + + for (size_t i = 0; i < 100; ++i) { + ASSERT_EQ(get_delete_bitmap_res.rowset_ids(i), "123"); + ASSERT_EQ(get_delete_bitmap_res.segment_ids(i), 1); + ASSERT_EQ(get_delete_bitmap_res.versions(i), i); + ASSERT_EQ(get_delete_bitmap_res.segment_delete_bitmaps(i), large_value); + } + + ASSERT_EQ(get_delete_bitmap_res.rowset_ids(100), "124"); + ASSERT_EQ(get_delete_bitmap_res.segment_ids(100), 0); + ASSERT_EQ(get_delete_bitmap_res.versions(100), 3); + ASSERT_EQ(get_delete_bitmap_res.segment_delete_bitmaps(100), "abcd4"); + } +} + +TEST(MetaServiceTest, UpdateDeleteBitmapWithException) { + auto meta_service = get_meta_service(); + brpc::Controller cntl; + + { + UpdateDeleteBitmapRequest update_delete_bitmap_req; + UpdateDeleteBitmapResponse update_delete_bitmap_res; + update_delete_bitmap_req.set_cloud_unique_id("test_cloud_unique_id"); + update_delete_bitmap_req.set_table_id(112); + update_delete_bitmap_req.set_partition_id(123); + update_delete_bitmap_req.set_lock_id(888); + update_delete_bitmap_req.set_initiator(-1); + update_delete_bitmap_req.set_tablet_id(333); + + update_delete_bitmap_req.add_rowset_ids("123"); + update_delete_bitmap_req.add_segment_ids(1); + update_delete_bitmap_req.add_versions(2); + update_delete_bitmap_req.add_segment_delete_bitmaps("abc0"); + + meta_service->update_delete_bitmap( + reinterpret_cast(&cntl), + &update_delete_bitmap_req, &update_delete_bitmap_res, nullptr); + ASSERT_EQ(update_delete_bitmap_res.status().code(), MetaServiceCode::LOCK_EXPIRED); + } + + // get delete bitmap update lock + GetDeleteBitmapUpdateLockRequest get_lock_req; + GetDeleteBitmapUpdateLockResponse get_lock_res; + get_lock_req.set_cloud_unique_id("test_cloud_unique_id"); + get_lock_req.set_table_id(112); + get_lock_req.add_partition_ids(123); + get_lock_req.set_expiration(5); + get_lock_req.set_lock_id(888); + get_lock_req.set_initiator(-1); + meta_service->get_delete_bitmap_update_lock( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &get_lock_req, + &get_lock_res, nullptr); + ASSERT_EQ(get_lock_res.status().code(), MetaServiceCode::OK); + + { + UpdateDeleteBitmapRequest update_delete_bitmap_req; + UpdateDeleteBitmapResponse update_delete_bitmap_res; + update_delete_bitmap_req.set_cloud_unique_id("test_cloud_unique_id"); + update_delete_bitmap_req.set_table_id(112); + update_delete_bitmap_req.set_partition_id(123); + update_delete_bitmap_req.set_lock_id(222); + update_delete_bitmap_req.set_initiator(-1); + update_delete_bitmap_req.set_tablet_id(333); + + update_delete_bitmap_req.add_rowset_ids("123"); + update_delete_bitmap_req.add_segment_ids(1); + update_delete_bitmap_req.add_versions(2); + update_delete_bitmap_req.add_segment_delete_bitmaps("abc0"); + + meta_service->update_delete_bitmap( + reinterpret_cast(&cntl), + &update_delete_bitmap_req, &update_delete_bitmap_res, nullptr); + ASSERT_EQ(update_delete_bitmap_res.status().code(), MetaServiceCode::LOCK_EXPIRED); + } + + { + UpdateDeleteBitmapRequest update_delete_bitmap_req; + UpdateDeleteBitmapResponse update_delete_bitmap_res; + update_delete_bitmap_req.set_cloud_unique_id("test_cloud_unique_id"); + update_delete_bitmap_req.set_table_id(112); + update_delete_bitmap_req.set_partition_id(123); + update_delete_bitmap_req.set_lock_id(888); + update_delete_bitmap_req.set_initiator(-2); + update_delete_bitmap_req.set_tablet_id(333); + + update_delete_bitmap_req.add_rowset_ids("123"); + update_delete_bitmap_req.add_segment_ids(1); + update_delete_bitmap_req.add_versions(2); + update_delete_bitmap_req.add_segment_delete_bitmaps("abc0"); + + meta_service->update_delete_bitmap( + reinterpret_cast(&cntl), + &update_delete_bitmap_req, &update_delete_bitmap_res, nullptr); + ASSERT_EQ(update_delete_bitmap_res.status().code(), MetaServiceCode::LOCK_EXPIRED); + } +} + +TEST(MetaServiceTest, GetDeleteBitmapWithIdx) { + auto meta_service = get_meta_service(); + extern std::string get_instance_id(const std::shared_ptr& rc_mgr, + const std::string& cloud_unique_id); + auto instance_id = get_instance_id(meta_service->resource_mgr(), "test_cloud_unique_id"); + int64_t db_id = 1; + int64_t table_id = 1; + int64_t index_id = 1; + int64_t partition_id = 1; + int64_t tablet_id = 123; + + brpc::Controller cntl; + GetDeleteBitmapRequest req; + GetDeleteBitmapResponse res; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_tablet_id(tablet_id); + TabletIndexPB idx; + idx.set_tablet_id(tablet_id); + idx.set_index_id(index_id); + idx.set_db_id(db_id); + idx.set_partition_id(partition_id); + idx.set_table_id(table_id); + *(req.mutable_idx()) = idx; + req.set_base_compaction_cnt(9); + req.set_cumulative_compaction_cnt(19); + req.set_cumulative_point(21); + + meta_service->get_delete_bitmap(reinterpret_cast(&cntl), &req, + &res, nullptr); + EXPECT_EQ(res.status().code(), MetaServiceCode::TABLET_NOT_FOUND); + + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string stats_key = + stats_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + TabletStatsPB stats; + stats.set_base_compaction_cnt(9); + stats.set_cumulative_compaction_cnt(19); + stats.set_cumulative_point(20); + txn->put(stats_key, stats.SerializeAsString()); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + meta_service->get_delete_bitmap(reinterpret_cast(&cntl), &req, + &res, nullptr); + EXPECT_EQ(res.status().code(), MetaServiceCode::ROWSETS_EXPIRED); + + req.set_cumulative_point(20); + meta_service->get_delete_bitmap(reinterpret_cast(&cntl), &req, + &res, nullptr); + EXPECT_EQ(res.status().code(), MetaServiceCode::OK); + + req.add_rowset_ids("1234"); + req.add_begin_versions(1); + req.add_end_versions(2); + req.add_end_versions(3); + meta_service->get_delete_bitmap(reinterpret_cast(&cntl), &req, + &res, nullptr); + EXPECT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); +} + +TEST(MetaServiceTest, DeleteBimapCommitTxnTest) { + auto meta_service = get_meta_service(); + extern std::string get_instance_id(const std::shared_ptr& rc_mgr, + const std::string& cloud_unique_id); + auto instance_id = get_instance_id(meta_service->resource_mgr(), "test_cloud_unique_id"); + + // case: first version of rowset + { + int64_t txn_id = -1; + int64_t table_id = 123456; // same as table_id of tmp rowset + int64_t db_id = 222; + int64_t tablet_id_base = 8113; + int64_t partition_id = 1234; + // begin txn + { + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + TxnInfoPB txn_info_pb; + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label("test_label"); + txn_info_pb.add_table_ids(table_id); + txn_info_pb.set_timeout_ms(36000); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + txn_id = res.txn_id(); + } + + // mock rowset and tablet + for (int i = 0; i < 5; ++i) { + create_tablet(meta_service.get(), table_id, 1235, partition_id, tablet_id_base + i); + auto tmp_rowset = create_rowset(txn_id, tablet_id_base + i); + tmp_rowset.set_partition_id(partition_id); + CreateRowsetResponse res; + commit_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // update delete bitmap + { + // get delete bitmap update lock + brpc::Controller cntl; + GetDeleteBitmapUpdateLockRequest get_lock_req; + GetDeleteBitmapUpdateLockResponse get_lock_res; + get_lock_req.set_cloud_unique_id("test_cloud_unique_id"); + get_lock_req.set_table_id(table_id); + get_lock_req.add_partition_ids(partition_id); + get_lock_req.set_expiration(5); + get_lock_req.set_lock_id(txn_id); + get_lock_req.set_initiator(-1); + meta_service->get_delete_bitmap_update_lock( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &get_lock_req, + &get_lock_res, nullptr); + ASSERT_EQ(get_lock_res.status().code(), MetaServiceCode::OK); + + // first update delete bitmap + UpdateDeleteBitmapRequest update_delete_bitmap_req; + UpdateDeleteBitmapResponse update_delete_bitmap_res; + update_delete_bitmap_req.set_cloud_unique_id("test_cloud_unique_id"); + update_delete_bitmap_req.set_table_id(table_id); + update_delete_bitmap_req.set_partition_id(partition_id); + update_delete_bitmap_req.set_lock_id(txn_id); + update_delete_bitmap_req.set_initiator(-1); + update_delete_bitmap_req.set_tablet_id(tablet_id_base); + + update_delete_bitmap_req.add_rowset_ids("123"); + update_delete_bitmap_req.add_segment_ids(1); + update_delete_bitmap_req.add_versions(2); + update_delete_bitmap_req.add_segment_delete_bitmaps("abc0"); + + meta_service->update_delete_bitmap( + reinterpret_cast(&cntl), + &update_delete_bitmap_req, &update_delete_bitmap_res, nullptr); + ASSERT_EQ(update_delete_bitmap_res.status().code(), MetaServiceCode::OK); + } + + // check delete bitmap update lock and pending delete bitmap + { + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string lock_key = meta_delete_bitmap_update_lock_key({instance_id, table_id, -1}); + std::string lock_val; + auto ret = txn->get(lock_key, &lock_val); + ASSERT_EQ(ret, TxnErrorCode::TXN_OK); + + std::string pending_key = meta_pending_delete_bitmap_key({instance_id, tablet_id_base}); + std::string pending_val; + ret = txn->get(pending_key, &pending_val); + ASSERT_EQ(ret, TxnErrorCode::TXN_OK); + } + + // commit txn + { + brpc::Controller cntl; + CommitTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(db_id); + req.set_txn_id(txn_id); + req.add_mow_table_ids(table_id); + CommitTxnResponse res; + meta_service->commit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + + // check delete bitmap update lock and pending delete bitmap + { + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string lock_key = meta_delete_bitmap_update_lock_key({instance_id, table_id, -1}); + std::string lock_val; + auto ret = txn->get(lock_key, &lock_val); + ASSERT_EQ(ret, TxnErrorCode::TXN_KEY_NOT_FOUND); + + std::string pending_key = meta_pending_delete_bitmap_key({instance_id, tablet_id_base}); + std::string pending_val; + ret = txn->get(pending_key, &pending_val); + ASSERT_EQ(ret, TxnErrorCode::TXN_KEY_NOT_FOUND); + } + } +} + +TEST(MetaServiceTest, GetVersion) { + auto service = get_meta_service(); + + int64_t table_id = 1; + int64_t partition_id = 1; + int64_t tablet_id = 1; + + // INVALID_ARGUMENT + { + brpc::Controller ctrl; + GetVersionRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_table_id(table_id); + req.set_partition_id(partition_id); + + GetVersionResponse resp; + service->get_version(&ctrl, &req, &resp, nullptr); + + ASSERT_EQ(resp.status().code(), MetaServiceCode::INVALID_ARGUMENT) + << " status is " << resp.status().DebugString(); + } + + { + brpc::Controller ctrl; + GetVersionRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(1); + req.set_table_id(table_id); + req.set_partition_id(partition_id); + + GetVersionResponse resp; + service->get_version(&ctrl, &req, &resp, nullptr); + + ASSERT_EQ(resp.status().code(), MetaServiceCode::VERSION_NOT_FOUND) + << " status is " << resp.status().DebugString(); + } + + create_tablet(service.get(), table_id, 1, partition_id, tablet_id); + insert_rowset(service.get(), 1, "get_version_label_1", table_id, partition_id, tablet_id); + + { + brpc::Controller ctrl; + GetVersionRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(1); + req.set_table_id(table_id); + req.set_partition_id(partition_id); + + GetVersionResponse resp; + service->get_version(&ctrl, &req, &resp, nullptr); + + ASSERT_EQ(resp.status().code(), MetaServiceCode::OK) + << " status is " << resp.status().DebugString(); + ASSERT_EQ(resp.version(), 2); + } +} + +TEST(MetaServiceTest, BatchGetVersion) { + struct TestCase { + std::vector table_ids; + std::vector partition_ids; + std::vector expected_versions; + std::vector< + std::tuple> + insert_rowsets; + }; + + std::vector cases = { + // all version are missing + {{1, 1, 2, 3}, {1, 2, 1, 2}, {-1, -1, -1, -1}, {}}, + // update table 1, partition 1 + {{1, 1, 2, 3}, {1, 2, 1, 2}, {2, -1, -1, -1}, {{1, 1, 1}}}, + // update table 2, partition 1 + // update table 3, partition 2 + {{1, 1, 2, 3}, {1, 2, 1, 2}, {2, -1, 2, 2}, {{2, 1, 3}, {3, 2, 4}}}, + // update table 1, partition 2 twice + {{1, 1, 2, 3}, {1, 2, 1, 2}, {2, 3, 2, 2}, {{1, 2, 2}, {1, 2, 2}}}, + }; + + auto service = get_meta_service(); + create_tablet(service.get(), 1, 1, 1, 1); + create_tablet(service.get(), 1, 1, 2, 2); + create_tablet(service.get(), 2, 1, 1, 3); + create_tablet(service.get(), 3, 1, 2, 4); + + size_t num_cases = cases.size(); + size_t label_index = 0; + for (size_t i = 0; i < num_cases; ++i) { + auto& [table_ids, partition_ids, expected_versions, insert_rowsets] = cases[i]; + for (auto [table_id, partition_id, tablet_id] : insert_rowsets) { + LOG(INFO) << "insert rowset for table " << table_id << " partition " << partition_id + << " table_id " << tablet_id; + insert_rowset(service.get(), 1, std::to_string(++label_index), table_id, partition_id, + tablet_id); + } + + brpc::Controller ctrl; + GetVersionRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(-1); + req.set_table_id(-1); + req.set_partition_id(-1); + req.set_batch_mode(true); + for (size_t i = 0; i < table_ids.size(); ++i) req.add_db_ids(1); + std::copy(table_ids.begin(), table_ids.end(), + google::protobuf::RepeatedFieldBackInserter(req.mutable_table_ids())); + std::copy(partition_ids.begin(), partition_ids.end(), + google::protobuf::RepeatedFieldBackInserter(req.mutable_partition_ids())); + + GetVersionResponse resp; + service->get_version(&ctrl, &req, &resp, nullptr); + + ASSERT_EQ(resp.status().code(), MetaServiceCode::OK) + << "case " << i << " status is " << resp.status().msg() + << ", code=" << resp.status().code(); + + std::vector versions(resp.versions().begin(), resp.versions().end()); + EXPECT_EQ(versions, expected_versions) << "case " << i; + } + + // INVALID_ARGUMENT + { + brpc::Controller ctrl; + GetVersionRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_batch_mode(true); + GetVersionResponse resp; + service->get_version(&ctrl, &req, &resp, nullptr); + ASSERT_EQ(resp.status().code(), MetaServiceCode::INVALID_ARGUMENT) + << " status is " << resp.status().msg() << ", code=" << resp.status().code(); + } +} + +TEST(MetaServiceTest, BatchGetVersionFallback) { + constexpr size_t N = 100; + size_t i = 0; + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("batch_get_version_err", [&](void* args) { + if (i++ == N / 10) { + *reinterpret_cast(args) = TxnErrorCode::TXN_TOO_OLD; + } + }); + + sp->enable_processing(); + + auto service = get_meta_service(); + for (int64_t i = 1; i <= N; ++i) { + create_tablet(service.get(), 1, 1, i, i); + insert_rowset(service.get(), 1, std::to_string(i), 1, i, i); + } + + brpc::Controller ctrl; + GetVersionRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(-1); + req.set_table_id(-1); + req.set_partition_id(-1); + req.set_batch_mode(true); + for (size_t i = 1; i <= N; ++i) { + req.add_db_ids(1); + req.add_table_ids(1); + req.add_partition_ids(i); + } + + GetVersionResponse resp; + service->get_version(&ctrl, &req, &resp, nullptr); + + ASSERT_EQ(resp.status().code(), MetaServiceCode::OK) + << "case " << i << " status is " << resp.status().msg() + << ", code=" << resp.status().code(); + + ASSERT_EQ(resp.versions_size(), N); +} + +extern bool is_dropped_tablet(Transaction* txn, const std::string& instance_id, int64_t index_id, + int64_t partition_id); + +TEST(MetaServiceTest, IsDroppedTablet) { + auto meta_service = get_meta_service(); + std::string instance_id = "IsDroppedTablet"; + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + meta_service = get_meta_service(); + auto reset_meta_service = [&meta_service] { meta_service = get_meta_service(); }; + + constexpr int64_t index_id = 10002; + constexpr int64_t partition_id = 10003; + + std::unique_ptr txn; + RecycleIndexPB index_pb; + auto index_key = recycle_index_key({instance_id, index_id}); + RecyclePartitionPB partition_pb; + auto partition_key = recycle_partition_key({instance_id, partition_id}); + std::string val; + // No recycle index and partition kv + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + EXPECT_FALSE(is_dropped_tablet(txn.get(), instance_id, index_id, partition_id)); + // Tablet in PREPARED index + index_pb.set_state(RecycleIndexPB::PREPARED); + val = index_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(index_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + EXPECT_FALSE(is_dropped_tablet(txn.get(), instance_id, index_id, partition_id)); + // Tablet in DROPPED/RECYCLING index + reset_meta_service(); + index_pb.set_state(RecycleIndexPB::DROPPED); + val = index_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(index_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + EXPECT_TRUE(is_dropped_tablet(txn.get(), instance_id, index_id, partition_id)); + reset_meta_service(); + index_pb.set_state(RecycleIndexPB::RECYCLING); + val = index_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(index_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + EXPECT_TRUE(is_dropped_tablet(txn.get(), instance_id, index_id, partition_id)); + // Tablet in PREPARED partition + reset_meta_service(); + partition_pb.set_state(RecyclePartitionPB::PREPARED); + val = partition_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(partition_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + EXPECT_FALSE(is_dropped_tablet(txn.get(), instance_id, index_id, partition_id)); + // Tablet in DROPPED/RECYCLING partition + reset_meta_service(); + partition_pb.set_state(RecyclePartitionPB::DROPPED); + val = partition_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(partition_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + EXPECT_TRUE(is_dropped_tablet(txn.get(), instance_id, index_id, partition_id)); + reset_meta_service(); + partition_pb.set_state(RecyclePartitionPB::RECYCLING); + val = partition_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(partition_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + EXPECT_TRUE(is_dropped_tablet(txn.get(), instance_id, index_id, partition_id)); +} + +TEST(MetaServiceTest, IndexRequest) { + auto meta_service = get_meta_service(); + std::string instance_id = "IndexRequest"; + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + auto reset_meta_service = [&meta_service] { meta_service = get_meta_service(); }; + constexpr int64_t table_id = 10001; + constexpr int64_t index_id = 10002; + constexpr int64_t partition_id = 10003; + constexpr int64_t tablet_id = 10004; + + std::unique_ptr txn; + doris::TabletMetaCloudPB tablet_pb; + tablet_pb.set_table_id(table_id); + tablet_pb.set_index_id(index_id); + tablet_pb.set_partition_id(partition_id); + tablet_pb.set_tablet_id(tablet_id); + auto tablet_key = meta_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + auto tablet_val = tablet_pb.SerializeAsString(); + RecycleIndexPB index_pb; + auto index_key = recycle_index_key({instance_id, index_id}); + std::string val; + + // ------------Test prepare index------------ + brpc::Controller ctrl; + IndexRequest req; + IndexResponse res; + meta_service->prepare_index(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + req.set_table_id(table_id); + req.add_index_ids(index_id); + // Last state UNKNOWN + res.Clear(); + meta_service->prepare_index(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(index_pb.ParseFromString(val)); + ASSERT_EQ(index_pb.state(), RecycleIndexPB::PREPARED); + // Last state PREPARED + res.Clear(); + meta_service->prepare_index(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(index_pb.ParseFromString(val)); + ASSERT_EQ(index_pb.state(), RecycleIndexPB::PREPARED); + // Last state DROPPED + reset_meta_service(); + index_pb.set_state(RecycleIndexPB::DROPPED); + val = index_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(index_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + meta_service->prepare_index(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(index_pb.ParseFromString(val)); + ASSERT_EQ(index_pb.state(), RecycleIndexPB::DROPPED); + // Last state RECYCLING + reset_meta_service(); + index_pb.set_state(RecycleIndexPB::RECYCLING); + val = index_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(index_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + meta_service->prepare_index(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(index_pb.ParseFromString(val)); + ASSERT_EQ(index_pb.state(), RecycleIndexPB::RECYCLING); + // Last state UNKNOWN but tablet meta existed + reset_meta_service(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->remove(index_key); + txn->put(tablet_key, tablet_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + meta_service->prepare_index(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::ALREADY_EXISTED); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); + // ------------Test commit index------------ + reset_meta_service(); + req.Clear(); + meta_service->commit_index(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + req.set_table_id(table_id); + req.add_index_ids(index_id); + // Last state UNKNOWN + res.Clear(); + meta_service->commit_index(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); + // Last state PREPARED + reset_meta_service(); + index_pb.set_state(RecycleIndexPB::PREPARED); + val = index_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(index_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + meta_service->commit_index(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); + // Last state DROPPED + reset_meta_service(); + index_pb.set_state(RecycleIndexPB::DROPPED); + val = index_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(index_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + meta_service->commit_index(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(index_pb.ParseFromString(val)); + ASSERT_EQ(index_pb.state(), RecycleIndexPB::DROPPED); + // Last state RECYCLING + reset_meta_service(); + index_pb.set_state(RecycleIndexPB::RECYCLING); + val = index_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(index_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + meta_service->commit_index(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(index_pb.ParseFromString(val)); + ASSERT_EQ(index_pb.state(), RecycleIndexPB::RECYCLING); + // Last state UNKNOWN but tablet meta existed + reset_meta_service(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(tablet_key, tablet_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + meta_service->commit_index(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); + // ------------Test drop index------------ + reset_meta_service(); + req.Clear(); + meta_service->drop_index(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + req.set_table_id(table_id); + req.add_index_ids(index_id); + // Last state UNKNOWN + res.Clear(); + meta_service->drop_index(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(index_pb.ParseFromString(val)); + ASSERT_EQ(index_pb.state(), RecycleIndexPB::DROPPED); + // Last state PREPARED + reset_meta_service(); + index_pb.set_state(RecycleIndexPB::PREPARED); + val = index_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(index_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + meta_service->drop_index(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(index_pb.ParseFromString(val)); + ASSERT_EQ(index_pb.state(), RecycleIndexPB::DROPPED); + // Last state DROPPED + reset_meta_service(); + index_pb.set_state(RecycleIndexPB::DROPPED); + val = index_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(index_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + meta_service->drop_index(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(index_pb.ParseFromString(val)); + ASSERT_EQ(index_pb.state(), RecycleIndexPB::DROPPED); + // Last state RECYCLING + reset_meta_service(); + index_pb.set_state(RecycleIndexPB::RECYCLING); + val = index_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(index_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + meta_service->drop_index(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(index_key, &val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(index_pb.ParseFromString(val)); + ASSERT_EQ(index_pb.state(), RecycleIndexPB::RECYCLING); +} + +TEST(MetaServiceTest, PartitionRequest) { + auto meta_service = get_meta_service(); + std::string instance_id = "PartitionRequest"; + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + auto reset_meta_service = [&meta_service] { meta_service = get_meta_service(); }; + constexpr int64_t table_id = 10001; + constexpr int64_t index_id = 10002; + constexpr int64_t partition_id = 10003; + constexpr int64_t tablet_id = 10004; + + std::unique_ptr txn; + doris::TabletMetaCloudPB tablet_pb; + tablet_pb.set_table_id(table_id); + tablet_pb.set_index_id(index_id); + tablet_pb.set_partition_id(partition_id); + tablet_pb.set_tablet_id(tablet_id); + auto tablet_key = meta_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + auto tablet_val = tablet_pb.SerializeAsString(); + RecyclePartitionPB partition_pb; + auto partition_key = recycle_partition_key({instance_id, partition_id}); + std::string val; + // ------------Test prepare partition------------ + brpc::Controller ctrl; + PartitionRequest req; + PartitionResponse res; + meta_service->prepare_partition(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + req.set_table_id(table_id); + req.add_index_ids(index_id); + req.add_partition_ids(partition_id); + // Last state UNKNOWN + res.Clear(); + meta_service->prepare_partition(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(partition_pb.ParseFromString(val)); + ASSERT_EQ(partition_pb.state(), RecyclePartitionPB::PREPARED); + // Last state PREPARED + res.Clear(); + meta_service->prepare_partition(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(partition_pb.ParseFromString(val)); + ASSERT_EQ(partition_pb.state(), RecyclePartitionPB::PREPARED); + // Last state DROPPED + reset_meta_service(); + partition_pb.set_state(RecyclePartitionPB::DROPPED); + val = partition_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(partition_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + meta_service->prepare_partition(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(partition_pb.ParseFromString(val)); + ASSERT_EQ(partition_pb.state(), RecyclePartitionPB::DROPPED); + // Last state RECYCLING + reset_meta_service(); + partition_pb.set_state(RecyclePartitionPB::RECYCLING); + val = partition_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(partition_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + meta_service->prepare_partition(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(partition_pb.ParseFromString(val)); + ASSERT_EQ(partition_pb.state(), RecyclePartitionPB::RECYCLING); + // Last state UNKNOWN but tablet meta existed + reset_meta_service(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->remove(partition_key); + txn->put(tablet_key, tablet_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + meta_service->prepare_partition(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::ALREADY_EXISTED); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); + // ------------Test commit partition------------ + reset_meta_service(); + req.Clear(); + meta_service->commit_partition(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + req.set_table_id(table_id); + req.add_index_ids(index_id); + req.add_partition_ids(partition_id); + // Last state UNKNOWN + res.Clear(); + meta_service->commit_partition(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); + // Last state PREPARED + reset_meta_service(); + partition_pb.set_state(RecyclePartitionPB::PREPARED); + val = partition_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(partition_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + meta_service->commit_partition(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); + // Last state DROPPED + reset_meta_service(); + partition_pb.set_state(RecyclePartitionPB::DROPPED); + val = partition_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(partition_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + meta_service->commit_partition(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(partition_pb.ParseFromString(val)); + ASSERT_EQ(partition_pb.state(), RecyclePartitionPB::DROPPED); + // Last state RECYCLING + reset_meta_service(); + partition_pb.set_state(RecyclePartitionPB::RECYCLING); + val = partition_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(partition_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + meta_service->commit_partition(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(partition_pb.ParseFromString(val)); + ASSERT_EQ(partition_pb.state(), RecyclePartitionPB::RECYCLING); + // Last state UNKNOWN but tablet meta existed + reset_meta_service(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(tablet_key, tablet_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + meta_service->commit_partition(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); + // Last state UNKNOWN and tablet meta existed, but request has no index ids + reset_meta_service(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(tablet_key, tablet_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + req.clear_index_ids(); + meta_service->commit_partition(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_KEY_NOT_FOUND); + req.add_index_ids(index_id); + // ------------Test drop partition------------ + reset_meta_service(); + req.Clear(); + meta_service->drop_partition(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + req.set_table_id(table_id); + req.add_index_ids(index_id); + req.add_partition_ids(partition_id); + // Last state UNKNOWN + res.Clear(); + meta_service->drop_partition(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(partition_pb.ParseFromString(val)); + ASSERT_EQ(partition_pb.state(), RecyclePartitionPB::DROPPED); + // Last state PREPARED + reset_meta_service(); + partition_pb.set_state(RecyclePartitionPB::PREPARED); + val = partition_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(partition_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + meta_service->drop_partition(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(partition_pb.ParseFromString(val)); + ASSERT_EQ(partition_pb.state(), RecyclePartitionPB::DROPPED); + // Last state DROPPED + reset_meta_service(); + partition_pb.set_state(RecyclePartitionPB::DROPPED); + val = partition_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(partition_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + meta_service->drop_partition(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(partition_pb.ParseFromString(val)); + ASSERT_EQ(partition_pb.state(), RecyclePartitionPB::DROPPED); + // Last state RECYCLING + reset_meta_service(); + partition_pb.set_state(RecyclePartitionPB::RECYCLING); + val = partition_pb.SerializeAsString(); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(partition_key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + res.Clear(); + meta_service->drop_partition(&ctrl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(partition_key, &val), TxnErrorCode::TXN_OK); + ASSERT_TRUE(partition_pb.ParseFromString(val)); + ASSERT_EQ(partition_pb.state(), RecyclePartitionPB::RECYCLING); +} + +TEST(MetaServiceTxnStoreRetryableTest, MockGetVersion) { + size_t index = 0; + SyncPoint::get_instance()->set_call_back("get_version_code", [&](void* arg) { + LOG(INFO) << "GET_VERSION_CODE"; + if (++index < 5) { + *reinterpret_cast(arg) = MetaServiceCode::KV_TXN_STORE_GET_RETRYABLE; + } + }); + SyncPoint::get_instance()->enable_processing(); + config::enable_txn_store_retry = true; + + auto service = get_meta_service(); + create_tablet(service.get(), 1, 1, 1, 1); + insert_rowset(service.get(), 1, std::to_string(1), 1, 1, 1); + + brpc::Controller ctrl; + GetVersionRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(1); + req.set_table_id(1); + req.set_partition_id(1); + + GetVersionResponse resp; + service->get_version(&ctrl, &req, &resp, nullptr); + + ASSERT_EQ(resp.status().code(), MetaServiceCode::OK) + << " status is " << resp.status().msg() << ", code=" << resp.status().code(); + EXPECT_EQ(resp.version(), 2); + EXPECT_GE(index, 5); + + SyncPoint::get_instance()->disable_processing(); + SyncPoint::get_instance()->clear_all_call_backs(); + config::enable_txn_store_retry = false; +} + +TEST(MetaServiceTxnStoreRetryableTest, DoNotReturnRetryableCode) { + SyncPoint::get_instance()->set_call_back("get_version_code", [&](void* arg) { + *reinterpret_cast(arg) = MetaServiceCode::KV_TXN_STORE_GET_RETRYABLE; + }); + SyncPoint::get_instance()->enable_processing(); + config::enable_txn_store_retry = true; + int32_t retry_times = config::txn_store_retry_times; + config::txn_store_retry_times = 3; + + auto service = get_meta_service(); + create_tablet(service.get(), 1, 1, 1, 1); + insert_rowset(service.get(), 1, std::to_string(1), 1, 1, 1); + + brpc::Controller ctrl; + GetVersionRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(1); + req.set_table_id(1); + req.set_partition_id(1); + + GetVersionResponse resp; + service->get_version(&ctrl, &req, &resp, nullptr); + + ASSERT_EQ(resp.status().code(), MetaServiceCode::KV_TXN_GET_ERR) + << " status is " << resp.status().msg() << ", code=" << resp.status().code(); + + SyncPoint::get_instance()->disable_processing(); + SyncPoint::get_instance()->clear_all_call_backs(); + config::enable_txn_store_retry = false; + config::txn_store_retry_times = retry_times; +} + +TEST(MetaServiceTest, GetClusterStatusTest) { + auto meta_service = get_meta_service(); + + // add cluster first + InstanceKeyInfo key_info {mock_instance}; + std::string key; + std::string val; + instance_key(key_info, &key); + + InstanceInfoPB instance; + instance.set_instance_id(mock_instance); + ClusterPB c1; + c1.set_type(ClusterPB::COMPUTE); + c1.set_cluster_name(mock_cluster_name); + c1.set_cluster_id(mock_cluster_id); + c1.add_mysql_user_name()->append("m1"); + c1.set_cluster_status(ClusterStatus::NORMAL); + ClusterPB c2; + c2.set_type(ClusterPB::COMPUTE); + c2.set_cluster_name(mock_cluster_name + "2"); + c2.set_cluster_id(mock_cluster_id + "2"); + c2.add_mysql_user_name()->append("m2"); + c2.set_cluster_status(ClusterStatus::SUSPENDED); + ClusterPB c3; + c3.set_type(ClusterPB::COMPUTE); + c3.set_cluster_name(mock_cluster_name + "3"); + c3.set_cluster_id(mock_cluster_id + "3"); + c3.add_mysql_user_name()->append("m3"); + c3.set_cluster_status(ClusterStatus::TO_RESUME); + instance.add_clusters()->CopyFrom(c1); + instance.add_clusters()->CopyFrom(c2); + instance.add_clusters()->CopyFrom(c3); + val = instance.SerializeAsString(); + + std::unique_ptr txn; + std::string get_val; + TxnErrorCode err = meta_service->txn_kv()->create_txn(&txn); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + txn->put(key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + // case: get all cluster + { + brpc::Controller cntl; + GetClusterStatusRequest req; + req.add_instance_ids(mock_instance); + GetClusterStatusResponse res; + meta_service->get_cluster_status( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.details().at(0).clusters().size(), 3); + } + + // get normal cluster + { + brpc::Controller cntl; + GetClusterStatusRequest req; + req.add_instance_ids(mock_instance); + req.set_status(NORMAL); + GetClusterStatusResponse res; + meta_service->get_cluster_status( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.details().at(0).clusters().size(), 1); + } +} + +TEST(MetaServiceTest, DecryptInfoTest) { + auto meta_service = get_meta_service(); + auto sp = cloud::SyncPoint::get_instance(); + sp->enable_processing(); + sp->set_call_back("decrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key", [](void* p) { + *reinterpret_cast(p) = "selectdbselectdbselectdbselectdb"; + }); + InstanceInfoPB instance; + + EncryptionInfoPB encryption_info; + encryption_info.set_encryption_method("AES_256_ECB"); + encryption_info.set_key_id(1); + + std::string cipher_sk = "JUkuTDctR+ckJtnPkLScWaQZRcOtWBhsLLpnCRxQLxr734qB8cs6gNLH6grE1FxO"; + std::string plain_sk = "Hx60p12123af234541nsVsffdfsdfghsdfhsdf34t"; + ObjectStoreInfoPB obj_info; + obj_info.mutable_encryption_info()->CopyFrom(encryption_info); + obj_info.set_ak("akak1"); + obj_info.set_sk(cipher_sk); + instance.add_obj_info()->CopyFrom(obj_info); + + RamUserPB ram_user; + ram_user.set_ak("akak2"); + ram_user.set_sk(cipher_sk); + ram_user.mutable_encryption_info()->CopyFrom(encryption_info); + instance.mutable_ram_user()->CopyFrom(ram_user); + + StagePB stage; + stage.mutable_obj_info()->CopyFrom(obj_info); + instance.add_stages()->CopyFrom(stage); + + auto checkcheck = [&](const InstanceInfoPB& instance) { + ASSERT_EQ(instance.obj_info(0).ak(), "akak1"); + ASSERT_EQ(instance.obj_info(0).sk(), plain_sk); + + ASSERT_EQ(instance.ram_user().ak(), "akak2"); + ASSERT_EQ(instance.ram_user().sk(), plain_sk); + + ASSERT_EQ(instance.stages(0).obj_info().ak(), "akak1"); + ASSERT_EQ(instance.stages(0).obj_info().sk(), plain_sk); + }; + + std::string instance_id = "i1"; + MetaServiceCode code; + std::string msg; + // No system_meta_service_arn_info_key + { + std::unique_ptr txn0; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn0), TxnErrorCode::TXN_OK); + std::shared_ptr txn(txn0.release()); + InstanceInfoPB decrypt_instance; + decrypt_instance.CopyFrom(instance); + int ret = decrypt_instance_info(decrypt_instance, instance_id, code, msg, txn); + ASSERT_EQ(ret, 0); + checkcheck(decrypt_instance); + ASSERT_EQ(decrypt_instance.iam_user().user_id(), config::arn_id); + ASSERT_EQ(decrypt_instance.iam_user().external_id(), instance_id); + ASSERT_EQ(decrypt_instance.iam_user().ak(), config::arn_ak); + ASSERT_EQ(decrypt_instance.iam_user().sk(), config::arn_sk); + } + + // With system_meta_service_arn_info_key + { + std::string key = system_meta_service_arn_info_key(); + std::string val; + std::unique_ptr txn2; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn2), TxnErrorCode::TXN_OK); + RamUserPB iam_user; + iam_user.set_user_id("1234"); + iam_user.set_ak("aksk3"); + iam_user.set_sk(cipher_sk); + iam_user.set_external_id(instance_id); + iam_user.mutable_encryption_info()->CopyFrom(encryption_info); + val = iam_user.SerializeAsString(); + txn2->put(key, val); + ASSERT_EQ(txn2->commit(), TxnErrorCode::TXN_OK); + std::unique_ptr txn0; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn0), TxnErrorCode::TXN_OK); + std::shared_ptr txn(txn0.release()); + InstanceInfoPB decrypt_instance; + decrypt_instance.CopyFrom(instance); + int ret = decrypt_instance_info(decrypt_instance, instance_id, code, msg, txn); + ASSERT_EQ(ret, 0); + checkcheck(decrypt_instance); + ASSERT_EQ(decrypt_instance.iam_user().user_id(), "1234"); + ASSERT_EQ(decrypt_instance.iam_user().external_id(), instance_id); + ASSERT_EQ(decrypt_instance.iam_user().ak(), "aksk3"); + ASSERT_EQ(decrypt_instance.iam_user().sk(), plain_sk); + } + SyncPoint::get_instance()->disable_processing(); + SyncPoint::get_instance()->clear_all_call_backs(); +} + +TEST(MetaServiceTest, LegacyUpdateAkSkTest) { + auto meta_service = get_meta_service(); + + auto sp = cloud::SyncPoint::get_instance(); + sp->enable_processing(); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", [](void* p) { + *reinterpret_cast(p) = "selectdbselectdbselectdbselectdb"; + }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_id", + [](void* p) { *reinterpret_cast(p) = 1; }); + + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string key; + std::string val; + InstanceKeyInfo key_info {"test_instance"}; + instance_key(key_info, &key); + + ObjectStoreInfoPB obj_info; + obj_info.set_id("1"); + obj_info.set_ak("ak"); + obj_info.set_sk("sk"); + InstanceInfoPB instance; + instance.add_obj_info()->CopyFrom(obj_info); + val = instance.SerializeAsString(); + txn->put(key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + auto get_test_instance = [&](InstanceInfoPB& i) { + std::string key; + std::string val; + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + InstanceKeyInfo key_info {"test_instance"}; + instance_key(key_info, &key); + ASSERT_EQ(txn->get(key, &val), TxnErrorCode::TXN_OK); + i.ParseFromString(val); + }; + + std::string cipher_sk = "JUkuTDctR+ckJtnPkLScWaQZRcOtWBhsLLpnCRxQLxr734qB8cs6gNLH6grE1FxO"; + std::string plain_sk = "Hx60p12123af234541nsVsffdfsdfghsdfhsdf34t"; + + // update failed + { + AlterObjStoreInfoRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_op(AlterObjStoreInfoRequest::LEGACY_UPDATE_AK_SK); + req.mutable_obj()->set_id("2"); + req.mutable_obj()->set_ak("new_ak"); + req.mutable_obj()->set_sk(plain_sk); + + brpc::Controller cntl; + AlterObjStoreInfoResponse res; + meta_service->alter_obj_store_info( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + InstanceInfoPB instance; + get_test_instance(instance); + ASSERT_EQ(instance.obj_info(0).id(), "1"); + ASSERT_EQ(instance.obj_info(0).ak(), "ak"); + ASSERT_EQ(instance.obj_info(0).sk(), "sk"); + } + + // update successful + { + AlterObjStoreInfoRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_op(AlterObjStoreInfoRequest::LEGACY_UPDATE_AK_SK); + req.mutable_obj()->set_id("1"); + req.mutable_obj()->set_ak("new_ak"); + req.mutable_obj()->set_sk(plain_sk); + + brpc::Controller cntl; + AlterObjStoreInfoResponse res; + meta_service->alter_obj_store_info( + reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + InstanceInfoPB instance; + get_test_instance(instance); + ASSERT_EQ(instance.obj_info(0).id(), "1"); + ASSERT_EQ(instance.obj_info(0).ak(), "new_ak"); + ASSERT_EQ(instance.obj_info(0).sk(), cipher_sk); + } + + SyncPoint::get_instance()->disable_processing(); + SyncPoint::get_instance()->clear_all_call_backs(); +} + +TEST(MetaServiceTest, UpdateAkSkTest) { + auto meta_service = get_meta_service(); + + auto sp = cloud::SyncPoint::get_instance(); + sp->enable_processing(); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", [](void* p) { + *reinterpret_cast(p) = "selectdbselectdbselectdbselectdb"; + }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_id", + [](void* p) { *reinterpret_cast(p) = 1; }); + + auto get_test_instance = [&](InstanceInfoPB& i) { + std::string key; + std::string val; + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + InstanceKeyInfo key_info {"test_instance"}; + instance_key(key_info, &key); + ASSERT_EQ(txn->get(key, &val), TxnErrorCode::TXN_OK); + i.ParseFromString(val); + }; + + std::string cipher_sk = "JUkuTDctR+ckJtnPkLScWaQZRcOtWBhsLLpnCRxQLxr734qB8cs6gNLH6grE1FxO"; + std::string plain_sk = "Hx60p12123af234541nsVsffdfsdfghsdfhsdf34t"; + + auto update = [&](bool with_user_id, bool with_wrong_user_id) { + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string key; + std::string val; + InstanceKeyInfo key_info {"test_instance"}; + instance_key(key_info, &key); + + ObjectStoreInfoPB obj_info; + if (with_user_id) { + obj_info.set_user_id("111"); + } + obj_info.set_ak("ak"); + obj_info.set_sk("sk"); + InstanceInfoPB instance; + instance.add_obj_info()->CopyFrom(obj_info); + val = instance.SerializeAsString(); + txn->put(key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + UpdateAkSkRequest req; + req.set_instance_id("test_instance"); + RamUserPB ram_user; + if (with_wrong_user_id) { + ram_user.set_user_id("222"); + } else { + ram_user.set_user_id("111"); + } + ram_user.set_ak("new_ak"); + ram_user.set_sk(plain_sk); + req.add_internal_bucket_user()->CopyFrom(ram_user); + + brpc::Controller cntl; + UpdateAkSkResponse res; + meta_service->update_ak_sk(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + if (with_wrong_user_id) { + ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT); + } else { + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + InstanceInfoPB update_instance; + get_test_instance(update_instance); + ASSERT_EQ(update_instance.obj_info(0).user_id(), "111"); + ASSERT_EQ(update_instance.obj_info(0).ak(), "new_ak"); + ASSERT_EQ(update_instance.obj_info(0).sk(), cipher_sk); + } + }; + + update(false, false); + update(true, false); + update(true, true); +} + +TEST(MetaServiceTest, AlterIamTest) { + auto meta_service = get_meta_service(); + + auto sp = cloud::SyncPoint::get_instance(); + sp->enable_processing(); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", [](void* p) { + *reinterpret_cast(p) = "selectdbselectdbselectdbselectdb"; + }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_id", + [](void* p) { *reinterpret_cast(p) = 1; }); + + std::string cipher_sk = "JUkuTDctR+ckJtnPkLScWaQZRcOtWBhsLLpnCRxQLxr734qB8cs6gNLH6grE1FxO"; + std::string plain_sk = "Hx60p12123af234541nsVsffdfsdfghsdfhsdf34t"; + + auto get_arn_info_key = [&](RamUserPB& i) { + std::string val; + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(system_meta_service_arn_info_key(), &val), TxnErrorCode::TXN_OK); + i.ParseFromString(val); + }; + + // add new system_meta_service_arn_info_key + { + AlterIamRequest req; + req.set_account_id("123"); + req.set_ak("ak1"); + req.set_sk(plain_sk); + + brpc::Controller cntl; + AlterIamResponse res; + meta_service->alter_iam(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + RamUserPB ram_user; + get_arn_info_key(ram_user); + ASSERT_EQ(ram_user.user_id(), "123"); + ASSERT_EQ(ram_user.ak(), "ak1"); + ASSERT_EQ(ram_user.sk(), cipher_sk); + } + // with old system_meta_service_arn_info_key + { + AlterIamRequest req; + req.set_account_id("321"); + req.set_ak("ak2"); + req.set_sk(plain_sk); + + brpc::Controller cntl; + AlterIamResponse res; + meta_service->alter_iam(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + RamUserPB ram_user; + get_arn_info_key(ram_user); + ASSERT_EQ(ram_user.user_id(), "321"); + ASSERT_EQ(ram_user.ak(), "ak2"); + ASSERT_EQ(ram_user.sk(), cipher_sk); + } + + SyncPoint::get_instance()->disable_processing(); + SyncPoint::get_instance()->clear_all_call_backs(); +} + +TEST(MetaServiceTest, UpdateTmpRowsetTest) { + auto meta_service = get_meta_service(); + + std::string instance_id = "update_rowset_meta_test_instance_id"; + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + { + // 1. normal path + constexpr auto db_id = 10000, table_id = 10001, index_id = 10002, partition_id = 10003, + tablet_id = 10004; + int64_t txn_id = 0; + std::string label = "update_rowset_meta_test_label1"; + CreateRowsetResponse res; + + ASSERT_NO_FATAL_FAILURE( + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id)); + + ASSERT_NO_FATAL_FAILURE(begin_txn(meta_service.get(), db_id, label, table_id, txn_id)); + auto rowset = create_rowset(txn_id, tablet_id, partition_id); + ASSERT_NO_FATAL_FAILURE(prepare_rowset(meta_service.get(), rowset, res)); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + res.Clear(); + + rowset.set_rowset_state(doris::BEGIN_PARTIAL_UPDATE); + ASSERT_NO_FATAL_FAILURE(commit_rowset(meta_service.get(), rowset, res)); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + res.Clear(); + + // simulate that there are new segments added to this rowset + rowset.set_num_segments(rowset.num_segments() + 3); + rowset.set_num_rows(rowset.num_rows() + 1000); + rowset.set_data_disk_size(rowset.data_disk_size() + 10000); + + ASSERT_NO_FATAL_FAILURE(update_tmp_rowset(meta_service.get(), rowset, res)); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + + std::string key; + std::string val; + MetaRowsetTmpKeyInfo key_info {instance_id, txn_id, tablet_id}; + meta_rowset_tmp_key(key_info, &key); + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(key, &val), TxnErrorCode::TXN_OK); + doris::RowsetMetaCloudPB fetchedRowsetMeta; + ASSERT_TRUE(fetchedRowsetMeta.ParseFromString(val)); + + ASSERT_EQ(doris::BEGIN_PARTIAL_UPDATE, fetchedRowsetMeta.rowset_state()); + ASSERT_EQ(rowset.num_segments(), fetchedRowsetMeta.num_segments()); + ASSERT_EQ(rowset.num_rows(), fetchedRowsetMeta.num_rows()); + ASSERT_EQ(rowset.data_disk_size(), fetchedRowsetMeta.data_disk_size()); + + ASSERT_NO_FATAL_FAILURE(commit_txn(meta_service.get(), db_id, txn_id, label)); + } + + { + // 2. rpc retryies due to network error will success + constexpr auto db_id = 20000, table_id = 20001, index_id = 20002, partition_id = 20003, + tablet_id = 20004; + int64_t txn_id = 0; + std::string label = "update_rowset_meta_test_label2"; + CreateRowsetResponse res; + + ASSERT_NO_FATAL_FAILURE( + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id)); + + ASSERT_NO_FATAL_FAILURE(begin_txn(meta_service.get(), db_id, label, table_id, txn_id)); + auto rowset = create_rowset(txn_id, tablet_id, partition_id); + ASSERT_NO_FATAL_FAILURE(prepare_rowset(meta_service.get(), rowset, res)); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + res.Clear(); + + rowset.set_rowset_state(doris::BEGIN_PARTIAL_UPDATE); + ASSERT_NO_FATAL_FAILURE(commit_rowset(meta_service.get(), rowset, res)); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + res.Clear(); + + // simulate that there are new segments added to this rowset + rowset.set_num_segments(rowset.num_segments() + 3); + rowset.set_num_rows(rowset.num_rows() + 1000); + rowset.set_data_disk_size(rowset.data_disk_size() + 10000); + + // repeated calls to update_tmp_rowset will all success + ASSERT_NO_FATAL_FAILURE(update_tmp_rowset(meta_service.get(), rowset, res)); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + ASSERT_NO_FATAL_FAILURE(update_tmp_rowset(meta_service.get(), rowset, res)); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + ASSERT_NO_FATAL_FAILURE(update_tmp_rowset(meta_service.get(), rowset, res)); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + ASSERT_NO_FATAL_FAILURE(update_tmp_rowset(meta_service.get(), rowset, res)); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + + std::string key; + std::string val; + MetaRowsetTmpKeyInfo key_info {instance_id, txn_id, tablet_id}; + meta_rowset_tmp_key(key_info, &key); + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(key, &val), TxnErrorCode::TXN_OK); + doris::RowsetMetaCloudPB fetchedRowsetMeta; + ASSERT_TRUE(fetchedRowsetMeta.ParseFromString(val)); + + ASSERT_EQ(doris::BEGIN_PARTIAL_UPDATE, fetchedRowsetMeta.rowset_state()); + ASSERT_EQ(rowset.num_segments(), fetchedRowsetMeta.num_segments()); + ASSERT_EQ(rowset.num_rows(), fetchedRowsetMeta.num_rows()); + ASSERT_EQ(rowset.data_disk_size(), fetchedRowsetMeta.data_disk_size()); + + ASSERT_NO_FATAL_FAILURE(commit_txn(meta_service.get(), db_id, txn_id, label)); + } + + { + // 3. call update_tmp_rowset without commit_rowset first will fail + constexpr auto db_id = 30000, table_id = 30001, index_id = 30002, partition_id = 30003, + tablet_id = 30004; + int64_t txn_id = 0; + std::string label = "update_rowset_meta_test_label1"; + CreateRowsetResponse res; + + ASSERT_NO_FATAL_FAILURE( + create_tablet(meta_service.get(), table_id, index_id, partition_id, tablet_id)); + + ASSERT_NO_FATAL_FAILURE(begin_txn(meta_service.get(), db_id, label, table_id, txn_id)); + auto rowset = create_rowset(txn_id, tablet_id, partition_id); + ASSERT_NO_FATAL_FAILURE(prepare_rowset(meta_service.get(), rowset, res)); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + res.Clear(); + + // simulate that there are new segments added to this rowset + rowset.set_num_segments(rowset.num_segments() + 3); + rowset.set_num_rows(rowset.num_rows() + 1000); + rowset.set_data_disk_size(rowset.data_disk_size() + 10000); + + ASSERT_NO_FATAL_FAILURE(update_tmp_rowset(meta_service.get(), rowset, res)); + ASSERT_EQ(res.status().code(), MetaServiceCode::ROWSET_META_NOT_FOUND) << label; + } +} + +} // namespace doris::cloud diff --git a/cloud/test/metric_test.cpp b/cloud/test/metric_test.cpp new file mode 100644 index 00000000000000..f46d512effffff --- /dev/null +++ b/cloud/test/metric_test.cpp @@ -0,0 +1,175 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "common/metric.h" + +#include + +#include +#include +#include +#include + +#include "common/bvars.h" +#include "common/config.h" +#include "meta-service/mem_txn_kv.h" +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" + +int main(int argc, char** argv) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + +TEST(MetricTest, FdbMetricExporterTest) { + using namespace doris::cloud; + + // normal to collect fdb metirc + { + std::string fdb_metric_example = "./fdb_metric_example.json"; + std::ifstream inFile(fdb_metric_example); + + ASSERT_TRUE(inFile.is_open()); + std::string fileContent((std::istreambuf_iterator(inFile)), + std::istreambuf_iterator()); + + std::shared_ptr txn_kv = std::make_shared(); + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put("\xff\xff/status/json", fileContent); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + FdbMetricExporter fdb_metric_exporter(txn_kv); + fdb_metric_exporter.sleep_interval_ms_ = 1; + fdb_metric_exporter.start(); + std::this_thread::sleep_for(std::chrono::milliseconds(20)); + fdb_metric_exporter.stop(); + ASSERT_EQ(g_bvar_fdb_machines_count.get_value(), 3); + ASSERT_EQ(g_bvar_fdb_client_count.get_value(), 8); + } + + // empty fdb_status + { + g_bvar_fdb_machines_count.set_value(BVAR_FDB_INVALID_VALUE); + g_bvar_fdb_client_count.set_value(BVAR_FDB_INVALID_VALUE); + std::shared_ptr txn_kv = std::make_shared(); + { + FdbMetricExporter fdb_metric_exporter(txn_kv); + fdb_metric_exporter.sleep_interval_ms_ = 1; + fdb_metric_exporter.start(); + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + } + ASSERT_EQ(g_bvar_fdb_machines_count.get_value(), BVAR_FDB_INVALID_VALUE); + ASSERT_EQ(g_bvar_fdb_client_count.get_value(), BVAR_FDB_INVALID_VALUE); + } + + // The cluster field is missing + { + g_bvar_fdb_machines_count.set_value(BVAR_FDB_INVALID_VALUE); + g_bvar_fdb_client_count.set_value(BVAR_FDB_INVALID_VALUE); + + std::string fdb_metric_example = "./fdb_metric_example.json"; + std::ifstream inFile(fdb_metric_example); + + ASSERT_TRUE(inFile.is_open()); + std::string fileContent((std::istreambuf_iterator(inFile)), + std::istreambuf_iterator()); + + std::string word_to_replace = "cluster"; + std::string new_word = "xxxx"; + + size_t start_pos = 0; + while ((start_pos = fileContent.find(word_to_replace, start_pos)) != std::string::npos) { + fileContent.replace(start_pos, word_to_replace.length(), new_word); + start_pos += new_word.length(); + } + std::shared_ptr txn_kv = std::make_shared(); + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put("\xff\xff/status/json", fileContent); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + FdbMetricExporter fdb_metric_exporter(txn_kv); + fdb_metric_exporter.sleep_interval_ms_ = 1; + fdb_metric_exporter.start(); + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + fdb_metric_exporter.stop(); + ASSERT_EQ(g_bvar_fdb_machines_count.get_value(), BVAR_FDB_INVALID_VALUE); + ASSERT_EQ(g_bvar_fdb_client_count.get_value(), BVAR_FDB_INVALID_VALUE); + } + + // The client field is missing + { + g_bvar_fdb_machines_count.set_value(BVAR_FDB_INVALID_VALUE); + g_bvar_fdb_client_count.set_value(BVAR_FDB_INVALID_VALUE); + + std::string fdb_metric_example = "./fdb_metric_example.json"; + std::ifstream inFile(fdb_metric_example); + + ASSERT_TRUE(inFile.is_open()); + std::string fileContent((std::istreambuf_iterator(inFile)), + std::istreambuf_iterator()); + + std::string word_to_replace = "machines"; + std::string new_word = "xxxx"; + + size_t start_pos = 0; + while ((start_pos = fileContent.find(word_to_replace, start_pos)) != std::string::npos) { + fileContent.replace(start_pos, word_to_replace.length(), new_word); + start_pos += new_word.length(); + } + std::shared_ptr txn_kv = std::make_shared(); + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put("\xff\xff/status/json", fileContent); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + FdbMetricExporter fdb_metric_exporter(txn_kv); + fdb_metric_exporter.sleep_interval_ms_ = 1; + fdb_metric_exporter.start(); + std::this_thread::sleep_for(std::chrono::milliseconds(15)); + fdb_metric_exporter.stop(); + ASSERT_EQ(g_bvar_fdb_machines_count.get_value(), BVAR_FDB_INVALID_VALUE); + ASSERT_EQ(g_bvar_fdb_client_count.get_value(), 8); + } + + // stop without start + { + g_bvar_fdb_machines_count.set_value(BVAR_FDB_INVALID_VALUE); + g_bvar_fdb_client_count.set_value(BVAR_FDB_INVALID_VALUE); + + std::string fdb_metric_example = "./fdb_metric_example.json"; + std::ifstream inFile(fdb_metric_example); + + ASSERT_TRUE(inFile.is_open()); + std::string fileContent((std::istreambuf_iterator(inFile)), + std::istreambuf_iterator()); + + std::shared_ptr txn_kv = std::make_shared(); + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put("\xff\xff/status/json", fileContent); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + FdbMetricExporter fdb_metric_exporter(txn_kv); + fdb_metric_exporter.sleep_interval_ms_ = 1; + std::this_thread::sleep_for(std::chrono::milliseconds(15)); + fdb_metric_exporter.stop(); + ASSERT_EQ(g_bvar_fdb_machines_count.get_value(), BVAR_FDB_INVALID_VALUE); + ASSERT_EQ(g_bvar_fdb_client_count.get_value(), BVAR_FDB_INVALID_VALUE); + } +} \ No newline at end of file diff --git a/cloud/test/mock_accessor.h b/cloud/test/mock_accessor.h new file mode 100644 index 00000000000000..2f6602b10bfe23 --- /dev/null +++ b/cloud/test/mock_accessor.h @@ -0,0 +1,142 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include + +#include +#include + +#include "common/sync_point.h" +#include "recycler/s3_accessor.h" + +namespace doris::cloud { + +class MockAccessor : public ObjStoreAccessor { +public: + explicit MockAccessor(const S3Conf& conf) { + path_ = conf.endpoint + '/' + conf.bucket + '/' + conf.prefix; + } + ~MockAccessor() override = default; + + const std::string& path() const override { return path_; } + + // returns 0 for success otherwise error + int init() override { return 0; } + + // returns 0 for success otherwise error + int delete_objects_by_prefix(const std::string& relative_path) override { + TEST_SYNC_POINT_CALLBACK("MockAccessor::delete_objects_by_prefix", nullptr); + LOG(INFO) << "delete object of prefix=" << relative_path; + std::lock_guard lock(mtx_); + if (relative_path.empty()) { + objects_.clear(); + return 0; + } + auto begin = objects_.lower_bound(relative_path); + if (begin == objects_.end()) { + return 0; + } + auto path1 = relative_path; + path1.back() += 1; + auto end = objects_.lower_bound(path1); + objects_.erase(begin, end); + return 0; + } + + // returns 0 for success otherwise error + int delete_objects(const std::vector& relative_paths) override { + TEST_SYNC_POINT_CALLBACK("MockAccessor::delete_objects", nullptr); + { + [[maybe_unused]] int ret = -1; + TEST_SYNC_POINT_RETURN_WITH_VALUE("MockAccessor::delete_objects_ret", &ret); + } + for (auto& path : relative_paths) { + LOG(INFO) << "delete object path=" << path; + } + std::lock_guard lock(mtx_); + for (auto& path : relative_paths) { + objects_.erase(path); + } + return 0; + } + + // returns 0 for success otherwise error + int delete_object(const std::string& relative_path) override { + LOG(INFO) << "delete object path=" << relative_path; + std::lock_guard lock(mtx_); + objects_.erase(relative_path); + return 0; + } + + // for test + // returns 0 for success otherwise error + int put_object(const std::string& relative_path, const std::string& content) override { + std::lock_guard lock(mtx_); + objects_.insert(relative_path); + return 0; + } + + // returns 0 for success otherwise error + int list(const std::string& relative_path, std::vector* paths) override { + std::lock_guard lock(mtx_); + if (relative_path == "") { + for (const auto& obj : objects_) { + paths->push_back({obj}); + } + return 0; + } + auto begin = objects_.lower_bound(relative_path); + if (begin == objects_.end()) { + return 0; + } + auto path1 = relative_path; + path1.back() += 1; + auto end = objects_.lower_bound(path1); + for (auto it = begin; it != end; ++it) { + paths->push_back({*it}); + } + return 0; + } + + int exist(const std::string& relative_path) override { + std::lock_guard lock(mtx_); + return !objects_.count(relative_path); + } + + // delete objects which last modified time is less than the input expired time and under the input relative path + // returns 0 for success otherwise error + int delete_expired_objects(const std::string& relative_path, int64_t expired_time) override { + return 0; + } + + int get_bucket_lifecycle(int64_t* expiration_days) override { + *expiration_days = 7; + return 0; + } + + int check_bucket_versioning() override { return 0; } + +private: + std::string path_; + + std::mutex mtx_; + std::set objects_; // store objects' relative path +}; + +} // namespace doris::cloud diff --git a/cloud/test/mock_resource_manager.h b/cloud/test/mock_resource_manager.h new file mode 100644 index 00000000000000..748947cb46a993 --- /dev/null +++ b/cloud/test/mock_resource_manager.h @@ -0,0 +1,77 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" +#include "resource-manager/resource_manager.h" + +using namespace doris::cloud; + +static std::string mock_instance = "test_instance"; +static std::string mock_cluster_name = "test_cluster"; +static std::string mock_cluster_id = "test_cluster_id"; + +class MockResourceManager : public ResourceManager { +public: + MockResourceManager(std::shared_ptr txn_kv) : ResourceManager(txn_kv) {}; + ~MockResourceManager() override = default; + + int init() override { return 0; } + + std::string get_node(const std::string& cloud_unique_id, + std::vector* nodes) override { + NodeInfo i {Role::COMPUTE_NODE, mock_instance, mock_cluster_name, mock_cluster_id}; + nodes->push_back(i); + return ""; + } + + std::pair add_cluster(const std::string& instance_id, + const ClusterInfo& cluster) override { + return std::make_pair(MetaServiceCode::OK, ""); + } + + std::pair drop_cluster(const std::string& instance_id, + const ClusterInfo& cluster) override { + return std::make_pair(MetaServiceCode::OK, ""); + } + + std::pair refresh_instance( + const std::string& instance_id) override { + return std::make_pair(MetaServiceCode::OK, ""); + } + + std::string update_cluster( + const std::string& instance_id, const ClusterInfo& cluster, + std::function filter, + std::function& cluster_names)> action) + override { + return ""; + } + + std::pair get_instance(std::shared_ptr txn, + const std::string& instance_id, + InstanceInfoPB* inst_pb) override { + return {TxnErrorCode::TXN_KEY_NOT_FOUND, ""}; + } + + std::string modify_nodes(const std::string& instance_id, const std::vector& to_add, + const std::vector& to_del) override { + return ""; + } +}; \ No newline at end of file diff --git a/cloud/test/rate_limiter_test.cpp b/cloud/test/rate_limiter_test.cpp new file mode 100644 index 00000000000000..2a10451a69fa1b --- /dev/null +++ b/cloud/test/rate_limiter_test.cpp @@ -0,0 +1,112 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "rate-limiter/rate_limiter.h" + +#include + +#include "common/config.h" +#include "common/util.h" +#include "meta-service/keys.h" +#include "meta-service/mem_txn_kv.h" +#include "meta-service/meta_service.h" +#include "meta-service/txn_kv_error.h" +#include "mock_resource_manager.h" + +int main(int argc, char** argv) { + doris::cloud::config::init(nullptr, true); + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + +using namespace doris::cloud; + +std::unique_ptr get_meta_service() { + auto txn_kv = std::dynamic_pointer_cast(std::make_shared()); + [&] { ASSERT_NE(txn_kv.get(), nullptr); }(); + auto rs = std::make_shared(txn_kv); + auto rl = std::make_shared(); + auto meta_service = std::make_unique(txn_kv, rs, rl); + return std::make_unique(std::move(meta_service)); +} + +TEST(RateLimiterTest, RateLimitGetClusterTest) { + auto meta_service = get_meta_service(); + // add cluster first + InstanceKeyInfo key_info {mock_instance}; + std::string key; + std::string val; + instance_key(key_info, &key); + + InstanceInfoPB instance; + instance.set_instance_id(mock_instance); + ClusterPB c1; + c1.set_cluster_name(mock_cluster_name); + c1.set_cluster_id(mock_cluster_id); + c1.add_mysql_user_name()->append("name1"); + instance.add_clusters()->CopyFrom(c1); + val = instance.SerializeAsString(); + + std::unique_ptr txn; + std::string get_val; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put(key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + auto get_cluster = [&](MetaServiceCode code) { + GetClusterRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_cluster_id(mock_cluster_id); + req.set_cluster_name(mock_cluster_name); + brpc::Controller cntl; + GetClusterResponse res; + meta_service->get_cluster(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + + ASSERT_EQ(res.status().code(), code); + }; + std::vector threads; + for (int i = 0; i < 20; ++i) { + threads.emplace_back(get_cluster, MetaServiceCode::OK); + } + for (auto& t : threads) { + t.join(); + } + threads.clear(); + + std::this_thread::sleep_for(std::chrono::seconds(1)); + meta_service->rate_limiter() + ->get_rpc_rate_limiter("get_cluster") + ->qps_limiter_[mock_instance] + ->max_qps_limit_ = 1; + threads.emplace_back(get_cluster, MetaServiceCode::MAX_QPS_LIMIT); + for (auto& t : threads) { + t.join(); + } + threads.clear(); + + std::this_thread::sleep_for(std::chrono::seconds(1)); + meta_service->rate_limiter() + ->get_rpc_rate_limiter("get_cluster") + ->qps_limiter_[mock_instance] + ->max_qps_limit_ = 10000; + threads.emplace_back(get_cluster, MetaServiceCode::OK); + for (auto& t : threads) { + t.join(); + } + threads.clear(); +} \ No newline at end of file diff --git a/cloud/test/recycler_test.cpp b/cloud/test/recycler_test.cpp new file mode 100644 index 00000000000000..cf6d606585d457 --- /dev/null +++ b/cloud/test/recycler_test.cpp @@ -0,0 +1,2422 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "recycler/recycler.h" + +#include +#include +#include + +#include +#include +#include +#include +#include + +#include "common/config.h" +#include "common/logging.h" +#include "common/sync_point.h" +#include "common/util.h" +#include "meta-service/keys.h" +#include "meta-service/mem_txn_kv.h" +#include "meta-service/meta_service.h" +#include "meta-service/txn_kv_error.h" +#include "mock_resource_manager.h" +#include "rate-limiter/rate_limiter.h" +#include "recycler/checker.h" +#include "recycler/s3_accessor.h" +#include "recycler/util.h" +#include "recycler/white_black_list.h" + +using namespace doris; + +static const std::string instance_id = "instance_id_recycle_test"; +static int64_t current_time = 0; +static constexpr int64_t db_id = 1000; + +int main(int argc, char** argv) { + auto conf_file = "doris_cloud.conf"; + if (!cloud::config::init(conf_file, true)) { + std::cerr << "failed to init config file, conf=" << conf_file << std::endl; + return -1; + } + if (!cloud::init_glog("recycler")) { + std::cerr << "failed to init glog" << std::endl; + return -1; + } + + using namespace std::chrono; + current_time = duration_cast(system_clock::now().time_since_epoch()).count(); + + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + +namespace doris::cloud { + +TEST(RecyclerTest, WhiteBlackList) { + WhiteBlackList filter; + EXPECT_FALSE(filter.filter_out("instance1")); + EXPECT_FALSE(filter.filter_out("instance2")); + filter.reset({}, {"instance1", "instance2"}); + EXPECT_TRUE(filter.filter_out("instance1")); + EXPECT_TRUE(filter.filter_out("instance2")); + EXPECT_FALSE(filter.filter_out("instance3")); + filter.reset({"instance1", "instance2"}, {}); + EXPECT_FALSE(filter.filter_out("instance1")); + EXPECT_FALSE(filter.filter_out("instance2")); + EXPECT_TRUE(filter.filter_out("instance3")); + filter.reset({"instance1"}, {"instance1"}); // whitelist overrides blacklist + EXPECT_FALSE(filter.filter_out("instance1")); + EXPECT_TRUE(filter.filter_out("instance2")); +} + +static std::string next_rowset_id() { + static int64_t cnt = 0; + return std::to_string(++cnt); +} + +static doris::RowsetMetaCloudPB create_rowset(const std::string& resource_id, int64_t tablet_id, + int64_t index_id, int num_segments, + const doris::TabletSchemaCloudPB& schema, + int64_t txn_id = 0) { + doris::RowsetMetaCloudPB rowset; + rowset.set_rowset_id(0); // useless but required + rowset.set_rowset_id_v2(next_rowset_id()); + rowset.set_txn_id(txn_id); + rowset.set_num_segments(num_segments); + rowset.set_tablet_id(tablet_id); + rowset.set_index_id(index_id); + rowset.set_resource_id(resource_id); + rowset.set_schema_version(schema.schema_version()); + rowset.mutable_tablet_schema()->CopyFrom(schema); + return rowset; +} + +static int create_recycle_rowset(TxnKv* txn_kv, ObjStoreAccessor* accessor, + const doris::RowsetMetaCloudPB& rowset, RecycleRowsetPB::Type type, + bool write_schema_kv) { + std::string key; + std::string val; + + RecycleRowsetKeyInfo key_info {instance_id, rowset.tablet_id(), rowset.rowset_id_v2()}; + recycle_rowset_key(key_info, &key); + + RecycleRowsetPB rowset_pb; + rowset_pb.set_creation_time(current_time); + if (type != RecycleRowsetPB::UNKNOWN) { + rowset_pb.set_type(type); + rowset_pb.mutable_rowset_meta()->CopyFrom(rowset); + if (write_schema_kv) { // Detach schema + rowset_pb.mutable_rowset_meta()->set_allocated_tablet_schema(nullptr); + } + } else { // old version RecycleRowsetPB + rowset_pb.set_tablet_id(rowset.tablet_id()); + rowset_pb.set_resource_id(rowset.resource_id()); + } + rowset_pb.SerializeToString(&val); + + std::unique_ptr txn; + if (txn_kv->create_txn(&txn) != TxnErrorCode::TXN_OK) { + return -1; + } + txn->put(key, val); + std::string schema_key, schema_val; + if (write_schema_kv) { + meta_schema_key({instance_id, rowset.index_id(), rowset.schema_version()}, &schema_key); + rowset.tablet_schema().SerializeToString(&schema_val); + txn->put(schema_key, schema_val); + } + if (txn->commit() != TxnErrorCode::TXN_OK) { + return -1; + } + + for (int i = 0; i < rowset.num_segments(); ++i) { + auto path = segment_path(rowset.tablet_id(), rowset.rowset_id_v2(), i); + accessor->put_object(path, path); + for (auto& index : rowset.tablet_schema().index()) { + auto path = inverted_index_path(rowset.tablet_id(), rowset.rowset_id_v2(), i, + index.index_id()); + accessor->put_object(path, path); + } + } + return 0; +} + +static int create_tmp_rowset(TxnKv* txn_kv, ObjStoreAccessor* accessor, + const doris::RowsetMetaCloudPB& rowset, bool write_schema_kv) { + std::string key, val; + meta_rowset_tmp_key({instance_id, rowset.txn_id(), rowset.tablet_id()}, &key); + if (write_schema_kv) { + auto rowset_copy = rowset; + rowset_copy.clear_tablet_schema(); + rowset_copy.SerializeToString(&val); + } else { + rowset.SerializeToString(&val); + } + std::unique_ptr txn; + if (txn_kv->create_txn(&txn) != TxnErrorCode::TXN_OK) { + return -1; + } + txn->put(key, val); + std::string schema_key, schema_val; + if (write_schema_kv) { + meta_schema_key({instance_id, rowset.index_id(), rowset.schema_version()}, &schema_key); + rowset.tablet_schema().SerializeToString(&schema_val); + txn->put(schema_key, schema_val); + } + if (txn->commit() != TxnErrorCode::TXN_OK) { + return -1; + } + + for (int i = 0; i < rowset.num_segments(); ++i) { + auto path = segment_path(rowset.tablet_id(), rowset.rowset_id_v2(), i); + accessor->put_object(path, path); + for (auto& index : rowset.tablet_schema().index()) { + auto path = inverted_index_path(rowset.tablet_id(), rowset.rowset_id_v2(), i, + index.index_id()); + accessor->put_object(path, path); + } + } + return 0; +} + +static int create_committed_rowset(TxnKv* txn_kv, ObjStoreAccessor* accessor, + const std::string& resource_id, int64_t tablet_id, + int64_t version, int num_segments = 1, + int num_inverted_indexes = 1) { + std::string key; + std::string val; + + auto rowset_id = next_rowset_id(); + MetaRowsetKeyInfo key_info {instance_id, tablet_id, version}; + meta_rowset_key(key_info, &key); + + doris::RowsetMetaCloudPB rowset_pb; + rowset_pb.set_rowset_id(0); // useless but required + rowset_pb.set_rowset_id_v2(rowset_id); + rowset_pb.set_num_segments(num_segments); + rowset_pb.set_tablet_id(tablet_id); + rowset_pb.set_resource_id(resource_id); + rowset_pb.set_creation_time(current_time); + if (num_inverted_indexes > 0) { + auto schema = rowset_pb.mutable_tablet_schema(); + for (int i = 0; i < num_inverted_indexes; ++i) { + schema->add_index()->set_index_id(i); + } + } + rowset_pb.SerializeToString(&val); + + std::unique_ptr txn; + if (txn_kv->create_txn(&txn) != TxnErrorCode::TXN_OK) { + return -1; + } + txn->put(key, val); + if (txn->commit() != TxnErrorCode::TXN_OK) { + return -1; + } + + for (int i = 0; i < num_segments; ++i) { + auto path = segment_path(tablet_id, rowset_id, i); + accessor->put_object(path, path); + for (int j = 0; j < num_inverted_indexes; ++j) { + auto path = inverted_index_path(tablet_id, rowset_id, i, j); + accessor->put_object(path, path); + } + } + return 0; +} + +static int create_tablet(TxnKv* txn_kv, int64_t table_id, int64_t index_id, int64_t partition_id, + int64_t tablet_id) { + std::unique_ptr txn; + if (txn_kv->create_txn(&txn) != TxnErrorCode::TXN_OK) { + return -1; + } + auto key = meta_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + doris::TabletMetaCloudPB tablet_meta; + tablet_meta.set_tablet_id(tablet_id); + auto val = tablet_meta.SerializeAsString(); + txn->put(key, val); + key = meta_tablet_idx_key({instance_id, tablet_id}); + txn->put(key, val); // val is not necessary + key = stats_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + txn->put(key, val); // val is not necessary + key = job_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); + txn->put(key, val); // val is not necessary + if (txn->commit() != TxnErrorCode::TXN_OK) { + return -1; + } + return 0; +} + +static int create_recycle_partiton(TxnKv* txn_kv, int64_t table_id, int64_t partition_id, + const std::vector& index_ids) { + std::string key; + std::string val; + + RecyclePartKeyInfo key_info {instance_id, partition_id}; + recycle_partition_key(key_info, &key); + + RecyclePartitionPB partition_pb; + partition_pb.set_db_id(db_id); + partition_pb.set_table_id(table_id); + for (auto index_id : index_ids) { + partition_pb.add_index_id(index_id); + } + partition_pb.set_creation_time(current_time); + partition_pb.set_state(RecyclePartitionPB::DROPPED); + partition_pb.SerializeToString(&val); + + std::unique_ptr txn; + if (txn_kv->create_txn(&txn) != TxnErrorCode::TXN_OK) { + return -1; + } + txn->put(key, val); + if (txn->commit() != TxnErrorCode::TXN_OK) { + return -1; + } + return 0; +} + +static int create_version_kv(TxnKv* txn_kv, int64_t table_id, int64_t partition_id) { + auto key = version_key({instance_id, db_id, table_id, partition_id}); + VersionPB version; + version.set_version(1); + auto val = version.SerializeAsString(); + std::unique_ptr txn; + if (txn_kv->create_txn(&txn) != TxnErrorCode::TXN_OK) { + return -1; + } + txn->put(key, val); + if (txn->commit() != TxnErrorCode::TXN_OK) { + return -1; + } + return 0; +} + +static int create_txn_label_kv(TxnKv* txn_kv, std::string label, int64_t db_id) { + std::string txn_label_key_; + std::string txn_label_val; + auto keyinfo = TxnLabelKeyInfo({instance_id, db_id, label}); + txn_label_key(keyinfo, &txn_label_key_); + std::unique_ptr txn; + if (txn_kv->create_txn(&txn) != TxnErrorCode::TXN_OK) { + return -1; + } + txn->put(txn_label_key_, label); + if (txn->commit() != TxnErrorCode::TXN_OK) { + return -1; + } + return 0; +} + +static int create_recycle_index(TxnKv* txn_kv, int64_t table_id, int64_t index_id) { + std::string key; + std::string val; + + RecycleIndexKeyInfo key_info {instance_id, index_id}; + recycle_index_key(key_info, &key); + + RecycleIndexPB index_pb; + + index_pb.set_table_id(table_id); + index_pb.set_creation_time(current_time); + index_pb.set_state(RecycleIndexPB::DROPPED); + index_pb.SerializeToString(&val); + + std::unique_ptr txn; + if (txn_kv->create_txn(&txn) != TxnErrorCode::TXN_OK) { + return -1; + } + txn->put(key, val); + if (txn->commit() != TxnErrorCode::TXN_OK) { + return -1; + } + return 0; +} + +static int get_txn_info(std::shared_ptr txn_kv, std::string instance_id, int64_t db_id, + int64_t txn_id, TxnInfoPB& txn_info_pb) { + std::string txn_inf_key; + std::string txn_inf_val; + TxnInfoKeyInfo txn_inf_key_info {instance_id, db_id, txn_id}; + + LOG(INFO) << instance_id << "|" << db_id << "|" << txn_id; + + std::unique_ptr txn; + if (txn_kv->create_txn(&txn) != TxnErrorCode::TXN_OK) { + return -1; + } + txn_info_key(txn_inf_key_info, &txn_inf_key); + LOG(INFO) << "txn_inf_key:" << hex(txn_inf_key); + TxnErrorCode err = txn->get(txn_inf_key, &txn_inf_val); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "txn->get failed, err=" << err; + return -2; + } + + if (!txn_info_pb.ParseFromString(txn_inf_val)) { + LOG(WARNING) << "ParseFromString failed"; + return -3; + } + LOG(INFO) << "txn_info_pb" << txn_info_pb.DebugString(); + err = txn->commit(); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "txn->commit failed, err=" << err; + return -4; + } + return 0; +} + +static int check_recycle_txn_keys(std::shared_ptr txn_kv, std::string instance_id, + int64_t db_id, int64_t txn_id, const std::string& label) { + std::string txn_inf_key; + std::string txn_inf_val; + TxnInfoKeyInfo txn_inf_key_info {instance_id, db_id, txn_id}; + + LOG(INFO) << instance_id << "|" << db_id << "|" << txn_id; + + std::unique_ptr txn; + if (txn_kv->create_txn(&txn) != TxnErrorCode::TXN_OK) { + return -1; + } + txn_info_key(txn_inf_key_info, &txn_inf_key); + TxnErrorCode err = txn->get(txn_inf_key, &txn_inf_val); + if (err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + return -2; + } + + std::string label_key, label_val; + txn_label_key({instance_id, db_id, label}, &label_key); + err = txn->get(label_key, &label_val); + if (err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + return -3; + } + + std::string index_key, index_val; + index_key = txn_index_key({instance_id, txn_id}); + err = txn->get(index_key, &index_val); + if (err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + return -4; + } + + std::string running_key; + std::string running_value; + TxnRunningKeyInfo running_key_info {instance_id, db_id, txn_id}; + txn_running_key(running_key_info, &running_key); + err = txn->get(running_key, &running_value); + if (err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + return -5; + } + + std::string rec_txn_key; + std::string rec_txn_val; + RecycleTxnKeyInfo recycle_txn_key_info {instance_id, db_id, txn_id}; + recycle_txn_key(recycle_txn_key_info, &rec_txn_key); + err = txn->get(rec_txn_key, &rec_txn_val); + if (err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + return -6; + } + + return 0; +} + +static int create_instance(const std::string& internal_stage_id, + const std::string& external_stage_id, InstanceInfoPB& instance_info) { + // create internal stage + { + std::string s3_prefix = "internal_prefix"; + std::string stage_prefix = fmt::format("{}/stage/root/{}/", s3_prefix, internal_stage_id); + ObjectStoreInfoPB object_info; + object_info.set_id(internal_stage_id); // test use accessor_map_ in recycle + object_info.set_ak("ak"); + object_info.set_sk("sk"); + object_info.set_bucket("bucket"); + object_info.set_endpoint("endpoint"); + object_info.set_region("region"); + // in real case, this is {s3_prefix} + object_info.set_prefix(stage_prefix); + object_info.set_provider(ObjectStoreInfoPB::OSS); + + StagePB internal_stage; + internal_stage.set_type(StagePB::INTERNAL); + internal_stage.set_stage_id(internal_stage_id); + ObjectStoreInfoPB internal_object_info; + internal_object_info.set_prefix(stage_prefix); + internal_object_info.set_id("0"); + internal_stage.mutable_obj_info()->CopyFrom(internal_object_info); + + instance_info.add_obj_info()->CopyFrom(object_info); + instance_info.add_stages()->CopyFrom(internal_stage); + } + + // create external stage + { + ObjectStoreInfoPB object_info; + object_info.set_id(external_stage_id); + object_info.set_ak("ak1"); + object_info.set_sk("sk1"); + object_info.set_bucket("bucket1"); + object_info.set_endpoint("endpoint1"); + object_info.set_region("region1"); + object_info.set_prefix("external_prefix"); + object_info.set_provider(ObjectStoreInfoPB::OSS); + + StagePB external_stage; + external_stage.set_type(StagePB::EXTERNAL); + external_stage.set_stage_id(external_stage_id); + external_stage.mutable_obj_info()->CopyFrom(object_info); + + instance_info.add_obj_info()->CopyFrom(object_info); + instance_info.add_stages()->CopyFrom(external_stage); + } + + instance_info.set_instance_id(instance_id); + return 0; +} + +static int create_copy_job(TxnKv* txn_kv, const std::string& stage_id, int64_t table_id, + StagePB::StageType stage_type, CopyJobPB::JobStatus job_status, + std::vector object_files, int64_t timeout_time, + int64_t start_time = 0, int64_t finish_time = 0) { + std::string key; + std::string val; + CopyJobKeyInfo key_info {instance_id, stage_id, table_id, "copy_id", 0}; + copy_job_key(key_info, &key); + + CopyJobPB copy_job; + copy_job.set_stage_type(stage_type); + copy_job.set_job_status(job_status); + copy_job.set_timeout_time_ms(timeout_time); + if (start_time != 0) { + copy_job.set_start_time_ms(start_time); + } + if (finish_time != 0) { + copy_job.set_finish_time_ms(finish_time); + } + for (const auto& file : object_files) { + copy_job.add_object_files()->CopyFrom(file); + } + copy_job.SerializeToString(&val); + + std::vector file_keys; + std::string file_val; + CopyFilePB copy_file; + copy_file.set_copy_id("copy_id"); + copy_file.set_group_id(0); + file_val = copy_file.SerializeAsString(); + + // create job files + for (const auto& file : object_files) { + CopyFileKeyInfo file_info {instance_id, stage_id, table_id, file.relative_path(), + file.etag()}; + std::string file_key; + copy_file_key(file_info, &file_key); + file_keys.push_back(file_key); + } + + std::unique_ptr txn; + if (txn_kv->create_txn(&txn) != TxnErrorCode::TXN_OK) { + return -1; + } + txn->put(key, val); + for (const auto& file_key : file_keys) { + txn->put(file_key, file_val); + } + if (txn->commit() != TxnErrorCode::TXN_OK) { + return -1; + } + return 0; +} + +static int copy_job_exists(TxnKv* txn_kv, const std::string& stage_id, int64_t table_id, + bool* exist) { + std::string key; + std::string val; + CopyJobKeyInfo key_info {instance_id, stage_id, table_id, "copy_id", 0}; + copy_job_key(key_info, &key); + + std::unique_ptr txn; + if (txn_kv->create_txn(&txn) != TxnErrorCode::TXN_OK) { + return -1; + } + TxnErrorCode err = txn->get(key, &val); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + return -1; + } + *exist = err == TxnErrorCode::TXN_OK; + return 0; +} + +static int create_object_files(ObjStoreAccessor* accessor, + std::vector* object_files) { + for (auto& file : *object_files) { + auto key = file.relative_path(); + if (accessor->put_object(key, key) != 0) { + return -1; + } + file.set_etag(""); + } + return 0; +} + +static int get_copy_file_num(TxnKv* txn_kv, const std::string& stage_id, int64_t table_id, + int* file_num) { + *file_num = 0; + std::string key0; + std::string key1; + CopyFileKeyInfo key_info0 {instance_id, stage_id, table_id, "", ""}; + CopyFileKeyInfo key_info1 {instance_id, stage_id, table_id + 1, "", ""}; + copy_file_key(key_info0, &key0); + copy_file_key(key_info1, &key1); + + std::unique_ptr txn; + if (txn_kv->create_txn(&txn) != TxnErrorCode::TXN_OK) { + return -1; + } + std::unique_ptr it; + do { + if (txn->get(key0, key1, &it) != TxnErrorCode::TXN_OK) { + return -1; + } + while (it->has_next()) { + it->next(); + ++(*file_num); + } + key0.push_back('\x00'); + } while (it->more()); + return 0; +} + +TEST(RecyclerTest, recycle_empty) { + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + + InstanceInfoPB instance; + instance.set_instance_id(instance_id); + auto obj_info = instance.add_obj_info(); + obj_info->set_id("recycle_empty"); + obj_info->set_ak(config::test_s3_ak); + obj_info->set_sk(config::test_s3_sk); + obj_info->set_endpoint(config::test_s3_endpoint); + obj_info->set_region(config::test_s3_region); + obj_info->set_bucket(config::test_s3_bucket); + obj_info->set_prefix("recycle_empty"); + + InstanceRecycler recycler(txn_kv, instance); + ASSERT_EQ(recycler.init(), 0); + + ASSERT_EQ(recycler.recycle_rowsets(), 0); +} + +TEST(RecyclerTest, recycle_rowsets) { + config::retention_seconds = 0; + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + + InstanceInfoPB instance; + instance.set_instance_id(instance_id); + auto obj_info = instance.add_obj_info(); + obj_info->set_id("recycle_rowsets"); + obj_info->set_ak(config::test_s3_ak); + obj_info->set_sk(config::test_s3_sk); + obj_info->set_endpoint(config::test_s3_endpoint); + obj_info->set_region(config::test_s3_region); + obj_info->set_bucket(config::test_s3_bucket); + obj_info->set_prefix("recycle_rowsets"); + + config::instance_recycler_worker_pool_size = 1; + + int insert_no_inverted_index = 0; + int insert_inverted_index = 0; + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("InvertedIndexIdCache::insert1", + [&](void* p) { ++insert_no_inverted_index; }); + sp->set_call_back("InvertedIndexIdCache::insert2", [&](void* p) { ++insert_inverted_index; }); + sp->enable_processing(); + + InstanceRecycler recycler(txn_kv, instance); + ASSERT_EQ(recycler.init(), 0); + + std::vector schemas; + for (int i = 0; i < 5; ++i) { + auto& schema = schemas.emplace_back(); + schema.set_schema_version(i); + for (int j = 0; j < i; ++j) { + schema.add_index()->set_index_id(j); + } + } + + auto accessor = recycler.accessor_map_.begin()->second; + constexpr int index_id = 10001, tablet_id = 10002; + for (int i = 0; i < 1000; ++i) { + auto rowset = create_rowset("recycle_rowsets", tablet_id, index_id, 5, schemas[i % 5]); + create_recycle_rowset( + txn_kv.get(), accessor.get(), rowset, + static_cast(i % (RecycleRowsetPB::Type_MAX + 1)), i & 1); + } + for (int i = 0; i < 1000; ++i) { + auto rowset = create_rowset("recycle_rowsets", tablet_id, index_id, 5, schemas[i % 5]); + create_recycle_rowset(txn_kv.get(), accessor.get(), rowset, RecycleRowsetPB::COMPACT, true); + } + + ASSERT_EQ(recycler.recycle_rowsets(), 0); + + // check rowset does not exist on obj store + std::vector files; + ASSERT_EQ(0, accessor->list(tablet_path_prefix(tablet_id), &files)); + EXPECT_TRUE(files.empty()); + // check all recycle rowset kv have been deleted + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::unique_ptr it; + auto begin_key = recycle_key_prefix(instance_id); + auto end_key = recycle_key_prefix(instance_id + '\xff'); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + EXPECT_EQ(it->size(), 0); + // Check InvertedIndexIdCache + EXPECT_EQ(insert_inverted_index, 4); + EXPECT_EQ(insert_no_inverted_index, 1); +} + +TEST(RecyclerTest, bench_recycle_rowsets) { + config::retention_seconds = 0; + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + + InstanceInfoPB instance; + instance.set_instance_id(instance_id); + auto obj_info = instance.add_obj_info(); + obj_info->set_id("recycle_rowsets"); + obj_info->set_ak(config::test_s3_ak); + obj_info->set_sk(config::test_s3_sk); + obj_info->set_endpoint(config::test_s3_endpoint); + obj_info->set_region(config::test_s3_region); + obj_info->set_bucket(config::test_s3_bucket); + obj_info->set_prefix("recycle_rowsets"); + + config::instance_recycler_worker_pool_size = 10; + InstanceRecycler recycler(txn_kv, instance); + ASSERT_EQ(recycler.init(), 0); + + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("memkv::Transaction::get", [](void* limit) { + *((int*)limit) = 100; + std::this_thread::sleep_for(std::chrono::milliseconds(5)); + }); + sp->set_call_back("MockAccessor::delete_objects", + [&](void* p) { std::this_thread::sleep_for(std::chrono::milliseconds(20)); }); + sp->set_call_back("MockAccessor::delete_objects_by_prefix", + [&](void* p) { std::this_thread::sleep_for(std::chrono::milliseconds(20)); }); + sp->enable_processing(); + + std::vector schemas; + for (int i = 0; i < 5; ++i) { + auto& schema = schemas.emplace_back(); + schema.set_schema_version(i); + for (int j = 0; j < i; ++j) { + schema.add_index()->set_index_id(j); + } + } + + auto accessor = recycler.accessor_map_.begin()->second; + constexpr int index_id = 10001, tablet_id = 10002; + for (int i = 0; i < 2000; ++i) { + auto rowset = create_rowset("recycle_rowsets", tablet_id, index_id, 5, schemas[i % 5]); + create_recycle_rowset(txn_kv.get(), accessor.get(), rowset, + i % 10 < 2 ? RecycleRowsetPB::PREPARE : RecycleRowsetPB::COMPACT, + i & 1); + } + + ASSERT_EQ(recycler.recycle_rowsets(), 0); + + // check rowset does not exist on obj store + std::vector files; + ASSERT_EQ(0, accessor->list(tablet_path_prefix(tablet_id), &files)); + ASSERT_TRUE(files.empty()); + // check all recycle rowset kv have been deleted + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::unique_ptr it; + auto begin_key = recycle_key_prefix(instance_id); + auto end_key = recycle_key_prefix(instance_id + '\xff'); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); +} + +TEST(RecyclerTest, recycle_tmp_rowsets) { + config::retention_seconds = 0; + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + + InstanceInfoPB instance; + instance.set_instance_id(instance_id); + auto obj_info = instance.add_obj_info(); + obj_info->set_id("recycle_tmp_rowsets"); + obj_info->set_ak(config::test_s3_ak); + obj_info->set_sk(config::test_s3_sk); + obj_info->set_endpoint(config::test_s3_endpoint); + obj_info->set_region(config::test_s3_region); + obj_info->set_bucket(config::test_s3_bucket); + obj_info->set_prefix("recycle_tmp_rowsets"); + + int insert_no_inverted_index = 0; + int insert_inverted_index = 0; + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("InvertedIndexIdCache::insert1", + [&](void* p) { ++insert_no_inverted_index; }); + sp->set_call_back("InvertedIndexIdCache::insert2", [&](void* p) { ++insert_inverted_index; }); + sp->enable_processing(); + + InstanceRecycler recycler(txn_kv, instance); + ASSERT_EQ(recycler.init(), 0); + + std::vector schemas; + for (int i = 0; i < 5; ++i) { + auto& schema = schemas.emplace_back(); + schema.set_schema_version(i); + for (int j = 0; j < i; ++j) { + schema.add_index()->set_index_id(j); + } + } + + auto accessor = recycler.accessor_map_.begin()->second; + int64_t txn_id_base = 114115; + int64_t tablet_id_base = 10015; + int64_t index_id_base = 1000; + for (int i = 0; i < 100; ++i) { + int64_t txn_id = txn_id_base + i; + for (int j = 0; j < 20; ++j) { + auto rowset = create_rowset("recycle_tmp_rowsets", tablet_id_base + j, + index_id_base + j % 4, 5, schemas[i % 5], txn_id); + create_tmp_rowset(txn_kv.get(), accessor.get(), rowset, i & 1); + } + } + + ASSERT_EQ(recycler.recycle_tmp_rowsets(), 0); + + // check rowset does not exist on obj store + std::vector files; + ASSERT_EQ(0, accessor->list("data/", &files)); + ASSERT_TRUE(files.empty()); + // check all tmp rowset kv have been deleted + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::unique_ptr it; + auto begin_key = meta_rowset_tmp_key({instance_id, 0, 0}); + auto end_key = meta_rowset_tmp_key({instance_id, INT64_MAX, 0}); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + // Check InvertedIndexIdCache + EXPECT_EQ(insert_inverted_index, 16); + EXPECT_EQ(insert_no_inverted_index, 4); +} + +TEST(RecyclerTest, recycle_tablet) { + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + + InstanceInfoPB instance; + instance.set_instance_id(instance_id); + auto obj_info = instance.add_obj_info(); + obj_info->set_id("recycle_tablet"); + obj_info->set_ak(config::test_s3_ak); + obj_info->set_sk(config::test_s3_sk); + obj_info->set_endpoint(config::test_s3_endpoint); + obj_info->set_region(config::test_s3_region); + obj_info->set_bucket(config::test_s3_bucket); + obj_info->set_prefix("recycle_tablet"); + + InstanceRecycler recycler(txn_kv, instance); + ASSERT_EQ(recycler.init(), 0); + + std::vector schemas; + for (int i = 0; i < 5; ++i) { + auto& schema = schemas.emplace_back(); + schema.set_schema_version(i); + for (int j = 0; j < i; ++j) { + schema.add_index()->set_index_id(j); + } + } + + constexpr int table_id = 10000, index_id = 10001, partition_id = 10002, tablet_id = 10003; + auto accessor = recycler.accessor_map_.begin()->second; + create_tablet(txn_kv.get(), table_id, index_id, partition_id, tablet_id); + for (int i = 0; i < 500; ++i) { + auto rowset = create_rowset("recycle_tablet", tablet_id, index_id, 5, schemas[i % 5]); + create_recycle_rowset(txn_kv.get(), accessor.get(), rowset, + i % 10 < 2 ? RecycleRowsetPB::PREPARE : RecycleRowsetPB::COMPACT, + i & 1); + } + for (int i = 0; i < 500; ++i) { + create_committed_rowset(txn_kv.get(), accessor.get(), "recycle_tablet", tablet_id, i); + } + + ASSERT_EQ(0, recycler.recycle_tablets(table_id, index_id)); + + // check rowset does not exist on s3 + std::vector files; + ASSERT_EQ(0, accessor->list(tablet_path_prefix(tablet_id), &files)); + ASSERT_TRUE(files.empty()); + // check all related kv have been deleted + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::unique_ptr it; + // meta_tablet_key, meta_tablet_idx_key, meta_rowset_key + auto begin_key = meta_key_prefix(instance_id); + auto end_key = meta_key_prefix(instance_id + '\xff'); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + // job_tablet_key + begin_key = job_tablet_key({instance_id, table_id, 0, 0, 0}); + end_key = job_tablet_key({instance_id, table_id + 1, 0, 0, 0}); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + // stats_tablet_key + begin_key = stats_tablet_key({instance_id, table_id, 0, 0, 0}); + end_key = stats_tablet_key({instance_id, table_id + 1, 0, 0, 0}); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + // recycle_rowset_key + begin_key = recycle_key_prefix(instance_id); + end_key = recycle_key_prefix(instance_id + '\xff'); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); +} + +TEST(RecyclerTest, recycle_indexes) { + config::retention_seconds = 0; + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + + InstanceInfoPB instance; + instance.set_instance_id(instance_id); + auto obj_info = instance.add_obj_info(); + obj_info->set_id("recycle_indexes"); + obj_info->set_ak(config::test_s3_ak); + obj_info->set_sk(config::test_s3_sk); + obj_info->set_endpoint(config::test_s3_endpoint); + obj_info->set_region(config::test_s3_region); + obj_info->set_bucket(config::test_s3_bucket); + obj_info->set_prefix("recycle_indexes"); + + InstanceRecycler recycler(txn_kv, instance); + ASSERT_EQ(recycler.init(), 0); + + std::vector schemas; + for (int i = 0; i < 5; ++i) { + auto& schema = schemas.emplace_back(); + schema.set_schema_version(i); + for (int j = 0; j < i; ++j) { + schema.add_index()->set_index_id(j); + } + } + + constexpr int table_id = 10000, index_id = 10001, partition_id = 10002; + auto accessor = recycler.accessor_map_.begin()->second; + int64_t tablet_id_base = 10100; + int64_t txn_id_base = 114115; + for (int i = 0; i < 100; ++i) { + int64_t tablet_id = tablet_id_base + i; + create_tablet(txn_kv.get(), table_id, index_id, partition_id, tablet_id); + for (int j = 0; j < 10; ++j) { + auto rowset = create_rowset("recycle_tablet", tablet_id, index_id, 5, schemas[j % 5]); + create_recycle_rowset(txn_kv.get(), accessor.get(), rowset, + j % 10 < 2 ? RecycleRowsetPB::PREPARE : RecycleRowsetPB::COMPACT, + j & 1); + auto tmp_rowset = create_rowset("recycle_tmp_rowsets", tablet_id, index_id, 5, + schemas[j % 5], txn_id_base + j); + create_tmp_rowset(txn_kv.get(), accessor.get(), tmp_rowset, j & 1); + } + for (int j = 0; j < 10; ++j) { + create_committed_rowset(txn_kv.get(), accessor.get(), "recycle_indexes", tablet_id, j); + } + } + create_recycle_index(txn_kv.get(), table_id, index_id); + ASSERT_EQ(recycler.recycle_indexes(), 0); + + // check rowset does not exist on s3 + std::vector files; + ASSERT_EQ(0, accessor->list("data/", &files)); + ASSERT_TRUE(files.empty()); + // check all related kv have been deleted + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::unique_ptr it; + // meta_rowset_key + auto begin_key = meta_rowset_key({instance_id, 0, 0}); + auto end_key = meta_rowset_key({instance_id, INT64_MAX, 0}); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + // meta_rowset_tmp_key + begin_key = meta_rowset_tmp_key({instance_id, 0, 0}); + end_key = meta_rowset_tmp_key({instance_id, INT64_MAX, 0}); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 1000); + // meta_tablet_idx_key + begin_key = meta_tablet_idx_key({instance_id, 0}); + end_key = meta_tablet_idx_key({instance_id, INT64_MAX}); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + // meta_tablet_key + begin_key = meta_tablet_key({instance_id, 0, 0, 0, 0}); + end_key = meta_tablet_key({instance_id, INT64_MAX, 0, 0, 0}); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + // meta_schema_key + begin_key = meta_schema_key({instance_id, 0, 0}); + end_key = meta_schema_key({instance_id, INT64_MAX, 0}); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + // job_tablet_key + begin_key = job_tablet_key({instance_id, table_id, 0, 0, 0}); + end_key = job_tablet_key({instance_id, table_id + 1, 0, 0, 0}); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + // stats_tablet_key + begin_key = stats_tablet_key({instance_id, table_id, 0, 0, 0}); + end_key = stats_tablet_key({instance_id, table_id + 1, 0, 0, 0}); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + // recycle_rowset_key + begin_key = recycle_key_prefix(instance_id); + end_key = recycle_key_prefix(instance_id + '\xff'); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + + // Test recycle tmp rowsets after recycle indexes + ASSERT_EQ(recycler.recycle_tmp_rowsets(), 0); + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + begin_key = meta_rowset_tmp_key({instance_id, 0, 0}); + end_key = meta_rowset_tmp_key({instance_id, INT64_MAX, 0}); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); +} + +TEST(RecyclerTest, recycle_partitions) { + config::retention_seconds = 0; + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + + InstanceInfoPB instance; + instance.set_instance_id(instance_id); + auto obj_info = instance.add_obj_info(); + obj_info->set_id("recycle_partitions"); + obj_info->set_ak(config::test_s3_ak); + obj_info->set_sk(config::test_s3_sk); + obj_info->set_endpoint(config::test_s3_endpoint); + obj_info->set_region(config::test_s3_region); + obj_info->set_bucket(config::test_s3_bucket); + obj_info->set_prefix("recycle_partitions"); + + InstanceRecycler recycler(txn_kv, instance); + ASSERT_EQ(recycler.init(), 0); + + std::vector schemas; + for (int i = 0; i < 5; ++i) { + auto& schema = schemas.emplace_back(); + schema.set_schema_version(i); + for (int j = 0; j < i; ++j) { + schema.add_index()->set_index_id(j); + } + } + + constexpr int table_id = 10000, partition_id = 30020; + auto accessor = recycler.accessor_map_.begin()->second; + std::vector index_ids {20200, 20201, 20202, 20203, 20204}; + + int64_t tablet_id_base = 10100; + for (auto index_id : index_ids) { + for (int i = 0; i < 20; ++i) { + int64_t tablet_id = tablet_id_base + i; + create_tablet(txn_kv.get(), table_id, index_id, partition_id, tablet_id); + for (int j = 0; j < 10; ++j) { + auto rowset = + create_rowset("recycle_tablet", tablet_id, index_id, 5, schemas[j % 5]); + create_recycle_rowset( + txn_kv.get(), accessor.get(), rowset, + j % 10 < 2 ? RecycleRowsetPB::PREPARE : RecycleRowsetPB::COMPACT, j & 1); + } + for (int j = 0; j < 10; ++j) { + create_committed_rowset(txn_kv.get(), accessor.get(), "recycle_partitions", + tablet_id, j); + } + } + } + create_recycle_partiton(txn_kv.get(), table_id, partition_id, index_ids); + ASSERT_EQ(recycler.recycle_partitions(), 0); + + // check rowset does not exist on s3 + std::vector files; + ASSERT_EQ(0, accessor->list("data/", &files)); + ASSERT_TRUE(files.empty()); + // check all related kv have been deleted + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::unique_ptr it; + // meta_rowset_key + auto begin_key = meta_rowset_key({instance_id, 0, 0}); + auto end_key = meta_rowset_key({instance_id, INT64_MAX, 0}); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + // meta_rowset_tmp_key + begin_key = meta_rowset_tmp_key({instance_id, 0, 0}); + end_key = meta_rowset_tmp_key({instance_id, INT64_MAX, 0}); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + // meta_tablet_idx_key + begin_key = meta_tablet_idx_key({instance_id, 0}); + end_key = meta_tablet_idx_key({instance_id, INT64_MAX}); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + // meta_tablet_key + begin_key = meta_tablet_key({instance_id, 0, 0, 0, 0}); + end_key = meta_tablet_key({instance_id, INT64_MAX, 0, 0, 0}); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + // job_tablet_key + begin_key = job_tablet_key({instance_id, table_id, 0, 0, 0}); + end_key = job_tablet_key({instance_id, table_id + 1, 0, 0, 0}); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + // stats_tablet_key + begin_key = stats_tablet_key({instance_id, table_id, 0, 0, 0}); + end_key = stats_tablet_key({instance_id, table_id + 1, 0, 0, 0}); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + // recycle_rowset_key + begin_key = recycle_key_prefix(instance_id); + end_key = recycle_key_prefix(instance_id + '\xff'); + ASSERT_EQ(txn->get(begin_key, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); +} + +TEST(RecyclerTest, recycle_versions) { + config::retention_seconds = 0; + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + + std::vector index_ids {20001, 20002, 20003, 20004, 20005}; + std::vector partition_ids {30001, 30002, 30003, 30004, 30005, 30006}; + constexpr int64_t table_id = 10000; + + int64_t tablet_id = 40000; + for (auto index_id : index_ids) { + for (auto partition_id : partition_ids) { + create_tablet(txn_kv.get(), table_id, index_id, partition_id, ++tablet_id); + } + } + for (auto partition_id : partition_ids) { + create_version_kv(txn_kv.get(), table_id, partition_id); + } + // Drop partitions + for (int i = 0; i < 5; ++i) { + create_recycle_partiton(txn_kv.get(), table_id, partition_ids[i], index_ids); + } + + InstanceInfoPB instance; + instance.set_instance_id(instance_id); + InstanceRecycler recycler(txn_kv, instance); + ASSERT_EQ(recycler.init(), 0); + // Recycle all partitions in table except 30006 + ASSERT_EQ(recycler.recycle_partitions(), 0); + ASSERT_EQ(recycler.recycle_versions(), 0); // `recycle_versions` should do nothing + // All version kvs except version of partition 30006 must have been deleted + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + auto key_begin = version_key({instance_id, db_id, table_id, 0}); + auto key_end = version_key({instance_id, db_id, table_id, INT64_MAX}); + std::unique_ptr iter; + ASSERT_EQ(txn->get(key_begin, key_end, &iter), TxnErrorCode::TXN_OK); + ASSERT_EQ(iter->size(), 1); + auto [k, v] = iter->next(); + EXPECT_EQ(k, version_key({instance_id, db_id, table_id, 30006})); + + // Drop indexes + for (auto index_id : index_ids) { + create_recycle_index(txn_kv.get(), table_id, index_id); + } + // Recycle all indexes of the table, that is, the table has been dropped + ASSERT_EQ(recycler.recycle_indexes(), 0); + // `recycle_versions` should delete all version kvs of the dropped table + ASSERT_EQ(recycler.recycle_versions(), 0); + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(key_begin, key_end, &iter), TxnErrorCode::TXN_OK); + ASSERT_EQ(iter->size(), 0); +} + +TEST(RecyclerTest, abort_timeout_txn) { + auto txn_kv = std::dynamic_pointer_cast(std::make_shared()); + ASSERT_NE(txn_kv.get(), nullptr); + auto rs = std::make_shared(txn_kv); + auto rl = std::make_shared(); + auto meta_service = std::make_unique(txn_kv, rs, rl); + ASSERT_EQ(txn_kv->init(), 0); + + int64_t db_id = 666; + int64_t table_id = 1234; + int64_t txn_id = -1; + { + brpc::Controller cntl; + BeginTxnRequest req; + + req.set_cloud_unique_id("test_cloud_unique_id"); + TxnInfoPB txn_info_pb; + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label("abort_timeout_txn"); + txn_info_pb.add_table_ids(table_id); + txn_info_pb.set_timeout_ms(1); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + txn_id = res.txn_id(); + ASSERT_GT(txn_id, -1); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + InstanceInfoPB instance; + instance.set_instance_id(mock_instance); + InstanceRecycler recycler(txn_kv, instance); + ASSERT_EQ(recycler.init(), 0); + sleep(1); + ASSERT_EQ(recycler.abort_timeout_txn(), 0); + TxnInfoPB txn_info_pb; + get_txn_info(txn_kv, mock_instance, db_id, txn_id, txn_info_pb); + ASSERT_EQ(txn_info_pb.status(), TxnStatusPB::TXN_STATUS_ABORTED); +} + +TEST(RecyclerTest, abort_timeout_txn_and_rebegin) { + config::label_keep_max_second = 0; + auto txn_kv = std::dynamic_pointer_cast(std::make_shared()); + ASSERT_NE(txn_kv.get(), nullptr); + auto rs = std::make_shared(txn_kv); + auto rl = std::make_shared(); + auto meta_service = std::make_unique(txn_kv, rs, rl); + ASSERT_EQ(txn_kv->init(), 0); + + int64_t db_id = 888; + int64_t table_id = 1234; + int64_t txn_id = -1; + std::string cloud_unique_id = "test_cloud_unique_id22131"; + std::string label = "abort_timeout_txn_and_rebegin"; + { + brpc::Controller cntl; + BeginTxnRequest req; + + req.set_cloud_unique_id(cloud_unique_id); + TxnInfoPB txn_info_pb; + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label(label); + txn_info_pb.add_table_ids(table_id); + txn_info_pb.set_timeout_ms(1); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + txn_id = res.txn_id(); + ASSERT_GT(txn_id, -1); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + InstanceInfoPB instance; + instance.set_instance_id(mock_instance); + InstanceRecycler recycler(txn_kv, instance); + ASSERT_EQ(recycler.init(), 0); + sleep(1); + ASSERT_EQ(recycler.abort_timeout_txn(), 0); + TxnInfoPB txn_info_pb; + get_txn_info(txn_kv, mock_instance, db_id, txn_id, txn_info_pb); + ASSERT_EQ(txn_info_pb.status(), TxnStatusPB::TXN_STATUS_ABORTED); + + { + brpc::Controller cntl; + BeginTxnRequest req; + + req.set_cloud_unique_id(cloud_unique_id); + TxnInfoPB txn_info_pb; + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label(label); + txn_info_pb.add_table_ids(table_id); + txn_info_pb.set_timeout_ms(1); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + txn_id = res.txn_id(); + ASSERT_GT(txn_id, -1); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } +} + +TEST(RecyclerTest, recycle_expired_txn_label) { + config::label_keep_max_second = 0; + auto txn_kv = std::dynamic_pointer_cast(std::make_shared()); + ASSERT_NE(txn_kv.get(), nullptr); + auto rs = std::make_shared(txn_kv); + auto rl = std::make_shared(); + auto meta_service = std::make_unique(txn_kv, rs, rl); + ASSERT_EQ(txn_kv->init(), 0); + + int64_t db_id = 88812123; + int64_t table_id = 12131234; + int64_t txn_id = -1; + std::string cloud_unique_id = "test_cloud_unique_id2"; + std::string label = "recycle_expired_txn_label"; + { + // 1. begin_txn + // 2. abort_txn by db_id and label + // 3. recycle_expired_txn_label + // 4. check + { + brpc::Controller cntl; + BeginTxnRequest req; + + req.set_cloud_unique_id(cloud_unique_id); + TxnInfoPB txn_info_pb; + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label(label); + txn_info_pb.add_table_ids(table_id); + txn_info_pb.set_timeout_ms(100000); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + txn_id = res.txn_id(); + ASSERT_GT(txn_id, -1); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + InstanceInfoPB instance; + instance.set_instance_id(mock_instance); + InstanceRecycler recycler(txn_kv, instance); + ASSERT_EQ(recycler.init(), 0); + recycler.abort_timeout_txn(); + TxnInfoPB txn_info_pb; + ASSERT_EQ(get_txn_info(txn_kv, mock_instance, db_id, txn_id, txn_info_pb), 0); + ASSERT_EQ(txn_info_pb.status(), TxnStatusPB::TXN_STATUS_PREPARED); + + // abort txn by db_id and label + { + brpc::Controller cntl; + AbortTxnRequest req; + req.set_cloud_unique_id(cloud_unique_id); + req.set_db_id(db_id); + req.set_label(label); + req.set_reason("test"); + AbortTxnResponse res; + meta_service->abort_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.txn_info().status(), TxnStatusPB::TXN_STATUS_ABORTED); + } + recycler.recycle_expired_txn_label(); + ASSERT_EQ(get_txn_info(txn_kv, mock_instance, db_id, txn_id, txn_info_pb), -2); + ASSERT_EQ(check_recycle_txn_keys(txn_kv, mock_instance, db_id, txn_id, label), 0); + } + + { + // 1. begin_txn + // 2. abort_txn by db_id and txn_id + // 3. recycle_expired_txn_label + // 4. check + { + brpc::Controller cntl; + BeginTxnRequest req; + + req.set_cloud_unique_id(cloud_unique_id); + TxnInfoPB txn_info_pb; + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label(label); + txn_info_pb.add_table_ids(table_id); + txn_info_pb.set_timeout_ms(10000); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + txn_id = res.txn_id(); + ASSERT_GT(txn_id, -1); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + InstanceInfoPB instance; + instance.set_instance_id(mock_instance); + InstanceRecycler recycler(txn_kv, instance); + ASSERT_EQ(recycler.init(), 0); + sleep(1); + recycler.abort_timeout_txn(); + TxnInfoPB txn_info_pb; + get_txn_info(txn_kv, mock_instance, db_id, txn_id, txn_info_pb); + ASSERT_EQ(txn_info_pb.status(), TxnStatusPB::TXN_STATUS_PREPARED); + + // abort txn by db_id and label + { + brpc::Controller cntl; + AbortTxnRequest req; + req.set_cloud_unique_id(cloud_unique_id); + req.set_db_id(db_id); + req.set_txn_id(txn_id); + req.set_reason("test"); + AbortTxnResponse res; + meta_service->abort_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + ASSERT_EQ(res.txn_info().status(), TxnStatusPB::TXN_STATUS_ABORTED); + } + recycler.recycle_expired_txn_label(); + ASSERT_EQ(get_txn_info(txn_kv, mock_instance, db_id, txn_id, txn_info_pb), -2); + ASSERT_EQ(check_recycle_txn_keys(txn_kv, mock_instance, db_id, txn_id, label), 0); + } + + { + // 1. begin_txn + // 2. commit_txn + // 3. recycle_expired_txn_label + // 4. check + { + brpc::Controller cntl; + BeginTxnRequest req; + + req.set_cloud_unique_id(cloud_unique_id); + TxnInfoPB txn_info_pb; + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label(label); + txn_info_pb.add_table_ids(table_id); + txn_info_pb.set_timeout_ms(10000); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + txn_id = res.txn_id(); + ASSERT_GT(txn_id, -1); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + InstanceInfoPB instance; + instance.set_instance_id(mock_instance); + InstanceRecycler recycler(txn_kv, instance); + ASSERT_EQ(recycler.init(), 0); + sleep(1); + recycler.abort_timeout_txn(); + TxnInfoPB txn_info_pb; + get_txn_info(txn_kv, mock_instance, db_id, txn_id, txn_info_pb); + ASSERT_EQ(txn_info_pb.status(), TxnStatusPB::TXN_STATUS_PREPARED); + + // commit_txn + { + brpc::Controller cntl; + CommitTxnRequest req; + req.set_cloud_unique_id(cloud_unique_id); + req.set_db_id(db_id); + req.set_txn_id(txn_id); + CommitTxnResponse res; + meta_service->commit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + recycler.recycle_expired_txn_label(); + ASSERT_EQ(get_txn_info(txn_kv, mock_instance, db_id, txn_id, txn_info_pb), -2); + ASSERT_EQ(check_recycle_txn_keys(txn_kv, mock_instance, db_id, txn_id, label), 0); + } +} + +void create_object_file_pb(std::string prefix, std::vector* object_files, + int file_num = 10) { + for (int i = 0; i < file_num; ++i) { + ObjectFilePB object_file; + // create object in S3, pay attention to the relative path + object_file.set_relative_path(prefix + "/obj_" + std::to_string(i)); + object_file.set_etag(""); + object_files->push_back(object_file); + } +} + +TEST(RecyclerTest, recycle_copy_jobs) { + using namespace std::chrono; + auto txn_kv = std::dynamic_pointer_cast(std::make_shared()); + ASSERT_NE(txn_kv.get(), nullptr); + ASSERT_EQ(txn_kv->init(), 0); + + // create internal/external stage + std::string internal_stage_id = "internal"; + std::string external_stage_id = "external"; + std::string nonexist_internal_stage_id = "non_exist_internal"; + std::string nonexist_external_stage_id = "non_exist_external"; + + InstanceInfoPB instance_info; + create_instance(internal_stage_id, external_stage_id, instance_info); + InstanceRecycler recycler(txn_kv, instance_info); + ASSERT_EQ(recycler.init(), 0); + auto internal_accessor = recycler.accessor_map_.find(internal_stage_id)->second; + + // create internal stage copy job with finish status + { + std::vector object_files; + create_object_file_pb("0", &object_files); + ASSERT_EQ(create_object_files(internal_accessor.get(), &object_files), 0); + create_copy_job(txn_kv.get(), internal_stage_id, 0, StagePB::INTERNAL, CopyJobPB::FINISH, + object_files, 0); + } + // create internal stage copy job and files with loading status which is timeout + { + std::vector object_files; + create_object_file_pb("5", &object_files); + ASSERT_EQ(create_object_files(internal_accessor.get(), &object_files), 0); + create_copy_job(txn_kv.get(), internal_stage_id, 5, StagePB::INTERNAL, CopyJobPB::LOADING, + object_files, 0); + } + // create internal stage copy job and files with loading status which is not timeout + { + std::vector object_files; + create_object_file_pb("6", &object_files); + ASSERT_EQ(create_object_files(internal_accessor.get(), &object_files), 0); + create_copy_job(txn_kv.get(), internal_stage_id, 6, StagePB::INTERNAL, CopyJobPB::LOADING, + object_files, 9963904963479L); + } + // create internal stage copy job with deleted stage id + { + std::vector object_files; + create_object_file_pb("8", &object_files); + ASSERT_EQ(create_object_files(internal_accessor.get(), &object_files), 0); + ASSERT_EQ(0, create_copy_job(txn_kv.get(), nonexist_internal_stage_id, 8, StagePB::INTERNAL, + CopyJobPB::FINISH, object_files, 0)); + } + // ----- external stage ---- + // + std::vector> + external_copy_jobs; + uint64_t current_time = + duration_cast(system_clock::now().time_since_epoch()).count(); + int64_t expire_time = current_time - config::copy_job_max_retention_second * 1000 - 1000; + int64_t not_expire_time = current_time - config::copy_job_max_retention_second * 1000 / 2; + // create external stage copy job with start time not expired and no finish time + external_copy_jobs.emplace_back(1, 0, 9963904963479L, 0, CopyJobPB::FINISH); + // create external stage copy job with start time expired and no finish time + external_copy_jobs.emplace_back(2, 0, expire_time, 0, CopyJobPB::FINISH); + // create external stage copy job with start time not expired and finish time not expired + external_copy_jobs.emplace_back(9, 0, expire_time, not_expire_time, CopyJobPB::FINISH); + // create external stage copy job with start time expired and finish time expired + external_copy_jobs.emplace_back(10, 0, expire_time, expire_time + 1, CopyJobPB::FINISH); + // create external stage copy job and files with loading status which is timeout + external_copy_jobs.emplace_back(3, 0, 0, 0, CopyJobPB::LOADING); + // create external stage copy job and files with loading status which is not timeout + external_copy_jobs.emplace_back(4, 9963904963479L, 0, 0, CopyJobPB::LOADING); + for (const auto& [table_id, timeout_time, start_time, finish_time, job_status] : + external_copy_jobs) { + std::vector object_files; + create_object_file_pb(external_stage_id + "_" + std::to_string(table_id), &object_files); + create_copy_job(txn_kv.get(), external_stage_id, table_id, StagePB::EXTERNAL, job_status, + object_files, timeout_time, start_time, finish_time); + } + // create external stage copy job with deleted stage id + { + std::vector object_files; + create_object_file_pb(nonexist_external_stage_id + "_7", &object_files); + ASSERT_EQ(0, create_copy_job(txn_kv.get(), nonexist_external_stage_id, 7, StagePB::EXTERNAL, + CopyJobPB::FINISH, object_files, 0)); + } + { + // + std::vector> stage_table_files; + stage_table_files.emplace_back(internal_stage_id, 0); + stage_table_files.emplace_back(nonexist_internal_stage_id, 8); + stage_table_files.emplace_back(external_stage_id, 1); + stage_table_files.emplace_back(external_stage_id, 2); + stage_table_files.emplace_back(external_stage_id, 9); + stage_table_files.emplace_back(external_stage_id, 10); + stage_table_files.emplace_back(external_stage_id, 3); + stage_table_files.emplace_back(external_stage_id, 4); + stage_table_files.emplace_back(external_stage_id, 9); + stage_table_files.emplace_back(nonexist_external_stage_id, 7); + // check copy files + for (const auto& [stage_id, table_id] : stage_table_files) { + int file_num = 0; + ASSERT_EQ(0, get_copy_file_num(txn_kv.get(), stage_id, table_id, &file_num)); + ASSERT_EQ(10, file_num); + } + } + + ASSERT_EQ(recycler.recycle_copy_jobs(), 0); + + // check object files + std::vector, std::string, int>> + prefix_and_files_list; + prefix_and_files_list.emplace_back(internal_accessor, "0/", 0); + prefix_and_files_list.emplace_back(internal_accessor, "5/", 10); + prefix_and_files_list.emplace_back(internal_accessor, "6/", 10); + prefix_and_files_list.emplace_back(internal_accessor, "8/", 10); + for (const auto& [accessor, relative_path, file_num] : prefix_and_files_list) { + std::vector object_files; + ASSERT_EQ(0, accessor->list(relative_path, &object_files)); + ASSERT_EQ(file_num, object_files.size()); + } + + // check fdb kvs + // + std::vector> stage_table_files; + stage_table_files.emplace_back(internal_stage_id, 0, 0, false); + stage_table_files.emplace_back(nonexist_internal_stage_id, 8, 0, false); + stage_table_files.emplace_back(internal_stage_id, 5, 0, false); + stage_table_files.emplace_back(internal_stage_id, 6, 10, true); + stage_table_files.emplace_back(external_stage_id, 1, 10, true); + stage_table_files.emplace_back(external_stage_id, 2, 0, false); + stage_table_files.emplace_back(external_stage_id, 9, 10, true); + stage_table_files.emplace_back(external_stage_id, 10, 0, false); + stage_table_files.emplace_back(external_stage_id, 3, 0, false); + stage_table_files.emplace_back(external_stage_id, 4, 10, true); + stage_table_files.emplace_back(nonexist_external_stage_id, 7, 0, false); + for (const auto& [stage_id, table_id, files, expected_job_exists] : stage_table_files) { + // check copy files + int file_num = 0; + ASSERT_EQ(0, get_copy_file_num(txn_kv.get(), stage_id, table_id, &file_num)) << table_id; + EXPECT_EQ(files, file_num) << table_id; + // check copy jobs + bool exist = false; + ASSERT_EQ(0, copy_job_exists(txn_kv.get(), stage_id, table_id, &exist)) << table_id; + EXPECT_EQ(expected_job_exists, exist) << table_id; + } +} + +TEST(RecyclerTest, recycle_batch_copy_jobs) { + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("MockAccessor::delete_objects_ret::pred", + [](void* p) { *((bool*)p) = true; }); + sp->enable_processing(); + using namespace std::chrono; + auto txn_kv = std::dynamic_pointer_cast(std::make_shared()); + ASSERT_NE(txn_kv.get(), nullptr); + ASSERT_EQ(txn_kv->init(), 0); + + // create internal/external stage + std::string internal_stage_id = "internal"; + std::string external_stage_id = "external"; + std::string nonexist_internal_stage_id = "non_exist_internal"; + std::string nonexist_external_stage_id = "non_exist_external"; + + InstanceInfoPB instance_info; + create_instance(internal_stage_id, external_stage_id, instance_info); + InstanceRecycler recycler(txn_kv, instance_info); + ASSERT_EQ(recycler.init(), 0); + const auto& internal_accessor = recycler.accessor_map_.find(internal_stage_id)->second; + + // create internal stage copy job with finish status + { + std::vector object_files; + create_object_file_pb("0", &object_files, 1000); + ASSERT_EQ(create_object_files(internal_accessor.get(), &object_files), 0); + create_copy_job(txn_kv.get(), internal_stage_id, 0, StagePB::INTERNAL, CopyJobPB::FINISH, + object_files, 0); + } + { + std::vector object_files; + create_object_file_pb("4", &object_files); + ASSERT_EQ(create_object_files(internal_accessor.get(), &object_files), 0); + create_copy_job(txn_kv.get(), internal_stage_id, 4, StagePB::INTERNAL, CopyJobPB::FINISH, + object_files, 0); + } + // create internal stage copy job with deleted stage id + { + std::vector object_files; + create_object_file_pb("8", &object_files); + ASSERT_EQ(create_object_files(internal_accessor.get(), &object_files), 0); + ASSERT_EQ(0, create_copy_job(txn_kv.get(), nonexist_internal_stage_id, 8, StagePB::INTERNAL, + CopyJobPB::FINISH, object_files, 0)); + } + + ASSERT_EQ(recycler.recycle_copy_jobs(), 0); + + // check object files + std::vector, std::string, int>> + prefix_and_files_list; + prefix_and_files_list.emplace_back(internal_accessor, "0/", 1000); + prefix_and_files_list.emplace_back(internal_accessor, "4/", 10); + prefix_and_files_list.emplace_back(internal_accessor, "8/", 10); + for (const auto& [accessor, relative_path, file_num] : prefix_and_files_list) { + std::vector object_files; + ASSERT_EQ(0, accessor->list(relative_path, &object_files)); + ASSERT_EQ(file_num, object_files.size()); + } + + // check fdb kvs + // + std::vector> stage_table_files; + stage_table_files.emplace_back(internal_stage_id, 0, 1000, true); + stage_table_files.emplace_back(internal_stage_id, 4, 10, true); + stage_table_files.emplace_back(nonexist_internal_stage_id, 8, 0, false); + for (const auto& [stage_id, table_id, files, expected_job_exists] : stage_table_files) { + // check copy files + int file_num = 0; + ASSERT_EQ(0, get_copy_file_num(txn_kv.get(), stage_id, table_id, &file_num)) << table_id; + EXPECT_EQ(files, file_num) << table_id; + // check copy jobs + bool exist = false; + ASSERT_EQ(0, copy_job_exists(txn_kv.get(), stage_id, table_id, &exist)) << table_id; + EXPECT_EQ(expected_job_exists, exist) << table_id; + } + + sp->clear_call_back("MockAccessor::delete_objects_ret::pred"); + ASSERT_EQ(recycler.recycle_copy_jobs(), 0); + + // check object files + prefix_and_files_list.clear(); + prefix_and_files_list.emplace_back(internal_accessor, "0/", 0); + prefix_and_files_list.emplace_back(internal_accessor, "4/", 0); + prefix_and_files_list.emplace_back(internal_accessor, "8/", 10); + for (const auto& [accessor, relative_path, file_num] : prefix_and_files_list) { + std::vector object_files; + ASSERT_EQ(0, accessor->list(relative_path, &object_files)); + ASSERT_EQ(file_num, object_files.size()); + } + + // check fdb kvs + // + stage_table_files.clear(); + stage_table_files.emplace_back(internal_stage_id, 0, 0, false); + stage_table_files.emplace_back(internal_stage_id, 4, 0, false); + stage_table_files.emplace_back(nonexist_internal_stage_id, 8, 0, false); + for (const auto& [stage_id, table_id, files, expected_job_exists] : stage_table_files) { + // check copy files + int file_num = 0; + ASSERT_EQ(0, get_copy_file_num(txn_kv.get(), stage_id, table_id, &file_num)) << table_id; + EXPECT_EQ(files, file_num) << table_id; + // check copy jobs + bool exist = false; + ASSERT_EQ(0, copy_job_exists(txn_kv.get(), stage_id, table_id, &exist)) << table_id; + EXPECT_EQ(expected_job_exists, exist) << table_id; + } +} + +TEST(RecyclerTest, recycle_stage) { + [[maybe_unused]] auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + auto txn_kv = std::dynamic_pointer_cast(std::make_shared()); + ASSERT_NE(txn_kv.get(), nullptr); + ASSERT_EQ(txn_kv->init(), 0); + + std::string stage_prefix = "prefix/stage/bob/bc9fff5e-5f91-4168-8eaa-0afd6667f7ef"; + ObjectStoreInfoPB object_info; + object_info.set_id("obj_id"); + object_info.set_ak("ak"); + object_info.set_sk("sk"); + object_info.set_bucket("bucket"); + object_info.set_endpoint("endpoint"); + object_info.set_region("region"); + object_info.set_prefix(stage_prefix); + object_info.set_provider(ObjectStoreInfoPB::OSS); + InstanceInfoPB instance; + instance.set_instance_id(mock_instance); + instance.add_obj_info()->CopyFrom(object_info); + + InstanceRecycler recycler(txn_kv, instance); + ASSERT_EQ(recycler.init(), 0); + auto accessor = recycler.accessor_map_.begin()->second; + for (int i = 0; i < 10; ++i) { + accessor->put_object(std::to_string(i) + ".csv", "abc"); + } + sp->set_call_back("recycle_stage:get_accessor", [&recycler](void* ret) { + *reinterpret_cast*>(ret) = + recycler.accessor_map_.begin()->second; + }); + sp->enable_processing(); + + std::string key; + std::string val; + RecycleStageKeyInfo key_info {mock_instance, "stage_id"}; + recycle_stage_key(key_info, &key); + StagePB stage; + stage.add_mysql_user_name("user_name"); + stage.add_mysql_user_id("user_id"); + stage.mutable_obj_info()->set_id("1"); + stage.mutable_obj_info()->set_prefix(stage_prefix); + RecycleStagePB recycle_stage; + recycle_stage.set_instance_id(mock_instance); + recycle_stage.mutable_stage()->CopyFrom(stage); + val = recycle_stage.SerializeAsString(); + std::unique_ptr txn; + ASSERT_EQ(TxnErrorCode::TXN_OK, txn_kv->create_txn(&txn)); + txn->put(key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + ASSERT_EQ(TxnErrorCode::TXN_OK, txn_kv->create_txn(&txn)); + ASSERT_EQ(TxnErrorCode::TXN_OK, txn->get(key, &val)); + + // recycle stage + ASSERT_EQ(0, recycler.recycle_stage()); + std::vector files; + ASSERT_EQ(0, accessor->list("", &files)); + ASSERT_EQ(0, files.size()); + ASSERT_EQ(TxnErrorCode::TXN_OK, txn_kv->create_txn(&txn)); + ASSERT_EQ(TxnErrorCode::TXN_KEY_NOT_FOUND, txn->get(key, &val)); +} + +TEST(RecyclerTest, recycle_deleted_instance) { + auto txn_kv = std::dynamic_pointer_cast(std::make_shared()); + ASSERT_NE(txn_kv.get(), nullptr); + ASSERT_EQ(txn_kv->init(), 0); + // create internal/external stage + std::string internal_stage_id = "internal"; + std::string external_stage_id = "external"; + std::string nonexist_internal_stage_id = "non_exist_internal"; + std::string nonexist_external_stage_id = "non_exist_external"; + + InstanceInfoPB instance_info; + create_instance(internal_stage_id, external_stage_id, instance_info); + InstanceRecycler recycler(txn_kv, instance_info); + ASSERT_EQ(recycler.init(), 0); + // create txn key + for (size_t i = 0; i < 100; i++) { + ASSERT_EQ(0, create_txn_label_kv(txn_kv.get(), fmt::format("fake_label{}", i), i)); + } + // create version key + for (size_t i = 101; i < 200; i += 2) { + ASSERT_EQ(0, create_version_kv(txn_kv.get(), i, i + 1)); + } + // create meta key + std::vector schemas; + for (int i = 0; i < 5; ++i) { + auto& schema = schemas.emplace_back(); + schema.set_schema_version(i); + for (int j = 0; j < i; ++j) { + schema.add_index()->set_index_id(j); + } + } + + constexpr int table_id = 10000, index_id = 10001, partition_id = 10002; + auto accessor = recycler.accessor_map_.begin()->second; + int64_t tablet_id_base = 10100; + int64_t txn_id_base = 114115; + for (int i = 0; i < 100; ++i) { + int64_t tablet_id = tablet_id_base + i; + // creare stats key + create_tablet(txn_kv.get(), table_id, index_id, partition_id, tablet_id); + for (int j = 0; j < 10; ++j) { + auto rowset = create_rowset("recycle_tablet", tablet_id, index_id, 5, schemas[j % 5]); + // create recycle key + create_recycle_rowset(txn_kv.get(), accessor.get(), rowset, + j % 10 < 2 ? RecycleRowsetPB::PREPARE : RecycleRowsetPB::COMPACT, + j & 1); + auto tmp_rowset = create_rowset("recycle_tmp_rowsets", tablet_id, index_id, 5, + schemas[j % 5], txn_id_base + j); + // create meta key + create_tmp_rowset(txn_kv.get(), accessor.get(), tmp_rowset, j & 1); + } + for (int j = 0; j < 10; ++j) { + // create meta key + create_committed_rowset(txn_kv.get(), accessor.get(), "recycle_indexes", tablet_id, j); + } + } + + ASSERT_EQ(0, recycler.recycle_deleted_instance()); + + // check if all the objects are deleted + std::vector files; + std::for_each(recycler.accessor_map_.begin(), recycler.accessor_map_.end(), + [&](const auto& entry) { + auto& acc = entry.second; + ASSERT_EQ(0, acc->list("", &files)); + ASSERT_EQ(0, files.size()); + }); + + // check if all the keys are deleted + // check all related kv have been deleted + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::unique_ptr it; + + std::string start_txn_key = txn_key_prefix(instance_id); + std::string end_txn_key = txn_key_prefix(instance_id + '\x00'); + ASSERT_EQ(txn->get(start_txn_key, end_txn_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + + std::string start_version_key = version_key({instance_id, 0, 0, 0}); + std::string end_version_key = version_key({instance_id, INT64_MAX, 0, 0}); + ASSERT_EQ(txn->get(start_version_key, end_version_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + + std::string start_meta_key = meta_key_prefix(instance_id); + std::string end_meta_key = meta_key_prefix(instance_id + '\x00'); + ASSERT_EQ(txn->get(start_meta_key, end_meta_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + + auto begin_recycle_key = recycle_key_prefix(instance_id); + auto end_recycle_key = recycle_key_prefix(instance_id + '\x00'); + ASSERT_EQ(txn->get(begin_recycle_key, end_recycle_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + + std::string start_stats_tablet_key = stats_tablet_key({instance_id, 0, 0, 0, 0}); + std::string end_stats_tablet_key = stats_tablet_key({instance_id, INT64_MAX, 0, 0, 0}); + ASSERT_EQ(txn->get(start_stats_tablet_key, end_stats_tablet_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); + + std::string start_copy_key = copy_key_prefix(instance_id); + std::string end_copy_key = copy_key_prefix(instance_id + '\x00'); + ASSERT_EQ(txn->get(start_copy_key, end_copy_key, &it), TxnErrorCode::TXN_OK); + ASSERT_EQ(it->size(), 0); +} + +TEST(RecyclerTest, multi_recycler) { + config::recycle_concurrency = 2; + config::recycle_interval_seconds = 10; + config::recycle_job_lease_expired_ms = 1000; + auto mem_kv = std::make_shared(); + ASSERT_EQ(mem_kv->init(), 0); + + std::atomic_int count {0}; + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("InstanceRecycler.do_recycle", [&count](void*) { + sleep(1); + ++count; + }); + sp->enable_processing(); + + std::unique_ptr txn; + ASSERT_EQ(TxnErrorCode::TXN_OK, mem_kv->create_txn(&txn)); + for (int i = 0; i < 10; ++i) { + InstanceInfoPB instance; + instance.set_instance_id(std::to_string(i)); + auto obj_info = instance.add_obj_info(); + obj_info->set_id("multi_recycler_test"); + obj_info->set_ak(config::test_s3_ak); + obj_info->set_sk(config::test_s3_sk); + obj_info->set_endpoint(config::test_s3_endpoint); + obj_info->set_region(config::test_s3_region); + obj_info->set_bucket(config::test_s3_bucket); + obj_info->set_prefix("multi_recycler_test"); + InstanceKeyInfo key_info {std::to_string(i)}; + std::string key; + instance_key(key_info, &key); + std::string val = instance.SerializeAsString(); + txn->put(key, val); + } + ASSERT_EQ(TxnErrorCode::TXN_OK, txn->commit()); + + Recycler r1(mem_kv); + r1.ip_port_ = "r1:p1"; + r1.start(nullptr); + Recycler r2(mem_kv); + r2.ip_port_ = "r2:p2"; + r2.start(nullptr); + Recycler r3(mem_kv); + r3.ip_port_ = "r3:p3"; + std::this_thread::sleep_for(std::chrono::milliseconds(500)); + r3.start(nullptr); + + std::this_thread::sleep_for(std::chrono::seconds(5)); + r1.stop(); + r2.stop(); + r3.stop(); + + ASSERT_EQ(TxnErrorCode::TXN_OK, mem_kv->create_txn(&txn)); + for (int i = 0; i < 10; ++i) { + JobRecycleKeyInfo key_info {std::to_string(i)}; + JobRecyclePB job_info; + std::string key; + std::string val; + job_recycle_key(key_info, &key); + ASSERT_EQ(TxnErrorCode::TXN_OK, txn->get(key, &val)) << i; + ASSERT_TRUE(job_info.ParseFromString(val)); + EXPECT_EQ(JobRecyclePB::IDLE, job_info.status()); + EXPECT_GT(job_info.last_finish_time_ms(), 0); + std::cout << "host: " << job_info.ip_port() << " finish recycle job of instance_id: " << i + << std::endl; + } + EXPECT_EQ(count, 10); +} + +TEST(RecyclerTest, safe_exit) { + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + auto checker_ = std::make_unique(txn_kv); + brpc::Server server; + int ret = checker_->start(&server); + ASSERT_TRUE(ret == 0); + checker_->stop(); +} + +TEST(CheckerTest, safe_exit) { + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + auto checker_ = std::make_unique(txn_kv); + int ret = checker_->start(); + ASSERT_TRUE(ret == 0); + checker_->stop(); +} + +TEST(CheckerTest, normal_inverted_check) { + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + + InstanceInfoPB instance; + instance.set_instance_id(instance_id); + auto obj_info = instance.add_obj_info(); + obj_info->set_id("1"); + obj_info->set_ak(config::test_s3_ak); + obj_info->set_sk(config::test_s3_sk); + obj_info->set_endpoint(config::test_s3_endpoint); + obj_info->set_region(config::test_s3_region); + obj_info->set_bucket(config::test_s3_bucket); + obj_info->set_prefix("CheckerTest"); + + auto sp = SyncPoint::get_instance(); + sp->set_call_back("InstanceChecker::do_inverted_check::pred", + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("InstanceChecker::do_inverted_check", [&](void* p) { *((int*)p) = 0; }); + sp->enable_processing(); + std::unique_ptr> defer((int*)0x01, [](int*) { + SyncPoint::get_instance()->clear_all_call_backs(); + SyncPoint::get_instance()->disable_processing(); + }); + + InstanceChecker checker(txn_kv, instance_id); + ASSERT_EQ(checker.init(instance), 0); + // Add some visible rowsets along with some rowsets that should be recycled + // call inverted check after do recycle which would sweep all the rowsets not visible + auto accessor = checker.accessor_map_.begin()->second; + for (int t = 10001; t <= 10100; ++t) { + for (int v = 0; v < 10; ++v) { + create_committed_rowset(txn_kv.get(), accessor.get(), "1", t, v, 1); + } + } + for (int t = 10101; t <= 10200; ++t) { + for (int v = 0; v < 10; ++v) { + create_committed_rowset(txn_kv.get(), accessor.get(), "1", t, v, 5); + } + } + ASSERT_EQ(checker.do_inverted_check(), 0); +} + +// TODO(Xiaocc): We need one mocked accessor which provides one async stream like list function +// to do the following test +TEST(CheckerTest, DISABLED_abnormal_inverted_check) { + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + + InstanceInfoPB instance; + instance.set_instance_id(instance_id); + auto obj_info = instance.add_obj_info(); + obj_info->set_id("1"); + obj_info->set_ak(config::test_s3_ak); + obj_info->set_sk(config::test_s3_sk); + obj_info->set_endpoint(config::test_s3_endpoint); + obj_info->set_region(config::test_s3_region); + obj_info->set_bucket(config::test_s3_bucket); + obj_info->set_prefix("CheckerTest"); + + auto sp = SyncPoint::get_instance(); + sp->set_call_back("InstanceChecker::do_inverted_check::pred", + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("InstanceChecker::do_inverted_check", [&](void* p) { *((int*)p) = 0; }); + sp->enable_processing(); + std::unique_ptr> defer((int*)0x01, [](int*) { + SyncPoint::get_instance()->clear_all_call_backs(); + SyncPoint::get_instance()->disable_processing(); + }); + + InstanceChecker checker(txn_kv, instance_id); + ASSERT_EQ(checker.init(instance), 0); + // Add some visible rowsets along with some rowsets that should be recycled + // call inverted check after do recycle which would sweep all the rowsets not visible + auto accessor = checker.accessor_map_.begin()->second; + for (int t = 10001; t <= 10100; ++t) { + for (int v = 0; v < 10; ++v) { + create_committed_rowset(txn_kv.get(), accessor.get(), "1", t, v, 1); + } + } + std::vector schemas; + for (int i = 0; i < 5; ++i) { + auto& schema = schemas.emplace_back(); + schema.set_schema_version(i); + for (int j = 0; j < i; ++j) { + schema.add_index()->set_index_id(j); + } + } + + // Create some rowsets not visible in S3 + constexpr int table_id = 10101, index_id = 10102, partition_id = 10103, tablet_id = 10104; + create_tablet(txn_kv.get(), table_id, index_id, partition_id, tablet_id); + for (int i = 0; i < 500; ++i) { + auto rowset = create_rowset("recycle_tablet", tablet_id, index_id, 5, schemas[i % 5]); + create_recycle_rowset(txn_kv.get(), accessor.get(), rowset, + i % 10 < 2 ? RecycleRowsetPB::PREPARE : RecycleRowsetPB::COMPACT, + i & 1); + } + ASSERT_NE(checker.do_inverted_check(), 0); +} + +TEST(CheckerTest, normal) { + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + + InstanceInfoPB instance; + instance.set_instance_id(instance_id); + auto obj_info = instance.add_obj_info(); + obj_info->set_id("1"); + obj_info->set_ak(config::test_s3_ak); + obj_info->set_sk(config::test_s3_sk); + obj_info->set_endpoint(config::test_s3_endpoint); + obj_info->set_region(config::test_s3_region); + obj_info->set_bucket(config::test_s3_bucket); + obj_info->set_prefix("CheckerTest"); + + InstanceChecker checker(txn_kv, instance_id); + ASSERT_EQ(checker.init(instance), 0); + + auto accessor = checker.accessor_map_.begin()->second; + for (int t = 10001; t <= 10100; ++t) { + for (int v = 0; v < 10; ++v) { + create_committed_rowset(txn_kv.get(), accessor.get(), "1", t, v, 1); + } + } + for (int t = 10101; t <= 10200; ++t) { + for (int v = 0; v < 10; ++v) { + create_committed_rowset(txn_kv.get(), accessor.get(), "1", t, v, 5); + } + } + ASSERT_EQ(checker.do_check(), 0); +} + +TEST(CheckerTest, abnormal) { + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + + InstanceInfoPB instance; + instance.set_instance_id(instance_id); + auto obj_info = instance.add_obj_info(); + obj_info->set_id("1"); + obj_info->set_ak(config::test_s3_ak); + obj_info->set_sk(config::test_s3_sk); + obj_info->set_endpoint(config::test_s3_endpoint); + obj_info->set_region(config::test_s3_region); + obj_info->set_bucket(config::test_s3_bucket); + obj_info->set_prefix("CheckerTest"); + + InstanceChecker checker(txn_kv, instance_id); + ASSERT_EQ(checker.init(instance), 0); + + auto accessor = checker.accessor_map_.begin()->second; + for (int t = 10001; t <= 10100; ++t) { + for (int v = 0; v < 10; ++v) { + create_committed_rowset(txn_kv.get(), accessor.get(), "1", t, v, 1, 0); + } + } + for (int t = 10101; t <= 10200; ++t) { + for (int v = 0; v < 10; ++v) { + create_committed_rowset(txn_kv.get(), accessor.get(), "1", t, v, 5, 0); + } + } + + // Delete some objects + std::mt19937 gen(std::chrono::system_clock::now().time_since_epoch().count()); + std::vector files; + std::vector deleted_paths; + ASSERT_EQ(0, accessor->list(tablet_path_prefix(10001 + gen() % 100), &files)); + deleted_paths.push_back(files[gen() % files.size()].path); + ASSERT_EQ(0, accessor->delete_object(deleted_paths.back())); + files.clear(); + ASSERT_EQ(0, accessor->list(tablet_path_prefix(10101 + gen() % 100), &files)); + deleted_paths.push_back(files[gen() % files.size()].path); + ASSERT_EQ(0, accessor->delete_object(deleted_paths.back())); + + std::vector lost_paths; + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("InstanceChecker.do_check1", + [&lost_paths](void* arg) { lost_paths.push_back(*(std::string*)arg); }); + sp->enable_processing(); + + ASSERT_NE(checker.do_check(), 0); + EXPECT_EQ(deleted_paths, lost_paths); +} + +TEST(CheckerTest, multi_checker) { + config::recycle_concurrency = 2; + config::scan_instances_interval_seconds = 10; + config::recycle_job_lease_expired_ms = 1000; + auto mem_kv = std::make_shared(); + ASSERT_EQ(mem_kv->init(), 0); + + std::atomic_int count {0}; + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("InstanceChecker.do_check", [&count](void*) { + sleep(1); + ++count; + }); + sp->enable_processing(); + + std::unique_ptr txn; + ASSERT_EQ(TxnErrorCode::TXN_OK, mem_kv->create_txn(&txn)); + for (int i = 0; i < 10; ++i) { + InstanceInfoPB instance; + instance.set_instance_id(std::to_string(i)); + auto obj_info = instance.add_obj_info(); + obj_info->set_id("1"); + obj_info->set_ak(config::test_s3_ak); + obj_info->set_sk(config::test_s3_sk); + obj_info->set_endpoint(config::test_s3_endpoint); + obj_info->set_region(config::test_s3_region); + obj_info->set_bucket(config::test_s3_bucket); + obj_info->set_prefix("CheckerTest"); + InstanceKeyInfo key_info {std::to_string(i)}; + std::string key; + instance_key(key_info, &key); + std::string val = instance.SerializeAsString(); + txn->put(key, val); + } + ASSERT_EQ(TxnErrorCode::TXN_OK, txn->commit()); + + Checker c1(mem_kv); + c1.ip_port_ = "r1:p1"; + c1.start(); + Checker c2(mem_kv); + c2.ip_port_ = "r2:p2"; + c2.start(); + Checker c3(mem_kv); + c3.ip_port_ = "r3:p3"; + std::this_thread::sleep_for(std::chrono::milliseconds(500)); + c3.start(); + + std::this_thread::sleep_for(std::chrono::seconds(5)); + c1.stop(); + c2.stop(); + c3.stop(); + + ASSERT_EQ(TxnErrorCode::TXN_OK, mem_kv->create_txn(&txn)); + for (int i = 0; i < 10; ++i) { + JobRecycleKeyInfo key_info {std::to_string(i)}; + JobRecyclePB job_info; + std::string key; + std::string val; + job_check_key(key_info, &key); + ASSERT_EQ(TxnErrorCode::TXN_OK, txn->get(key, &val)) << i; + ASSERT_TRUE(job_info.ParseFromString(val)); + EXPECT_EQ(JobRecyclePB::IDLE, job_info.status()); + EXPECT_GT(job_info.last_finish_time_ms(), 0); + std::cout << "host: " << job_info.ip_port() << " finish check job of instance_id: " << i + << std::endl; + } + EXPECT_EQ(count, 10); +} + +TEST(CheckerTest, do_inspect) { + using namespace std::chrono; + { + auto mem_kv = std::make_shared(); + ASSERT_EQ(mem_kv->init(), 0); + + InstanceInfoPB instance; + instance.set_instance_id(instance_id); + instance.set_ctime(11111); + auto obj_info = instance.add_obj_info(); + obj_info->set_id("1"); + obj_info->set_ak(config::test_s3_ak); + obj_info->set_sk(config::test_s3_sk); + obj_info->set_endpoint(config::test_s3_endpoint); + obj_info->set_region(config::test_s3_region); + obj_info->set_bucket(config::test_s3_bucket); + obj_info->set_prefix("Test"); + + Checker checker(mem_kv); + checker.do_inspect(instance); + + { + // empty job info + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("Checker:do_inspect", [](void* p) { + ASSERT_TRUE(*reinterpret_cast(p) == 11111); + std::cout << "last_ctime: " << *reinterpret_cast(p) << std::endl; + }); + sp->enable_processing(); + } + + { + // add job_info but no last ctime + std::unique_ptr txn; + ASSERT_EQ(TxnErrorCode::TXN_OK, mem_kv->create_txn(&txn)); + JobRecyclePB job_info; + job_info.set_instance_id(instance_id); + std::string key = job_check_key({instance_id}); + std::string val = job_info.SerializeAsString(); + txn->put(key, val); + ASSERT_EQ(TxnErrorCode::TXN_OK, txn->commit()); + checker.do_inspect(instance); + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("Checker:do_inspect", [](void* p) { + ASSERT_TRUE(*reinterpret_cast(p) == 11111); + }); + sp->enable_processing(); + } + + { + // add job_info with last ctime + std::unique_ptr txn; + ASSERT_EQ(TxnErrorCode::TXN_OK, mem_kv->create_txn(&txn)); + JobRecyclePB job_info; + job_info.set_instance_id(instance_id); + job_info.set_last_ctime_ms(12345); + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("Checker:do_inspect", [](void* p) { + ASSERT_TRUE(*reinterpret_cast(p) == 12345); + }); + sp->enable_processing(); + std::string key = job_check_key({instance_id}); + std::string val = job_info.SerializeAsString(); + txn->put(key, val); + ASSERT_EQ(TxnErrorCode::TXN_OK, txn->commit()); + checker.do_inspect(instance); + } + { + // alarm + int64_t expiration_ms = 7 > config::reserved_buffer_days + ? (7 - config::reserved_buffer_days) * 3600000 + : 7 * 3600000; + auto now = duration_cast(system_clock::now().time_since_epoch()).count(); + std::unique_ptr txn; + ASSERT_EQ(TxnErrorCode::TXN_OK, mem_kv->create_txn(&txn)); + JobRecyclePB job_info; + job_info.set_instance_id(instance_id); + job_info.set_last_ctime_ms(now - expiration_ms - 10); + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + + bool alarm = false; + sp->set_call_back("Checker:do_inspect", [&alarm](void*) { alarm = true; }); + sp->enable_processing(); + std::string key = job_check_key({instance_id}); + std::string val = job_info.SerializeAsString(); + txn->put(key, val); + ASSERT_EQ(TxnErrorCode::TXN_OK, txn->commit()); + checker.do_inspect(instance); + ASSERT_TRUE(alarm); + } + } +} + +TEST(RecyclerTest, delete_rowset_data) { + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + + InstanceInfoPB instance; + instance.set_instance_id(instance_id); + auto obj_info = instance.add_obj_info(); + obj_info->set_id("recycle_tmp_rowsets"); + obj_info->set_ak(config::test_s3_ak); + obj_info->set_sk(config::test_s3_sk); + obj_info->set_endpoint(config::test_s3_endpoint); + obj_info->set_region(config::test_s3_region); + obj_info->set_bucket(config::test_s3_bucket); + obj_info->set_prefix("recycle_tmp_rowsets"); + + std::vector schemas; + for (int i = 0; i < 5; ++i) { + auto& schema = schemas.emplace_back(); + schema.set_schema_version(i); + for (int j = 0; j < i; ++j) { + schema.add_index()->set_index_id(j); + } + } + + { + InstanceRecycler recycler(txn_kv, instance); + ASSERT_EQ(recycler.init(), 0); + auto accessor = recycler.accessor_map_.begin()->second; + int64_t txn_id_base = 114115; + int64_t tablet_id_base = 10015; + int64_t index_id_base = 1000; + // Delete each rowset directly using one RowsetPB + for (int i = 0; i < 100; ++i) { + int64_t txn_id = txn_id_base + i; + for (int j = 0; j < 20; ++j) { + auto rowset = create_rowset("recycle_tmp_rowsets", tablet_id_base + j, + index_id_base + j % 4, 5, schemas[i % 5], txn_id); + create_tmp_rowset(txn_kv.get(), accessor.get(), rowset, i & 1); + ASSERT_EQ(0, recycler.delete_rowset_data(rowset)); + } + } + std::vector files; + accessor->list("", &files); + ASSERT_EQ(0, files.size()); + } + { + InstanceInfoPB tmp_instance; + std::string resource_id = "recycle_tmp_rowsets"; + tmp_instance.set_instance_id(instance_id); + auto tmp_obj_info = tmp_instance.add_obj_info(); + tmp_obj_info->set_id(resource_id); + tmp_obj_info->set_ak(config::test_s3_ak); + tmp_obj_info->set_sk(config::test_s3_sk); + tmp_obj_info->set_endpoint(config::test_s3_endpoint); + tmp_obj_info->set_region(config::test_s3_region); + tmp_obj_info->set_bucket(config::test_s3_bucket); + tmp_obj_info->set_prefix(resource_id); + + InstanceRecycler recycler(txn_kv, tmp_instance); + ASSERT_EQ(recycler.init(), 0); + auto accessor = recycler.accessor_map_.begin()->second; + // Delete multiple rowset files using one series of RowsetPB + constexpr int index_id = 10001, tablet_id = 10002; + std::vector rowset_pbs; + for (int i = 0; i < 10; ++i) { + auto rowset = create_rowset(resource_id, tablet_id, index_id, 5, schemas[i % 5]); + create_recycle_rowset( + txn_kv.get(), accessor.get(), rowset, + static_cast(i % (RecycleRowsetPB::Type_MAX + 1)), true); + + rowset_pbs.emplace_back(std::move(rowset)); + } + ASSERT_EQ(0, recycler.delete_rowset_data(rowset_pbs)); + std::vector files; + accessor->list("", &files); + ASSERT_EQ(0, files.size()); + } + { + InstanceRecycler recycler(txn_kv, instance); + ASSERT_EQ(recycler.init(), 0); + auto accessor = recycler.accessor_map_.begin()->second; + // Delete multiple rowset files using one series of RowsetPB + constexpr int index_id = 20001, tablet_id = 20002; + // Delete each rowset file directly using it's id to construct one path + for (int i = 0; i < 1000; ++i) { + auto rowset = + create_rowset("recycle_tmp_rowsets", tablet_id, index_id, 5, schemas[i % 5]); + create_recycle_rowset(txn_kv.get(), accessor.get(), rowset, RecycleRowsetPB::COMPACT, + true); + ASSERT_EQ(0, recycler.delete_rowset_data(rowset.resource_id(), rowset.tablet_id(), + rowset.rowset_id_v2())); + } + std::vector files; + accessor->list("", &files); + ASSERT_EQ(0, files.size()); + } +} + +} // namespace doris::cloud diff --git a/cloud/test/resource_test.cpp b/cloud/test/resource_test.cpp new file mode 100644 index 00000000000000..242de8b3a51b57 --- /dev/null +++ b/cloud/test/resource_test.cpp @@ -0,0 +1,463 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//#define private public +#include "meta-service/meta_service.h" +//#undef private + +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +#include "common/config.h" +#include "common/logging.h" +#include "common/sync_point.h" +#include "common/util.h" +#include "meta-service/keys.h" +#include "meta-service/mem_txn_kv.h" +#include "meta-service/txn_kv_error.h" +#include "rate-limiter/rate_limiter.h" +#include "resource-manager/resource_manager.h" + +int main(int argc, char** argv) { + const std::string conf_file = "doris_cloud.conf"; + if (!doris::cloud::config::init(conf_file.c_str(), true)) { + std::cerr << "failed to init config file, conf=" << conf_file << std::endl; + return -1; + } + + if (!doris::cloud::init_glog("resource_test")) { + std::cerr << "failed to init glog" << std::endl; + return -1; + } + + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + +namespace doris::cloud { + +static std::shared_ptr create_txn_kv() { + // MemKv + int ret = 0; + auto txn_kv = std::dynamic_pointer_cast(std::make_shared()); + if (txn_kv != nullptr) { + ret = txn_kv->init(); + [&] { ASSERT_EQ(ret, 0); }(); + } + [&] { ASSERT_NE(txn_kv.get(), nullptr); }(); + + std::unique_ptr txn; + EXPECT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->remove("\x00", "\xfe"); // This is dangerous if the fdb is not correctly set + EXPECT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + return txn_kv; +} + +std::unique_ptr get_meta_service(std::shared_ptr txn_kv = {}) { + if (!txn_kv) { + txn_kv = create_txn_kv(); + } + + auto rs = std::make_shared(txn_kv); + EXPECT_EQ(rs->init(), 0); + auto rl = std::make_shared(); + auto meta_service = std::make_unique(txn_kv, rs, rl); + return std::make_unique(std::move(meta_service)); +} + +static void create_args_to_add(std::vector* to_add, std::vector* to_del, + bool is_host = false) { + to_add->clear(); + to_del->clear(); + auto ni_1 = NodeInfo {.role = Role::COMPUTE_NODE, + .instance_id = "test-resource-instance", + .cluster_name = "cluster_name_1", + .cluster_id = "cluster_id_1", + .node_info = NodeInfoPB {}}; + is_host ? ni_1.node_info.set_host("host1") : ni_1.node_info.set_ip("127.0.0.1"); + ni_1.node_info.set_cloud_unique_id("test_cloud_unique_id_1"); + ni_1.node_info.set_heartbeat_port(9999); + to_add->push_back(ni_1); + + auto ni_2 = NodeInfo {.role = Role::COMPUTE_NODE, + .instance_id = "test-resource-instance", + .cluster_name = "cluster_name_1", + .cluster_id = "cluster_id_1", + .node_info = NodeInfoPB {}}; + is_host ? ni_2.node_info.set_host("host2") : ni_2.node_info.set_ip("127.0.0.2"); + ni_2.node_info.set_cloud_unique_id("test_cloud_unique_id_1"); + ni_2.node_info.set_heartbeat_port(9999); + to_add->push_back(ni_2); + + auto ni_3 = NodeInfo {.role = Role::COMPUTE_NODE, + .instance_id = "test-resource-instance", + .cluster_name = "cluster_name_2", + .cluster_id = "cluster_id_2", + .node_info = NodeInfoPB {}}; + is_host ? ni_3.node_info.set_host("host3") : ni_3.node_info.set_ip("127.0.0.3"); + ni_3.node_info.set_cloud_unique_id("test_cloud_unique_id_2"); + ni_3.node_info.set_heartbeat_port(9999); + to_add->push_back(ni_3); +} + +static void create_args_to_del(std::vector* to_add, std::vector* to_del, + bool is_host = false) { + to_add->clear(); + to_del->clear(); + auto ni_1 = NodeInfo {.role = Role::COMPUTE_NODE, + .instance_id = "test-resource-instance", + .cluster_name = "cluster_name_1", + .cluster_id = "cluster_id_1", + .node_info = NodeInfoPB {}}; + is_host ? ni_1.node_info.set_host("host2") : ni_1.node_info.set_ip("127.0.0.2"); + ni_1.node_info.set_cloud_unique_id("test_cloud_unique_id_1"); + ni_1.node_info.set_heartbeat_port(9999); + to_del->push_back(ni_1); +} + +static void get_instance_info(MetaServiceProxy* ms, InstanceInfoPB* instance, + std::string_view instance_id = "test-resource-instance") { + InstanceKeyInfo key_info {instance_id}; + std::string key; + std::string val; + instance_key(key_info, &key); + std::unique_ptr txn; + EXPECT_EQ(ms->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + EXPECT_EQ(txn->get(key, &val), TxnErrorCode::TXN_OK); + instance->ParseFromString(val); +} + +static void create_instance(MetaServiceProxy* ms, const std::string& instance_id) { + brpc::Controller ctrl; + CreateInstanceRequest req; + req.set_instance_id(instance_id); + req.set_user_id("test_user"); + req.set_name(instance_id + "name"); + ObjectStoreInfoPB obj; + obj.set_ak("123"); + obj.set_sk("321"); + obj.set_bucket("456"); + obj.set_prefix("654"); + obj.set_endpoint("789"); + obj.set_region("987"); + obj.set_external_endpoint("888"); + obj.set_provider(ObjectStoreInfoPB::BOS); + req.mutable_obj_info()->CopyFrom(obj); + CreateInstanceResponse resp; + ms->create_instance(&ctrl, &req, &resp, brpc::DoNothing()); + ASSERT_FALSE(ctrl.Failed()); + ASSERT_EQ(resp.status().code(), MetaServiceCode::OK); +} + +static void create_cluster(MetaServiceProxy* ms, const std::string& instance_id, + const std::string& cluster_id, const std::string& cluster_name, + ClusterPB_Type type) { + brpc::Controller cntl; + AlterClusterRequest req; + req.set_instance_id(instance_id); + req.mutable_cluster()->set_cluster_id(cluster_id); + req.mutable_cluster()->set_cluster_name(cluster_name); + req.mutable_cluster()->set_type(type); + if (type == ClusterPB::SQL) { + auto* node = req.mutable_cluster()->add_nodes(); + node->set_node_type(NodeInfoPB::FE_MASTER); + node->set_ip("127.0.0.1"); + node->set_edit_log_port(10000); + node->set_name("sql_node"); + } else { + auto* node = req.mutable_cluster()->add_nodes(); + node->set_ip("127.0.0.1"); + node->set_heartbeat_port(10000); + node->set_name("sql_node"); + } + req.set_op(AlterClusterRequest::ADD_CLUSTER); + AlterClusterResponse res; + ms->alter_cluster(&cntl, &req, &res, brpc::DoNothing()); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); +} + +static void drop_cluster(MetaServiceProxy* ms, const std::string& instance_id, + const std::string& cluster_id) { + brpc::Controller cntl; + AlterClusterRequest req; + req.set_instance_id(instance_id); + req.mutable_cluster()->set_cluster_id(cluster_id); + req.set_op(AlterClusterRequest::DROP_CLUSTER); + AlterClusterResponse res; + ms->alter_cluster(&cntl, &req, &res, brpc::DoNothing()); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); +} + +// test cluster's node addr use ip +TEST(ResourceTest, ModifyNodesIpTest) { + auto meta_service = get_meta_service(); + std::vector to_add = {}; + std::vector to_del = {}; + auto ins = InstanceInfoPB {}; + create_args_to_add(&to_add, &to_del); + auto sp = SyncPoint::get_instance(); + sp->set_call_back("modify_nodes:get_instance", + [](void* p) { *reinterpret_cast(p) = TxnErrorCode::TXN_OK; }); + sp->set_call_back("modify_nodes:get_instance_ret", [&](void* p) { + ins.set_instance_id("test-resource-instance"); + ins.set_status(InstanceInfoPB::NORMAL); + auto c = ins.mutable_clusters()->Add(); + c->set_cluster_name("cluster_name_1"); + c->set_cluster_id("cluster_id_1"); + auto c1 = ins.mutable_clusters()->Add(); + c1->set_cluster_name("cluster_name_2"); + c1->set_cluster_id("cluster_id_2"); + *reinterpret_cast(p) = ins; + }); + sp->enable_processing(); + + // test cluster add nodes + auto r = meta_service->resource_mgr()->modify_nodes("test-resource-instance", to_add, to_del); + ASSERT_EQ(r, ""); + InstanceInfoPB instance; + get_instance_info(meta_service.get(), &instance); + std::cout << "after to add = " << proto_to_json(instance) << std::endl; + ASSERT_EQ(instance.clusters().size(), 2); + // after add assert cluster_name_1 has 2 nodes + ASSERT_EQ(instance.clusters(0).nodes().size(), 2); + // after add assert cluster_name_2 has 1 nodes + ASSERT_EQ(instance.clusters(1).nodes().size(), 1); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); + + sp->set_call_back("modify_nodes:get_instance", + [](void* p) { *reinterpret_cast(p) = TxnErrorCode::TXN_OK; }); + sp->set_call_back("modify_nodes:get_instance_ret", + [&](void* p) { *reinterpret_cast(p) = instance; }); + sp->enable_processing(); + create_args_to_del(&to_add, &to_del); + // test cluster del node + r = meta_service->resource_mgr()->modify_nodes("test-resource-instance", to_add, to_del); + InstanceInfoPB instance1; + get_instance_info(meta_service.get(), &instance1); + ASSERT_EQ(r, ""); + std::cout << "after to del = " << proto_to_json(instance1) << std::endl; + ASSERT_EQ(instance1.clusters().size(), 2); + // after del assert cluster_name_1 has 1 nodes + ASSERT_EQ(instance1.clusters(0).nodes().size(), 1); + // after del assert cluster_name_2 has 1 nodes + ASSERT_EQ(instance1.clusters(1).nodes().size(), 1); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); +} + +// test cluster's node addr use host +TEST(ResourceTest, ModifyNodesHostTest) { + auto meta_service = get_meta_service(); + std::vector to_add = {}; + std::vector to_del = {}; + auto ins = InstanceInfoPB {}; + create_args_to_add(&to_add, &to_del, true); + auto sp = SyncPoint::get_instance(); + sp->set_call_back("modify_nodes:get_instance", + [](void* p) { *reinterpret_cast(p) = TxnErrorCode::TXN_OK; }); + sp->set_call_back("modify_nodes:get_instance_ret", [&](void* p) { + ins.set_instance_id("test-resource-instance"); + ins.set_status(InstanceInfoPB::NORMAL); + auto c = ins.mutable_clusters()->Add(); + c->set_cluster_name("cluster_name_1"); + c->set_cluster_id("cluster_id_1"); + auto c1 = ins.mutable_clusters()->Add(); + c1->set_cluster_name("cluster_name_2"); + c1->set_cluster_id("cluster_id_2"); + *reinterpret_cast(p) = ins; + }); + sp->enable_processing(); + + // test cluster add nodes + auto r = meta_service->resource_mgr()->modify_nodes("test-resource-instance", to_add, to_del); + ASSERT_EQ(r, ""); + InstanceInfoPB instance; + get_instance_info(meta_service.get(), &instance); + std::cout << "after to add = " << proto_to_json(instance) << std::endl; + ASSERT_EQ(instance.clusters().size(), 2); + // after add assert cluster_name_1 has 2 nodes + ASSERT_EQ(instance.clusters(0).nodes().size(), 2); + // after add assert cluster_name_2 has 1 nodes + ASSERT_EQ(instance.clusters(1).nodes().size(), 1); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); + + sp->set_call_back("modify_nodes:get_instance", + [](void* p) { *reinterpret_cast(p) = TxnErrorCode::TXN_OK; }); + sp->set_call_back("modify_nodes:get_instance_ret", + [&](void* p) { *reinterpret_cast(p) = instance; }); + sp->enable_processing(); + create_args_to_del(&to_add, &to_del, true); + r = meta_service->resource_mgr()->modify_nodes("test-resource-instance", to_add, to_del); + InstanceInfoPB instance1; + get_instance_info(meta_service.get(), &instance1); + ASSERT_EQ(r, ""); + std::cout << "after to del = " << proto_to_json(instance1) << std::endl; + ASSERT_EQ(instance1.clusters().size(), 2); + // after del assert cluster_name_1 has 1 nodes + ASSERT_EQ(instance1.clusters(0).nodes().size(), 1); + // after del assert cluster_name_2 has 1 nodes + ASSERT_EQ(instance1.clusters(1).nodes().size(), 1); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); +} + +// test restart meta service +TEST(ResourceTest, RestartResourceManager) { + auto sp = cloud::SyncPoint::get_instance(); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_id", + [](void* p) { *reinterpret_cast(p) = 1; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->enable_processing(); + + auto txn_kv = create_txn_kv(); + + { + auto meta_service = get_meta_service(txn_kv); + create_instance(meta_service.get(), "test_instance_id"); + create_instance(meta_service.get(), "test_instance_id_2"); + create_cluster(meta_service.get(), "test_instance_id", "cluster_id", "cluster_name", + ClusterPB::SQL); + } + + { + auto meta_service = get_meta_service(txn_kv); + { + InstanceInfoPB info; + auto [code, msg] = + meta_service->resource_mgr()->get_instance(nullptr, "test_instance_id", &info); + ASSERT_EQ(code, TxnErrorCode::TXN_OK) << msg; + ASSERT_EQ(info.name(), "test_instance_idname"); + } + { + InstanceInfoPB info; + auto [code, msg] = meta_service->resource_mgr()->get_instance( + nullptr, "test_instance_id_2", &info); + ASSERT_EQ(code, TxnErrorCode::TXN_OK) << msg; + ASSERT_EQ(info.name(), "test_instance_id_2name"); + } + } + sp->disable_processing(); + sp->clear_all_call_backs(); +} + +// test add/drop cluster +TEST(ResourceTest, AddDropCluster) { + auto sp = cloud::SyncPoint::get_instance(); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_id", + [](void* p) { *reinterpret_cast(p) = 1; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->enable_processing(); + + auto meta_service = get_meta_service(); + create_instance(meta_service.get(), "test_instance_id"); + create_cluster(meta_service.get(), "test_instance_id", "sql_id", "sql_cluster", ClusterPB::SQL); + create_cluster(meta_service.get(), "test_instance_id", "compute_id", "compute_cluster", + ClusterPB::COMPUTE); + + InstanceInfoPB info; + get_instance_info(meta_service.get(), &info, "test_instance_id"); + ASSERT_EQ(info.clusters_size(), 2); + + drop_cluster(meta_service.get(), "test_instance_id", "sql_id"); + + get_instance_info(meta_service.get(), &info, "test_instance_id"); + ASSERT_EQ(info.clusters_size(), 1); + ASSERT_EQ(info.clusters(0).cluster_name(), "compute_cluster"); + ASSERT_EQ(info.clusters(0).cluster_id(), "compute_id"); + + sp->disable_processing(); + sp->clear_all_call_backs(); +} + +TEST(ResourceTest, InitScanRetry) { + auto sp = cloud::SyncPoint::get_instance(); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->set_call_back("encrypt_ak_sk:get_encryption_key_id", + [](void* p) { *reinterpret_cast(p) = 1; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key_ret", + [](void* p) { *reinterpret_cast(p) = 0; }); + sp->set_call_back("decrypt_ak_sk:get_encryption_key", + [](void* p) { *reinterpret_cast(p) = "test"; }); + sp->enable_processing(); + + constexpr size_t NUM_BATCH_SIZE = 100; + sp->set_call_back("ResourceManager:init:limit", + [&](void* raw) { *reinterpret_cast(raw) = NUM_BATCH_SIZE; }); + + auto txn_kv = create_txn_kv(); + { + auto meta_service = get_meta_service(txn_kv); + for (size_t i = 0; i < NUM_BATCH_SIZE * 2; i++) { + std::string instance_id = "test_instance_id_" + std::to_string(i); + create_instance(meta_service.get(), instance_id); + } + } + + { + size_t count = 0; + sp->set_call_back("ResourceManager:init:get_err", [&](void* raw) { + if (++count == 2) { + *reinterpret_cast(raw) = TxnErrorCode::TXN_TOO_OLD; + } + }); + auto meta_service = get_meta_service(txn_kv); + ASSERT_GT(count, 1) << count; + for (size_t i = 0; i < NUM_BATCH_SIZE * 2; i++) { + InstanceInfoPB info; + std::string instance_id = "test_instance_id_" + std::to_string(i); + get_instance_info(meta_service.get(), &info, instance_id); + } + } + + sp->disable_processing(); + sp->clear_all_call_backs(); +} + +} // namespace doris::cloud diff --git a/cloud/test/s3_accessor_test.cpp b/cloud/test/s3_accessor_test.cpp new file mode 100644 index 00000000000000..c8b95e25e234d5 --- /dev/null +++ b/cloud/test/s3_accessor_test.cpp @@ -0,0 +1,829 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "recycler/s3_accessor.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "common/configbase.h" +#include "common/logging.h" +#include "common/sync_point.h" +#include "mock_accessor.h" + +using namespace doris; + +std::unique_ptr _mock_fs; + +class S3ClientInterface { +public: + S3ClientInterface() = default; + virtual ~S3ClientInterface() = default; + virtual Aws::S3::Model::ListObjectsV2Outcome ListObjectsV2( + const Aws::S3::Model::ListObjectsV2Request& req) = 0; + virtual Aws::S3::Model::DeleteObjectsOutcome DeleteObjects( + const Aws::S3::Model::DeleteObjectsRequest& req) = 0; + virtual Aws::S3::Model::PutObjectOutcome PutObject( + const Aws::S3::Model::PutObjectRequest& req) = 0; + virtual Aws::S3::Model::HeadObjectOutcome HeadObject( + const Aws::S3::Model::HeadObjectRequest& req) = 0; + virtual Aws::S3::Model::GetBucketLifecycleConfigurationOutcome GetBucketLifecycleConfiguration( + const Aws::S3::Model::GetBucketLifecycleConfigurationRequest& req) = 0; + virtual Aws::S3::Model::GetBucketVersioningOutcome GetBucketVersioning( + const Aws::S3::Model::GetBucketVersioningRequest& req) = 0; +}; + +static bool list_object_v2_with_expire_time = false; +static int64_t expire_time = 0; +static bool set_bucket_lifecycle = false; +static bool set_bucket_versioning_status_error = false; + +class S3Client : public S3ClientInterface { +public: + S3Client() = default; + ~S3Client() override = default; + Aws::S3::Model::ListObjectsV2Outcome ListObjectsV2( + const Aws::S3::Model::ListObjectsV2Request& req) override { + auto prefix = req.GetPrefix(); + auto continuation_token = + req.ContinuationTokenHasBeenSet() ? req.GetContinuationToken() : ""; + bool truncated = true; + std::vector files; + size_t num = 0; + do { + _mock_fs->list(prefix, &files); + if (num == files.size()) { + truncated = false; + break; + } + num = files.size(); + auto path1 = files.back().path; + prefix = path1.back() += 1; + } while (files.size() <= 1000); + Aws::S3::Model::ListObjectsV2Result result; + result.SetIsTruncated(truncated); + std::vector objects; + std::for_each(files.begin(), files.end(), [&](const cloud::ObjectMeta& file) { + Aws::S3::Model::Object obj; + obj.SetKey(file.path); + Aws::Utils::DateTime date; + if (list_object_v2_with_expire_time) { + date = Aws::Utils::DateTime(expire_time); + } + obj.SetLastModified(date); + objects.emplace_back(std::move(obj)); + }); + result.SetContents(std::move(objects)); + return Aws::S3::Model::ListObjectsV2Outcome(std::move(result)); + } + + Aws::S3::Model::DeleteObjectsOutcome DeleteObjects( + const Aws::S3::Model::DeleteObjectsRequest& req) override { + Aws::S3::Model::DeleteObjectsResult result; + const auto& deletes = req.GetDelete(); + for (const auto& obj : deletes.GetObjects()) { + _mock_fs->delete_object(obj.GetKey()); + } + return Aws::S3::Model::DeleteObjectsOutcome(std::move(result)); + } + + Aws::S3::Model::PutObjectOutcome PutObject( + const Aws::S3::Model::PutObjectRequest& req) override { + Aws::S3::Model::PutObjectResult result; + const auto& key = req.GetKey(); + _mock_fs->put_object(key, ""); + return Aws::S3::Model::PutObjectOutcome(std::move(result)); + } + + Aws::S3::Model::HeadObjectOutcome HeadObject( + const Aws::S3::Model::HeadObjectRequest& req) override { + Aws::S3::Model::HeadObjectResult result; + const auto& key = req.GetKey(); + auto v = _mock_fs->exist(key); + if (v == 1) { + auto err = Aws::Client::AWSError( + Aws::S3::S3Errors::RESOURCE_NOT_FOUND, false); + err.SetResponseCode(Aws::Http::HttpResponseCode::NOT_FOUND); + + return Aws::S3::Model::HeadObjectOutcome(std::move(err)); + } + return Aws::S3::Model::HeadObjectOutcome(std::move(result)); + } + + Aws::S3::Model::GetBucketLifecycleConfigurationOutcome GetBucketLifecycleConfiguration( + const Aws::S3::Model::GetBucketLifecycleConfigurationRequest& req) override { + Aws::S3::Model::GetBucketLifecycleConfigurationResult result; + Aws::Vector rules; + if (set_bucket_lifecycle) { + Aws::S3::Model::LifecycleRule rule; + Aws::S3::Model::NoncurrentVersionExpiration expiration; + expiration.SetNoncurrentDays(1000); + rule.SetNoncurrentVersionExpiration(expiration); + rules.emplace_back(std::move(rule)); + } + result.SetRules(std::move(rules)); + return Aws::S3::Model::GetBucketLifecycleConfigurationOutcome(std::move(result)); + } + + Aws::S3::Model::GetBucketVersioningOutcome GetBucketVersioning( + const Aws::S3::Model::GetBucketVersioningRequest& req) override { + Aws::S3::Model::GetBucketVersioningResult result; + if (set_bucket_versioning_status_error) { + result.SetStatus(Aws::S3::Model::BucketVersioningStatus::Suspended); + } else { + result.SetStatus(Aws::S3::Model::BucketVersioningStatus::Enabled); + } + return Aws::S3::Model::GetBucketVersioningOutcome(std::move(result)); + } +}; + +static bool return_error_for_error_s3_client = false; +static bool delete_objects_return_part_error = false; + +class ErrorS3Client : public S3ClientInterface { +public: + ErrorS3Client() : _correct_impl(std::make_unique()) {} + ~ErrorS3Client() override = default; + Aws::S3::Model::ListObjectsV2Outcome ListObjectsV2( + const Aws::S3::Model::ListObjectsV2Request& req) override { + if (!return_error_for_error_s3_client) { + return _correct_impl->ListObjectsV2(req); + } + auto err = Aws::Client::AWSError(Aws::S3::S3Errors::RESOURCE_NOT_FOUND, + false); + err.SetResponseCode(Aws::Http::HttpResponseCode::NOT_FOUND); + return Aws::S3::Model::ListObjectsV2Outcome(std::move(err)); + } + + Aws::S3::Model::DeleteObjectsOutcome DeleteObjects( + const Aws::S3::Model::DeleteObjectsRequest& req) override { + if (!delete_objects_return_part_error) { + Aws::S3::Model::DeleteObjectsResult result; + Aws::Vector errors; + Aws::S3::Model::Error error; + errors.emplace_back(std::move(error)); + result.SetErrors(std::move(errors)); + return Aws::S3::Model::DeleteObjectsOutcome(std::move(result)); + } + auto err = Aws::Client::AWSError(Aws::S3::S3Errors::RESOURCE_NOT_FOUND, + false); + err.SetResponseCode(Aws::Http::HttpResponseCode::NOT_FOUND); + // return -1 + return Aws::S3::Model::DeleteObjectsOutcome(std::move(err)); + } + + Aws::S3::Model::PutObjectOutcome PutObject( + const Aws::S3::Model::PutObjectRequest& req) override { + if (!return_error_for_error_s3_client) { + return _correct_impl->PutObject(req); + } + auto err = Aws::Client::AWSError(Aws::S3::S3Errors::RESOURCE_NOT_FOUND, + false); + err.SetResponseCode(Aws::Http::HttpResponseCode::NOT_FOUND); + return Aws::S3::Model::PutObjectOutcome(std::move(err)); + } + + Aws::S3::Model::HeadObjectOutcome HeadObject( + const Aws::S3::Model::HeadObjectRequest& req) override { + if (!return_error_for_error_s3_client) { + return _correct_impl->HeadObject(req); + } + auto err = Aws::Client::AWSError(Aws::S3::S3Errors::RESOURCE_NOT_FOUND, + false); + err.SetResponseCode(Aws::Http::HttpResponseCode::INTERNAL_SERVER_ERROR); + return Aws::S3::Model::HeadObjectOutcome(std::move(err)); + } + + Aws::S3::Model::GetBucketLifecycleConfigurationOutcome GetBucketLifecycleConfiguration( + const Aws::S3::Model::GetBucketLifecycleConfigurationRequest& req) override { + if (!return_error_for_error_s3_client) { + return _correct_impl->GetBucketLifecycleConfiguration(req); + } + auto err = Aws::Client::AWSError(Aws::S3::S3Errors::RESOURCE_NOT_FOUND, + false); + err.SetResponseCode(Aws::Http::HttpResponseCode::INTERNAL_SERVER_ERROR); + return Aws::S3::Model::GetBucketLifecycleConfigurationOutcome(std::move(err)); + } + + Aws::S3::Model::GetBucketVersioningOutcome GetBucketVersioning( + const Aws::S3::Model::GetBucketVersioningRequest& req) override { + if (!return_error_for_error_s3_client) { + return _correct_impl->GetBucketVersioning(req); + } + auto err = Aws::Client::AWSError(Aws::S3::S3Errors::RESOURCE_NOT_FOUND, + false); + err.SetResponseCode(Aws::Http::HttpResponseCode::INTERNAL_SERVER_ERROR); + return Aws::S3::Model::GetBucketVersioningOutcome(std::move(err)); + } + +private: + std::unique_ptr _correct_impl; +}; + +class MockS3Client { +public: + MockS3Client(std::unique_ptr impl = std::make_unique()) + : _impl(std::move(impl)) {} + auto ListObjectsV2(const Aws::S3::Model::ListObjectsV2Request& req) { + return _impl->ListObjectsV2(req); + } + + auto DeleteObjects(const Aws::S3::Model::DeleteObjectsRequest& req) { + return _impl->DeleteObjects(req); + } + + auto PutObject(const Aws::S3::Model::PutObjectRequest& req) { return _impl->PutObject(req); } + + auto HeadObject(const Aws::S3::Model::HeadObjectRequest& req) { return _impl->HeadObject(req); } + + auto GetBucketLifecycleConfiguration( + const Aws::S3::Model::GetBucketLifecycleConfigurationRequest& req) { + return _impl->GetBucketLifecycleConfiguration(req); + } + + auto GetBucketVersioning(const Aws::S3::Model::GetBucketVersioningRequest& req) { + return _impl->GetBucketVersioning(req); + } + +private: + std::unique_ptr _impl; +}; + +std::unique_ptr _mock_client; + +struct MockCallable { + std::string point_name; + std::function func; +}; + +static auto callbacks = std::array { + MockCallable {"s3_client::list_objects_v2", + [](void* p) { + auto pair = *(std::pair*)p; + *pair.first = (*_mock_client).ListObjectsV2(*pair.second); + }}, + MockCallable {"s3_client::delete_objects", + [](void* p) { + auto pair = *(std::pair*)p; + *pair.first = (*_mock_client).DeleteObjects(*pair.second); + }}, + MockCallable {"s3_client::put_object", + [](void* p) { + auto pair = *(std::pair*)p; + *pair.first = (*_mock_client).PutObject(*pair.second); + }}, + MockCallable {"s3_client::head_object", + [](void* p) { + auto pair = *(std::pair*)p; + *pair.first = (*_mock_client).HeadObject(*pair.second); + }}, + MockCallable { + "s3_client::get_bucket_lifecycle_configuration", + [](void* p) { + auto pair = + *(std::pair*)p; + *pair.first = (*_mock_client).GetBucketLifecycleConfiguration(*pair.second); + }}, + MockCallable {"s3_client::get_bucket_versioning", [](void* p) { + auto pair = *(std::pair*)p; + *pair.first = (*_mock_client).GetBucketVersioning(*pair.second); + }}}; + +int main(int argc, char** argv) { + const std::string conf_file = "doris_cloud.conf"; + if (!cloud::config::init(conf_file.c_str(), true)) { + std::cerr << "failed to init config file, conf=" << conf_file << std::endl; + return -1; + } + + if (!cloud::init_glog("s3_accessor_test")) { + std::cerr << "failed to init glog" << std::endl; + return -1; + } + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + +namespace doris::cloud { + +std::string get_key(const std::string& relative_path) { + return fmt::format("/{}", relative_path); +} + +void create_file_under_prefix(std::string_view prefix, size_t file_nums) { + for (size_t i = 0; i < file_nums; i++) { + _mock_fs->put_object(get_key(fmt::format("{}{}", prefix, i)), ""); + } +} + +TEST(S3AccessorTest, init) { + auto accessor = std::make_unique(S3Conf {}); + ASSERT_EQ(0, accessor->init()); +} + +TEST(S3AccessorTest, check_bucket_versioning) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + set_bucket_versioning_status_error = false; + }); + { ASSERT_EQ(0, accessor->check_bucket_versioning()); } + { + set_bucket_versioning_status_error = true; + ASSERT_EQ(-1, accessor->check_bucket_versioning()); + } +} + +TEST(S3AccessorTest, check_bucket_versioning_error) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(std::make_unique()); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + return_error_for_error_s3_client = true; + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + return_error_for_error_s3_client = false; + }); + ASSERT_EQ(-1, accessor->check_bucket_versioning()); +} + +TEST(S3AccessorTest, get_bucket_lifecycle) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + set_bucket_lifecycle = false; + }); + { + int64_t expiration_time = 0; + ASSERT_EQ(-1, accessor->get_bucket_lifecycle(&expiration_time)); + } + { + set_bucket_lifecycle = true; + int64_t expiration_time = 0; + ASSERT_EQ(0, accessor->get_bucket_lifecycle(&expiration_time)); + } +} + +TEST(S3AccessorTest, get_bucket_lifecycle_error) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(std::make_unique()); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + return_error_for_error_s3_client = true; + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + return_error_for_error_s3_client = false; + }); + int64_t expiration_time = 0; + ASSERT_EQ(-1, accessor->get_bucket_lifecycle(&expiration_time)); +} + +TEST(S3AccessorTest, list) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + }); + create_file_under_prefix("test_list", 300); + std::vector files; + ASSERT_EQ(0, accessor->list("test_list", &files)); + ASSERT_EQ(300, files.size()); +} + +TEST(S3AccessorTest, list_error) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(std::make_unique()); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + return_error_for_error_s3_client = true; + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + return_error_for_error_s3_client = false; + }); + create_file_under_prefix("test_list", 300); + std::vector files; + ASSERT_EQ(-1, accessor->list("test_list", &files)); +} + +TEST(S3AccessorTest, put) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + }); + std::string prefix = "test_put"; + for (size_t i = 0; i < 300; i++) { + ASSERT_EQ(0, accessor->put_object(fmt::format("{}{}", prefix, i), "")); + } + std::vector files; + ASSERT_EQ(0, accessor->list("test_put", &files)); + ASSERT_EQ(300, files.size()); +} + +TEST(S3AccessorTest, put_error) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(std::make_unique()); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + return_error_for_error_s3_client = false; + }); + std::string prefix = "test_put_error"; + for (size_t i = 0; i < 300; i++) { + if (i % 2) { + return_error_for_error_s3_client = true; + ASSERT_EQ(-1, accessor->put_object(fmt::format("{}{}", prefix, i), "")); + return_error_for_error_s3_client = false; + break; + } + ASSERT_EQ(0, accessor->put_object(fmt::format("{}{}", prefix, i), "")); + } + std::vector files; + ASSERT_EQ(0, accessor->list("test_put_error", &files)); +} + +TEST(S3AccessorTest, exist) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + }); + std::string prefix = "test_exist"; + ASSERT_EQ(1, accessor->exist(prefix)); + ASSERT_EQ(0, accessor->put_object(prefix, "")); + ASSERT_EQ(0, accessor->exist(prefix)); +} + +TEST(S3AccessorTest, exist_error) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(std::make_unique()); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + return_error_for_error_s3_client = false; + }); + std::string prefix = "test_exist_error"; + ASSERT_EQ(1, accessor->exist(prefix)); + ASSERT_EQ(0, accessor->put_object(prefix, "")); + return_error_for_error_s3_client = true; + ASSERT_EQ(-1, accessor->exist(prefix)); +} + +// function is not implemented +TEST(S3AccessorTest, DISABLED_delete_object) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + }); + std::string prefix = "test_delete_object"; + create_file_under_prefix(prefix, 200); + for (size_t i = 0; i < 200; i++) { + auto path = fmt::format("{}{}", prefix, i); + ASSERT_EQ(0, accessor->delete_object(path)); + ASSERT_EQ(1, accessor->exist(path)); + } +} + +TEST(S3AccessorTest, delete_objects) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + }); + std::string prefix = "test_delete_objects"; + std::vector paths; + size_t num = 300; + for (size_t i = 0; i < num; i++) { + auto path = fmt::format("{}{}", prefix, i); + _mock_fs->put_object(path, ""); + paths.emplace_back(std::move(path)); + } + ASSERT_EQ(0, accessor->delete_objects(paths)); + for (size_t i = 0; i < num; i++) { + auto path = fmt::format("{}{}", prefix, i); + ASSERT_EQ(1, accessor->exist(path)); + } +} + +TEST(S3AccessorTest, delete_objects_error) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(std::make_unique()); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + return_error_for_error_s3_client = false; + delete_objects_return_part_error = false; + }); + std::string prefix = "test_delete_objects"; + std::vector paths_first_half; + std::vector paths_second_half; + size_t num = 300; + for (size_t i = 0; i < num; i++) { + auto path = fmt::format("{}{}", prefix, i); + _mock_fs->put_object(path, ""); + if (i < 150) { + paths_first_half.emplace_back(std::move(path)); + } else { + paths_second_half.emplace_back(std::move(path)); + } + } + std::vector empty; + ASSERT_EQ(0, accessor->delete_objects(empty)); + return_error_for_error_s3_client = true; + delete_objects_return_part_error = true; + ASSERT_EQ(-1, accessor->delete_objects(paths_first_half)); + delete_objects_return_part_error = false; + ASSERT_EQ(-2, accessor->delete_objects(paths_second_half)); +} + +TEST(S3AccessorTest, delete_expired_objects) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + }); + { + std::string prefix = "atest_delete_expired_objects"; + size_t num = 2000; + create_file_under_prefix(prefix, num); + list_object_v2_with_expire_time = true; + expire_time = 50; + ASSERT_EQ(0, accessor->delete_expired_objects(prefix, 100)); + for (size_t i = 0; i < num; i++) { + auto path = fmt::format("{}{}", prefix, i); + ASSERT_EQ(1, accessor->exist(path)); + } + } + { + std::string prefix = "btest_delete_expired_objects"; + size_t num = 2000; + create_file_under_prefix(prefix, num); + list_object_v2_with_expire_time = true; + expire_time = 150; + ASSERT_EQ(0, accessor->delete_expired_objects(prefix, 100)); + for (size_t i = 0; i < num; i++) { + auto path = fmt::format("{}{}", prefix, i); + ASSERT_EQ(1, accessor->exist(path)); + } + } + { + std::string prefix = "ctest_delete_expired_objects"; + size_t num = 2000; + create_file_under_prefix(prefix, num); + list_object_v2_with_expire_time = true; + expire_time = 150; + return_error_for_error_s3_client = true; + std::unique_ptr> defer( + (int*)0x01, [&](int*) { return_error_for_error_s3_client = false; }); + ASSERT_EQ(0, accessor->delete_expired_objects(prefix, 100)); + } +} + +TEST(S3AccessorTest, delete_object_by_prefix) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + }); + std::string prefix = "test_delete_objects_by_prefix"; + size_t num = 2000; + create_file_under_prefix(prefix, num); + ASSERT_EQ(0, accessor->delete_objects_by_prefix(prefix)); + for (size_t i = 0; i < num; i++) { + auto path = fmt::format("{}{}", prefix, i); + ASSERT_EQ(1, accessor->exist(path)); + } +} + +TEST(S3AccessorTest, delete_object_by_prefix_error) { + _mock_fs = std::make_unique(cloud::S3Conf {}); + _mock_client = std::make_unique(std::make_unique()); + auto accessor = std::make_unique(S3Conf {}); + auto sp = SyncPoint::get_instance(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->set_call_back(fmt::format("{}::pred", mock_callback.point_name), + [](void* p) { *((bool*)p) = true; }); + sp->set_call_back(mock_callback.point_name, mock_callback.func); + }); + sp->enable_processing(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + sp->disable_processing(); + std::for_each(callbacks.begin(), callbacks.end(), [&](const MockCallable& mock_callback) { + sp->clear_call_back(mock_callback.point_name); + }); + return_error_for_error_s3_client = false; + delete_objects_return_part_error = false; + }); + std::string prefix = "test_delete_objects_by_prefix"; + size_t num = 2000; + create_file_under_prefix(prefix, num); + delete_objects_return_part_error = true; + return_error_for_error_s3_client = true; + ASSERT_EQ(-1, accessor->delete_objects_by_prefix(prefix)); + return_error_for_error_s3_client = false; + ASSERT_EQ(-2, accessor->delete_objects_by_prefix(prefix)); + delete_objects_return_part_error = false; + ASSERT_EQ(-3, accessor->delete_objects_by_prefix(prefix)); +} + +} // namespace doris::cloud diff --git a/cloud/test/schema_kv_test.cpp b/cloud/test/schema_kv_test.cpp new file mode 100644 index 00000000000000..65e51f67e079a8 --- /dev/null +++ b/cloud/test/schema_kv_test.cpp @@ -0,0 +1,606 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include +#include +#include +#include +#include + +#include + +#include "common/config.h" +#include "common/sync_point.h" +#include "meta-service/keys.h" +#include "meta-service/meta_service.h" +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" + +static std::string instance_id = "schema_kv_test"; + +namespace doris::cloud { +extern std::unique_ptr get_meta_service(); + +static std::string next_rowset_id() { + static int cnt = 0; + return std::to_string(++cnt); +} + +static void add_tablet(CreateTabletsRequest& req, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id, const std::string& rowset_id, + int32_t schema_version) { + auto tablet = req.add_tablet_metas(); + tablet->set_table_id(table_id); + tablet->set_index_id(index_id); + tablet->set_partition_id(partition_id); + tablet->set_tablet_id(tablet_id); + auto schema = tablet->mutable_schema(); + schema->set_schema_version(schema_version); + auto first_rowset = tablet->add_rs_metas(); + first_rowset->set_rowset_id(0); // required + first_rowset->set_rowset_id_v2(rowset_id); + first_rowset->set_start_version(0); + first_rowset->set_end_version(1); + first_rowset->mutable_tablet_schema()->CopyFrom(*schema); +} + +static void create_tablet(MetaServiceProxy* meta_service, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id, const std::string& rowset_id, + int32_t schema_version) { + brpc::Controller cntl; + CreateTabletsRequest req; + CreateTabletsResponse res; + add_tablet(req, table_id, index_id, partition_id, tablet_id, rowset_id, schema_version); + meta_service->create_tablets(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << tablet_id; +} + +static void get_rowset(MetaServiceProxy* meta_service, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id, GetRowsetResponse& res) { + brpc::Controller cntl; + GetRowsetRequest req; + auto tablet_idx = req.mutable_idx(); + tablet_idx->set_table_id(table_id); + tablet_idx->set_index_id(index_id); + tablet_idx->set_partition_id(partition_id); + tablet_idx->set_tablet_id(tablet_id); + req.set_start_version(0); + req.set_end_version(-1); + req.set_cumulative_compaction_cnt(0); + req.set_base_compaction_cnt(0); + req.set_cumulative_point(2); + meta_service->get_rowset(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << tablet_id; +} + +static void check_get_tablet(MetaServiceProxy* meta_service, int64_t tablet_id, + int32_t schema_version) { + brpc::Controller cntl; + GetTabletRequest req; + GetTabletResponse res; + req.set_tablet_id(tablet_id); + meta_service->get_tablet(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << tablet_id; + ASSERT_TRUE(res.has_tablet_meta()) << tablet_id; + EXPECT_TRUE(res.tablet_meta().has_schema()) << tablet_id; + EXPECT_EQ(res.tablet_meta().schema_version(), schema_version) << tablet_id; +}; + +TEST(DetachSchemaKVTest, TabletTest) { + auto meta_service = get_meta_service(); + // meta_service->resource_mgr().reset(); // Do not use resource manager + + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + // new MS write with write_schema_kv=false, old MS read + { + constexpr auto table_id = 10001, index_id = 10002, partition_id = 10003, tablet_id = 10004; + config::write_schema_kv = false; + ASSERT_NO_FATAL_FAILURE(create_tablet(meta_service.get(), table_id, index_id, partition_id, + tablet_id, next_rowset_id(), 1)); + // check saved values in txn_kv + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string tablet_key, tablet_val; + meta_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}, &tablet_key); + ASSERT_EQ(txn->get(tablet_key, &tablet_val), TxnErrorCode::TXN_OK); + doris::TabletMetaCloudPB saved_tablet; + ASSERT_TRUE(saved_tablet.ParseFromString(tablet_val)); + EXPECT_TRUE(saved_tablet.has_schema()); + std::string rowset_key, rowset_val; + meta_rowset_key({instance_id, tablet_id, 1}, &rowset_key); + ASSERT_EQ(txn->get(rowset_key, &rowset_val), TxnErrorCode::TXN_OK); + doris::RowsetMetaCloudPB saved_rowset; + ASSERT_TRUE(saved_rowset.ParseFromString(rowset_val)); + EXPECT_TRUE(saved_rowset.has_tablet_schema()); + } + + // old MS write, new MS read + { + constexpr auto table_id = 10011, index_id = 10012, partition_id = 10013, tablet_id = 10014; + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + doris::TabletMetaCloudPB saved_tablet; + saved_tablet.set_table_id(table_id); + saved_tablet.set_index_id(index_id); + saved_tablet.set_partition_id(partition_id); + saved_tablet.set_tablet_id(tablet_id); + saved_tablet.mutable_schema()->set_schema_version(1); + std::string tablet_key, tablet_val; + meta_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}, &tablet_key); + ASSERT_TRUE(saved_tablet.SerializeToString(&tablet_val)); + txn->put(tablet_key, tablet_val); + TabletIndexPB saved_tablet_idx; + saved_tablet_idx.set_table_id(table_id); + saved_tablet_idx.set_index_id(index_id); + saved_tablet_idx.set_partition_id(partition_id); + saved_tablet_idx.set_tablet_id(tablet_id); + std::string tablet_idx_key, tablet_idx_val; + meta_tablet_idx_key({instance_id, tablet_id}, &tablet_idx_key); + ASSERT_TRUE(saved_tablet_idx.SerializeToString(&tablet_idx_val)); + txn->put(tablet_idx_key, tablet_idx_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + // check get tablet response + check_get_tablet(meta_service.get(), tablet_id, 1); + } + + auto check_new_saved_tablet_val = [](Transaction* txn, int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id, + int32_t schema_version) { + std::string tablet_key, tablet_val; + meta_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}, &tablet_key); + ASSERT_EQ(txn->get(tablet_key, &tablet_val), TxnErrorCode::TXN_OK); + doris::TabletMetaCloudPB saved_tablet; + ASSERT_TRUE(saved_tablet.ParseFromString(tablet_val)); + EXPECT_FALSE(saved_tablet.has_schema()) << tablet_id; + EXPECT_EQ(saved_tablet.schema_version(), schema_version) << tablet_id; + }; + + // new MS write with write_schema_kv=true, new MS read + { + constexpr auto table_id = 10021, index_id = 10022, partition_id = 10023, tablet_id = 10024; + config::write_schema_kv = true; + ASSERT_NO_FATAL_FAILURE(create_tablet(meta_service.get(), table_id, index_id, partition_id, + tablet_id, next_rowset_id(), 1)); + // check saved values in txn_kv + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + check_new_saved_tablet_val(txn.get(), table_id, index_id, partition_id, tablet_id, 1); + std::string rowset_key, rowset_val; + meta_rowset_key({instance_id, tablet_id, 1}, &rowset_key); + ASSERT_EQ(txn->get(rowset_key, &rowset_val), TxnErrorCode::TXN_OK); + doris::RowsetMetaCloudPB saved_rowset; + ASSERT_TRUE(saved_rowset.ParseFromString(rowset_val)); + EXPECT_FALSE(saved_rowset.has_tablet_schema()); + EXPECT_EQ(saved_rowset.index_id(), index_id); + EXPECT_EQ(saved_rowset.schema_version(), 1); + // check get tablet response + check_get_tablet(meta_service.get(), tablet_id, 1); + // check get rowset response + GetRowsetResponse get_rowset_res; + get_rowset(meta_service.get(), table_id, index_id, partition_id, tablet_id, get_rowset_res); + ASSERT_EQ(get_rowset_res.rowset_meta_size(), 1); + EXPECT_TRUE(get_rowset_res.rowset_meta(0).has_tablet_schema()); + EXPECT_EQ(get_rowset_res.rowset_meta(0).index_id(), index_id); + EXPECT_EQ(get_rowset_res.rowset_meta(0).schema_version(), 1); + ASSERT_TRUE(get_rowset_res.has_stats()); + EXPECT_EQ(get_rowset_res.stats().num_rows(), 0); + EXPECT_EQ(get_rowset_res.stats().num_rowsets(), 1); + EXPECT_EQ(get_rowset_res.stats().num_segments(), 0); + EXPECT_EQ(get_rowset_res.stats().data_size(), 0); + } + + // new MS batch create tablets with write_schema_kv=true + { + config::write_schema_kv = true; + brpc::Controller cntl; + CreateTabletsRequest req; + CreateTabletsResponse res; + add_tablet(req, 10031, 10032, 10033, 100031, next_rowset_id(), 1); + add_tablet(req, 10031, 10032, 10033, 100032, next_rowset_id(), 2); + add_tablet(req, 10031, 10032, 10033, 100033, next_rowset_id(), 2); + add_tablet(req, 10031, 10032, 10033, 100034, next_rowset_id(), 3); + add_tablet(req, 10031, 10032, 10033, 100035, next_rowset_id(), 3); + add_tablet(req, 10031, 10032, 10033, 100036, next_rowset_id(), 3); + add_tablet(req, 10031, 10034, 10033, 100037, next_rowset_id(), 1); + add_tablet(req, 10031, 10034, 10033, 100038, next_rowset_id(), 2); + add_tablet(req, 10031, 10034, 10033, 100039, next_rowset_id(), 2); + meta_service->create_tablets(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + // check saved values in txn_kv + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + check_new_saved_tablet_val(txn.get(), 10031, 10032, 10033, 100031, 1); + check_new_saved_tablet_val(txn.get(), 10031, 10032, 10033, 100032, 2); + check_new_saved_tablet_val(txn.get(), 10031, 10032, 10033, 100033, 2); + check_new_saved_tablet_val(txn.get(), 10031, 10032, 10033, 100034, 3); + check_new_saved_tablet_val(txn.get(), 10031, 10032, 10033, 100035, 3); + check_new_saved_tablet_val(txn.get(), 10031, 10032, 10033, 100036, 3); + check_new_saved_tablet_val(txn.get(), 10031, 10034, 10033, 100037, 1); + check_new_saved_tablet_val(txn.get(), 10031, 10034, 10033, 100038, 2); + check_new_saved_tablet_val(txn.get(), 10031, 10034, 10033, 100039, 2); + // check get tablet response + check_get_tablet(meta_service.get(), 100031, 1); + check_get_tablet(meta_service.get(), 100032, 2); + check_get_tablet(meta_service.get(), 100033, 2); + check_get_tablet(meta_service.get(), 100034, 3); + check_get_tablet(meta_service.get(), 100035, 3); + check_get_tablet(meta_service.get(), 100036, 3); + check_get_tablet(meta_service.get(), 100037, 1); + check_get_tablet(meta_service.get(), 100038, 2); + check_get_tablet(meta_service.get(), 100039, 2); + } +} + +static void begin_txn(MetaServiceProxy* meta_service, int64_t db_id, const std::string& label, + int64_t table_id, int64_t& txn_id) { + brpc::Controller cntl; + BeginTxnRequest req; + BeginTxnResponse res; + auto txn_info = req.mutable_txn_info(); + txn_info->set_db_id(db_id); + txn_info->set_label(label); + txn_info->add_table_ids(table_id); + txn_info->set_timeout_ms(36000); + meta_service->begin_txn(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + ASSERT_TRUE(res.has_txn_id()) << label; + txn_id = res.txn_id(); +} + +static void commit_txn(MetaServiceProxy* meta_service, int64_t db_id, int64_t txn_id, + const std::string& label) { + brpc::Controller cntl; + CommitTxnRequest req; + CommitTxnResponse res; + req.set_db_id(db_id); + req.set_txn_id(txn_id); + meta_service->commit_txn(&cntl, &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; +} + +static doris::RowsetMetaCloudPB create_rowset(int64_t txn_id, int64_t tablet_id, + const std::string& rowset_id, int32_t schema_version, + int64_t version = -1) { + doris::RowsetMetaCloudPB rowset; + rowset.set_rowset_id(0); // required + rowset.set_rowset_id_v2(rowset_id); + rowset.set_tablet_id(tablet_id); + rowset.set_txn_id(txn_id); + rowset.set_num_rows(100); + rowset.set_num_segments(1); + rowset.set_data_disk_size(10000); + if (version > 0) { + rowset.set_start_version(version); + rowset.set_end_version(version); + } + rowset.mutable_tablet_schema()->set_schema_version(schema_version); + rowset.set_txn_expiration(::time(nullptr)); // Required by DCHECK + return rowset; +} + +static void prepare_rowset(MetaServiceProxy* meta_service, const doris::RowsetMetaCloudPB& rowset, + CreateRowsetResponse& res) { + brpc::Controller cntl; + auto arena = res.GetArena(); + auto req = google::protobuf::Arena::CreateMessage(arena); + req->set_temporary(true); + req->mutable_rowset_meta()->CopyFrom(rowset); + meta_service->prepare_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; +} + +static void commit_rowset(MetaServiceProxy* meta_service, const doris::RowsetMetaCloudPB& rowset, + CreateRowsetResponse& res) { + brpc::Controller cntl; + auto arena = res.GetArena(); + auto req = google::protobuf::Arena::CreateMessage(arena); + req->set_temporary(true); + req->mutable_rowset_meta()->CopyFrom(rowset); + meta_service->commit_rowset(&cntl, req, &res, nullptr); + if (!arena) delete req; +} + +static void insert_rowset(MetaServiceProxy* meta_service, int64_t db_id, const std::string& label, + int64_t table_id, int64_t tablet_id, int32_t schema_version) { + int64_t txn_id = 0; + ASSERT_NO_FATAL_FAILURE(begin_txn(meta_service, db_id, label, table_id, txn_id)); + CreateRowsetResponse res; + auto rowset = create_rowset(txn_id, tablet_id, next_rowset_id(), schema_version); + prepare_rowset(meta_service, rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + res.Clear(); + ASSERT_NO_FATAL_FAILURE(commit_rowset(meta_service, rowset, res)); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + commit_txn(meta_service, db_id, txn_id, label); +} + +TEST(DetachSchemaKVTest, RowsetTest) { + auto meta_service = get_meta_service(); + // meta_service->resource_mgr().reset(); // Do not use resource manager + + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + constexpr int64_t db_id = 10000; + + // new MS write with write_schema_kv=false, old MS read + { + constexpr auto table_id = 10001, index_id = 10002, partition_id = 10003, tablet_id = 10004; + config::write_schema_kv = false; + ASSERT_NO_FATAL_FAILURE(create_tablet(meta_service.get(), table_id, index_id, partition_id, + tablet_id, next_rowset_id(), 1)); + ASSERT_NO_FATAL_FAILURE( + insert_rowset(meta_service.get(), db_id, "101", table_id, tablet_id, 2)); // [2-2] + // check saved values in txn_kv + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string rowset_key, rowset_val; + meta_rowset_key({instance_id, tablet_id, 2}, &rowset_key); // [2-2] + ASSERT_EQ(txn->get(rowset_key, &rowset_val), TxnErrorCode::TXN_OK); + doris::RowsetMetaCloudPB saved_rowset; + ASSERT_TRUE(saved_rowset.ParseFromString(rowset_val)); + ASSERT_TRUE(saved_rowset.has_tablet_schema()); + EXPECT_EQ(saved_rowset.tablet_schema().schema_version(), 2); + } + + // old MS write, new MS read + { + constexpr auto table_id = 10011, index_id = 10012, partition_id = 10013, tablet_id = 10014; + config::write_schema_kv = false; + ASSERT_NO_FATAL_FAILURE(create_tablet(meta_service.get(), table_id, index_id, partition_id, + tablet_id, next_rowset_id(), 1)); + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + auto saved_rowset = create_rowset(10015, tablet_id, next_rowset_id(), 2, 2); + std::string rowset_key, rowset_val; + meta_rowset_key({instance_id, tablet_id, 2}, &rowset_key); // version=[2-2] + ASSERT_TRUE(saved_rowset.SerializeToString(&rowset_val)); + txn->put(rowset_key, rowset_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + // check get rowset response + GetRowsetResponse get_rowset_res; + get_rowset(meta_service.get(), table_id, index_id, partition_id, tablet_id, get_rowset_res); + ASSERT_EQ(get_rowset_res.rowset_meta_size(), 2); + ASSERT_EQ(get_rowset_res.rowset_meta(0).end_version(), 1); // [0-1] + ASSERT_TRUE(get_rowset_res.rowset_meta(0).has_tablet_schema()); + EXPECT_EQ(get_rowset_res.rowset_meta(0).tablet_schema().schema_version(), 1); + EXPECT_EQ(get_rowset_res.rowset_meta(0).index_id(), index_id); + EXPECT_EQ(get_rowset_res.rowset_meta(0).schema_version(), 1); + ASSERT_EQ(get_rowset_res.rowset_meta(1).end_version(), 2); // [2-2] + ASSERT_TRUE(get_rowset_res.rowset_meta(1).has_tablet_schema()); + EXPECT_EQ(get_rowset_res.rowset_meta(1).tablet_schema().schema_version(), 2); + EXPECT_EQ(get_rowset_res.rowset_meta(1).index_id(), index_id); + EXPECT_EQ(get_rowset_res.rowset_meta(1).schema_version(), 2); + } + + // new MS write with write_schema_kv=true, new MS read + { + constexpr auto table_id = 10021, index_id = 10022, partition_id = 10023, tablet_id = 10024; + config::write_schema_kv = true; + ASSERT_NO_FATAL_FAILURE(create_tablet(meta_service.get(), table_id, index_id, partition_id, + tablet_id, next_rowset_id(), 1)); + ASSERT_NO_FATAL_FAILURE( + insert_rowset(meta_service.get(), db_id, "201", table_id, tablet_id, 2)); // [2-2] + // check saved values in txn_kv + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string rowset_key, rowset_val; + meta_rowset_key({instance_id, 10024, 2}, &rowset_key); // [2-2] + ASSERT_EQ(txn->get(rowset_key, &rowset_val), TxnErrorCode::TXN_OK); + doris::RowsetMetaCloudPB saved_rowset; + ASSERT_TRUE(saved_rowset.ParseFromString(rowset_val)); + EXPECT_FALSE(saved_rowset.has_tablet_schema()); + EXPECT_EQ(saved_rowset.index_id(), index_id); + EXPECT_EQ(saved_rowset.schema_version(), 2); + // check get rowset response + GetRowsetResponse get_rowset_res; + get_rowset(meta_service.get(), table_id, index_id, partition_id, tablet_id, get_rowset_res); + ASSERT_EQ(get_rowset_res.rowset_meta_size(), 2); + ASSERT_EQ(get_rowset_res.rowset_meta(0).end_version(), 1); // [0-1] + ASSERT_TRUE(get_rowset_res.rowset_meta(0).has_tablet_schema()); + EXPECT_EQ(get_rowset_res.rowset_meta(0).tablet_schema().schema_version(), 1); + EXPECT_EQ(get_rowset_res.rowset_meta(0).index_id(), index_id); + EXPECT_EQ(get_rowset_res.rowset_meta(0).schema_version(), 1); + ASSERT_EQ(get_rowset_res.rowset_meta(1).end_version(), 2); // [2-2] + ASSERT_TRUE(get_rowset_res.rowset_meta(1).has_tablet_schema()); + EXPECT_EQ(get_rowset_res.rowset_meta(1).tablet_schema().schema_version(), 2); + EXPECT_EQ(get_rowset_res.rowset_meta(1).index_id(), index_id); + EXPECT_EQ(get_rowset_res.rowset_meta(1).schema_version(), 2); + ASSERT_TRUE(get_rowset_res.has_stats()); + EXPECT_EQ(get_rowset_res.stats().num_rows(), 100); + EXPECT_EQ(get_rowset_res.stats().num_rowsets(), 2); + EXPECT_EQ(get_rowset_res.stats().num_segments(), 1); + EXPECT_EQ(get_rowset_res.stats().data_size(), 10000); + } + + // new MS read rowsets committed by both old and new MS + auto insert_and_get_rowset = [&meta_service](int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id, + int label_base, + google::protobuf::Arena* arena = nullptr) { + config::write_schema_kv = false; + std::mt19937 rng(std::chrono::system_clock::now().time_since_epoch().count()); + std::uniform_int_distribution dist1(1, 4); + std::uniform_int_distribution dist2(2, 7); + std::vector schema_versions {1}; + ASSERT_NO_FATAL_FAILURE(create_tablet(meta_service.get(), table_id, index_id, partition_id, + tablet_id, next_rowset_id(), schema_versions[0])); + for (int i = 0; i < 10; ++i) { + schema_versions.push_back(dist1(rng)); + ASSERT_NO_FATAL_FAILURE(insert_rowset(meta_service.get(), db_id, + std::to_string(++label_base), table_id, tablet_id, + schema_versions.back())); + } + config::write_schema_kv = true; + for (int i = 0; i < 15; ++i) { + schema_versions.push_back(dist2(rng)); + ASSERT_NO_FATAL_FAILURE(insert_rowset(meta_service.get(), db_id, + std::to_string(++label_base), table_id, tablet_id, + schema_versions.back())); + } + // check get rowset response + auto get_rowset_res = google::protobuf::Arena::CreateMessage(arena); + std::unique_ptr> defer((int*)0x01, [&](int*) { + if (!arena) delete get_rowset_res; + }); + get_rowset(meta_service.get(), table_id, index_id, partition_id, tablet_id, + *get_rowset_res); + ASSERT_EQ(get_rowset_res->rowset_meta_size(), schema_versions.size()); + for (int i = 0; i < schema_versions.size(); ++i) { + auto& rowset = get_rowset_res->rowset_meta(i); + ASSERT_EQ(rowset.end_version(), i + 1); + ASSERT_TRUE(rowset.has_tablet_schema()); + EXPECT_EQ(rowset.tablet_schema().schema_version(), schema_versions[i]); + EXPECT_EQ(rowset.index_id(), index_id); + EXPECT_EQ(rowset.schema_version(), schema_versions[i]); + } + ASSERT_TRUE(get_rowset_res->has_stats()); + EXPECT_EQ(get_rowset_res->stats().num_rows(), 2500); + EXPECT_EQ(get_rowset_res->stats().num_rowsets(), 26); + EXPECT_EQ(get_rowset_res->stats().num_segments(), 25); + EXPECT_EQ(get_rowset_res->stats().data_size(), 250000); + }; + insert_and_get_rowset(10031, 10032, 10033, 10034, 300); + // use arena + google::protobuf::Arena arena; + insert_and_get_rowset(10041, 10042, 10043, 10044, 400, &arena); +} + +TEST(DetachSchemaKVTest, InsertExistedRowsetTest) { + auto meta_service = get_meta_service(); + // meta_service->resource_mgr().reset(); // Do not use resource manager + + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + // old MS commit rowset, new MS commit rowset again + { + constexpr auto table_id = 10001, index_id = 10002, partition_id = 10003, tablet_id = 10004; + config::write_schema_kv = false; + ASSERT_NO_FATAL_FAILURE(create_tablet(meta_service.get(), table_id, index_id, partition_id, + tablet_id, next_rowset_id(), 1)); + std::unique_ptr txn; + ASSERT_EQ(meta_service->txn_kv()->create_txn(&txn), TxnErrorCode::TXN_OK); + auto committed_rowset = create_rowset(10005, tablet_id, next_rowset_id(), 2, 2); + std::string tmp_rowset_key, tmp_rowset_val; + // 0:instance_id 1:txn_id 2:tablet_id + meta_rowset_tmp_key({instance_id, 10005, tablet_id}, &tmp_rowset_key); + ASSERT_TRUE(committed_rowset.SerializeToString(&tmp_rowset_val)); + txn->put(tmp_rowset_key, tmp_rowset_val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + CreateRowsetResponse res; + auto new_rowset = create_rowset(10005, tablet_id, next_rowset_id(), 2, 2); + prepare_rowset(meta_service.get(), new_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::ALREADY_EXISTED); + ASSERT_TRUE(res.has_existed_rowset_meta()); + EXPECT_EQ(res.existed_rowset_meta().rowset_id_v2(), committed_rowset.rowset_id_v2()); + EXPECT_EQ(res.existed_rowset_meta().index_id(), index_id); + EXPECT_EQ(res.existed_rowset_meta().schema_version(), 2); + ASSERT_TRUE(res.existed_rowset_meta().has_tablet_schema()); + EXPECT_EQ(res.existed_rowset_meta().tablet_schema().schema_version(), 2); + res.Clear(); + commit_rowset(meta_service.get(), new_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::ALREADY_EXISTED); + ASSERT_TRUE(res.has_existed_rowset_meta()); + EXPECT_EQ(res.existed_rowset_meta().rowset_id_v2(), committed_rowset.rowset_id_v2()); + EXPECT_EQ(res.existed_rowset_meta().index_id(), index_id); + EXPECT_EQ(res.existed_rowset_meta().schema_version(), 2); + ASSERT_TRUE(res.existed_rowset_meta().has_tablet_schema()); + EXPECT_EQ(res.existed_rowset_meta().tablet_schema().schema_version(), 2); + } + + // new MS commit rowset, new MS commit rowset again + auto insert_existed_rowset = [&meta_service](int64_t table_id, int64_t index_id, + int64_t partition_id, int64_t tablet_id, + int64_t txn_id, + google::protobuf::Arena* arena = nullptr) { + config::write_schema_kv = true; + ASSERT_NO_FATAL_FAILURE(create_tablet(meta_service.get(), table_id, index_id, partition_id, + tablet_id, next_rowset_id(), 1)); + auto committed_rowset = create_rowset(txn_id, tablet_id, next_rowset_id(), 2, 2); + auto res = google::protobuf::Arena::CreateMessage(arena); + std::unique_ptr> defer((int*)0x01, [&](int*) { + if (!arena) delete res; + }); + prepare_rowset(meta_service.get(), committed_rowset, *res); + ASSERT_EQ(res->status().code(), MetaServiceCode::OK); + res->Clear(); + commit_rowset(meta_service.get(), committed_rowset, *res); + ASSERT_EQ(res->status().code(), MetaServiceCode::OK); + res->Clear(); + auto new_rowset = create_rowset(txn_id, tablet_id, next_rowset_id(), 2, 2); + prepare_rowset(meta_service.get(), new_rowset, *res); + ASSERT_EQ(res->status().code(), MetaServiceCode::ALREADY_EXISTED); + ASSERT_TRUE(res->has_existed_rowset_meta()); + EXPECT_EQ(res->existed_rowset_meta().rowset_id_v2(), committed_rowset.rowset_id_v2()); + EXPECT_EQ(res->existed_rowset_meta().index_id(), index_id); + EXPECT_EQ(res->existed_rowset_meta().schema_version(), 2); + ASSERT_TRUE(res->existed_rowset_meta().has_tablet_schema()); + EXPECT_EQ(res->existed_rowset_meta().tablet_schema().schema_version(), 2); + res->Clear(); + commit_rowset(meta_service.get(), new_rowset, *res); + ASSERT_EQ(res->status().code(), MetaServiceCode::ALREADY_EXISTED); + ASSERT_TRUE(res->has_existed_rowset_meta()); + EXPECT_EQ(res->existed_rowset_meta().rowset_id_v2(), committed_rowset.rowset_id_v2()); + EXPECT_EQ(res->existed_rowset_meta().index_id(), index_id); + EXPECT_EQ(res->existed_rowset_meta().schema_version(), 2); + ASSERT_TRUE(res->existed_rowset_meta().has_tablet_schema()); + EXPECT_EQ(res->existed_rowset_meta().tablet_schema().schema_version(), 2); + }; + insert_existed_rowset(10011, 10012, 10013, 10014, 10015); + google::protobuf::Arena arena; + insert_existed_rowset(10021, 10022, 10023, 10024, 10025, &arena); +} + +TEST(SchemaKVTest, InsertExistedRowsetTest) { + auto meta_service = get_meta_service(); + // meta_service->resource_mgr().reset(); // Do not use resource manager + + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("get_instance_id::pred", [](void* p) { *((bool*)p) = true; }); + sp->set_call_back("get_instance_id", [&](void* p) { *((std::string*)p) = instance_id; }); + sp->enable_processing(); + + config::write_schema_kv = true; + config::meta_schema_value_version = 0; + ASSERT_NO_FATAL_FAILURE( + create_tablet(meta_service.get(), 10001, 10002, 10003, 10004, next_rowset_id(), 1)); + check_get_tablet(meta_service.get(), 10004, 1); + + config::meta_schema_value_version = 1; + ASSERT_NO_FATAL_FAILURE( + create_tablet(meta_service.get(), 10001, 10002, 10003, 10005, next_rowset_id(), 2)); + check_get_tablet(meta_service.get(), 10005, 2); +} + +} // namespace doris::cloud diff --git a/cloud/test/stopwatch_test.cpp b/cloud/test/stopwatch_test.cpp new file mode 100644 index 00000000000000..f4ecb1d82f2b1b --- /dev/null +++ b/cloud/test/stopwatch_test.cpp @@ -0,0 +1,51 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "common/stopwatch.h" + +#include + +#include +#include + +using namespace doris::cloud; + +int main(int argc, char** argv) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + +TEST(StopWatchTest, SimpleTest) { + { + StopWatch s; + s.start(); + std::this_thread::sleep_for(std::chrono::microseconds(1000)); + ASSERT_TRUE(s.elapsed_us() >= 1000); + + s.pause(); + std::this_thread::sleep_for(std::chrono::microseconds(1000)); + ASSERT_TRUE(s.elapsed_us() >= 1000 && s.elapsed_us() < 1500); + + s.resume(); + std::this_thread::sleep_for(std::chrono::microseconds(1000)); + ASSERT_TRUE(s.elapsed_us() >= 1000 && s.elapsed_us() < 2500); + + s.reset(); + std::this_thread::sleep_for(std::chrono::microseconds(1000)); + ASSERT_TRUE(s.elapsed_us() >= 1000 && s.elapsed_us() < 1500); + } +} \ No newline at end of file diff --git a/cloud/test/sync_point_test.cpp b/cloud/test/sync_point_test.cpp new file mode 100644 index 00000000000000..6841fff0ecb663 --- /dev/null +++ b/cloud/test/sync_point_test.cpp @@ -0,0 +1,340 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +// switch on syncpoint macro +#ifndef UNIT_TEST +#define UNIT_TEST +#endif +#include "common/sync_point.h" + +#include +#include +#include +#include + +#define STDOUT (std::cout << __PRETTY_FUNCTION__ << " ") + +int g_data = 12315; + +void foo() { + int a = 10086; + + STDOUT << "a: " << a << std::endl; + + TEST_SYNC_POINT_CALLBACK("foo:a_assigned", &a); // a may be accessed + + STDOUT << "a: " << a << std::endl; + + int b = 100; + + STDOUT << "b: " << b << std::endl; + TEST_SYNC_POINT_CALLBACK("foo:b_assigned", &b); // b may be accessed + + STDOUT << "b: " << b << std::endl; + + a += b; + + STDOUT << "a: " << a << std::endl; + + if (a == 10086 + 100) { + STDOUT << "expected branch taken, a: " << a << std::endl; + } else { + STDOUT << "exceptional branch taken, a: " << a << std::endl; + } +} + +void test_foo_single_thread() { + auto sp = doris::cloud::SyncPoint::get_instance(); + std::cout << "========== default behavior ==========" << std::endl; + foo(); // first run, nothing changed + + std::cout << "========== change default behavior ==========" << std::endl; + sp->set_call_back("foo:a_assigned", [](void* arg) { + int& a = *reinterpret_cast(arg); // we known what arg is + STDOUT << "original a: " << a << std::endl; + a = 10010; + STDOUT << "change a: " << a << std::endl; + }); + + sp->set_call_back("foo:b_assigned", [](void* arg) { + int& b = *reinterpret_cast(arg); // we known what arg is + STDOUT << "original b: " << b << std::endl; + b = 200; + STDOUT << "change b: " << b << std::endl; + }); + + sp->enable_processing(); // turn on + foo(); + + sp->clear_all_call_backs(); + sp->disable_processing(); + sp->clear_trace(); +} + +void foo(int x) { + // multi-thread data race + g_data = x; + + TEST_SYNC_POINT("foo:assigned:1"); + // concurrent stuff may be executed by other thread here, + // and this thread may race with it + TEST_SYNC_POINT("foo:assigned:2"); + + STDOUT << x << std::endl; +} + +void bar() { + TEST_SYNC_POINT("bar:assigned:1"); + g_data = 10086; + TEST_SYNC_POINT("bar:assigned:2"); +} + +void test_foo_data_race() { + std::cout << "========== data race ==========" << std::endl; + + std::mutex mtx; + std::condition_variable cv; + auto sp = doris::cloud::SyncPoint::get_instance(); + + bool go = false; + std::unique_lock lk(mtx); + g_data = 0; + + // =========================================================================== + // FORCE threads concurrent execution with sequence: + // + // thread1 thread2 thread3 + // | | | + // | | foo(3) + // | | | + // | foo(2) | + // | | | + // foo(1) | | + // | | | + // v v v + // + std::thread th1([&] { + { + std::unique_lock lk(mtx); + cv.wait(lk, [&] { return go; }); + } + TEST_SYNC_POINT("test_foo_data_race:1:start"); + assert(g_data == 2); + foo(1); + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + assert(g_data == 1); + TEST_SYNC_POINT("test_foo_data_race:1:end"); + }); + std::thread th2([&] { + { + std::unique_lock lk(mtx); + cv.wait(lk, [&] { return go; }); + } + TEST_SYNC_POINT("test_foo_data_race:2:start"); + assert(g_data == 3); + foo(2); + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + assert(g_data == 2); + TEST_SYNC_POINT("test_foo_data_race:2:end"); + }); + std::thread th3([&] { + { + std::unique_lock lk(mtx); + cv.wait(lk, [&] { return go; }); + } + TEST_SYNC_POINT("test_foo_data_race:3:start"); + assert(g_data == 0); + foo(3); + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + assert(g_data == 3); + TEST_SYNC_POINT("test_foo_data_race:3:end"); + }); + + // prepare dependency + sp->enable_processing(); + // run foo(int) in sequence 3->2->1 + sp->load_dependency({ + {"test_foo_data_race:3:end", "test_foo_data_race:2:start"}, + {"test_foo_data_race:2:end", "test_foo_data_race:1:start"}, + }); + + // set and go + go = true; + lk.unlock(); + cv.notify_all(); + + th1.join(); + th2.join(); + th3.join(); + sp->clear_all_call_backs(); + sp->clear_trace(); + sp->disable_processing(); + + // =========================================================================== + // FORCE to run bar() in the middle of foo() + // + // thread4 thread5 + // | | + // | foo():assigned:1 + // | | + // bar() | + // | | + // | foo():assigned:2 + // | | + // v v + // + lk.lock(); + go = false; + + std::thread th4([&] { + { + std::unique_lock lk(mtx); + cv.wait(lk, [&] { return go; }); + } + bar(); + }); + std::thread th5([&] { + { + std::unique_lock lk(mtx); + cv.wait(lk, [&] { return go; }); + } + foo(10010); // try to set g_data to 10010 + assert(g_data == 10086); // foo() is racing with bar() + }); + + // prepare dependency + sp->enable_processing(); + sp->load_dependency({ + {"foo:assigned:1", "bar:assigned:1"}, // no need to specify bar1->bar2, + {"bar:assigned:2", "foo:assigned:2"}, // because they a natually sequenced + }); + + // set and go + go = true; + lk.unlock(); + cv.notify_all(); + + th4.join(); + th5.join(); + sp->clear_all_call_backs(); + sp->clear_trace(); + + sp->disable_processing(); +} + +int foo_return_with_value() { + int ctx = 1; + (void)ctx; + // for those return values are not explictly declared + { + // for ctx capture + TEST_SYNC_POINT_CALLBACK("foo_return_with_value1_ctx", &ctx); + + int tmp_ret = -1; + (void)tmp_ret; // supress `unused` warning when build in release mode + TEST_SYNC_POINT_RETURN_WITH_VALUE("foo_return_with_value1", &tmp_ret); + } + + // for those return valuse are explicitly declared + { + int ret = -1; + TEST_SYNC_POINT_RETURN_WITH_VALUE("foo_return_with_value2", &ret); + return ret; + } +} + +void foo_return_with_void(int* in) { + *in = 10000; + TEST_SYNC_POINT_RETURN_WITH_VOID("foo_return_with_void1"); + *in = 10010; + TEST_SYNC_POINT_RETURN_WITH_VOID("foo_return_with_void2"); + *in = 10086; +} + +void test_return_point() { + auto sp = doris::cloud::SyncPoint::get_instance(); + sp->clear_all_call_backs(); + sp->enable_processing(); + + // test pred == false, nothing happens + { + sp->clear_all_call_backs(); + + sp->set_call_back("foo_return_with_value1", + [](void* ret) { *reinterpret_cast(ret) = -1; }); + sp->set_call_back("foo_return_with_value1::pred", + [](void* pred) { *reinterpret_cast(pred) = false; }); + + sp->set_call_back("foo_return_with_void1::pred", + [](void* pred) { *reinterpret_cast(pred) = false; }); + + int ret = foo_return_with_value(); + assert(ret == -1); + + foo_return_with_void(&ret); + assert(ret == 10086); + } + + // test pred == true, get the value and return point we want + { + sp->clear_all_call_backs(); + sp->set_call_back("foo_return_with_value2", + [](void* ret) { *reinterpret_cast(ret) = 10086; }); + sp->set_call_back("foo_return_with_value2::pred", + [](void* pred) { *reinterpret_cast(pred) = true; }); + + sp->set_call_back("foo_return_with_void2::pred", + [](void* pred) { *reinterpret_cast(pred) = true; }); + + int ret = foo_return_with_value(); + assert(ret == 10086); + + foo_return_with_void(&ret); + assert(ret == 10010); + } + + // pred depends on tested thread's context + { + sp->clear_all_call_backs(); + int* ctx; + // "steal" context from tested thread to this testing thread in order to + // change behaviors of the tested thread without changing it's code + sp->set_call_back("foo_return_with_value1_ctx", + [&ctx](void* tested_ctx) { ctx = reinterpret_cast(tested_ctx); }); + sp->set_call_back("foo_return_with_value1", + [](void* ret) { *reinterpret_cast(ret) = 10086; }); + // use the context from tested thread to do more checks and modifications + sp->set_call_back("foo_return_with_value1::pred", [&ctx](void* pred) { + // we can change the return logic of the tested thread + *reinterpret_cast(pred) = (*ctx > 0); // true + }); + + [[maybe_unused]] int ret = foo_return_with_value(); + assert(ret == 10086); + } +} + +void test() { + // test_foo_single_thread(); + // test_foo_data_race(); + test_return_point(); +} + +int main() { + test(); + return 0; +} diff --git a/cloud/test/txn_kv_test.cpp b/cloud/test/txn_kv_test.cpp new file mode 100644 index 00000000000000..63a89a0743438b --- /dev/null +++ b/cloud/test/txn_kv_test.cpp @@ -0,0 +1,547 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "meta-service/txn_kv.h" + +#include +#include +#include +#include +#include +#include + +#include + +#include "common/config.h" +#include "common/stopwatch.h" +#include "common/sync_point.h" +#include "common/util.h" +#include "meta-service/doris_txn.h" +#include "meta-service/keys.h" +#include "meta-service/mem_txn_kv.h" +#include "meta-service/txn_kv.h" +#include "meta-service/txn_kv_error.h" + +using namespace doris::cloud; + +std::shared_ptr txn_kv; + +void init_txn_kv() { + config::fdb_cluster_file_path = "fdb.cluster"; + txn_kv = std::dynamic_pointer_cast(std::make_shared()); + ASSERT_NE(txn_kv.get(), nullptr); + int ret = txn_kv->init(); + ASSERT_EQ(ret, 0); +} + +int main(int argc, char** argv) { + config::init(nullptr, true); + ::testing::InitGoogleTest(&argc, argv); + init_txn_kv(); + return RUN_ALL_TESTS(); +} + +TEST(TxnKvTest, Network) { + fdb::Network network(FDBNetworkOption {}); + network.init(); + network.stop(); +} + +TEST(TxnKvTest, GetVersionTest) { + std::unique_ptr txn; + std::string key; + std::string val; + { + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + key.push_back('\xfe'); + key.append(" unit_test_prefix "); + key.append(" GetVersionTest "); + txn->atomic_set_ver_value(key, ""); + TxnErrorCode err = txn->commit(); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + + int64_t ver0 = 0; + ASSERT_EQ(txn->get_committed_version(&ver0), TxnErrorCode::TXN_OK); + ASSERT_GT(ver0, 0); + + err = txn_kv->create_txn(&txn); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + err = txn->get(key, &val); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + + int64_t ver1 = 0; + ASSERT_EQ(txn->get_read_version(&ver1), TxnErrorCode::TXN_OK); + ASSERT_GE(ver1, ver0); + + int64_t ver2; + int64_t txn_id; + int ret = get_txn_id_from_fdb_ts(val, &txn_id); + ASSERT_EQ(ret, 0); + ver2 = txn_id >> 10; + + std::cout << "ver0=" << ver0 << " ver1=" << ver1 << " ver2=" << ver2 << std::endl; + } +} + +TEST(TxnKvTest, ConflictTest) { + std::unique_ptr txn, txn1, txn2; + std::string key = "unit_test"; + std::string val, val1, val2; + + // Historical data + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->put("unit_test", "xxxxxxxxxxxxx"); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + // txn1 begin + ASSERT_EQ(txn_kv->create_txn(&txn1), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn1->get(key, &val1), TxnErrorCode::TXN_OK); + std::cout << "val1=" << val1 << std::endl; + + // txn2 begin + ASSERT_EQ(txn_kv->create_txn(&txn2), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn2->get(key, &val2), TxnErrorCode::TXN_OK); + std::cout << "val2=" << val2 << std::endl; + + // txn2 commit + val2 = "zzzzzzzzzzzzzzz"; + txn2->put(key, val2); + ASSERT_EQ(txn2->commit(), TxnErrorCode::TXN_OK); + + // txn1 commit, intend to fail + val1 = "yyyyyyyyyyyyyyy"; + txn1->put(key, val1); + ASSERT_NE(txn1->commit(), TxnErrorCode::TXN_OK); + + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(key, &val), TxnErrorCode::TXN_OK); + ASSERT_EQ(val, val2); // First wins + std::cout << "final val=" << val << std::endl; +} + +TEST(TxnKvTest, AtomicSetVerKeyTest) { + std::string key_prefix = "key_1"; + + std::string versionstamp_1; + { + // write key_1 + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->atomic_set_ver_key(key_prefix, "1"); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + // read key_1 + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string end_key = key_prefix + "\xFF"; + std::unique_ptr it; + ASSERT_EQ(txn->get(key_prefix, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_TRUE(it->has_next()); + auto&& [key_1, _1] = it->next(); + ASSERT_EQ(key_1.length(), key_prefix.size() + 10); // versionstamp = 10bytes + key_1.remove_prefix(key_prefix.size()); + versionstamp_1 = key_1; + } + + std::string versionstamp_2; + { + // write key_2 + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + key_prefix = "key_2"; + txn->atomic_set_ver_key(key_prefix, "2"); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + // read key_2 + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string end_key = key_prefix + "\xFF"; + std::unique_ptr it; + ASSERT_EQ(txn->get(key_prefix, end_key, &it), TxnErrorCode::TXN_OK); + ASSERT_TRUE(it->has_next()); + auto&& [key_2, _2] = it->next(); + ASSERT_EQ(key_2.length(), key_prefix.size() + 10); // versionstamp = 10bytes + key_2.remove_prefix(key_prefix.size()); + versionstamp_2 = key_2; + } + + ASSERT_LT(versionstamp_1, versionstamp_2); +} + +TEST(TxnKvTest, AtomicAddTest) { + std::unique_ptr txn, txn1, txn2; + std::string key = "counter"; + // clear counter + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + txn->remove(key); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + // txn1 atomic add + ASSERT_EQ(txn_kv->create_txn(&txn1), TxnErrorCode::TXN_OK); + txn1->atomic_add(key, 10); + // txn2 atomic add + ASSERT_EQ(txn_kv->create_txn(&txn2), TxnErrorCode::TXN_OK); + txn2->atomic_add(key, 20); + // txn1 commit success + ASSERT_EQ(txn1->commit(), TxnErrorCode::TXN_OK); + // txn2 commit success + ASSERT_EQ(txn2->commit(), TxnErrorCode::TXN_OK); + // Check counter val + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + std::string val; + ASSERT_EQ(txn->get(key, &val), TxnErrorCode::TXN_OK); + ASSERT_EQ(val.size(), 8); + ASSERT_EQ(*(int64_t*)val.data(), 30); + + // txn1 atomic add + ASSERT_EQ(txn_kv->create_txn(&txn1), TxnErrorCode::TXN_OK); + txn1->atomic_add(key, 30); + // txn2 get and put + ASSERT_EQ(txn_kv->create_txn(&txn2), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn2->get(key, &val), TxnErrorCode::TXN_OK); + ASSERT_EQ(val.size(), 8); + ASSERT_EQ(*(int64_t*)val.data(), 30); + *(int64_t*)val.data() = 100; + txn2->put(key, val); + // txn1 commit success + ASSERT_EQ(txn1->commit(), TxnErrorCode::TXN_OK); + // txn2 commit, intend to fail + ASSERT_NE(txn2->commit(), TxnErrorCode::TXN_OK); + // Check counter val + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(txn->get(key, &val), TxnErrorCode::TXN_OK); + ASSERT_EQ(val.size(), 8); + ASSERT_EQ(*(int64_t*)val.data(), 60); +} + +TEST(TxnKvTest, CompatibleGetTest) { + auto txn_kv = std::make_shared(); + doris::TabletSchemaCloudPB schema; + schema.set_schema_version(1); + for (int i = 0; i < 1000; ++i) { + auto column = schema.add_column(); + column->set_unique_id(i); + column->set_name("col" + std::to_string(i)); + column->set_type("VARCHAR"); + column->set_aggregation("NONE"); + column->set_length(100); + column->set_index_length(80); + } + std::string instance_id = "compatible_get_test_" + std::to_string(::time(nullptr)); + auto key = meta_schema_key({instance_id, 10005, 1}); + auto val = schema.SerializeAsString(); + + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(doris::cloud::key_exists(txn.get(), key), TxnErrorCode::TXN_KEY_NOT_FOUND); + ValueBuf val_buf; + ASSERT_EQ(doris::cloud::get(txn.get(), key, &val_buf), TxnErrorCode::TXN_KEY_NOT_FOUND); + txn->put(key, val); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + // Check get + TxnErrorCode err = txn_kv->create_txn(&txn); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + err = doris::cloud::key_exists(txn.get(), key); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + err = doris::cloud::get(txn.get(), key, &val_buf); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + EXPECT_EQ(val_buf.ver, 0); + doris::TabletSchemaCloudPB saved_schema; + ASSERT_TRUE(val_buf.to_pb(&saved_schema)); + ASSERT_EQ(saved_schema.column_size(), schema.column_size()); + for (size_t i = 0; i < saved_schema.column_size(); ++i) { + auto& saved_col = saved_schema.column(i); + auto& col = schema.column(i); + EXPECT_EQ(saved_col.name(), col.name()); + } + + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + val_buf.remove(txn.get()); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + // Check remove + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(doris::cloud::key_exists(txn.get(), key), TxnErrorCode::TXN_KEY_NOT_FOUND); + ASSERT_EQ(doris::cloud::get(txn.get(), key, &val_buf), TxnErrorCode::TXN_KEY_NOT_FOUND); +} + +TEST(TxnKvTest, PutLargeValueTest) { + auto txn_kv = std::make_shared(); + + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->enable_processing(); + + doris::TabletSchemaCloudPB schema; + schema.set_schema_version(1); + for (int i = 0; i < 10000; ++i) { + auto column = schema.add_column(); + column->set_unique_id(i); + column->set_name("col" + std::to_string(i)); + column->set_type("VARCHAR"); + column->set_aggregation("NONE"); + column->set_length(100); + column->set_index_length(80); + } + std::cout << "value size=" << schema.SerializeAsString().size() << std::endl; + + std::string instance_id = "put_large_value_" + std::to_string(::time(nullptr)); + auto key = meta_schema_key({instance_id, 10005, 1}); + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + doris::cloud::put(txn.get(), key, schema, 1, 100); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + + // Check get + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + ValueBuf val_buf; + doris::TabletSchemaCloudPB saved_schema; + ASSERT_EQ(doris::cloud::key_exists(txn.get(), key), TxnErrorCode::TXN_OK); + TxnErrorCode err = doris::cloud::get(txn.get(), key, &val_buf); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + std::cout << "num iterators=" << val_buf.iters.size() << std::endl; + EXPECT_EQ(val_buf.ver, 1); + ASSERT_TRUE(val_buf.to_pb(&saved_schema)); + ASSERT_EQ(saved_schema.column_size(), schema.column_size()); + for (size_t i = 0; i < saved_schema.column_size(); ++i) { + auto& saved_col = saved_schema.column(i); + auto& col = schema.column(i); + EXPECT_EQ(saved_col.name(), col.name()); + } + // Check multi range get + sp->set_call_back("memkv::Transaction::get", [](void* limit) { *((int*)limit) = 100; }); + err = doris::cloud::get(txn.get(), key, &val_buf); + ASSERT_EQ(err, TxnErrorCode::TXN_OK); + std::cout << "num iterators=" << val_buf.iters.size() << std::endl; + EXPECT_EQ(val_buf.ver, 1); + ASSERT_TRUE(val_buf.to_pb(&saved_schema)); + ASSERT_EQ(saved_schema.column_size(), schema.column_size()); + for (size_t i = 0; i < saved_schema.column_size(); ++i) { + auto& saved_col = saved_schema.column(i); + auto& col = schema.column(i); + EXPECT_EQ(saved_col.name(), col.name()); + } + // Check keys + auto& iters = val_buf.iters; + size_t i = 0; + std::vector, int, int>> fields; + for (auto&& it : iters) { + it->reset(); + while (it->has_next()) { + auto [k, _] = it->next(); + k.remove_prefix(1); + fields.clear(); + int ret = decode_key(&k, &fields); + ASSERT_EQ(ret, 0); + int64_t* suffix = std::get_if(&std::get<0>(fields.back())); + ASSERT_TRUE(suffix); + EXPECT_EQ(*suffix, (1L << 56) + i++); + } + } + + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + val_buf.remove(txn.get()); + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + // Check remove + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + ASSERT_EQ(doris::cloud::key_exists(txn.get(), key), TxnErrorCode::TXN_KEY_NOT_FOUND); + ASSERT_EQ(doris::cloud::get(txn.get(), key, &val_buf), TxnErrorCode::TXN_KEY_NOT_FOUND); +} + +TEST(TxnKvTest, RangeGetIteratorContinue) { + // insert data + std::string prefix("range_get_iterator_continue"); + { + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + for (size_t i = 0; i < 1000; ++i) { + txn->put(fmt::format("{}-{:05}", prefix, i), std::to_string(i)); + } + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + } + + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + + std::unique_ptr it; + std::string end = prefix + "\xFF"; + ASSERT_EQ(txn->get(prefix, end, &it, true, 500), TxnErrorCode::TXN_OK); + + size_t i = 0; + while (true) { + while (it->has_next()) { + auto [k, v] = it->next(); + ASSERT_EQ(k, fmt::format("{}-{:05}", prefix, i)); + ASSERT_EQ(v, std::to_string(i)); + i += 1; + } + if (!it->more()) { + break; + } + std::string begin = it->next_begin_key(); + ASSERT_EQ(txn->get(begin, end, &it, true, 500), TxnErrorCode::TXN_OK); + } + ASSERT_EQ(i, 1000); +} + +TEST(TxnKvTest, RangeGetIteratorSeek) { + // insert data + std::string prefix("range_get_iterator_seek"); + { + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + for (size_t i = 0; i < 10; ++i) { + txn->put(fmt::format("{}-{:05}", prefix, i), std::to_string(i)); + } + ASSERT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + } + + // Seek to MID + { + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + + std::unique_ptr it; + std::string end = prefix + "\xFF"; + ASSERT_EQ(txn->get(prefix, end, &it, true, 500), TxnErrorCode::TXN_OK); + + it->seek(5); + std::vector values; + while (it->has_next()) { + auto [_, v] = it->next(); + values.push_back(v); + } + + std::vector expected_values {"5", "6", "7", "8", "9"}; + ASSERT_EQ(values, expected_values); + + // reset + it->reset(); + values.clear(); + while (it->has_next()) { + auto [_, v] = it->next(); + values.push_back(v); + } + expected_values = {"0", "1", "2", "3", "4", "5", "6", "7", "8", "9"}; + ASSERT_EQ(values, expected_values); + } + + // Seek out of range? + { + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + + std::unique_ptr it; + std::string end = prefix + "\xFF"; + ASSERT_EQ(txn->get(prefix, end, &it, true, 500), TxnErrorCode::TXN_OK); + + it->seek(10); + ASSERT_FALSE(it->has_next()); + } +} + +TEST(TxnKvTest, AbortTxn) { + std::unique_ptr txn; + ASSERT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + + txn->atomic_set_ver_key("prefix", "value"); + ASSERT_EQ(txn->abort(), TxnErrorCode::TXN_OK); +} + +TEST(TxnKvTest, RunInBthread) { + bthread_t tid; + auto thread = +[](void*) -> void* { + std::unique_ptr txn; + EXPECT_EQ(txn_kv->create_txn(&txn), TxnErrorCode::TXN_OK); + + std::string value; + EXPECT_EQ(txn->get("not_exists_key", &value), TxnErrorCode::TXN_KEY_NOT_FOUND); + txn->remove("not_exists_key"); + EXPECT_EQ(txn->commit(), TxnErrorCode::TXN_OK); + return nullptr; + }; + bthread_start_background(&tid, nullptr, thread, nullptr); + bthread_join(tid, nullptr); +} + +TEST(TxnKvTest, KvErrorCodeFormat) { + std::vector> codes { + {TxnErrorCode::TXN_OK, "Ok"}, + {TxnErrorCode::TXN_KEY_NOT_FOUND, "KeyNotFound"}, + {TxnErrorCode::TXN_CONFLICT, "Conflict"}, + {TxnErrorCode::TXN_TOO_OLD, "TxnTooOld"}, + {TxnErrorCode::TXN_MAYBE_COMMITTED, "MaybeCommitted"}, + {TxnErrorCode::TXN_RETRYABLE_NOT_COMMITTED, "RetryableNotCommitted"}, + {TxnErrorCode::TXN_TIMEOUT, "Timeout"}, + {TxnErrorCode::TXN_INVALID_ARGUMENT, "InvalidArgument"}, + {TxnErrorCode::TXN_UNIDENTIFIED_ERROR, "Unknown"}, + }; + for (auto&& [code, expect] : codes) { + std::string msg = fmt::format("{}", code); + ASSERT_EQ(msg, expect); + std::stringstream out; + out << code; + msg = out.str(); + ASSERT_EQ(msg, expect); + } +} + +TEST(TxnKvTest, BatchGet) { + std::vector keys; + std::vector values; + constexpr int nums = 100; + for (int i = 0; i < nums; ++i) { + keys.push_back("BatchGet_" + std::to_string(i)); + } + std::unique_ptr txn; + // put kv + { + auto ret = txn_kv->create_txn(&txn); + ASSERT_EQ(ret, TxnErrorCode::TXN_OK); + for (const auto& k : keys) { + txn->put(k, k); + values.push_back(k); + } + ret = txn->commit(); + ASSERT_EQ(ret, TxnErrorCode::TXN_OK); + } + // batch get + { + auto ret = txn_kv->create_txn(&txn); + ASSERT_EQ(ret, TxnErrorCode::TXN_OK); + std::vector> res; + ret = txn->batch_get(&res, keys); + ASSERT_EQ(ret, TxnErrorCode::TXN_OK); + ASSERT_EQ(res.size(), values.size()); + for (auto i = 0; i < res.size(); ++i) { + ASSERT_EQ(res[i].has_value(), true); + ASSERT_EQ(res[i].value(), values[i]); + } + } + + // batch get with no-exists keys + { + auto ret = txn_kv->create_txn(&txn); + ASSERT_EQ(ret, TxnErrorCode::TXN_OK); + std::vector> res; + std::vector keys {"BatchGet_empty1", "BatchGet_empty2", "BatchGet_empty3"}; + ret = txn->batch_get(&res, keys); + ASSERT_EQ(ret, TxnErrorCode::TXN_OK); + ASSERT_EQ(res.size(), keys.size()); + for (const auto& r : res) { + ASSERT_EQ(r.has_value(), false); + } + } +} diff --git a/gensrc/script/gen_build_version.sh b/gensrc/script/gen_build_version.sh index 53e7dcd5b9d30f..b1a1f394941a3e 100755 --- a/gensrc/script/gen_build_version.sh +++ b/gensrc/script/gen_build_version.sh @@ -50,8 +50,7 @@ if [[ -z "${DORIS_HOME}" ]]; then fi if [[ -z "${DORIS_TEST_BINARY_DIR}" ]]; then - if [[ -e "${DORIS_HOME}/fe/fe-core/target/generated-sources/build/org/apache/doris/common/Version.java" && - -e "${DORIS_HOME}/gensrc/build/gen_cpp/version.h" ]]; then + if [[ -e "${DORIS_HOME}/fe/fe-core/target/generated-sources/build/org/apache/doris/common/Version.java" && -e "${DORIS_HOME}/gensrc/build/gen_cpp/version.h" && -e "${DORIS_HOME}/gensrc/build/gen_cpp/cloud_version.h" ]]; then exit fi fi @@ -196,3 +195,64 @@ namespace doris { #endif EOF + +################################################################################ +# doris cloud version info +################################################################################ + +build_version_prefix="doris_cloud" +build_version_major=0 +build_version_minor=0 +build_version_patch=0 +build_version_rc_version="" + +if [[ -f /etc/os-release ]]; then + build_os_version=$(head -n2 "${GEN_CPP_DIR}/cloud_version.h" < +# Optional options: +# --clean clean and build ut +# --run build and run all ut +# --run --filter=xx build and run specified ut +# -j build parallel +# -h print this help message +# +# All BE tests must use "_test" as the file suffix, and add the file +# to cloud/test/CMakeLists.txt. +# +# GTest result xml files will be in "cloud/ut_build_ASAN/gtest_output/" +##################################################################### + +ROOT=$(dirname "$0") +ROOT=$( + cd "${ROOT}" + pwd +) + +export DORIS_HOME="${ROOT}" + +# Check args +usage() { + echo " +Usage: $0 + Optional options: + --clean clean and build ut + --run build and run all ut + --coverage coverage after run ut + --run --filter=xx build and run specified ut + -j build parallel + -h print this help message + + Eg. + $0 build tests + $0 --run build and run all tests + $0 --run --filter=* also runs everything + $0 --run --filter=FooTest.* runs everything in test suite FooTest + $0 --run --filter=*Null*:*Constructor* runs any test whose full name contains either 'Null' or 'Constructor' + $0 --run --filter=-*DeathTest.* runs all non-death tests + $0 --run --filter=FooTest.*-FooTest.Bar runs everything in test suite FooTest except FooTest.Bar + $0 --run --filter=FooTest.*:BarTest.*-FooTest.Bar:BarTest.Foo runs everything in test suite FooTest except FooTest.Bar and everything in test suite BarTest except BarTest.Foo + $0 --clean clean and build tests + $0 --clean --run clean, build and run all tests + " + exit 1 +} + +if ! OPTS=$(getopt -n "$0" -o vhj:f: -l run,clean,filter:,fdb:,coverage -- "$@"); then + usage +fi + +set -eo pipefail + +eval set -- "${OPTS}" + +PARALLEL=$(($(nproc) / 5 + 1)) + +CLEAN=0 +RUN=0 +FILTER="" +FDB="" +ENABLE_CLANG_COVERAGE=OFF +echo "===================== filter: ${FILTER}" +if [[ $# != 1 ]]; then + while true; do + case "$1" in + --clean) + CLEAN=1 + shift + ;; + --run) + RUN=1 + shift + ;; + --coverage) + ENABLE_CLANG_COVERAGE="ON" + shift + ;; + --fdb) + FDB="$2" + shift 2 + ;; + -f | --filter) + FILTER="$2" + shift 2 + ;; + -j) + PARALLEL=$2 + shift 2 + ;; + --) + shift + break + ;; + *) + usage + ;; + esac + done +fi + +echo "===================== filter: ${FILTER}" + +CMAKE_BUILD_TYPE=${BUILD_TYPE:-ASAN} +CMAKE_BUILD_TYPE="${CMAKE_BUILD_TYPE^^}" + +echo "Get params: + PARALLEL -- ${PARALLEL} + CLEAN -- ${CLEAN} +" +echo "Build SelectDB Cloud UT" + +if [[ "_${ENABLE_CLANG_COVERAGE}" == "_ON" ]]; then + sed -i "s/DORIS_TOOLCHAIN=gcc/DORIS_TOOLCHAIN=clang/g" env.sh + echo "export DORIS_TOOLCHAIN=clang" >>custom_env.sh +fi + +. "${DORIS_HOME}/env.sh" + +CMAKE_BUILD_DIR="${DORIS_HOME}/cloud/ut_build_${CMAKE_BUILD_TYPE}" +if [[ "${CLEAN}" -eq 1 ]]; then + rm "${CMAKE_BUILD_DIR}" -rf + rm "${DORIS_HOME}/cloud/output/" -rf +fi + +if [[ ! -d "${CMAKE_BUILD_DIR}" ]]; then + mkdir -p "${CMAKE_BUILD_DIR}" +fi + +if [[ -z "${GLIBC_COMPATIBILITY}" ]]; then + GLIBC_COMPATIBILITY=ON +fi + +if [[ -z "${USE_DWARF}" ]]; then + USE_DWARF=OFF +fi + +MAKE_PROGRAM="$(command -v "${BUILD_SYSTEM}")" +echo "-- Make program: ${MAKE_PROGRAM}" + +cd "${CMAKE_BUILD_DIR}" +find . -name "*.gcda" -exec rm {} \; + +"${CMAKE_CMD}" -G "${GENERATOR}" \ + -DCMAKE_MAKE_PROGRAM="${MAKE_PROGRAM}" \ + -DCMAKE_BUILD_TYPE="${CMAKE_BUILD_TYPE}" \ + -DCMAKE_EXPORT_COMPILE_COMMANDS=ON \ + -DMAKE_TEST=ON \ + -DGLIBC_COMPATIBILITY="${GLIBC_COMPATIBILITY}" \ + -DUSE_DWARF="${USE_DWARF}" \ + -DUSE_MEM_TRACKER=ON \ + -DUSE_JEMALLOC=OFF \ + -DSTRICT_MEMORY_USE=OFF \ + -DENABLE_CLANG_COVERAGE="${ENABLE_CLANG_COVERAGE}" \ + "${CMAKE_USE_CCACHE}" \ + "${DORIS_HOME}/cloud/" +"${BUILD_SYSTEM}" -j "${PARALLEL}" +"${BUILD_SYSTEM}" install + +mkdir -p "${CMAKE_BUILD_DIR}/test/log" + +if [[ "${RUN}" -ne 1 ]]; then + echo "Finished" + exit 0 +fi + +echo "**********************************" +echo " Running MetaService Unit Test " +echo "**********************************" + +# test binary output dir +cd test +# FILTER: binary_name:gtest_filter +# FILTER: meta_service_test:DetachSchemaKVTest.* +# ./run_all_tests.sh --test "\"$(echo "${FILTER}" | awk -F: '{print $1}')\"" --filter "\"$(echo "${FILTER}" | awk -F: '{print $2}')\"" --fdb "\"${FDB}\"" +if [[ "_${ENABLE_CLANG_COVERAGE}" == "_ON" ]]; then + bash -x ./run_all_tests.sh --coverage --test "$(echo "${FILTER}" | awk -F: '{print $1}')" --filter "$(echo "${FILTER}" | awk -F: '{print $2}')" --fdb "${FDB}" +else + bash ./run_all_tests.sh --test "$(echo "${FILTER}" | awk -F: '{print $1}')" --filter "$(echo "${FILTER}" | awk -F: '{print $2}')" --fdb "${FDB}" +fi diff --git a/thirdparty/build-thirdparty.sh b/thirdparty/build-thirdparty.sh index 68e29eac154c97..ba661f6390a085 100755 --- a/thirdparty/build-thirdparty.sh +++ b/thirdparty/build-thirdparty.sh @@ -1735,6 +1735,45 @@ build_streamvbyte() { "${BUILD_SYSTEM}" install } +# jsoncpp +build_jsoncpp() { + check_if_source_exist "${JSONCPP_SOURCE}" + cd "${TP_SOURCE_DIR}/${JSONCPP_SOURCE}" + rm -rf "${BUILD_DIR}" + mkdir -p "${BUILD_DIR}" + cd "${BUILD_DIR}" + "${CMAKE_CMD}" -G "${GENERATOR}" -DBUILD_STATIC_LIBS=ON -DBUILD_SHARED_LIBS=OFF -DCMAKE_BUILD_TYPE=Release -DCMAKE_INSTALL_PREFIX="${TP_INSTALL_DIR}" .. + "${BUILD_SYSTEM}" -j "${PARALLEL}" + "${BUILD_SYSTEM}" install +} + +# libuuid +build_libuuid() { + check_if_source_exist "${LIBUUID_SOURCE}" + cd "${TP_SOURCE_DIR}/${LIBUUID_SOURCE}" + CC=gcc ./configure --prefix="${TP_INSTALL_DIR}" --disable-shared --enable-static + make -j "${PARALLEL}" + make install +} + +# ali_sdk +build_ali_sdk() { + build_jsoncpp + build_libuuid + check_if_source_exist "${ALI_SDK_SOURCE}" + cd "${TP_SOURCE_DIR}/${ALI_SDK_SOURCE}" + rm -rf "${BUILD_DIR}" + mkdir -p "${BUILD_DIR}" + cd "${BUILD_DIR}" + + CPPFLAGS="-I${TP_INCLUDE_DIR}" \ + CXXFLAGS="-I${TP_INCLUDE_DIR}" \ + LDFLAGS="-L${TP_LIB_DIR}" \ + "${CMAKE_CMD}" -G "${GENERATOR}" -DBUILD_SHARED_LIBS=OFF -DBUILD_PRODUCT=core -DCMAKE_BUILD_TYPE=Release -DCMAKE_INSTALL_PREFIX="${TP_INSTALL_DIR}" .. + "${BUILD_SYSTEM}" -j "${PARALLEL}" + "${BUILD_SYSTEM}" install +} + if [[ "${#packages[@]}" -eq 0 ]]; then packages=( libunixodbc @@ -1800,6 +1839,7 @@ if [[ "${#packages[@]}" -eq 0 ]]; then avx2neon libdeflate streamvbyte + ali_sdk ) if [[ "$(uname -s)" == 'Darwin' ]]; then read -r -a packages <<<"binutils gettext ${packages[*]}" diff --git a/thirdparty/vars.sh b/thirdparty/vars.sh index ad127fbd6a254a..2c815949a1b40f 100644 --- a/thirdparty/vars.sh +++ b/thirdparty/vars.sh @@ -490,6 +490,24 @@ STREAMVBYTE_NAME=streamvbyte-1.0.0.tar.gz STREAMVBYTE_SOURCE=streamvbyte-1.0.0 STREAMVBYTE_MD5SUM="f334219db5a832b6dae3589a56a29563" +# jsoncpp +JSONCPP_DOWNLOAD="https://github.com/open-source-parsers/jsoncpp/archive/refs/tags/1.9.5.tar.gz" +JSONCPP_NAME="1.9.5.tar.gz" +JSONCPP_SOURCE="jsoncpp-1.9.5" +JSONCPP_MD5SUM="d6c8c609f2162eff373db62b90a051c7" + +# libuuid +LIBUUID_DOWNLOAD="https://nchc.dl.sourceforge.net/project/libuuid/libuuid-1.0.3.tar.gz" +LIBUUID_NAME="libuuid-1.0.3.tar.gz" +LIBUUID_SOURCE="libuuid-1.0.3" +LIBUUID_MD5SUM="d44d866d06286c08ba0846aba1086d68" + +# ali +ALI_SDK_DOWNLOAD="https://github.com/aliyun/aliyun-openapi-cpp-sdk/archive/refs/tags/1.36.1586.tar.gz" +ALI_SDK_NAME="1.36.1586.tar.gz" +ALI_SDK_SOURCE="aliyun-openapi-cpp-sdk-1.36.1586" +ALI_SDK_MD5SUM="14623ee8e87c89615477d9bf8cb30bea" + # all thirdparties which need to be downloaded is set in array TP_ARCHIVES export TP_ARCHIVES=( 'LIBEVENT' @@ -562,6 +580,9 @@ export TP_ARCHIVES=( 'AVX2NEON' 'LIBDEFLATE' 'STREAMVBYTE' + 'JSONCPP' + 'LIBUUID' + 'ALI_SDK' ) if [[ "$(uname -s)" == 'Darwin' ]]; then From 389dde16d38968d249ba4ed08fdb5a95bbc16a78 Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Thu, 18 Jan 2024 14:25:34 +0800 Subject: [PATCH 081/200] [fix](Nereids) update assignment column name should case insensitive (#30071) --- .../nereids/rules/analysis/SlotBinder.java | 10 +++--- .../trees/plans/commands/UpdateCommand.java | 33 ++++++++++++++++++- .../update/update_unique_table.groovy | 14 +++++++- 3 files changed, 50 insertions(+), 7 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SlotBinder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SlotBinder.java index 6f5f11b0a77ff9..e25aa20262738c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SlotBinder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SlotBinder.java @@ -262,10 +262,6 @@ && compareDbName(qualifierStar.get(1), boundSlotQualifier.get(1)) return new BoundStar(slots); } - private boolean compareDbName(String unBoundDbName, String boundedDbName) { - return unBoundDbName.equalsIgnoreCase(boundedDbName); - } - private List bindSlot(UnboundSlot unboundSlot, List boundSlots) { return boundSlots.stream().distinct().filter(boundSlot -> { List nameParts = unboundSlot.getNameParts(); @@ -305,7 +301,11 @@ && sameTableName(qualifierTableName, nameParts.get(2)) }).collect(Collectors.toList()); } - private boolean sameTableName(String boundSlot, String unboundSlot) { + public static boolean compareDbName(String boundedDbName, String unBoundDbName) { + return unBoundDbName.equalsIgnoreCase(boundedDbName); + } + + public static boolean sameTableName(String boundSlot, String unboundSlot) { if (GlobalVariable.lowerCaseTableNames != 1) { return boundSlot.equals(unboundSlot); } else { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java index 6949b124e5dfe4..0023b4bd59c168 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java @@ -27,6 +27,7 @@ import org.apache.doris.nereids.analyzer.UnboundTableSink; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.parser.NereidsParser; +import org.apache.doris.nereids.rules.analysis.SlotBinder; import org.apache.doris.nereids.trees.expressions.EqualTo; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; @@ -101,9 +102,10 @@ public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { public LogicalPlan completeQueryPlan(ConnectContext ctx, LogicalPlan logicalQuery) { checkTable(ctx); - Map colNameToExpression = Maps.newHashMap(); + Map colNameToExpression = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); for (EqualTo equalTo : assignments) { List nameParts = ((UnboundSlot) equalTo.left()).getNameParts(); + checkAssignmentColumn(ctx, nameParts); colNameToExpression.put(nameParts.get(nameParts.size() - 1), equalTo.right()); } List selectItems = Lists.newArrayList(); @@ -118,6 +120,7 @@ public LogicalPlan completeQueryPlan(ConnectContext ctx, LogicalPlan logicalQuer selectItems.add(expr instanceof UnboundSlot ? ((NamedExpression) expr) : new UnboundAlias(expr)); + colNameToExpression.remove(column.getName()); } else { if (column.hasOnUpdateDefaultValue()) { Expression defualtValueExpression = @@ -129,6 +132,10 @@ public LogicalPlan completeQueryPlan(ConnectContext ctx, LogicalPlan logicalQuer } } } + if (!colNameToExpression.isEmpty()) { + throw new AnalysisException("unknown column in assignment list: " + + String.join(", ", colNameToExpression.keySet())); + } logicalQuery = new LogicalProject<>(selectItems, logicalQuery); if (cte.isPresent()) { @@ -143,6 +150,30 @@ public LogicalPlan completeQueryPlan(ConnectContext ctx, LogicalPlan logicalQuer false, ImmutableList.of(), isPartialUpdate, DMLCommandType.UPDATE, logicalQuery); } + private void checkAssignmentColumn(ConnectContext ctx, List columnNameParts) { + if (columnNameParts.size() <= 1) { + return; + } + String dbName = null; + String tableName = null; + if (columnNameParts.size() == 3) { + dbName = columnNameParts.get(0); + tableName = columnNameParts.get(1); + } else if (columnNameParts.size() == 2) { + tableName = columnNameParts.get(0); + } else { + throw new AnalysisException("column in assignment list is invalid, " + String.join(".", columnNameParts)); + } + if (dbName != null && this.tableAlias != null) { + throw new AnalysisException("column in assignment list is invalid, " + String.join(".", columnNameParts)); + } + List tableQualifier = RelationUtil.getQualifierName(ctx, nameParts); + if (!SlotBinder.sameTableName(tableAlias == null ? tableQualifier.get(2) : tableAlias, tableName) + || (dbName != null && SlotBinder.compareDbName(tableQualifier.get(1), dbName))) { + throw new AnalysisException("column in assignment list is invalid, " + String.join(".", columnNameParts)); + } + } + private void checkTable(ConnectContext ctx) { if (ctx.getSessionVariable().isInDebugMode()) { throw new AnalysisException("Update is forbidden since current session is in debug mode." diff --git a/regression-test/suites/nereids_p0/update/update_unique_table.groovy b/regression-test/suites/nereids_p0/update/update_unique_table.groovy index 59ea06b10b87d9..8689f16f9e815e 100644 --- a/regression-test/suites/nereids_p0/update/update_unique_table.groovy +++ b/regression-test/suites/nereids_p0/update/update_unique_table.groovy @@ -95,10 +95,22 @@ suite('update_unique_table') { sql ''' update t1 - set t1.c1 = t2.c1, t1.c3 = t2.c3 * 100 + set t1.C1 = t2.c1, t1.c3 = t2.c3 * 100 from t2 inner join t3 on t2.id = t3.id where t1.id = t2.id; ''' qt_sql 'select * from t1 order by id' + + test { + sql '''update t1 set t.c1 = 1 where t1.c1 = 1;''' + exception "" + } + + test { + sql '''update t1 t set t1.c1 = 1 where t1.c1 = 1;''' + exception "" + } + + } From d138b523f8da74ce710d5dec3c6beb4b33670f13 Mon Sep 17 00:00:00 2001 From: deardeng <565620795@qq.com> Date: Thu, 18 Jan 2024 14:46:55 +0800 Subject: [PATCH 082/200] =?UTF-8?q?[improvement](create=20tablet)=20backen?= =?UTF-8?q?d=20create=20tablet=20round=20robin=20among=20=E2=80=A6=20(#298?= =?UTF-8?q?18)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- be/src/agent/task_worker_pool.cpp | 2 +- be/src/common/config.cpp | 3 + be/src/common/config.h | 3 + be/src/olap/storage_engine.cpp | 135 +++++++++++---- be/src/olap/storage_engine.h | 89 ++++++---- be/src/olap/task/engine_clone_task.cpp | 2 +- be/src/runtime/memory/cache_policy.h | 3 + .../test_partition_create_tablet_rr.groovy | 162 ++++++++++++++++++ 8 files changed, 331 insertions(+), 68 deletions(-) create mode 100644 regression-test/suites/partition_p0/test_partition_create_tablet_rr.groovy diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp index f40fe73758f253..7d11b6c75e0763 100644 --- a/be/src/agent/task_worker_pool.cpp +++ b/be/src/agent/task_worker_pool.cpp @@ -257,7 +257,7 @@ Status check_migrate_request(StorageEngine& engine, const TStorageMediumMigrateR storage_medium); } // get a random store of specified storage medium - auto stores = engine.get_stores_for_create_tablet(storage_medium); + auto stores = engine.get_stores_for_create_tablet(tablet->partition_id(), storage_medium); if (stores.empty()) { return Status::InternalError("failed to get root path for create tablet"); } diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 8b7edb860655c6..04e537cd3241bc 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -1166,6 +1166,9 @@ DEFINE_mInt32(report_query_statistics_interval_ms, "3000"); // 30s DEFINE_mInt32(query_statistics_reserve_timeout_ms, "30000"); +// create tablet in partition random robin idx lru size, default 10000 +DEFINE_Int32(partition_disk_index_lru_size, "10000"); + // clang-format off #ifdef BE_TEST // test s3 diff --git a/be/src/common/config.h b/be/src/common/config.h index 6d5b54a270a32b..661192c1178700 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -1242,6 +1242,9 @@ DECLARE_Int32(ignore_invalid_partition_id_rowset_num); DECLARE_mInt32(report_query_statistics_interval_ms); DECLARE_mInt32(query_statistics_reserve_timeout_ms); +// create tablet in partition random robin idx lru size, default 10000 +DECLARE_Int32(partition_disk_index_lru_size); + #ifdef BE_TEST // test s3 DECLARE_String(test_s3_resource); diff --git a/be/src/olap/storage_engine.cpp b/be/src/olap/storage_engine.cpp index be987de32712a5..82ab02ea39e76b 100644 --- a/be/src/olap/storage_engine.cpp +++ b/be/src/olap/storage_engine.cpp @@ -18,6 +18,7 @@ #include "olap/storage_engine.h" // IWYU pragma: no_include +#include #include // IWYU pragma: keep #include #include @@ -95,7 +96,8 @@ using std::vector; namespace doris { using namespace ErrorCode; - +extern void get_round_robin_stores(int64 curr_index, const std::vector& dir_infos, + std::vector& stores); DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(unused_rowsets_count, MetricUnit::ROWSETS); static Status _validate_options(const EngineOptions& options) { @@ -129,7 +131,10 @@ StorageEngine::StorageEngine(const EngineOptions& options) _memtable_flush_executor(nullptr), _calc_delete_bitmap_executor(nullptr), _default_rowset_type(BETA_ROWSET), - _stream_load_recorder(nullptr) { + _heartbeat_flags(nullptr), + _stream_load_recorder(nullptr), + _create_tablet_idx_lru_cache( + new CreateTabletIdxCache(config::partition_disk_index_lru_size)) { REGISTER_HOOK_METRIC(unused_rowsets_count, [this]() { // std::lock_guard lock(_gc_mutex); return _unused_rowsets.size(); @@ -430,46 +435,80 @@ Status StorageEngine::set_cluster_id(int32_t cluster_id) { return Status::OK(); } +StorageEngine::DiskRemainingLevel get_available_level(double disk_usage_percent) { + assert(disk_usage_percent <= 1); + if (disk_usage_percent < 0.7) { + return StorageEngine::DiskRemainingLevel::LOW; + } else if (disk_usage_percent < 0.85) { + return StorageEngine::DiskRemainingLevel::MID; + } + return StorageEngine::DiskRemainingLevel::HIGH; +} + +int StorageEngine::_get_and_set_next_disk_index(int64 partition_id, + TStorageMedium::type storage_medium) { + auto key = CreateTabletIdxCache::get_key(partition_id, storage_medium); + int curr_index = _create_tablet_idx_lru_cache->get_index(key); + // -1, lru can't find key + if (curr_index == -1) { + curr_index = std::max(0, _last_use_index[storage_medium] + 1); + } + _last_use_index[storage_medium] = curr_index; + _create_tablet_idx_lru_cache->set_index(key, std::max(0, curr_index + 1)); + return curr_index; +} + +void StorageEngine::_get_candidate_stores(TStorageMedium::type storage_medium, + std::vector& dir_infos) { + for (auto& it : _store_map) { + DataDir* data_dir = it.second.get(); + if (data_dir->is_used()) { + if ((_available_storage_medium_type_count == 1 || + data_dir->storage_medium() == storage_medium) && + !data_dir->reach_capacity_limit(0)) { + DirInfo dir_info; + dir_info.data_dir = data_dir; + dir_info.available_level = get_available_level(data_dir->get_usage(0)); + dir_infos.push_back(dir_info); + } + } + } +} + std::vector StorageEngine::get_stores_for_create_tablet( - TStorageMedium::type storage_medium) { + int64 partition_id, TStorageMedium::type storage_medium) { + std::vector dir_infos; + int curr_index = 0; std::vector stores; { std::lock_guard l(_store_lock); - for (auto&& [_, store] : _store_map) { - if (store->is_used()) { - if ((_available_storage_medium_type_count == 1 || - store->storage_medium() == storage_medium) && - !store->reach_capacity_limit(0)) { - stores.push_back(store.get()); - } - } - } + curr_index = _get_and_set_next_disk_index(partition_id, storage_medium); + _get_candidate_stores(storage_medium, dir_infos); } - std::sort(stores.begin(), stores.end(), - [](DataDir* a, DataDir* b) { return a->get_usage(0) < b->get_usage(0); }); + std::sort(dir_infos.begin(), dir_infos.end()); + get_round_robin_stores(curr_index, dir_infos, stores); - size_t seventy_percent_index = stores.size(); - size_t eighty_five_percent_index = stores.size(); - for (size_t index = 0; index < stores.size(); index++) { - // If the usage of the store is less than 70%, we choose disk randomly. - if (stores[index]->get_usage(0) > 0.7 && seventy_percent_index == stores.size()) { - seventy_percent_index = index; + return stores; +} + +// maintain in stores LOW,MID,HIGH level round robin +void get_round_robin_stores(int64 curr_index, const std::vector& dir_infos, + std::vector& stores) { + for (size_t i = 0; i < dir_infos.size();) { + size_t end = i + 1; + while (end < dir_infos.size() && + dir_infos[i].available_level == dir_infos[end].available_level) { + end++; } - if (stores[index]->get_usage(0) > 0.85 && eighty_five_percent_index == stores.size()) { - eighty_five_percent_index = index; - break; + // data dirs [i, end) have the same tablet size, round robin range [i, end) + size_t count = end - i; + for (size_t k = 0; k < count; k++) { + size_t index = i + (k + curr_index) % count; + stores.push_back(dir_infos[index].data_dir); } + i = end; } - - std::random_device rd; - std::mt19937 g(rd()); - std::shuffle(stores.begin(), stores.begin() + seventy_percent_index, g); - std::shuffle(stores.begin() + seventy_percent_index, stores.begin() + eighty_five_percent_index, - g); - std::shuffle(stores.begin() + eighty_five_percent_index, stores.end(), g); - - return stores; } DataDir* StorageEngine::get_store(const std::string& path) { @@ -1034,7 +1073,7 @@ Status StorageEngine::create_tablet(const TCreateTabletReq& request, RuntimeProf std::vector stores; { SCOPED_TIMER(ADD_TIMER(profile, "GetStores")); - stores = get_stores_for_create_tablet(request.storage_medium); + stores = get_stores_for_create_tablet(request.partition_id, request.storage_medium); } if (stores.empty()) { return Status::Error( @@ -1044,7 +1083,8 @@ Status StorageEngine::create_tablet(const TCreateTabletReq& request, RuntimeProf } Status StorageEngine::obtain_shard_path(TStorageMedium::type storage_medium, int64_t path_hash, - std::string* shard_path, DataDir** store) { + std::string* shard_path, DataDir** store, + int64_t partition_id) { LOG(INFO) << "begin to process obtain root path. storage_medium=" << storage_medium; if (shard_path == nullptr) { @@ -1052,7 +1092,7 @@ Status StorageEngine::obtain_shard_path(TStorageMedium::type storage_medium, int "invalid output parameter which is null pointer."); } - auto stores = get_stores_for_create_tablet(storage_medium); + auto stores = get_stores_for_create_tablet(partition_id, storage_medium); if (stores.empty()) { return Status::Error( "no available disk can be used to create tablet."); @@ -1343,4 +1383,29 @@ void StorageEngine::_decrease_low_priority_task_nums(DataDir* dir) { } } +int CreateTabletIdxCache::get_index(const std::string& key) { + auto lru_handle = cache()->lookup(key); + if (lru_handle) { + Defer release([cache = cache(), lru_handle] { cache->release(lru_handle); }); + auto value = (CacheValue*)cache()->value(lru_handle); + value->last_visit_time = UnixMillis(); + VLOG_DEBUG << "use create tablet idx cache key=" << key << " value=" << value->idx; + return value->idx; + } + return -1; +} + +void CreateTabletIdxCache::set_index(const std::string& key, int next_idx) { + assert(next_idx >= 0); + CacheValue* value = new CacheValue; + value->last_visit_time = UnixMillis(); + value->idx = next_idx; + auto deleter = [](const doris::CacheKey& key, void* value) { + CacheValue* cache_value = (CacheValue*)value; + delete cache_value; + }; + auto lru_handle = cache()->insert(key, value, 1, deleter, CachePriority::NORMAL, sizeof(int)); + cache()->release(lru_handle); +} + } // namespace doris diff --git a/be/src/olap/storage_engine.h b/be/src/olap/storage_engine.h index eca5212b5aa1e7..002d7d67159854 100644 --- a/be/src/olap/storage_engine.h +++ b/be/src/olap/storage_engine.h @@ -70,6 +70,8 @@ class Thread; class ThreadPool; class TxnManager; class ReportWorker; +class CreateTabletIdxCache; +struct DirInfo; using SegCompactionCandidates = std::vector; using SegCompactionCandidatesSharedPtr = std::shared_ptr; @@ -83,6 +85,8 @@ class StorageEngine { StorageEngine(const EngineOptions& options); ~StorageEngine(); + enum class DiskRemainingLevel { LOW, MID, HIGH }; + [[nodiscard]] Status open(); static StorageEngine* instance() { return ExecEnv::GetInstance()->get_storage_engine(); } @@ -104,9 +108,11 @@ class StorageEngine { int64_t get_file_or_directory_size(const std::string& file_path); - // get root path for creating tablet. The returned vector of root path should be random, + // get root path for creating tablet. The returned vector of root path should be round robin, // for avoiding that all the tablet would be deployed one disk. - std::vector get_stores_for_create_tablet(TStorageMedium::type storage_medium); + std::vector get_stores_for_create_tablet(int64 partition_id, + TStorageMedium::type storage_medium); + DataDir* get_store(const std::string& path); uint32_t available_storage_medium_type_count() const { @@ -124,7 +130,7 @@ class StorageEngine { // @param [out] shard_path choose an available root_path to clone new tablet // @return error code Status obtain_shard_path(TStorageMedium::type storage_medium, int64_t path_hash, - std::string* shared_path, DataDir** store); + std::string* shared_path, DataDir** store, int64_t partition_id); // Load new tablet to make it effective. // @@ -328,36 +334,12 @@ class StorageEngine { void _decrease_low_priority_task_nums(DataDir* dir); -private: - struct CompactionCandidate { - CompactionCandidate(uint32_t nicumulative_compaction_, int64_t tablet_id_, uint32_t index_) - : nice(nicumulative_compaction_), tablet_id(tablet_id_), disk_index(index_) {} - uint32_t nice; // priority - int64_t tablet_id; - uint32_t disk_index = -1; - }; - - // In descending order - struct CompactionCandidateComparator { - bool operator()(const CompactionCandidate& a, const CompactionCandidate& b) { - return a.nice > b.nice; - } - }; + void _get_candidate_stores(TStorageMedium::type storage_medium, + std::vector& dir_infos); - struct CompactionDiskStat { - CompactionDiskStat(std::string path, uint32_t index, bool used) - : storage_path(path), - disk_index(index), - task_running(0), - task_remaining(0), - is_used(used) {} - const std::string storage_path; - const uint32_t disk_index; - uint32_t task_running; - uint32_t task_remaining; - bool is_used; - }; + int _get_and_set_next_disk_index(int64 partition_id, TStorageMedium::type storage_medium); +private: EngineOptions _options; std::mutex _store_lock; std::mutex _trash_sweep_lock; @@ -488,6 +470,51 @@ class StorageEngine { bool _clear_segment_cache = false; std::atomic _need_clean_trash {false}; + + // next index for create tablet + std::map _last_use_index; + + std::unique_ptr _create_tablet_idx_lru_cache; + + DISALLOW_COPY_AND_ASSIGN(StorageEngine); +}; + +// lru cache for create tabelt round robin in disks +// key: partitionId_medium +// value: index +class CreateTabletIdxCache : public LRUCachePolicy { +public: + // get key, delimiter with DELIMITER '-' + static std::string get_key(int64_t partition_id, TStorageMedium::type medium) { + return fmt::format("{}-{}", partition_id, medium); + } + + // -1 not found key in lru + int get_index(const std::string& key); + + void set_index(const std::string& key, int next_idx); + + struct CacheValue : public LRUCacheValueBase { + int idx = 0; + }; + + CreateTabletIdxCache(size_t capacity) + : LRUCachePolicy(CachePolicy::CacheType::CREATE_TABLET_RR_IDX_CACHE, capacity, + LRUCacheType::NUMBER, + /*stale_sweep_time_s*/ 30 * 60) {} +}; + +struct DirInfo { + DataDir* data_dir; + + StorageEngine::DiskRemainingLevel available_level; + + bool operator<(const DirInfo& other) const { + if (available_level != other.available_level) { + return available_level < other.available_level; + } + return data_dir->path_hash() < other.data_dir->path_hash(); + } }; } // namespace doris diff --git a/be/src/olap/task/engine_clone_task.cpp b/be/src/olap/task/engine_clone_task.cpp index 75509bb2635e80..d8c7a54cb74658 100644 --- a/be/src/olap/task/engine_clone_task.cpp +++ b/be/src/olap/task/engine_clone_task.cpp @@ -254,7 +254,7 @@ Status EngineCloneTask::_do_clone() { DataDir* store = nullptr; RETURN_IF_ERROR(StorageEngine::instance()->obtain_shard_path( _clone_req.storage_medium, _clone_req.dest_path_hash, &local_shard_root_path, - &store)); + &store, _clone_req.partition_id)); auto tablet_dir = fmt::format("{}/{}/{}", local_shard_root_path, _clone_req.tablet_id, _clone_req.schema_hash); diff --git a/be/src/runtime/memory/cache_policy.h b/be/src/runtime/memory/cache_policy.h index e965802ed2b81c..9a9f2c36e84d35 100644 --- a/be/src/runtime/memory/cache_policy.h +++ b/be/src/runtime/memory/cache_policy.h @@ -42,6 +42,7 @@ class CachePolicy { COMMON_OBJ_LRU_CACHE = 12, FOR_UT = 13, TABLET_SCHEMA_CACHE = 14, + CREATE_TABLET_RR_IDX_CACHE = 15 }; static std::string type_string(CacheType type) { @@ -76,6 +77,8 @@ class CachePolicy { return "ForUT"; case CacheType::TABLET_SCHEMA_CACHE: return "TabletSchemaCache"; + case CacheType::CREATE_TABLET_RR_IDX_CACHE: + return "CreateTabletRRIdxCache"; default: LOG(FATAL) << "not match type of cache policy :" << static_cast(type); } diff --git a/regression-test/suites/partition_p0/test_partition_create_tablet_rr.groovy b/regression-test/suites/partition_p0/test_partition_create_tablet_rr.groovy new file mode 100644 index 00000000000000..f7e77f06f38c23 --- /dev/null +++ b/regression-test/suites/partition_p0/test_partition_create_tablet_rr.groovy @@ -0,0 +1,162 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. +import org.apache.doris.regression.suite.ClusterOptions +import org.apache.doris.regression.util.NodeType +import org.apache.doris.regression.suite.SuiteCluster + +suite("test_partition_create_tablet_rr") { + def options = new ClusterOptions() + options.beNum = 1 + options.feConfigs.add('disable_balance=true') + def partition_disk_index_lru_size = 50 + options.beConfigs += [ + 'report_tablet_interval_seconds=1', + 'report_disk_state_interval_seconds=1', + "partition_disk_index_lru_size=$partition_disk_index_lru_size" + ] + options.beDisks = ['HDD=4','SSD=4'] + options.enableDebugPoints() + + def checkTabletOnDiskTabletNumEq = {tbl -> + sleep 5000 + + def tablets = sql_return_maparray "SHOW TABLETS FROM $tbl" + def pathTabletNum = [:] + tablets.each { + def num = pathTabletNum.get(it.PathHash) + if (num) { + pathTabletNum.put(it.PathHash, ++num) + } else { + pathTabletNum.put(it.PathHash, 1) + } + } + + log.info("table ${tbl} tablet in path ${pathTabletNum.values()}") + def count = pathTabletNum.values().stream().distinct().count() + assertEquals(count, 1) + } + + docker(options) { + sleep 2000 + def single_hdd_tbl = "single_HDD_tbl" + def single_ssd_tbl = "single_SDD_tbl" + def single_partition_tbl = "single_partition_tbl" + sql """drop table if exists $single_hdd_tbl""" + sql """drop table if exists $single_ssd_tbl""" + sql """drop table if exists $single_partition_tbl""" + for (def j = 0; j < partition_disk_index_lru_size + 10; j++) { + def tbl = single_partition_tbl + j.toString() + sql """drop table if exists $tbl""" + } + try { + // 1. test single partition table + // a. create 1 table, has 100 buckets + // b. check disk's tablet num + + sql """ + CREATE TABLE $single_hdd_tbl ( + `k1` int(11) NULL, + `k2` int(11) NULL + ) DUPLICATE KEY(`k1`, `k2`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 12000 + PROPERTIES ( + "replication_num"="1", + "storage_medium" = "HDD" + ); + """ + + checkTabletOnDiskTabletNumEq single_hdd_tbl + + sql """ + CREATE TABLE $single_ssd_tbl ( + `k1` int(11) NULL, + `k2` int(11) NULL + ) DUPLICATE KEY(`k1`, `k2`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 12000 + PROPERTIES ( + "replication_num"="1", + "storage_medium" = "SSD" + ); + """ + checkTabletOnDiskTabletNumEq single_ssd_tbl + + sql """ + CREATE TABLE $single_partition_tbl + ( + k1 DATE, + k2 DECIMAL(10, 2) DEFAULT "10.5", + k3 CHAR(10) COMMENT "string column", + k4 INT NOT NULL DEFAULT "1" COMMENT "int column" + ) + DUPLICATE KEY(k1, k2) + PARTITION BY RANGE(k1) + ( + PARTITION p1 VALUES LESS THAN ("2020-02-01"), + PARTITION p2 VALUES LESS THAN ("2020-03-01"), + PARTITION p3 VALUES LESS THAN ("2020-04-01") + ) + DISTRIBUTED BY HASH(k1) BUCKETS 320 + PROPERTIES ( + "replication_num" = "1" + ); + """ + checkTabletOnDiskTabletNumEq single_partition_tbl + + // 2. test multi thread create single partition tables + // a. multi thread create partition_disk_index_lru_size + 10 table + // b. check disk's tablet num + def futures = [] + for (def i = 0; i < partition_disk_index_lru_size + 10; i++) { + def tblMulti = single_partition_tbl + i.toString() + futures.add(thread { + sql """ + CREATE TABLE $tblMulti + ( + k1 DATE, + k2 DECIMAL(10, 2) DEFAULT "10.5", + k3 CHAR(10) COMMENT "string column", + k4 INT NOT NULL DEFAULT "1" COMMENT "int column" + ) + DUPLICATE KEY(k1, k2) + PARTITION BY RANGE(k1) + ( + PARTITION p1 VALUES LESS THAN ("2020-02-01"), + PARTITION p2 VALUES LESS THAN ("2020-03-01"), + PARTITION p3 VALUES LESS THAN ("2020-04-01") + ) + DISTRIBUTED BY HASH(k1) BUCKETS 320 + PROPERTIES ( + "replication_num" = "1" + ); + """ + checkTabletOnDiskTabletNumEq tblMulti + }) + } + futures.each { it.get() } + } finally { + sql """drop table if exists $single_hdd_tbl""" + sql """drop table if exists $single_ssd_tbl""" + sql """drop table if exists $single_partition_tbl""" + for (def j = 0; j < partition_disk_index_lru_size + 10; j++) { + def tbl = single_partition_tbl + j.toString() + sql """drop table if exists $tbl""" + } + } + } +} From 0c4e83f5fd36144aeb4475ea8ffe63b92c40016c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=A2=E5=81=A5?= Date: Thu, 18 Jan 2024 15:55:34 +0800 Subject: [PATCH 083/200] [fix](Nereids): eliminate redundant join condition after inferring condition (#30093) eliminate redundant join when find hashing join condition such as for plan: ``` T1 join T2 on T1.id = T2.id join T3 on T1.id = T3.id and T2.id = T3.id ``` we infer a new predicate T1.id = T2.id which is redundant. Therefore we need to eliminate it when find hash condition --- .../nereids/rules/rewrite/FindHashConditionForJoin.java | 9 +++++---- .../jobs/joinorder/hypergraph/CompareOuterJoinTest.java | 5 ++++- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/FindHashConditionForJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/FindHashConditionForJoin.java index fe51a7d830b075..aeeeefb593bcb2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/FindHashConditionForJoin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/FindHashConditionForJoin.java @@ -28,6 +28,7 @@ import org.apache.doris.nereids.util.JoinUtils; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Streams; import java.util.List; @@ -61,10 +62,10 @@ public Rule build() { return join; } - List combinedHashJoinConjuncts = new ImmutableList.Builder() - .addAll(join.getHashJoinConjuncts()) - .addAll(extractedHashJoinConjuncts) - .build(); + List combinedHashJoinConjuncts = Streams + .concat(join.getHashJoinConjuncts().stream(), extractedHashJoinConjuncts.stream()) + .distinct() + .collect(ImmutableList.toImmutableList()); JoinType joinType = join.getJoinType(); if (joinType == JoinType.CROSS_JOIN && !combinedHashJoinConjuncts.isEmpty()) { joinType = JoinType.INNER_JOIN; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/CompareOuterJoinTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/CompareOuterJoinTest.java index f73d8a2923e67f..bbf7746ec6a57d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/CompareOuterJoinTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/CompareOuterJoinTest.java @@ -28,10 +28,12 @@ import org.apache.doris.nereids.rules.exploration.mv.mapping.SlotMapping; import org.apache.doris.nereids.sqltest.SqlTestBase; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.plans.JoinType; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.util.HyperGraphBuilder; import org.apache.doris.nereids.util.PlanChecker; +import com.google.common.collect.Sets; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -69,7 +71,8 @@ void testStarGraphWithInnerJoin() { @Test void testRandomQuery() { - Plan p1 = new HyperGraphBuilder().randomBuildPlanWith(3, 3); + Plan p1 = new HyperGraphBuilder(Sets.newHashSet(JoinType.INNER_JOIN)) + .randomBuildPlanWith(3, 3); p1 = PlanChecker.from(connectContext, p1) .analyze() .rewrite() From 110b55a2091dab39acc2d11cb02004873b2cf3d6 Mon Sep 17 00:00:00 2001 From: Pxl Date: Thu, 18 Jan 2024 16:47:03 +0800 Subject: [PATCH 084/200] [Improvement](aggregate) optimize for small string aggregate (#29919) --- .../exec/aggregation_sink_operator.cpp | 4 +- .../exec/aggregation_source_operator.cpp | 9 +- .../exec/partition_sort_sink_operator.cpp | 4 +- be/src/vec/common/hash_table/hash_map.h | 6 - .../vec/common/hash_table/hash_map_context.h | 20 +- be/src/vec/common/hash_table/hash_table.h | 28 --- .../hash_table/partitioned_hash_table.h | 11 - be/src/vec/common/hash_table/ph_hash_map.h | 8 - .../vec/common/hash_table/string_hash_map.h | 66 ++---- .../vec/common/hash_table/string_hash_table.h | 219 +++++++++--------- be/src/vec/common/memcpy_small.h | 32 +++ .../vec/exec/distinct_vaggregation_node.cpp | 4 +- be/src/vec/exec/vaggregation_node.cpp | 30 +-- be/src/vec/exec/vpartition_sort_node.cpp | 2 +- 14 files changed, 202 insertions(+), 241 deletions(-) diff --git a/be/src/pipeline/exec/aggregation_sink_operator.cpp b/be/src/pipeline/exec/aggregation_sink_operator.cpp index 3ba4dd05dc7d8f..8d3a70a3a3a99d 100644 --- a/be/src/pipeline/exec/aggregation_sink_operator.cpp +++ b/be/src/pipeline/exec/aggregation_sink_operator.cpp @@ -536,7 +536,7 @@ void AggSinkLocalState::_emplace_into_hash_table( agg_method.init_serialized_keys(key_columns, num_rows); auto creator = [this](const auto& ctor, auto& key, auto& origin) { - HashMethodType::try_presis_key(key, origin, *_agg_arena_pool); + HashMethodType::try_presis_key_and_origin(key, origin, *_agg_arena_pool); auto mapped = Base::_shared_state->aggregate_data_container->append_data(origin); auto st = _create_agg_status(mapped); @@ -686,7 +686,7 @@ Status AggSinkLocalState::_reset_hash_table() { ((ss.total_size_of_aggregate_states + ss.align_aggregate_states - 1) / ss.align_aggregate_states) * ss.align_aggregate_states)); - hash_table = HashTableType(); + agg_method.hash_table.reset(new HashTableType()); ss.agg_arena_pool.reset(new vectorized::Arena); return Status::OK(); }, diff --git a/be/src/pipeline/exec/aggregation_source_operator.cpp b/be/src/pipeline/exec/aggregation_source_operator.cpp index dc476cf63a0634..7e2fca9bc67de9 100644 --- a/be/src/pipeline/exec/aggregation_source_operator.cpp +++ b/be/src/pipeline/exec/aggregation_source_operator.cpp @@ -17,6 +17,7 @@ #include "aggregation_source_operator.h" +#include #include #include "common/exception.h" @@ -141,13 +142,13 @@ Status AggLocalState::_reset_hash_table() { } }); - ss.aggregate_data_container.reset(new vectorized::AggregateDataContainer( + ss.aggregate_data_container = std::make_unique( sizeof(typename HashTableType::key_type), ((ss.total_size_of_aggregate_states + ss.align_aggregate_states - 1) / ss.align_aggregate_states) * - ss.align_aggregate_states)); - hash_table = HashTableType(); - ss.agg_arena_pool.reset(new vectorized::Arena); + ss.align_aggregate_states); + agg_method.hash_table.reset(new HashTableType()); + ss.agg_arena_pool = std::make_unique(); return Status::OK(); }, ss.agg_data->method_variant); diff --git a/be/src/pipeline/exec/partition_sort_sink_operator.cpp b/be/src/pipeline/exec/partition_sort_sink_operator.cpp index 478a3eedb20c76..14d917d62e3ab3 100644 --- a/be/src/pipeline/exec/partition_sort_sink_operator.cpp +++ b/be/src/pipeline/exec/partition_sort_sink_operator.cpp @@ -189,7 +189,7 @@ void PartitionSortSinkOperatorX::_emplace_into_hash_table( auto creator = [&](const auto& ctor, auto& key, auto& origin) { HashMethodType::try_presis_key(key, origin, *local_state._agg_arena_pool); - auto aggregate_data = _pool->add(new vectorized::PartitionBlocks()); + auto* aggregate_data = _pool->add(new vectorized::PartitionBlocks()); local_state._value_places.push_back(aggregate_data); ctor(key, aggregate_data); local_state._num_partition++; @@ -206,7 +206,7 @@ void PartitionSortSinkOperatorX::_emplace_into_hash_table( agg_method.lazy_emplace(state, row, creator, creator_for_null_key); mapped->add_row_idx(row); } - for (auto place : local_state._value_places) { + for (auto* place : local_state._value_places) { SCOPED_TIMER(local_state._selector_block_timer); place->append_block_by_selector(input_block, _child_x->row_desc(), _has_global_limit, _partition_inner_limit, diff --git a/be/src/vec/common/hash_table/hash_map.h b/be/src/vec/common/hash_table/hash_map.h index bd7c413dd97802..382f46acb7408d 100644 --- a/be/src/vec/common/hash_table/hash_map.h +++ b/be/src/vec/common/hash_table/hash_map.h @@ -155,12 +155,6 @@ class HashMapTable : public HashTable { using HashTable::HashTable; - /// Call func(const Key &, Mapped &) for each hash map element. - template - void for_each_value(Func&& func) { - for (auto& v : *this) func(v.get_first(), v.get_second()); - } - /// Call func(Mapped &) for each hash map element. template void for_each_mapped(Func&& func) { diff --git a/be/src/vec/common/hash_table/hash_map_context.h b/be/src/vec/common/hash_table/hash_map_context.h index 3784379902bfdf..d96aa2d7c65f7c 100644 --- a/be/src/vec/common/hash_table/hash_map_context.h +++ b/be/src/vec/common/hash_table/hash_map_context.h @@ -128,20 +128,22 @@ struct MethodBase { template ALWAYS_INLINE auto find(State& state, size_t i) { - prefetch(i); + if constexpr (!is_string_hash_map()) { + prefetch(i); + } return state.find_key_with_hash(*hash_table, hash_values[i], keys[i]); } template ALWAYS_INLINE auto& lazy_emplace(State& state, size_t i, F&& creator, FF&& creator_for_null_key) { - prefetch(i); + if constexpr (!is_string_hash_map()) { + prefetch(i); + } return state.lazy_emplace_key(*hash_table, i, keys[i], hash_values[i], creator, creator_for_null_key); } - static constexpr bool need_presis() { return std::is_same_v; } - static constexpr bool is_string_hash_map() { return std::is_same_v, HashMap> || std::is_same_v>, HashMap>; @@ -149,7 +151,14 @@ struct MethodBase { template static void try_presis_key(Key& key, Origin& origin, Arena& arena) { - if constexpr (need_presis()) { + if constexpr (std::is_same_v) { + key.data = arena.insert(key.data, key.size); + } + } + + template + static void try_presis_key_and_origin(Key& key, Origin& origin, Arena& arena) { + if constexpr (std::is_same_v) { origin.data = arena.insert(origin.data, origin.size); if constexpr (!is_string_hash_map()) { key = origin; @@ -303,7 +312,6 @@ struct MethodOneNumber : public MethodBase { ->get_raw_data() .data : key_columns[0]->get_raw_data().data); - std::string name = key_columns[0]->get_name(); if (is_join) { Base::init_join_bucket_num(num_rows, bucket_size, null_map); } else { diff --git a/be/src/vec/common/hash_table/hash_table.h b/be/src/vec/common/hash_table/hash_table.h index fcc682a49cb599..7e669f275e291b 100644 --- a/be/src/vec/common/hash_table/hash_table.h +++ b/be/src/vec/common/hash_table/hash_table.h @@ -353,29 +353,6 @@ class alignas(64) HashTableGrowerWithPrecalculation { } }; -static_assert(sizeof(HashTableGrowerWithPrecalculation<>) == 64); - -/** When used as a Grower, it turns a hash table into something like a lookup table. - * It remains non-optimal - the cells store the keys. - * Also, the compiler can not completely remove the code of passing through the collision resolution chain, although it is not needed. - * TODO Make a proper lookup table. - */ -template -struct HashTableFixedGrower { - size_t buf_size() const { return 1ULL << key_bits; } - size_t place(size_t x) const { return x; } - /// You could write __builtin_unreachable(), but the compiler does not optimize everything, and it turns out less efficiently. - size_t next(size_t pos) const { return pos + 1; } - bool overflow(size_t /*elems*/) const { return false; } - - void increase_size() { - LOG(FATAL) << "__builtin_unreachable"; - __builtin_unreachable(); - } - void set(size_t /*num_elems*/) {} - void set_buf_size(size_t /*buf_size_*/) {} -}; - /** If you want to store the zero key separately - a place to store it. */ template struct ZeroValueStorage; @@ -573,11 +550,6 @@ class HashTable : private boost::noncopyable, auto get_ptr() const { return ptr; } size_t get_hash() const { return ptr->get_hash(*container); } - size_t get_collision_chain_length() const { - return container->grower.place((ptr - container->buf) - - container->grower.place(get_hash())); - } - /** * A hack for HashedDictionary. * diff --git a/be/src/vec/common/hash_table/partitioned_hash_table.h b/be/src/vec/common/hash_table/partitioned_hash_table.h index c7626f1fc84a80..c6a19b36d3a0c3 100644 --- a/be/src/vec/common/hash_table/partitioned_hash_table.h +++ b/be/src/vec/common/hash_table/partitioned_hash_table.h @@ -97,17 +97,6 @@ class PartitionedHashTable : private boost::noncopyable, Impl::Hash { } } - template - void ALWAYS_INLINE for_each_value(Func&& func) { - if (_is_partitioned) { - for (auto i = 0u; i < NUM_LEVEL1_SUB_TABLES; ++i) { - level1_sub_tables[i].for_each_value(func); - } - } else { - level0_sub_table.for_each_value(func); - } - } - size_t size() { size_t count = 0; if (_is_partitioned) { diff --git a/be/src/vec/common/hash_table/ph_hash_map.h b/be/src/vec/common/hash_table/ph_hash_map.h index f1db30f41a5223..8f61ee966c5c0a 100644 --- a/be/src/vec/common/hash_table/ph_hash_map.h +++ b/be/src/vec/common/hash_table/ph_hash_map.h @@ -102,8 +102,6 @@ class PHHashMap : private boost::noncopyable { auto get_ptr() const { return this; } size_t get_hash() const { return base_iterator->get_hash(); } - - size_t get_collision_chain_length() const { return 0; } }; class iterator : public iterator_base { @@ -202,12 +200,6 @@ class PHHashMap : private boost::noncopyable { _hash_map.prefetch_hash(hash_value); } - /// Call func(const Key &, Mapped &) for each hash map element. - template - void for_each_value(Func&& func) { - for (auto& v : *this) func(v.get_first(), v.get_second()); - } - /// Call func(Mapped &) for each hash map element. template void for_each_mapped(Func&& func) { diff --git a/be/src/vec/common/hash_table/string_hash_map.h b/be/src/vec/common/hash_table/string_hash_map.h index 97f63553f4346b..f1efd0fab12126 100644 --- a/be/src/vec/common/hash_table/string_hash_map.h +++ b/be/src/vec/common/hash_table/string_hash_map.h @@ -55,26 +55,6 @@ struct StringHashMapCell static const StringKey16& get_key(const value_type& value_) { return value_.first; } }; -template -struct StringHashMapCell - : public HashMapCell { - using Base = HashMapCell; - using value_type = typename Base::value_type; - using Base::Base; - static constexpr bool need_zero_value_storage = false; - bool is_zero(const HashTableNoState& state) const { return is_zero(this->value.first, state); } - - // Zero means unoccupied cells in hash table. Use key with last word = 0 as - // zero keys, because such keys are unrepresentable (no way to encode length). - static bool is_zero(const StringKey24& key, const HashTableNoState&) { return key.c == 0; } - void set_zero() { this->value.first.c = 0; } - - // external - const doris::StringRef get_key() const { return to_string_ref(this->value.first); } /// NOLINT - // internal - static const StringKey24& get_key(const value_type& value_) { return value_.first; } -}; - template struct StringHashMapCell : public HashMapCellWithSavedHash template struct StringHashMapSubMaps { using T0 = StringHashTableEmpty>; - using T1 = HashMapTable, StringHashTableHash, - StringHashTableGrower<>, Allocator>; - using T2 = HashMapTable, + using T1 = HashMapTable, + StringHashTableHash, StringHashTableGrower<4>, Allocator>; + using T2 = HashMapTable, + StringHashTableHash, StringHashTableGrower<>, Allocator>; + using T3 = HashMapTable, StringHashTableHash, StringHashTableGrower<>, Allocator>; - using T3 = HashMapTable, + using T4 = HashMapTable, StringHashTableHash, StringHashTableGrower<>, Allocator>; using Ts = HashMapTable, StringHashTableHash, StringHashTableGrower<>, Allocator>; @@ -132,42 +118,34 @@ class StringHashMap : public StringHashTableemplace(x, it, inserted); - if (inserted) new (&it->get_mapped()) TMapped(); + if (inserted) { + new (&it->get_mapped()) TMapped(); + } return it->get_mapped(); } template - void ALWAYS_INLINE for_each_value(Func&& func) { + void ALWAYS_INLINE for_each_mapped(Func&& func) { if (this->m0.size()) { - func(doris::StringRef {}, this->m0.zero_value()->get_second()); + func(this->m0.zero_value()->get_second()); } - for (auto& v : this->m1) { - func(v.get_key(), v.get_second()); + func(v.get_second()); } - for (auto& v : this->m2) { - func(v.get_key(), v.get_second()); + func(v.get_second()); } - for (auto& v : this->m3) { - func(v.get_key(), v.get_second()); + func(v.get_second()); + } + for (auto& v : this->m4) { + func(v.get_second()); } - for (auto& v : this->ms) { - func(v.get_key(), v.get_second()); + func(v.get_second()); } } - - template - void ALWAYS_INLINE for_each_mapped(Func&& func) { - if (this->m0.size()) func(this->m0.zero_value()->get_second()); - for (auto& v : this->m1) func(v.get_second()); - for (auto& v : this->m2) func(v.get_second()); - for (auto& v : this->m3) func(v.get_second()); - for (auto& v : this->ms) func(v.get_second()); - } template char* get_null_key_data() { return nullptr; diff --git a/be/src/vec/common/hash_table/string_hash_table.h b/be/src/vec/common/hash_table/string_hash_table.h index 09e326761c4a3a..5d98ff0b720c4b 100644 --- a/be/src/vec/common/hash_table/string_hash_table.h +++ b/be/src/vec/common/hash_table/string_hash_table.h @@ -24,41 +24,43 @@ #include #include "vec/common/hash_table/hash.h" +#include "vec/common/hash_table/hash_table.h" +#include "vec/common/memcpy_small.h" +using StringKey2 = doris::vectorized::UInt16; +using StringKey4 = doris::vectorized::UInt32; using StringKey8 = doris::vectorized::UInt64; using StringKey16 = doris::vectorized::UInt128; -struct StringKey24 { - doris::vectorized::UInt64 a; - doris::vectorized::UInt64 b; - doris::vectorized::UInt64 c; - bool operator==(const StringKey24 rhs) const { return a == rhs.a && b == rhs.b && c == rhs.c; } +struct StringHashMapSubKeys { + using T1 = StringKey2; + using T2 = StringKey4; + using T3 = StringKey8; + using T4 = StringKey16; }; template -StringKey toStringKey(const doris::StringRef& key) { +StringKey to_string_key(const doris::StringRef& key) { DCHECK_LE(key.size, sizeof(StringKey)); StringKey string_key {}; - memcpy((char*)&string_key, key.data, key.size); + memcpy_small((char*)&string_key, key.data, key.size); return string_key; } -inline doris::StringRef ALWAYS_INLINE to_string_ref(const StringKey8& n) { +template +inline doris::StringRef ALWAYS_INLINE to_string_ref(const T& n) { assert(n != 0); - return {reinterpret_cast(&n), 8ul - (__builtin_clzll(n) >> 3)}; + return {reinterpret_cast(&n), sizeof(T) - (__builtin_clzll(n) >> 3)}; } inline doris::StringRef ALWAYS_INLINE to_string_ref(const StringKey16& n) { assert(n.high != 0); return {reinterpret_cast(&n), 16ul - (__builtin_clzll(n.high) >> 3)}; } -inline doris::StringRef ALWAYS_INLINE to_string_ref(const StringKey24& n) { - assert(n.c != 0); - return {reinterpret_cast(&n), 24ul - (__builtin_clzll(n.c) >> 3)}; -} struct StringHashTableHash { #if defined(__SSE4_2__) || defined(__aarch64__) - size_t ALWAYS_INLINE operator()(StringKey8 key) const { + template + size_t ALWAYS_INLINE operator()(T key) const { size_t res = -1ULL; res = _mm_crc32_u64(res, key); return res; @@ -69,31 +71,23 @@ struct StringHashTableHash { res = _mm_crc32_u64(res, key.high); return res; } - size_t ALWAYS_INLINE operator()(StringKey24 key) const { - size_t res = -1ULL; - res = _mm_crc32_u64(res, key.a); - res = _mm_crc32_u64(res, key.b); - res = _mm_crc32_u64(res, key.c); - return res; - } #else - size_t ALWAYS_INLINE operator()(StringKey8 key) const { - return util_hash::CityHash64(reinterpret_cast(&key), 8); - } - size_t ALWAYS_INLINE operator()(StringKey16 key) const { - return util_hash::CityHash64(reinterpret_cast(&key), 16); - } - size_t ALWAYS_INLINE operator()(StringKey24 key) const { - return util_hash::CityHash64(reinterpret_cast(&key), 24); + template + size_t ALWAYS_INLINE operator()(T key) const { + return util_hash::CityHash64(reinterpret_cast(&key), sizeof(T)); } #endif size_t ALWAYS_INLINE operator()(doris::StringRef key) const { - if (key.size <= 8) { - return StringHashTableHash()(toStringKey(key)); - } else if (key.size <= 16) { - return StringHashTableHash()(toStringKey(key)); - } else if (key.size <= 24) { - return StringHashTableHash()(toStringKey(key)); + if (key.size == 0) { + return 0; + } else if (key.size <= sizeof(StringHashMapSubKeys::T1)) { + return StringHashTableHash()(to_string_key(key)); + } else if (key.size <= sizeof(StringHashMapSubKeys::T2)) { + return StringHashTableHash()(to_string_key(key)); + } else if (key.size <= sizeof(StringHashMapSubKeys::T3)) { + return StringHashTableHash()(to_string_key(key)); + } else if (key.size <= sizeof(StringHashMapSubKeys::T4)) { + return StringHashTableHash()(to_string_key(key)); } return doris::StringRefHash()(key); } @@ -194,10 +188,9 @@ struct StringHashTableGrower : public HashTableGrowerWithPrecalculation struct StringHashTableLookupResult { Mapped* mapped_ptr; - StringHashTableLookupResult() : mapped_ptr(nullptr) {} /// NOLINT - StringHashTableLookupResult(Mapped* mapped_ptr_) : mapped_ptr(mapped_ptr_) {} /// NOLINT - StringHashTableLookupResult(std::nullptr_t) {} /// NOLINT - const VoidKey getKey() const { return {}; } /// NOLINT + StringHashTableLookupResult() : mapped_ptr(nullptr) {} + StringHashTableLookupResult(Mapped* mapped_ptr_) : mapped_ptr(mapped_ptr_) {} + StringHashTableLookupResult(std::nullptr_t) {} auto& get_mapped() { return *mapped_ptr; } auto& operator*() { return *this; } auto& operator*() const { return *this; } @@ -226,7 +219,6 @@ ALWAYS_INLINE inline auto lookup_result_get_mapped(StringHashTableLookupResult class StringHashTable : private boost::noncopyable { protected: - static constexpr size_t NUM_MAPS = 5; // Map for storing empty string using T0 = typename SubMaps::T0; @@ -234,18 +226,17 @@ class StringHashTable : private boost::noncopyable { using T1 = typename SubMaps::T1; using T2 = typename SubMaps::T2; using T3 = typename SubMaps::T3; + using T4 = typename SubMaps::T4; // Long strings are stored as doris::StringRef along with saved hash using Ts = typename SubMaps::Ts; using Self = StringHashTable; - template - friend class TwoLevelStringHashTable; - T0 m0; T1 m1; T2 m2; T3 m3; + T4 m4; Ts ms; using Cell = typename Ts::cell_type; @@ -259,7 +250,8 @@ class StringHashTable : private boost::noncopyable { typename T1::iterator iterator1; typename T2::iterator iterator2; typename T3::iterator iterator3; - typename Ts::iterator iterator4; + typename T4::iterator iterator4; + typename Ts::iterator iterator5; typename Ts::cell_type cell; @@ -269,39 +261,52 @@ class StringHashTable : private boost::noncopyable { iterator_base() = default; iterator_base(Container* container_, bool end = false) : container(container_) { if (end) { - sub_table_index = 4; - iterator4 = container->ms.end(); + sub_table_index = 5; + iterator5 = container->ms.end(); } else { sub_table_index = 0; - if (container->m0.size() == 0) + if (container->m0.size() == 0) { sub_table_index++; - else + } else { return; + } iterator1 = container->m1.begin(); - if (iterator1 == container->m1.end()) + if (iterator1 == container->m1.end()) { sub_table_index++; - else + } else { return; + } iterator2 = container->m2.begin(); - if (iterator2 == container->m2.end()) + if (iterator2 == container->m2.end()) { sub_table_index++; - else + } else { return; + } iterator3 = container->m3.begin(); - if (iterator3 == container->m3.end()) + if (iterator3 == container->m3.end()) { + sub_table_index++; + } else { + return; + } + + iterator4 = container->m4.begin(); + if (iterator4 == container->m4.end()) { sub_table_index++; - else + } else { return; + } - iterator4 = container->ms.begin(); + iterator5 = container->ms.begin(); } } bool operator==(const iterator_base& rhs) const { - if (sub_table_index != rhs.sub_table_index) return false; + if (sub_table_index != rhs.sub_table_index) { + return false; + } switch (sub_table_index) { case 0: { return true; @@ -318,6 +323,9 @@ class StringHashTable : private boost::noncopyable { case 4: { return iterator4 == rhs.iterator4; } + case 5: { + return iterator5 == rhs.iterator5; + } } LOG(FATAL) << "__builtin_unreachable"; __builtin_unreachable(); @@ -355,6 +363,13 @@ class StringHashTable : private boost::noncopyable { } case 4: { ++iterator4; + if (iterator4 == container->m4.end()) { + need_switch_to_next = true; + } + break; + } + case 5: { + ++iterator5; break; } } @@ -385,7 +400,14 @@ class StringHashTable : private boost::noncopyable { break; } case 4: { - iterator4 = container->ms.begin(); + iterator4 = container->m4.begin(); + if (iterator4 == container->m4.end()) { + need_switch_to_next = true; + } + break; + } + case 5: { + iterator5 = container->ms.begin(); break; } } @@ -416,6 +438,10 @@ class StringHashTable : private boost::noncopyable { const_cast(this)->cell = *iterator4; break; } + case 5: { + const_cast(this)->cell = *iterator5; + break; + } } return cell; } @@ -438,13 +464,14 @@ class StringHashTable : private boost::noncopyable { return iterator3->get_hash(container->m3); } case 4: { - return iterator4->get_hash(container->ms); + return iterator4->get_hash(container->m4); + } + case 5: { + return iterator5->get_hash(container->ms); } } } - size_t get_collision_chain_length() const { return 0; } - /** * A hack for HashedDictionary. * @@ -476,25 +503,11 @@ class StringHashTable : private boost::noncopyable { StringHashTable() = default; explicit StringHashTable(size_t reserve_for_num_elements) - : m1 {reserve_for_num_elements / 4}, - m2 {reserve_for_num_elements / 4}, - m3 {reserve_for_num_elements / 4}, - ms {reserve_for_num_elements / 4} {} - - StringHashTable(StringHashTable&& rhs) noexcept - : m1(std::move(rhs.m1)), - m2(std::move(rhs.m2)), - m3(std::move(rhs.m3)), - ms(std::move(rhs.ms)) {} - - StringHashTable& operator=(StringHashTable&& other) { - std::swap(m0, other.m0); - std::swap(m1, other.m1); - std::swap(m2, other.m2); - std::swap(m3, other.m3); - std::swap(ms, other.ms); - return *this; - } + : m1 {reserve_for_num_elements / 5}, + m2 {reserve_for_num_elements / 5}, + m3 {reserve_for_num_elements / 5}, + m4 {reserve_for_num_elements / 5}, + ms {reserve_for_num_elements / 5} {} ~StringHashTable() = default; @@ -524,24 +537,20 @@ class StringHashTable : private boost::noncopyable { return func(self.ms, std::forward(key), key, hash_value); } - switch ((sz - 1) >> 3) { - case 0: // 1..8 bytes - { - return func(self.m1, toStringKey(key), key, hash_value); - } - case 1: // 9..16 bytes - { - return func(self.m2, toStringKey(key), key, hash_value); + if (sz <= sizeof(StringHashMapSubKeys::T1)) { + return func(self.m1, to_string_key(key), key, hash_value); } - case 2: // 17..24 bytes - { - return func(self.m3, toStringKey(key), key, hash_value); + if (sz <= sizeof(StringHashMapSubKeys::T2)) { + return func(self.m2, to_string_key(key), key, hash_value); } - default: // >= 25 bytes - { - return func(self.ms, std::forward(key), key, hash_value); + if (sz <= sizeof(StringHashMapSubKeys::T3)) { + return func(self.m3, to_string_key(key), key, hash_value); } + if (sz <= sizeof(StringHashMapSubKeys::T4)) { + return func(self.m4, to_string_key(key), key, hash_value); } + + return func(self.ms, std::forward(key), key, hash_value); } struct EmplaceCallable { @@ -594,12 +603,14 @@ class StringHashTable : private boost::noncopyable { if (!key.size) { return; } - if (key.size <= 8) { + if (key.size <= sizeof(StringHashMapSubKeys::T1)) { m1.template prefetch(hash_value); - } else if (key.size <= 16) { + } else if (key.size <= sizeof(StringHashMapSubKeys::T2)) { m2.template prefetch(hash_value); - } else if (key.size <= 24) { + } else if (key.size <= sizeof(StringHashMapSubKeys::T3)) { m3.template prefetch(hash_value); + } else if (key.size <= sizeof(StringHashMapSubKeys::T4)) { + m4.template prefetch(hash_value); } else { ms.template prefetch(hash_value); } @@ -613,10 +624,11 @@ class StringHashTable : private boost::noncopyable { auto ALWAYS_INLINE operator()(Submap& map, const SubmapKey& key, const Origin& origin, size_t hash) { auto it = map.find(key, hash); - if (!it) + if (!it) { return decltype(&it->get_mapped()) {}; - else + } else { return &it->get_mapped(); + } } }; @@ -628,14 +640,12 @@ class StringHashTable : private boost::noncopyable { return dispatch(*this, x, hash_value, FindCallable {}); } - bool ALWAYS_INLINE has(const Key& x, size_t = 0) const { - return dispatch(*this, x, FindCallable {}) != nullptr; + size_t size() const { + return m0.size() + m1.size() + m2.size() + m3.size() + m4.size() + ms.size(); } - size_t size() const { return m0.size() + m1.size() + m2.size() + m3.size() + ms.size(); } - bool empty() const { - return m0.empty() && m1.empty() && m2.empty() && m3.empty() && ms.empty(); + return m0.empty() && m1.empty() && m2.empty() && m3.empty() && m4.empty() && ms.empty(); } size_t get_buffer_size_in_bytes() const { @@ -666,6 +676,7 @@ class StringHashTable : private boost::noncopyable { bool add_elem_size_overflow(size_t add_size) const { return m1.add_elem_size_overflow(add_size) || m2.add_elem_size_overflow(add_size) || - m3.add_elem_size_overflow(add_size) || ms.add_elem_size_overflow(add_size); + m3.add_elem_size_overflow(add_size) || m4.add_elem_size_overflow(add_size) || + ms.add_elem_size_overflow(add_size); } }; diff --git a/be/src/vec/common/memcpy_small.h b/be/src/vec/common/memcpy_small.h index 473900663186e6..62a093e8b628ed 100644 --- a/be/src/vec/common/memcpy_small.h +++ b/be/src/vec/common/memcpy_small.h @@ -20,8 +20,11 @@ #pragma once +#include #include +#include + #if defined(__SSE2__) || defined(__aarch64__) #include "util/sse_util.hpp" @@ -91,3 +94,32 @@ void memcpy_fixed(char* lhs, const char* rhs) { memcpy(lhs, rhs, sizeof(T)); } } + +template +inline void memcpy_small(char* lhs, const char* rhs, size_t n) { + DCHECK_NE(n, 0); + if constexpr (max_size >= 4) { + if (n >= 4) { + memcpy_fixed(lhs, rhs); + lhs += 4; + rhs += 4; + n -= 4; + } + } + while (n >= 1) { + memcpy_fixed(lhs, rhs); + lhs++; + rhs++; + n--; + } +} + +template <> +inline void memcpy_small<2>(char* lhs, const char* rhs, size_t n) { + DCHECK_NE(n, 0); + if (n == 2) { + memcpy_fixed(lhs, rhs); + } else { + memcpy_fixed(lhs, rhs); + } +} \ No newline at end of file diff --git a/be/src/vec/exec/distinct_vaggregation_node.cpp b/be/src/vec/exec/distinct_vaggregation_node.cpp index a5c57792ba3aca..094390103279ab 100644 --- a/be/src/vec/exec/distinct_vaggregation_node.cpp +++ b/be/src/vec/exec/distinct_vaggregation_node.cpp @@ -95,12 +95,12 @@ void DistinctAggregationNode::_emplace_into_hash_table_to_distinct(IColumn::Sele auto creator = [&](const auto& ctor, auto& key, auto& origin) { HashMethodType::try_presis_key(key, origin, *_agg_arena_pool); ctor(key, dummy_mapped_data); - distinct_row.push_back(row); + distinct_row.push_back_without_reserve(row); }; auto creator_for_null_key = [&](auto& mapped) { mapped = dummy_mapped_data; - distinct_row.push_back(row); + distinct_row.push_back_without_reserve(row); }; SCOPED_TIMER(_hash_table_emplace_timer); diff --git a/be/src/vec/exec/vaggregation_node.cpp b/be/src/vec/exec/vaggregation_node.cpp index 020763ecff75a4..5e0f9762668412 100644 --- a/be/src/vec/exec/vaggregation_node.cpp +++ b/be/src/vec/exec/vaggregation_node.cpp @@ -102,26 +102,10 @@ static constexpr int STREAMING_HT_MIN_REDUCTION_SIZE = AggregationNode::AggregationNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) : ExecNode(pool, tnode, descs), - _hash_table_compute_timer(nullptr), - _hash_table_input_counter(nullptr), - _expr_timer(nullptr), _intermediate_tuple_id(tnode.agg_node.intermediate_tuple_id), - _intermediate_tuple_desc(nullptr), _output_tuple_id(tnode.agg_node.output_tuple_id), - _output_tuple_desc(nullptr), _needs_finalize(tnode.agg_node.need_finalize), - _is_merge(false), - _serialize_key_timer(nullptr), - _merge_timer(nullptr), - _get_results_timer(nullptr), - _serialize_data_timer(nullptr), - _serialize_result_timer(nullptr), - _deserialize_data_timer(nullptr), - _hash_table_iterate_timer(nullptr), - _insert_keys_to_column_timer(nullptr), - _streaming_agg_timer(nullptr), - _hash_table_size_counter(nullptr), - _max_row_size_counter(nullptr) { + _is_merge(false) { if (tnode.agg_node.__isset.use_streaming_preaggregation) { _is_streaming_preagg = tnode.agg_node.use_streaming_preaggregation; if (_is_streaming_preagg) { @@ -593,7 +577,7 @@ Status AggregationNode::_get_without_key_result(RuntimeState* state, Block* bloc } for (int i = 0; i < _aggregate_evaluators.size(); ++i) { - auto column = columns[i].get(); + auto* column = columns[i].get(); _aggregate_evaluators[i]->insert_result_info( _agg_data->without_key + _offsets_of_aggregate_states[i], column); } @@ -816,13 +800,13 @@ Status AggregationNode::_reset_hash_table() { } }); - _aggregate_data_container.reset(new AggregateDataContainer( + _aggregate_data_container = std::make_unique( sizeof(typename HashTableType::key_type), ((_total_size_of_aggregate_states + _align_aggregate_states - 1) / _align_aggregate_states) * - _align_aggregate_states)); - hash_table = HashTableType(); - _agg_arena_pool.reset(new Arena); + _align_aggregate_states); + agg_method.hash_table.reset(new HashTableType()); + _agg_arena_pool = std::make_unique(); return Status::OK(); }, _agg_data->method_variant); @@ -845,7 +829,7 @@ void AggregationNode::_emplace_into_hash_table(AggregateDataPtr* places, ColumnR auto creator = [this](const auto& ctor, auto& key, auto& origin) { try { - HashMethodType::try_presis_key(key, origin, *_agg_arena_pool); + HashMethodType::try_presis_key_and_origin(key, origin, *_agg_arena_pool); auto mapped = _aggregate_data_container->append_data(origin); auto st = _create_agg_status(mapped); if (!st) { diff --git a/be/src/vec/exec/vpartition_sort_node.cpp b/be/src/vec/exec/vpartition_sort_node.cpp index 6db90763fa74c5..6a7ebcffd1e5b2 100644 --- a/be/src/vec/exec/vpartition_sort_node.cpp +++ b/be/src/vec/exec/vpartition_sort_node.cpp @@ -116,7 +116,7 @@ void VPartitionSortNode::_emplace_into_hash_table(const ColumnRawPtrs& key_colum auto creator = [&](const auto& ctor, auto& key, auto& origin) { HashMethodType::try_presis_key(key, origin, *_agg_arena_pool); - auto aggregate_data = _pool->add(new PartitionBlocks()); + auto* aggregate_data = _pool->add(new PartitionBlocks()); _value_places.push_back(aggregate_data); ctor(key, aggregate_data); _num_partition++; From 9c7d4b518e2baea8bdb3527af44664c38ca96c03 Mon Sep 17 00:00:00 2001 From: jakevin Date: Thu, 18 Jan 2024 17:18:41 +0800 Subject: [PATCH 085/200] [fix](Nereids): fix AssertNumRows StatsCalculator (#30053) --- .../doris/nereids/stats/StatsCalculator.java | 44 +++++-- .../shape/query54.out | 112 ++++++++-------- .../shape/query6.out | 68 +++++----- .../noStatsRfPrune/query58.out | 21 ++- .../no_stats_shape/query58.out | 21 ++- .../rf_prune/query54.out | 120 +++++++++--------- .../rf_prune/query6.out | 7 +- .../shape/query54.out | 120 +++++++++--------- .../shape/query6.out | 9 +- 9 files changed, 267 insertions(+), 255 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java index 14ead9990aa4d7..76587bcffb55e5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java @@ -26,6 +26,7 @@ import org.apache.doris.nereids.memo.Group; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.AssertNumRowsElement; import org.apache.doris.nereids.trees.expressions.CTEId; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; @@ -167,7 +168,7 @@ public class StatsCalculator extends DefaultPlanVisitor { private CascadesContext cascadesContext; private StatsCalculator(GroupExpression groupExpression, boolean forbidUnknownColStats, - Map columnStatisticMap, boolean isPlayNereidsDump, + Map columnStatisticMap, boolean isPlayNereidsDump, Map cteIdToStats, CascadesContext context) { this.groupExpression = groupExpression; this.forbidUnknownColStats = forbidUnknownColStats; @@ -193,7 +194,7 @@ public Map getTotalColumnStatisticMap() { * estimate stats */ public static StatsCalculator estimate(GroupExpression groupExpression, boolean forbidUnknownColStats, - Map columnStatisticMap, boolean isPlayNereidsDump, + Map columnStatisticMap, boolean isPlayNereidsDump, Map cteIdToStats, CascadesContext context) { StatsCalculator statsCalculator = new StatsCalculator( groupExpression, forbidUnknownColStats, columnStatisticMap, isPlayNereidsDump, cteIdToStats, context); @@ -369,7 +370,7 @@ public Statistics visitLogicalJoin(LogicalJoin j @Override public Statistics visitLogicalAssertNumRows( LogicalAssertNumRows assertNumRows, Void context) { - return computeAssertNumRows(assertNumRows.getAssertNumRowsElement().getDesiredNumOfRows()); + return computeAssertNumRows(assertNumRows.getAssertNumRowsElement()); } @Override @@ -533,7 +534,7 @@ public Statistics visitPhysicalDistribute(PhysicalDistribute dis @Override public Statistics visitPhysicalAssertNumRows(PhysicalAssertNumRows assertNumRows, Void context) { - return computeAssertNumRows(assertNumRows.getAssertNumRowsElement().getDesiredNumOfRows()); + return computeAssertNumRows(assertNumRows.getAssertNumRowsElement()); } @Override @@ -556,11 +557,34 @@ public Statistics visitPhysicalGenerate(PhysicalGenerate generat return computeGenerate(generate); } - private Statistics computeAssertNumRows(long desiredNumOfRows) { + private Statistics computeAssertNumRows(AssertNumRowsElement assertNumRowsElement) { Statistics statistics = groupExpression.childStatistics(0); - statistics.withRowCountAndEnforceValid(Math.min(1, statistics.getRowCount())); - statistics = new StatisticsBuilder(statistics).setWidthInJoinCluster(1).build(); - return statistics; + long newRowCount; + long rowCount = (long) statistics.getRowCount(); + long desiredNumOfRows = assertNumRowsElement.getDesiredNumOfRows(); + switch (assertNumRowsElement.getAssertion()) { + case EQ: + newRowCount = desiredNumOfRows; + break; + case GE: + newRowCount = statistics.getRowCount() >= desiredNumOfRows ? rowCount : desiredNumOfRows; + break; + case GT: + newRowCount = statistics.getRowCount() > desiredNumOfRows ? rowCount : desiredNumOfRows; + break; + case LE: + newRowCount = statistics.getRowCount() <= desiredNumOfRows ? rowCount : desiredNumOfRows; + break; + case LT: + newRowCount = statistics.getRowCount() < desiredNumOfRows ? rowCount : desiredNumOfRows; + break; + case NE: + return statistics; + default: + throw new IllegalArgumentException("Unknown assertion: " + assertNumRowsElement.getAssertion()); + } + Statistics newStatistics = statistics.withRowCountAndEnforceValid(newRowCount); + return new StatisticsBuilder(newStatistics).setWidthInJoinCluster(1).build(); } private Statistics computeFilter(Filter filter) { @@ -610,7 +634,7 @@ private ColumnStatistic getColumnStatistic(TableIf table, String colName) { } } else { return Env.getCurrentEnv().getStatisticsCache().getColumnStatistics( - catalogId, dbId, table.getId(), colName); + catalogId, dbId, table.getId(), colName); } } @@ -701,7 +725,7 @@ private Statistics computePartitionTopN(PartitionTopN partitionTopN) { rowCount = rowCount * DEFAULT_COLUMN_NDV_RATIO; } else { rowCount = Math.min(rowCount, partitionByKeyStats.stream().map(s -> s.ndv) - .max(Double::compare).get() * partitionTopN.getPartitionLimit()); + .max(Double::compare).get() * partitionTopN.getPartitionLimit()); } } else { rowCount = Math.min(rowCount, partitionTopN.getPartitionLimit()); diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query54.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query54.out index 42c40cc0279084..cbc7818f70ccd2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query54.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query54.out @@ -13,12 +13,64 @@ PhysicalResultSink --------------------PhysicalDistribute[DistributionSpecHash] ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF7 ss_sold_date_sk->[d_date_sk] +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[ss_sold_date_sk] ----------------------------PhysicalProject -------------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) <= (d_month_seq + 3)) ---------------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) >= (d_month_seq + 1)) +------------------------------hashJoin[INNER_JOIN] hashCondition=((my_customers.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ss_customer_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF7 +------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_county->[ca_county];RF5 s_state->[ca_state] +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((my_customers.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 c_current_addr_sk->[ca_address_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF3 RF4 RF5 +----------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------PhysicalProject +--------------------------------------------hashAgg[GLOBAL] +----------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_customer_sk = cs_or_ws_sales.customer_sk)) otherCondition=() build RFs:RF2 customer_sk->[c_customer_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------PhysicalOlapScan[customer] apply RFs: RF2 +------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((cs_or_ws_sales.sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk,ws_sold_date_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((cs_or_ws_sales.item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk,ws_item_sk] +----------------------------------------------------------------PhysicalUnion +------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 +----------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------filter((item.i_category = 'Music') and (item.i_class = 'country')) +----------------------------------------------------------------------PhysicalOlapScan[item] +------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------------------------------------PhysicalProject +----------------------------------------------------------------filter((date_dim.d_moy = 1) and (date_dim.d_year = 1999)) +------------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) <= (d_month_seq + 3)) +----------------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) >= (d_month_seq + 1)) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------------PhysicalAssertNumRows +----------------------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------------------hashAgg[GLOBAL] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------hashAgg[LOCAL] +------------------------------------------------PhysicalProject +--------------------------------------------------filter((date_dim.d_moy = 1) and (date_dim.d_year = 1999)) +----------------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------------PhysicalAssertNumRows --------------------------------------PhysicalDistribute[DistributionSpecGather] @@ -28,56 +80,4 @@ PhysicalResultSink ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_moy = 1) and (date_dim.d_year = 1999)) --------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------PhysicalAssertNumRows -------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------hashAgg[GLOBAL] -----------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------hashAgg[LOCAL] ---------------------------------------------PhysicalProject -----------------------------------------------filter((date_dim.d_moy = 1) and (date_dim.d_year = 1999)) -------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN] hashCondition=((my_customers.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ss_customer_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 -----------------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_county->[ca_county];RF5 s_state->[ca_state] -----------------------------------------hashJoin[INNER_JOIN] hashCondition=((my_customers.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 c_current_addr_sk->[ca_address_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF3 RF4 RF5 -------------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------------PhysicalProject -----------------------------------------------hashAgg[GLOBAL] -------------------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------------------hashAgg[LOCAL] -----------------------------------------------------PhysicalProject -------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_customer_sk = cs_or_ws_sales.customer_sk)) otherCondition=() build RFs:RF2 customer_sk->[c_customer_sk] ---------------------------------------------------------PhysicalProject -----------------------------------------------------------PhysicalOlapScan[customer] apply RFs: RF2 ---------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((cs_or_ws_sales.sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk,ws_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((cs_or_ws_sales.item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk,ws_item_sk] -------------------------------------------------------------------PhysicalUnion ---------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 ---------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 -------------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------filter((item.i_category = 'Music') and (item.i_class = 'country')) -------------------------------------------------------------------------PhysicalOlapScan[item] ---------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------filter((date_dim.d_moy = 1) and (date_dim.d_year = 1999)) ---------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query6.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query6.out index b09c0da2bb30bb..889a34c0eb0ca0 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query6.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query6.out @@ -10,44 +10,44 @@ PhysicalResultSink --------------PhysicalDistribute[DistributionSpecHash] ----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] +--------------------hashJoin[INNER_JOIN] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[ss_item_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 RF5 ---------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------hashJoin[INNER_JOIN] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF2 d_month_seq->[d_month_seq] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF2 -------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------hashJoin[INNER_JOIN] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF1 i_category->[i_category] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[item] apply RFs: RF1 ---------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------hashJoin[INNER_JOIN] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] ----------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------PhysicalOlapScan[customer] apply RFs: RF4 ----------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer_address] +--------------------------------hashJoin[INNER_JOIN] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +----------------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------------hashJoin[INNER_JOIN] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 +--------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------------PhysicalAssertNumRows +------------------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashJoin[INNER_JOIN] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[item] apply RFs: RF0 +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query58.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query58.out index c59907df29ca36..c0082e8cea084e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query58.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query58.out @@ -28,10 +28,9 @@ PhysicalResultSink ------------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------------PhysicalProject ----------------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF8 d_week_seq->[d_week_seq] -------------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF8 -------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF8 +------------------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------------------PhysicalAssertNumRows ----------------------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------------------PhysicalProject @@ -59,10 +58,9 @@ PhysicalResultSink ------------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------------PhysicalProject ----------------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF4 d_week_seq->[d_week_seq] -------------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 -------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +------------------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------------------PhysicalAssertNumRows ----------------------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------------------PhysicalProject @@ -90,10 +88,9 @@ PhysicalResultSink --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------PhysicalProject ------------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF0 d_week_seq->[d_week_seq] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 ---------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------------------PhysicalAssertNumRows ------------------------------------------PhysicalDistribute[DistributionSpecGather] --------------------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query58.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query58.out index a449bf2addb1b8..2814b83c308809 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query58.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query58.out @@ -28,10 +28,9 @@ PhysicalResultSink ------------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------------PhysicalProject ----------------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF8 d_week_seq->[d_week_seq] -------------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF8 RF9 -------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF8 RF9 +------------------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------------------PhysicalAssertNumRows ----------------------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------------------PhysicalProject @@ -59,10 +58,9 @@ PhysicalResultSink ------------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------------PhysicalProject ----------------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF4 d_week_seq->[d_week_seq] -------------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 RF5 -------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 RF5 +------------------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------------------PhysicalAssertNumRows ----------------------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------------------PhysicalProject @@ -90,10 +88,9 @@ PhysicalResultSink --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------PhysicalProject ------------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF0 d_week_seq->[d_week_seq] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 RF1 ---------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 RF1 +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------------------PhysicalAssertNumRows ------------------------------------------PhysicalDistribute[DistributionSpecGather] --------------------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query54.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query54.out index a1548688624a42..7dc2907cd0f45f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query54.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query54.out @@ -13,73 +13,71 @@ PhysicalResultSink --------------------PhysicalDistribute[DistributionSpecHash] ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) <= (d_month_seq + 3)) +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[ss_sold_date_sk] ----------------------------PhysicalProject -------------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) >= (d_month_seq + 1)) +------------------------------hashJoin[INNER_JOIN] hashCondition=((my_customers.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ss_customer_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN] hashCondition=((my_customers.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ss_customer_sk] +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_county->[ca_county];RF5 s_state->[ca_state] +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((my_customers.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 c_current_addr_sk->[ca_address_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF3 RF4 RF5 +----------------------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 -------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_county->[ca_county];RF5 s_state->[ca_state] -------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((my_customers.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 c_current_addr_sk->[ca_address_sk] +--------------------------------------------hashAgg[GLOBAL] +----------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------hashAgg[LOCAL] --------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF3 RF4 RF5 ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------PhysicalProject -------------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------------hashAgg[LOCAL] +----------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_customer_sk = cs_or_ws_sales.customer_sk)) otherCondition=() build RFs:RF2 customer_sk->[c_customer_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------PhysicalOlapScan[customer] apply RFs: RF2 +------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((cs_or_ws_sales.sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk,ws_sold_date_sk] ------------------------------------------------------------PhysicalProject ---------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_customer_sk = cs_or_ws_sales.customer_sk)) otherCondition=() build RFs:RF2 customer_sk->[c_customer_sk] -----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------PhysicalOlapScan[customer] apply RFs: RF2 +--------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((cs_or_ws_sales.item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk,ws_item_sk] +----------------------------------------------------------------PhysicalUnion +------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 ----------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((cs_or_ws_sales.sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk,ws_sold_date_sk] -----------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((cs_or_ws_sales.item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk,ws_item_sk] ---------------------------------------------------------------------------PhysicalUnion -----------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -----------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 ---------------------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------------filter((item.i_category = 'Women') and (item.i_class = 'maternity')) ---------------------------------------------------------------------------------PhysicalOlapScan[item] -----------------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 1998)) -----------------------------------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------PhysicalAssertNumRows -------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------hashAgg[GLOBAL] -----------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------hashAgg[LOCAL] ---------------------------------------------PhysicalProject -----------------------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 1998)) -------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] +--------------------------------------------------------------------filter((item.i_category = 'Women') and (item.i_class = 'maternity')) +----------------------------------------------------------------------PhysicalOlapScan[item] +------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------------------------------------PhysicalProject +----------------------------------------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 1998)) +------------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 1998)) ---------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------PhysicalOlapScan[store] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) <= (d_month_seq + 3)) +----------------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) >= (d_month_seq + 1)) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------------PhysicalAssertNumRows +----------------------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------------------hashAgg[GLOBAL] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------hashAgg[LOCAL] +------------------------------------------------PhysicalProject +--------------------------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 1998)) +----------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------PhysicalAssertNumRows +--------------------------------------PhysicalDistribute[DistributionSpecGather] +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] +----------------------------------------------PhysicalProject +------------------------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 1998)) +--------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query6.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query6.out index 07fa54e46bae91..28f53e05bbee95 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query6.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query6.out @@ -10,7 +10,9 @@ PhysicalResultSink --------------PhysicalDistribute[DistributionSpecHash] ----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() +--------------------hashJoin[INNER_JOIN] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] @@ -48,7 +50,4 @@ PhysicalResultSink --------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject ------------------------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out index e76a6a45d8588b..7dc2907cd0f45f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out @@ -13,73 +13,71 @@ PhysicalResultSink --------------------PhysicalDistribute[DistributionSpecHash] ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) <= (d_month_seq + 3)) +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[ss_sold_date_sk] ----------------------------PhysicalProject -------------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) >= (d_month_seq + 1)) +------------------------------hashJoin[INNER_JOIN] hashCondition=((my_customers.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ss_customer_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[ss_sold_date_sk] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN] hashCondition=((my_customers.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ss_customer_sk] +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_county->[ca_county];RF5 s_state->[ca_state] +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((my_customers.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 c_current_addr_sk->[ca_address_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF3 RF4 RF5 +----------------------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 -------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_county->[ca_county];RF5 s_state->[ca_state] -------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((my_customers.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 c_current_addr_sk->[ca_address_sk] +--------------------------------------------hashAgg[GLOBAL] +----------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------hashAgg[LOCAL] --------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF3 RF4 RF5 ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------PhysicalProject -------------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------------hashAgg[LOCAL] +----------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_customer_sk = cs_or_ws_sales.customer_sk)) otherCondition=() build RFs:RF2 customer_sk->[c_customer_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------PhysicalOlapScan[customer] apply RFs: RF2 +------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((cs_or_ws_sales.sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk,ws_sold_date_sk] ------------------------------------------------------------PhysicalProject ---------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_customer_sk = cs_or_ws_sales.customer_sk)) otherCondition=() build RFs:RF2 customer_sk->[c_customer_sk] -----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------PhysicalOlapScan[customer] apply RFs: RF2 +--------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((cs_or_ws_sales.item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk,ws_item_sk] +----------------------------------------------------------------PhysicalUnion +------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 ----------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((cs_or_ws_sales.sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk,ws_sold_date_sk] -----------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((cs_or_ws_sales.item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk,ws_item_sk] ---------------------------------------------------------------------------PhysicalUnion -----------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -----------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 ---------------------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------------filter((item.i_category = 'Women') and (item.i_class = 'maternity')) ---------------------------------------------------------------------------------PhysicalOlapScan[item] -----------------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 1998)) -----------------------------------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------PhysicalAssertNumRows -------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------hashAgg[GLOBAL] -----------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------hashAgg[LOCAL] ---------------------------------------------PhysicalProject -----------------------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 1998)) -------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] +--------------------------------------------------------------------filter((item.i_category = 'Women') and (item.i_class = 'maternity')) +----------------------------------------------------------------------PhysicalOlapScan[item] +------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------------------------------------PhysicalProject +----------------------------------------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 1998)) +------------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 1998)) ---------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------PhysicalOlapScan[store] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) <= (d_month_seq + 3)) +----------------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) >= (d_month_seq + 1)) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------------PhysicalAssertNumRows +----------------------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------------------hashAgg[GLOBAL] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------hashAgg[LOCAL] +------------------------------------------------PhysicalProject +--------------------------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 1998)) +----------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------PhysicalAssertNumRows +--------------------------------------PhysicalDistribute[DistributionSpecGather] +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] +----------------------------------------------PhysicalProject +------------------------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 1998)) +--------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query6.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query6.out index 000b390d0238a3..889a34c0eb0ca0 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query6.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query6.out @@ -10,13 +10,15 @@ PhysicalResultSink --------------PhysicalDistribute[DistributionSpecHash] ----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] +--------------------hashJoin[INNER_JOIN] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] ----------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 +--------------------------------PhysicalOlapScan[customer] apply RFs: RF4 ----------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------PhysicalProject --------------------------------hashJoin[INNER_JOIN] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] @@ -48,7 +50,4 @@ PhysicalResultSink --------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject ------------------------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer_address] From e49027a5e6657d27c066f87094984dd0d0462f61 Mon Sep 17 00:00:00 2001 From: plat1ko Date: Fri, 19 Jan 2024 00:09:06 +0800 Subject: [PATCH 086/200] [feature](cloud) Add CloudTabletMgr (#30089) --- be/src/cloud/cloud_storage_engine.h | 3 + be/src/cloud/cloud_tablet.cpp | 13 + be/src/cloud/cloud_tablet.h | 11 + be/src/cloud/cloud_tablet_mgr.cpp | 356 +++++++++++++++++++++++++++ be/src/cloud/cloud_tablet_mgr.h | 75 ++++++ be/src/cloud/config.cpp | 9 + be/src/cloud/config.h | 12 + be/src/runtime/memory/cache_policy.h | 5 +- 8 files changed, 483 insertions(+), 1 deletion(-) create mode 100644 be/src/cloud/cloud_tablet_mgr.cpp create mode 100644 be/src/cloud/cloud_tablet_mgr.h diff --git a/be/src/cloud/cloud_storage_engine.h b/be/src/cloud/cloud_storage_engine.h index 87e3ed52d39041..eb76d5e94effc9 100644 --- a/be/src/cloud/cloud_storage_engine.h +++ b/be/src/cloud/cloud_storage_engine.h @@ -24,6 +24,8 @@ namespace cloud { class CloudMetaMgr; } +class CloudTabletMgr; + class CloudStorageEngine { public: CloudStorageEngine(); @@ -34,6 +36,7 @@ class CloudStorageEngine { private: std::unique_ptr _meta_mgr; + std::unique_ptr _tablet_mgr; }; } // namespace doris diff --git a/be/src/cloud/cloud_tablet.cpp b/be/src/cloud/cloud_tablet.cpp index 03670df78d10d3..6beb1e45d94165 100644 --- a/be/src/cloud/cloud_tablet.cpp +++ b/be/src/cloud/cloud_tablet.cpp @@ -23,6 +23,8 @@ #include #include +#include + #include "cloud/cloud_meta_mgr.h" #include "cloud/cloud_storage_engine.h" #include "io/cache/block/block_file_cache_factory.h" @@ -365,6 +367,17 @@ Result> CloudTablet::create_rowset_writer( Status::NotSupported("CloudTablet::create_rowset_writer is not implemented")); } +int64_t CloudTablet::get_cloud_base_compaction_score() const { + return _approximate_num_rowsets.load(std::memory_order_relaxed) - + _approximate_cumu_num_rowsets.load(std::memory_order_relaxed); +} + +int64_t CloudTablet::get_cloud_cumu_compaction_score() const { + // TODO(plat1ko): Propose an algorithm that considers tablet's key type, number of delete rowsets, + // number of tablet versions simultaneously. + return _approximate_cumu_num_deltas.load(std::memory_order_relaxed); +} + // return a json string to show the compaction status of this tablet void CloudTablet::get_compaction_status(std::string* json_result) { rapidjson::Document root; diff --git a/be/src/cloud/cloud_tablet.h b/be/src/cloud/cloud_tablet.h index 537c8fe134d13f..bf8db3c9451932 100644 --- a/be/src/cloud/cloud_tablet.h +++ b/be/src/cloud/cloud_tablet.h @@ -93,6 +93,17 @@ class CloudTablet final : public BaseTablet { // Return number of deleted stale rowsets int delete_expired_stale_rowsets(); + bool has_stale_rowsets() const { return !_stale_rs_version_map.empty(); } + + int64_t get_cloud_base_compaction_score() const; + int64_t get_cloud_cumu_compaction_score() const; + + int64_t last_sync_time_s = 0; + int64_t last_load_time_ms = 0; + int64_t last_base_compaction_success_time_ms = 0; + int64_t last_cumu_compaction_success_time_ms = 0; + int64_t last_cumu_no_suitable_version_ms = 0; + private: Versions calc_missed_versions(int64_t spec_version); diff --git a/be/src/cloud/cloud_tablet_mgr.cpp b/be/src/cloud/cloud_tablet_mgr.cpp new file mode 100644 index 00000000000000..b3ad35257cc1a8 --- /dev/null +++ b/be/src/cloud/cloud_tablet_mgr.cpp @@ -0,0 +1,356 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include "cloud/cloud_tablet_mgr.h" + +#include + +#include "cloud/cloud_meta_mgr.h" +#include "cloud/cloud_storage_engine.h" +#include "cloud/cloud_tablet.h" +#include "cloud/config.h" +#include "common/status.h" +#include "olap/lru_cache.h" +#include "runtime/memory/cache_policy.h" + +namespace doris { +namespace { + +// port from +// https://github.com/golang/groupcache/blob/master/singleflight/singleflight.go +template +class SingleFlight { +public: + SingleFlight() = default; + + SingleFlight(const SingleFlight&) = delete; + void operator=(const SingleFlight&) = delete; + + using Loader = std::function; + + // Do executes and returns the results of the given function, making + // sure that only one execution is in-flight for a given key at a + // time. If a duplicate comes in, the duplicate caller waits for the + // original to complete and receives the same results. + Val load(const Key& key, Loader loader) { + std::unique_lock lock(_call_map_mtx); + + auto it = _call_map.find(key); + if (it != _call_map.end()) { + auto call = it->second; + lock.unlock(); + if (int ec = call->event.wait(); ec != 0) { + throw std::system_error(std::error_code(ec, std::system_category()), + "CountdownEvent wait failed"); + } + return call->val; + } + auto call = std::make_shared(); + _call_map.emplace(key, call); + lock.unlock(); + + call->val = loader(key); + call->event.signal(); + + lock.lock(); + _call_map.erase(key); + lock.unlock(); + + return call->val; + } + +private: + // `Call` is an in-flight or completed `load` call + struct Call { + bthread::CountdownEvent event; + Val val; + }; + + std::mutex _call_map_mtx; + std::unordered_map> _call_map; +}; + +SingleFlight> s_singleflight_load_tablet; + +} // namespace + +// tablet_id -> cached tablet +// This map owns all cached tablets. The lifetime of tablet can be longer than the LRU handle. +// It's also used for scenarios where users want to access the tablet by `tablet_id` without changing the LRU order. +// TODO(plat1ko): multi shard to increase concurrency +class CloudTabletMgr::TabletMap { +public: + void put(std::shared_ptr tablet) { + std::lock_guard lock(_mtx); + _map[tablet->tablet_id()] = std::move(tablet); + } + + void erase(CloudTablet* tablet) { + std::lock_guard lock(_mtx); + auto it = _map.find(tablet->tablet_id()); + // According to the implementation of `LRUCache`, `deleter` may be called after a tablet + // with same tablet id insert into cache and `TabletMap`. So we MUST check if the tablet + // instance to be erased is the same one in the map. + if (it != _map.end() && it->second.get() == tablet) { + _map.erase(it); + } + } + + std::shared_ptr get(int64_t tablet_id) { + std::lock_guard lock(_mtx); + if (auto it = _map.find(tablet_id); it != _map.end()) { + return it->second; + } + return nullptr; + } + + size_t size() { return _map.size(); } + + void traverse(std::function&)> visitor) { + std::lock_guard lock(_mtx); + for (auto& [_, tablet] : _map) { + visitor(tablet); + } + } + +private: + std::mutex _mtx; + std::unordered_map> _map; +}; + +// TODO(plat1ko): Prune cache +CloudTabletMgr::CloudTabletMgr(CloudStorageEngine& engine) + : _engine(engine), + _tablet_map(std::make_unique()), + _cache(std::make_unique( + CachePolicy::CacheType::CLOUD_TABLET_CACHE, config::tablet_cache_capacity, + LRUCacheType::NUMBER, 0, config::tablet_cache_shards)) {} + +CloudTabletMgr::~CloudTabletMgr() = default; + +Result> CloudTabletMgr::get_tablet(int64_t tablet_id, + bool warmup_data) { + // LRU value type. `Value`'s lifetime MUST NOT be longer than `CloudTabletMgr` + struct Value { + // FIXME(plat1ko): The ownership of tablet seems to belong to 'TabletMap', while `Value` + // only requires a reference. + std::shared_ptr tablet; + TabletMap& tablet_map; + }; + + auto tablet_id_str = std::to_string(tablet_id); + CacheKey key(tablet_id_str); + auto* cache = _cache->cache(); + auto* handle = cache->lookup(key); + if (handle == nullptr) { + auto load_tablet = [this, cache, &key, + warmup_data](int64_t tablet_id) -> std::shared_ptr { + TabletMetaSharedPtr tablet_meta; + auto st = _engine.meta_mgr().get_tablet_meta(tablet_id, &tablet_meta); + if (!st.ok()) { + LOG(WARNING) << "failed to tablet " << tablet_id << ": " << st; + return nullptr; + } + + auto tablet = std::make_shared(_engine, std::move(tablet_meta)); + auto value = std::make_unique(Value { + .tablet = tablet, + .tablet_map = *_tablet_map, + }); + // MUST sync stats to let compaction scheduler work correctly + st = _engine.meta_mgr().sync_tablet_rowsets(tablet.get(), warmup_data); + if (!st.ok()) { + LOG(WARNING) << "failed to sync tablet " << tablet_id << ": " << st; + return nullptr; + } + + auto deleter = [](const CacheKey& key, void* value) { + auto* value1 = reinterpret_cast(value); + // tablet has been evicted, release it from `tablet_map` + value1->tablet_map.erase(value1->tablet.get()); + delete value1; + }; + + auto* handle = cache->insert(key, value.release(), 1, deleter); + auto ret = std::shared_ptr( + tablet.get(), [cache, handle](...) { cache->release(handle); }); + _tablet_map->put(std::move(tablet)); + return ret; + }; + + auto tablet = s_singleflight_load_tablet.load(tablet_id, std::move(load_tablet)); + if (tablet == nullptr) { + return ResultError(Status::InternalError("failed to get tablet {}", tablet_id)); + } + return tablet; + } + + CloudTablet* tablet_raw_ptr = reinterpret_cast(cache->value(handle))->tablet.get(); + auto tablet = std::shared_ptr(tablet_raw_ptr, + [cache, handle](...) { cache->release(handle); }); + return tablet; +} + +void CloudTabletMgr::erase_tablet(int64_t tablet_id) { + auto tablet_id_str = std::to_string(tablet_id); + CacheKey key(tablet_id_str.data(), tablet_id_str.size()); + _cache->cache()->erase(key); +} + +void CloudTabletMgr::vacuum_stale_rowsets() { + LOG_INFO("begin to vacuum stale rowsets"); + std::vector> tablets_to_vacuum; + tablets_to_vacuum.reserve(_tablet_map->size()); + _tablet_map->traverse([&tablets_to_vacuum](auto&& t) { + if (t->has_stale_rowsets()) { + tablets_to_vacuum.push_back(t); + } + }); + int num_vacuumed = 0; + for (auto& t : tablets_to_vacuum) { + num_vacuumed += t->delete_expired_stale_rowsets(); + } + LOG_INFO("finish vacuum stale rowsets").tag("num_vacuumed", num_vacuumed); +} + +std::vector> CloudTabletMgr::get_weak_tablets() { + std::vector> weak_tablets; + weak_tablets.reserve(_tablet_map->size()); + _tablet_map->traverse([&weak_tablets](auto& t) { weak_tablets.push_back(t); }); + return weak_tablets; +} + +void CloudTabletMgr::sync_tablets() { + LOG_INFO("begin to sync tablets"); + int64_t last_sync_time_bound = ::time(nullptr) - config::tablet_sync_interval_seconds; + + auto weak_tablets = get_weak_tablets(); + + // sort by last_sync_time + static auto cmp = [](const auto& a, const auto& b) { return a.first < b.first; }; + std::multiset>, decltype(cmp)> + sync_time_tablet_set(cmp); + + for (auto& weak_tablet : weak_tablets) { + if (auto tablet = weak_tablet.lock()) { + if (tablet->tablet_state() != TABLET_RUNNING) { + continue; + } + int64_t last_sync_time = tablet->last_sync_time_s; + if (last_sync_time <= last_sync_time_bound) { + sync_time_tablet_set.emplace(last_sync_time, weak_tablet); + } + } + } + + int num_sync = 0; + for (auto&& [_, weak_tablet] : sync_time_tablet_set) { + if (auto tablet = weak_tablet.lock()) { + if (tablet->last_sync_time_s > last_sync_time_bound) { + continue; + } + + ++num_sync; + auto st = tablet->sync_meta(); + if (!st) { + LOG_WARNING("failed to sync tablet meta {}", tablet->tablet_id()).error(st); + if (st.is()) { + continue; + } + } + + st = tablet->sync_rowsets(-1); + if (!st) { + LOG_WARNING("failed to sync tablet rowsets {}", tablet->tablet_id()).error(st); + } + } + } + LOG_INFO("finish sync tablets").tag("num_sync", num_sync); +} + +Status CloudTabletMgr::get_topn_tablets_to_compact( + int n, CompactionType compaction_type, const std::function& filter_out, + std::vector>* tablets, int64_t* max_score) { + DCHECK(compaction_type == CompactionType::BASE_COMPACTION || + compaction_type == CompactionType::CUMULATIVE_COMPACTION); + *max_score = 0; + int64_t max_score_tablet_id = 0; + // clang-format off + auto score = [compaction_type](CloudTablet* t) { + return compaction_type == CompactionType::BASE_COMPACTION ? t->get_cloud_base_compaction_score() + : compaction_type == CompactionType::CUMULATIVE_COMPACTION ? t->get_cloud_cumu_compaction_score() + : 0; + }; + + using namespace std::chrono; + auto now = duration_cast(system_clock::now().time_since_epoch()).count(); + auto skip = [now, compaction_type](CloudTablet* t) { + if (compaction_type == CompactionType::BASE_COMPACTION) { + return now - t->last_base_compaction_success_time_ms < config::base_compaction_freeze_interval_s * 1000; + } + // If tablet has too many rowsets but not be compacted for a long time, compaction should be performed + // regardless of whether there is a load job recently. + return now - t->last_cumu_no_suitable_version_ms < config::min_compaction_failure_interval_ms || + (now - t->last_load_time_ms > config::cu_compaction_freeze_interval_s * 1000 + && now - t->last_cumu_compaction_success_time_ms < config::cumu_compaction_interval_s * 1000 + && t->fetch_add_approximate_num_rowsets(0) < config::max_tablet_version_num / 2); + }; + // We don't schedule tablets that are disabled for compaction + auto disable = [](CloudTablet* t) { return t->tablet_meta()->tablet_schema()->disable_auto_compaction(); }; + + auto [num_filtered, num_disabled, num_skipped] = std::make_tuple(0, 0, 0); + + auto weak_tablets = get_weak_tablets(); + std::vector, int64_t>> buf; + buf.reserve(n + 1); + for (auto& weak_tablet : weak_tablets) { + auto t = weak_tablet.lock(); + if (t == nullptr) { continue; } + + int64_t s = score(t.get()); + if (s > *max_score) { + max_score_tablet_id = t->tablet_id(); + *max_score = s; + } + + if (filter_out(t.get())) { ++num_filtered; continue; } + if (disable(t.get())) { ++num_disabled; continue; } + if (skip(t.get())) { ++num_skipped; continue; } + + buf.emplace_back(std::move(t), s); + std::sort(buf.begin(), buf.end(), [](auto& a, auto& b) { return a.second > b.second; }); + if (buf.size() > n) { buf.pop_back(); } + } + + LOG_EVERY_N(INFO, 1000) << "get_topn_compaction_score, n=" << n << " type=" << compaction_type + << " num_tablets=" << weak_tablets.size() << " num_skipped=" << num_skipped + << " num_disabled=" << num_disabled << " num_filtered=" << num_filtered + << " max_score=" << *max_score << " max_score_tablet=" << max_score_tablet_id + << " tablets=[" << [&buf] { std::stringstream ss; for (auto& i : buf) ss << i.first->tablet_id() << ":" << i.second << ","; return ss.str(); }() << "]" + ; + // clang-format on + + tablets->clear(); + tablets->reserve(n + 1); + for (auto& [t, _] : buf) { + tablets->emplace_back(std::move(t)); + } + + return Status::OK(); +} + +} // namespace doris diff --git a/be/src/cloud/cloud_tablet_mgr.h b/be/src/cloud/cloud_tablet_mgr.h new file mode 100644 index 00000000000000..6e8ae2c17b0fb4 --- /dev/null +++ b/be/src/cloud/cloud_tablet_mgr.h @@ -0,0 +1,75 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#pragma once + +#include +#include +#include + +#include "common/status.h" +#include "olap/olap_common.h" + +namespace doris { + +class CloudTablet; +class CloudStorageEngine; +class LRUCachePolicy; + +class CloudTabletMgr { +public: + CloudTabletMgr(CloudStorageEngine& engine); + ~CloudTabletMgr(); + + // If the tablet is in cache, return this tablet directly; otherwise will get tablet meta first, + // sync rowsets after, and download segment data in background if `warmup_data` is true. + Result> get_tablet(int64_t tablet_id, bool warmup_data = false); + + void erase_tablet(int64_t tablet_id); + + void vacuum_stale_rowsets(); + + // Return weak ptr of all cached tablets. + // We return weak ptr to avoid extend lifetime of tablets that are no longer cached. + std::vector> get_weak_tablets(); + + void sync_tablets(); + + /** + * Gets top N tablets that are considered to be compacted first + * + * @param n max number of tablets to get, all of them are comapction enabled + * @param filter_out a filter takes a tablet and return bool to check + * whether skipping the tablet, true for skip + * @param tablets output param + * @param max_score output param, max score of existed tablets + * @return status of this call + */ + Status get_topn_tablets_to_compact(int n, CompactionType compaction_type, + const std::function& filter_out, + std::vector>* tablets, + int64_t* max_score); + +private: + CloudStorageEngine& _engine; + + class TabletMap; + std::unique_ptr _tablet_map; + std::unique_ptr _cache; +}; + +} // namespace doris diff --git a/be/src/cloud/config.cpp b/be/src/cloud/config.cpp index d7513574037474..b222ec9d517883 100644 --- a/be/src/cloud/config.cpp +++ b/be/src/cloud/config.cpp @@ -30,4 +30,13 @@ DEFINE_mInt32(meta_service_idle_connection_timeout_ms, "0"); DEFINE_mInt32(meta_service_rpc_retry_times, "200"); DEFINE_mInt32(meta_service_brpc_timeout_ms, "10000"); +DEFINE_Int64(tablet_cache_capacity, "100000"); +DEFINE_Int64(tablet_cache_shards, "16"); +DEFINE_mInt64(tablet_sync_interval_seconds, "1800"); + +DEFINE_mInt64(min_compaction_failure_interval_ms, "5000"); +DEFINE_mInt64(base_compaction_freeze_interval_s, "86400"); +DEFINE_mInt64(cu_compaction_freeze_interval_s, "1200"); +DEFINE_mInt64(cumu_compaction_interval_s, "1800"); + } // namespace doris::config diff --git a/be/src/cloud/config.h b/be/src/cloud/config.h index 0a2ceab3e5aa5e..ccb7b834f9de91 100644 --- a/be/src/cloud/config.h +++ b/be/src/cloud/config.h @@ -41,4 +41,16 @@ DECLARE_mInt32(meta_service_rpc_retry_times); // default brpc timeout DECLARE_mInt32(meta_service_brpc_timeout_ms); +// CloudTabletMgr config +DECLARE_Int64(tablet_cache_capacity); +DECLARE_Int64(tablet_cache_shards); +DECLARE_mInt64(tablet_sync_interval_seconds); + +// Cloud compaction config +DECLARE_mInt64(min_compaction_failure_interval_ms); +// For cloud read/write seperate mode +DECLARE_mInt64(base_compaction_freeze_interval_s); +DECLARE_mInt64(cu_compaction_freeze_interval_s); +DECLARE_mInt64(cumu_compaction_interval_s); + } // namespace doris::config diff --git a/be/src/runtime/memory/cache_policy.h b/be/src/runtime/memory/cache_policy.h index 9a9f2c36e84d35..c19066b0be217c 100644 --- a/be/src/runtime/memory/cache_policy.h +++ b/be/src/runtime/memory/cache_policy.h @@ -42,7 +42,8 @@ class CachePolicy { COMMON_OBJ_LRU_CACHE = 12, FOR_UT = 13, TABLET_SCHEMA_CACHE = 14, - CREATE_TABLET_RR_IDX_CACHE = 15 + CREATE_TABLET_RR_IDX_CACHE = 15, + CLOUD_TABLET_CACHE = 16, }; static std::string type_string(CacheType type) { @@ -79,6 +80,8 @@ class CachePolicy { return "TabletSchemaCache"; case CacheType::CREATE_TABLET_RR_IDX_CACHE: return "CreateTabletRRIdxCache"; + case CacheType::CLOUD_TABLET_CACHE: + return "CloudTabletCache"; default: LOG(FATAL) << "not match type of cache policy :" << static_cast(type); } From c2b7de8f77dbfb2d687c102f50e4000a49197c32 Mon Sep 17 00:00:00 2001 From: Calvin Kirs Date: Fri, 19 Jan 2024 09:51:34 +0800 Subject: [PATCH 087/200] [Fix](Show-Delete)Missing Delete job information causes query exception (#30092) --- .../org/apache/doris/load/DeleteInfo.java | 16 ++--- .../java/org/apache/doris/load/DeleteJob.java | 6 +- .../suites/show_p0/test_show_delete.groovy | 67 +++++++++++++++++++ 3 files changed, 80 insertions(+), 9 deletions(-) create mode 100644 regression-test/suites/show_p0/test_show_delete.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteInfo.java b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteInfo.java index 05c6c4b1a86bdc..89114d5beedb3b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteInfo.java @@ -60,12 +60,19 @@ public class DeleteInfo implements Writable, GsonPostProcessable { @SerializedName(value = "partitionName") private String partitionName; - public DeleteInfo(long dbId, long tableId, String tableName, List deleteConditions) { + public DeleteInfo(long dbId, long tableId, String tableName, List deleteConditions, + boolean noPartitionSpecified, List partitionIds, List partitionNames) { this.dbId = dbId; this.tableId = tableId; this.tableName = tableName; this.deleteConditions = deleteConditions; this.createTimeMs = System.currentTimeMillis(); + this.noPartitionSpecified = noPartitionSpecified; + if (!noPartitionSpecified) { + Preconditions.checkState(partitionIds.size() == partitionNames.size()); + this.partitionIds = partitionIds; + this.partitionNames = partitionNames; + } } public long getDbId() { @@ -92,13 +99,6 @@ public boolean isNoPartitionSpecified() { return noPartitionSpecified; } - public void setPartitions(boolean noPartitionSpecified, List partitionIds, List partitionNames) { - this.noPartitionSpecified = noPartitionSpecified; - Preconditions.checkState(partitionIds.size() == partitionNames.size()); - this.partitionIds = partitionIds; - this.partitionNames = partitionNames; - } - public List getPartitionIds() { return partitionIds; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteJob.java index 5bb07dd7b68c9e..a764b42773b0c1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteJob.java @@ -490,6 +490,7 @@ public List getDeleteConditions() { public static class Builder { public DeleteJob buildWith(BuildParams params) throws Exception { + boolean noPartitionSpecified = params.getPartitionNames().isEmpty(); List partitions = getSelectedPartitions(params.getTable(), params.getPartitionNames(), params.getDeleteConditions()); Map partitionReplicaNum = partitions.stream() @@ -504,8 +505,11 @@ public DeleteJob buildWith(BuildParams params) throws Exception { String label = DELETE_PREFIX + UUID.randomUUID(); //generate jobId long jobId = Env.getCurrentEnv().getNextId(); + List partitionNames = partitions.stream().map(Partition::getName).collect(Collectors.toList()); + List partitionIds = partitions.stream().map(Partition::getId).collect(Collectors.toList()); DeleteInfo deleteInfo = new DeleteInfo(params.getDb().getId(), params.getTable().getId(), - params.getTable().getName(), getDeleteCondString(params.getDeleteConditions())); + params.getTable().getName(), getDeleteCondString(params.getDeleteConditions()), + noPartitionSpecified, partitionIds, partitionNames); DeleteJob deleteJob = new DeleteJob(jobId, -1, label, partitionReplicaNum, deleteInfo); long replicaNum = partitions.stream().mapToLong(Partition::getAllReplicaCount).sum(); deleteJob.setPartitions(partitions); diff --git a/regression-test/suites/show_p0/test_show_delete.groovy b/regression-test/suites/show_p0/test_show_delete.groovy new file mode 100644 index 00000000000000..bc0c0c561ef3d5 --- /dev/null +++ b/regression-test/suites/show_p0/test_show_delete.groovy @@ -0,0 +1,67 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +suite("test_show_delete") { + def tableName = "test_show_delete_table" + sql """drop table if exists ${tableName}""" + + sql """ + CREATE TABLE IF NOT EXISTS ${tableName} + ( + `datetime` DATE NOT NULL COMMENT "['0000-01-01', '9999-12-31']", + `type` TINYINT NOT NULL COMMENT "[-128, 127]", + `user_id` decimal(9,3) COMMENT "[-9223372036854775808, 9223372036854775807]" + ) + UNIQUE KEY(`datetime`) + PARTITION BY RANGE(`datetime`) + ( + PARTITION `Feb` VALUES LESS THAN ("2022-03-01"), + PARTITION `Mar` VALUES LESS THAN ("2022-04-01") + ) + DISTRIBUTED BY HASH(`datetime`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql """insert into ${tableName} values ('2022-02-01', 1, 1.1), ('2022-03-01', 2, 2.2)""" + + sql """ set delete_without_partition = true""" + // don't care nereids planner + sql """ delete from ${tableName} PARTITION Mar where type ='2'""" + sql """ delete from ${tableName} where type ='1'""" + def showDeleteResult = sql """ show delete""" + //When we test locally, multiple history results will be included, so size will be >= 2 + assert showDeleteResult.size() >= 2 + def count = 0 + showDeleteResult.each { row -> + + if (row[3] == 'type EQ "2"') { + assert row[1] == 'Mar' + count++ + return + } + if (row[3] == 'type EQ "1"') { + assert row[1] == '*' + count++ + return + } + + } + assert count == showDeleteResult.size() + +} \ No newline at end of file From 53a3d787f34d1229f94702f50d005ee8b8b053d0 Mon Sep 17 00:00:00 2001 From: qiye Date: Fri, 19 Jan 2024 10:05:05 +0800 Subject: [PATCH 088/200] [opt](docker)optimize ES docker compose (#30068) 1. add volume for es logs 2. optimize health check, waiting for es status to be green 3. fix es6 valume path error 4. optimize disk watermark to avoid es disk watermark error 5. fix es6 create index error 6. add custom elasticsearch.yml for es6 7. add log4j2.properties for es6, es7, es8 --- .../config/es6/elasticsearch.yml | 62 ++++ .../config/es6/log4j2.properties | 205 ++++++++++++ .../config/es7/log4j2.properties | 299 ++++++++++++++++++ .../config/es8/log4j2.properties | 273 ++++++++++++++++ .../docker-compose/elasticsearch/es.yaml.tpl | 28 +- .../scripts/index/es6_test1.json | 2 +- .../scripts/index/es6_test2.json | 2 +- .../thirdparties/run-thirdparties-docker.sh | 8 + .../external_table_p0/es/test_es_query.out | 42 +-- .../es/test_es_query_no_http_url.out | 2 +- 10 files changed, 894 insertions(+), 29 deletions(-) create mode 100755 docker/thirdparties/docker-compose/elasticsearch/config/es6/elasticsearch.yml create mode 100755 docker/thirdparties/docker-compose/elasticsearch/config/es6/log4j2.properties create mode 100755 docker/thirdparties/docker-compose/elasticsearch/config/es7/log4j2.properties create mode 100755 docker/thirdparties/docker-compose/elasticsearch/config/es8/log4j2.properties diff --git a/docker/thirdparties/docker-compose/elasticsearch/config/es6/elasticsearch.yml b/docker/thirdparties/docker-compose/elasticsearch/config/es6/elasticsearch.yml new file mode 100755 index 00000000000000..6cf99d51b8fb60 --- /dev/null +++ b/docker/thirdparties/docker-compose/elasticsearch/config/es6/elasticsearch.yml @@ -0,0 +1,62 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +cluster: + name: elasticsearch6 + routing: + allocation: + disk: + threshold_enabled: true + watermark: + flood_stage: 200mb + low: 500mb + high: 300mb + +node: + name: 2bf5838228d8 + master: true + data: true + ingest: true + +path: + data: /var/lib/elasticsearch/data + logs: /var/lib/elasticsearch/logs + +network: + host: 0.0.0.0 + + +http: + compression: true + cors: + enabled: false + + +bootstrap: + memory_lock: false + +discovery: + zen: + minimum_master_nodes: 1 + + +action: + destructive_requires_name: false + +xpack: + security: + enabled: false diff --git a/docker/thirdparties/docker-compose/elasticsearch/config/es6/log4j2.properties b/docker/thirdparties/docker-compose/elasticsearch/config/es6/log4j2.properties new file mode 100755 index 00000000000000..3ab4e253748683 --- /dev/null +++ b/docker/thirdparties/docker-compose/elasticsearch/config/es6/log4j2.properties @@ -0,0 +1,205 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +status = error + +# log action execution errors for easier debugging +logger.action.name = org.elasticsearch.action +logger.action.level = debug + +appender.console.type = Console +appender.console.name = console +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%d{ISO8601}][%-5p][%-25c{1.}] [%node_name]%marker %m%n + +appender.rolling.type = RollingFile +appender.rolling.name = rolling +appender.rolling.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}.log +appender.rolling.layout.type = PatternLayout +appender.rolling.layout.pattern = [%d{ISO8601}][%-5p][%-25c{1.}] [%node_name]%marker %.-10000m%n +appender.rolling.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}-%d{yyyy-MM-dd}-%i.log.gz +appender.rolling.policies.type = Policies +appender.rolling.policies.time.type = TimeBasedTriggeringPolicy +appender.rolling.policies.time.interval = 1 +appender.rolling.policies.time.modulate = true +appender.rolling.policies.size.type = SizeBasedTriggeringPolicy +appender.rolling.policies.size.size = 128MB +appender.rolling.strategy.type = DefaultRolloverStrategy +appender.rolling.strategy.fileIndex = nomax +appender.rolling.strategy.action.type = Delete +appender.rolling.strategy.action.basepath = ${sys:es.logs.base_path} +appender.rolling.strategy.action.condition.type = IfFileName +appender.rolling.strategy.action.condition.glob = ${sys:es.logs.cluster_name}-* +appender.rolling.strategy.action.condition.nested_condition.type = IfAccumulatedFileSize +appender.rolling.strategy.action.condition.nested_condition.exceeds = 2GB + +rootLogger.level = info +rootLogger.appenderRef.console.ref = console +rootLogger.appenderRef.rolling.ref = rolling + +appender.deprecation_rolling.type = RollingFile +appender.deprecation_rolling.name = deprecation_rolling +appender.deprecation_rolling.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_deprecation.log +appender.deprecation_rolling.layout.type = PatternLayout +appender.deprecation_rolling.layout.pattern = [%d{ISO8601}][%-5p][%-25c{1.}] [%node_name]%marker %.-10000m%n +appender.deprecation_rolling.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_deprecation-%i.log.gz +appender.deprecation_rolling.policies.type = Policies +appender.deprecation_rolling.policies.size.type = SizeBasedTriggeringPolicy +appender.deprecation_rolling.policies.size.size = 1GB +appender.deprecation_rolling.strategy.type = DefaultRolloverStrategy +appender.deprecation_rolling.strategy.max = 4 + +logger.deprecation.name = org.elasticsearch.deprecation +logger.deprecation.level = warn +logger.deprecation.appenderRef.deprecation_rolling.ref = deprecation_rolling +logger.deprecation.additivity = false + +appender.index_search_slowlog_rolling.type = RollingFile +appender.index_search_slowlog_rolling.name = index_search_slowlog_rolling +appender.index_search_slowlog_rolling.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_index_search_slowlog.log +appender.index_search_slowlog_rolling.layout.type = PatternLayout +appender.index_search_slowlog_rolling.layout.pattern = [%d{ISO8601}][%-5p][%-25c] [%node_name]%marker %.-10000m%n +appender.index_search_slowlog_rolling.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_index_search_slowlog-%d{yyyy-MM-dd}.log +appender.index_search_slowlog_rolling.policies.type = Policies +appender.index_search_slowlog_rolling.policies.time.type = TimeBasedTriggeringPolicy +appender.index_search_slowlog_rolling.policies.time.interval = 1 +appender.index_search_slowlog_rolling.policies.time.modulate = true + +logger.index_search_slowlog_rolling.name = index.search.slowlog +logger.index_search_slowlog_rolling.level = trace +logger.index_search_slowlog_rolling.appenderRef.index_search_slowlog_rolling.ref = index_search_slowlog_rolling +logger.index_search_slowlog_rolling.additivity = false + +appender.index_indexing_slowlog_rolling.type = RollingFile +appender.index_indexing_slowlog_rolling.name = index_indexing_slowlog_rolling +appender.index_indexing_slowlog_rolling.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_index_indexing_slowlog.log +appender.index_indexing_slowlog_rolling.layout.type = PatternLayout +appender.index_indexing_slowlog_rolling.layout.pattern = [%d{ISO8601}][%-5p][%-25c] [%node_name]%marker %.-10000m%n +appender.index_indexing_slowlog_rolling.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_index_indexing_slowlog-%d{yyyy-MM-dd}.log +appender.index_indexing_slowlog_rolling.policies.type = Policies +appender.index_indexing_slowlog_rolling.policies.time.type = TimeBasedTriggeringPolicy +appender.index_indexing_slowlog_rolling.policies.time.interval = 1 +appender.index_indexing_slowlog_rolling.policies.time.modulate = true + +logger.index_indexing_slowlog.name = index.indexing.slowlog.index +logger.index_indexing_slowlog.level = trace +logger.index_indexing_slowlog.appenderRef.index_indexing_slowlog_rolling.ref = index_indexing_slowlog_rolling +logger.index_indexing_slowlog.additivity = false + + +appender.audit_rolling.type = RollingFile +appender.audit_rolling.name = audit_rolling +appender.audit_rolling.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_audit.log +appender.audit_rolling.layout.type = PatternLayout +appender.audit_rolling.layout.pattern = {\ + "@timestamp":"%d{ISO8601}"\ + %varsNotEmpty{, "node.name":"%enc{%map{node.name}}{JSON}"}\ + %varsNotEmpty{, "node.id":"%enc{%map{node.id}}{JSON}"}\ + %varsNotEmpty{, "host.name":"%enc{%map{host.name}}{JSON}"}\ + %varsNotEmpty{, "host.ip":"%enc{%map{host.ip}}{JSON}"}\ + %varsNotEmpty{, "event.type":"%enc{%map{event.type}}{JSON}"}\ + %varsNotEmpty{, "event.action":"%enc{%map{event.action}}{JSON}"}\ + %varsNotEmpty{, "user.name":"%enc{%map{user.name}}{JSON}"}\ + %varsNotEmpty{, "user.run_by.name":"%enc{%map{user.run_by.name}}{JSON}"}\ + %varsNotEmpty{, "user.run_as.name":"%enc{%map{user.run_as.name}}{JSON}"}\ + %varsNotEmpty{, "user.realm":"%enc{%map{user.realm}}{JSON}"}\ + %varsNotEmpty{, "user.run_by.realm":"%enc{%map{user.run_by.realm}}{JSON}"}\ + %varsNotEmpty{, "user.run_as.realm":"%enc{%map{user.run_as.realm}}{JSON}"}\ + %varsNotEmpty{, "user.roles":%map{user.roles}}\ + %varsNotEmpty{, "origin.type":"%enc{%map{origin.type}}{JSON}"}\ + %varsNotEmpty{, "origin.address":"%enc{%map{origin.address}}{JSON}"}\ + %varsNotEmpty{, "realm":"%enc{%map{realm}}{JSON}"}\ + %varsNotEmpty{, "url.path":"%enc{%map{url.path}}{JSON}"}\ + %varsNotEmpty{, "url.query":"%enc{%map{url.query}}{JSON}"}\ + %varsNotEmpty{, "request.method":"%enc{%map{request.method}}{JSON}"}\ + %varsNotEmpty{, "request.body":"%enc{%map{request.body}}{JSON}"}\ + %varsNotEmpty{, "request.id":"%enc{%map{request.id}}{JSON}"}\ + %varsNotEmpty{, "action":"%enc{%map{action}}{JSON}"}\ + %varsNotEmpty{, "request.name":"%enc{%map{request.name}}{JSON}"}\ + %varsNotEmpty{, "indices":%map{indices}}\ + %varsNotEmpty{, "opaque_id":"%enc{%map{opaque_id}}{JSON}"}\ + %varsNotEmpty{, "x_forwarded_for":"%enc{%map{x_forwarded_for}}{JSON}"}\ + %varsNotEmpty{, "transport.profile":"%enc{%map{transport.profile}}{JSON}"}\ + %varsNotEmpty{, "rule":"%enc{%map{rule}}{JSON}"}\ + %varsNotEmpty{, "event.category":"%enc{%map{event.category}}{JSON}"}\ + }%n +# "node.name" node name from the `elasticsearch.yml` settings +# "node.id" node id which should not change between cluster restarts +# "host.name" unresolved hostname of the local node +# "host.ip" the local bound ip (i.e. the ip listening for connections) +# "event.type" a received REST request is translated into one or more transport requests. This indicates which processing layer generated the event "rest" or "transport" (internal) +# "event.action" the name of the audited event, eg. "authentication_failed", "access_granted", "run_as_granted", etc. +# "user.name" the subject name as authenticated by a realm +# "user.run_by.name" the original authenticated subject name that is impersonating another one. +# "user.run_as.name" if this "event.action" is of a run_as type, this is the subject name to be impersonated as. +# "user.realm" the name of the realm that authenticated "user.name" +# "user.run_by.realm" the realm name of the impersonating subject ("user.run_by.name") +# "user.run_as.realm" if this "event.action" is of a run_as type, this is the realm name the impersonated user is looked up from +# "user.roles" the roles array of the user; these are the roles that are granting privileges +# "origin.type" it is "rest" if the event is originating (is in relation to) a REST request; possible other values are "transport" and "ip_filter" +# "origin.address" the remote address and port of the first network hop, i.e. a REST proxy or another cluster node +# "realm" name of a realm that has generated an "authentication_failed" or an "authentication_successful"; the subject is not yet authenticated +# "url.path" the URI component between the port and the query string; it is percent (URL) encoded +# "url.query" the URI component after the path and before the fragment; it is percent (URL) encoded +# "request.method" the method of the HTTP request, i.e. one of GET, POST, PUT, DELETE, OPTIONS, HEAD, PATCH, TRACE, CONNECT +# "request.body" the content of the request body entity, JSON escaped +# "request.id" a synthentic identifier for the incoming request, this is unique per incoming request, and consistent across all audit events generated by that request +# "action" an action is the most granular operation that is authorized and this identifies it in a namespaced way (internal) +# "request.name" if the event is in connection to a transport message this is the name of the request class, similar to how rest requests are identified by the url path (internal) +# "indices" the array of indices that the "action" is acting upon +# "opaque_id" opaque value conveyed by the "X-Opaque-Id" request header +# "x_forwarded_for" the addresses from the "X-Forwarded-For" request header, as a verbatim string value (not an array) +# "transport.profile" name of the transport profile in case this is a "connection_granted" or "connection_denied" event +# "rule" name of the applied rulee if the "origin.type" is "ip_filter" +# "event.category" fixed value "elasticsearch-audit" + +appender.audit_rolling.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_audit-%d{yyyy-MM-dd}.log +appender.audit_rolling.policies.type = Policies +appender.audit_rolling.policies.time.type = TimeBasedTriggeringPolicy +appender.audit_rolling.policies.time.interval = 1 +appender.audit_rolling.policies.time.modulate = true + +appender.deprecated_audit_rolling.type = RollingFile +appender.deprecated_audit_rolling.name = deprecated_audit_rolling +appender.deprecated_audit_rolling.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_access.log +appender.deprecated_audit_rolling.layout.type = PatternLayout +appender.deprecated_audit_rolling.layout.pattern = [%d{ISO8601}] %m%n +appender.deprecated_audit_rolling.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_access-%d{yyyy-MM-dd}.log +appender.deprecated_audit_rolling.policies.type = Policies +appender.deprecated_audit_rolling.policies.time.type = TimeBasedTriggeringPolicy +appender.deprecated_audit_rolling.policies.time.interval = 1 +appender.deprecated_audit_rolling.policies.time.modulate = true + +logger.xpack_security_audit_logfile.name = org.elasticsearch.xpack.security.audit.logfile.LoggingAuditTrail +logger.xpack_security_audit_logfile.level = info +logger.xpack_security_audit_logfile.appenderRef.audit_rolling.ref = audit_rolling +logger.xpack_security_audit_logfile.additivity = false + +logger.xpack_security_audit_deprecated_logfile.name = org.elasticsearch.xpack.security.audit.logfile.DeprecatedLoggingAuditTrail +# set this to "off" instead of "info" to disable the deprecated appender +# in the 6.x releases both the new and the previous appenders are enabled +# for the logfile auditing +logger.xpack_security_audit_deprecated_logfile.level = info +logger.xpack_security_audit_deprecated_logfile.appenderRef.deprecated_audit_rolling.ref = deprecated_audit_rolling +logger.xpack_security_audit_deprecated_logfile.additivity = false + +logger.xmlsig.name = org.apache.xml.security.signature.XMLSignature +logger.xmlsig.level = error +logger.samlxml_decrypt.name = org.opensaml.xmlsec.encryption.support.Decrypter +logger.samlxml_decrypt.level = fatal +logger.saml2_decrypt.name = org.opensaml.saml.saml2.encryption.Decrypter +logger.saml2_decrypt.level = fatal \ No newline at end of file diff --git a/docker/thirdparties/docker-compose/elasticsearch/config/es7/log4j2.properties b/docker/thirdparties/docker-compose/elasticsearch/config/es7/log4j2.properties new file mode 100755 index 00000000000000..0e033f21dce725 --- /dev/null +++ b/docker/thirdparties/docker-compose/elasticsearch/config/es7/log4j2.properties @@ -0,0 +1,299 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +status = error + +appender.console.type = Console +appender.console.name = console +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%d{ISO8601}][%-5p][%-25c{1.}] [%node_name]%marker %m%n + +######## Server JSON ############################ +appender.rolling.type = RollingFile +appender.rolling.name = rolling +appender.rolling.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_server.json +appender.rolling.layout.type = ESJsonLayout +appender.rolling.layout.type_name = server + +appender.rolling.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}-%d{yyyy-MM-dd}-%i.json.gz +appender.rolling.policies.type = Policies +appender.rolling.policies.time.type = TimeBasedTriggeringPolicy +appender.rolling.policies.time.interval = 1 +appender.rolling.policies.time.modulate = true +appender.rolling.policies.size.type = SizeBasedTriggeringPolicy +appender.rolling.policies.size.size = 128MB +appender.rolling.strategy.type = DefaultRolloverStrategy +appender.rolling.strategy.fileIndex = nomax +appender.rolling.strategy.action.type = Delete +appender.rolling.strategy.action.basepath = ${sys:es.logs.base_path} +appender.rolling.strategy.action.condition.type = IfFileName +appender.rolling.strategy.action.condition.glob = ${sys:es.logs.cluster_name}-* +appender.rolling.strategy.action.condition.nested_condition.type = IfAccumulatedFileSize +appender.rolling.strategy.action.condition.nested_condition.exceeds = 2GB +################################################ +######## Server - old style pattern ########### +appender.rolling_old.type = RollingFile +appender.rolling_old.name = rolling_old +appender.rolling_old.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}.log +appender.rolling_old.layout.type = PatternLayout +appender.rolling_old.layout.pattern = [%d{ISO8601}][%-5p][%-25c{1.}] [%node_name]%marker %m%n + +appender.rolling_old.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}-%d{yyyy-MM-dd}-%i.log.gz +appender.rolling_old.policies.type = Policies +appender.rolling_old.policies.time.type = TimeBasedTriggeringPolicy +appender.rolling_old.policies.time.interval = 1 +appender.rolling_old.policies.time.modulate = true +appender.rolling_old.policies.size.type = SizeBasedTriggeringPolicy +appender.rolling_old.policies.size.size = 128MB +appender.rolling_old.strategy.type = DefaultRolloverStrategy +appender.rolling_old.strategy.fileIndex = nomax +appender.rolling_old.strategy.action.type = Delete +appender.rolling_old.strategy.action.basepath = ${sys:es.logs.base_path} +appender.rolling_old.strategy.action.condition.type = IfFileName +appender.rolling_old.strategy.action.condition.glob = ${sys:es.logs.cluster_name}-* +appender.rolling_old.strategy.action.condition.nested_condition.type = IfAccumulatedFileSize +appender.rolling_old.strategy.action.condition.nested_condition.exceeds = 2GB +################################################ + +rootLogger.level = info +rootLogger.appenderRef.console.ref = console +rootLogger.appenderRef.rolling.ref = rolling +rootLogger.appenderRef.rolling_old.ref = rolling_old + +######## Deprecation JSON ####################### +appender.deprecation_rolling.type = RollingFile +appender.deprecation_rolling.name = deprecation_rolling +appender.deprecation_rolling.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_deprecation.json +appender.deprecation_rolling.layout.type = ESJsonLayout +appender.deprecation_rolling.layout.type_name = deprecation.elasticsearch +appender.deprecation_rolling.layout.esmessagefields=x-opaque-id,key,category,elasticsearch.elastic_product_origin +appender.deprecation_rolling.filter.rate_limit.type = RateLimitingFilter + +appender.deprecation_rolling.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_deprecation-%i.json.gz +appender.deprecation_rolling.policies.type = Policies +appender.deprecation_rolling.policies.size.type = SizeBasedTriggeringPolicy +appender.deprecation_rolling.policies.size.size = 1GB +appender.deprecation_rolling.strategy.type = DefaultRolloverStrategy +appender.deprecation_rolling.strategy.max = 4 + +appender.header_warning.type = HeaderWarningAppender +appender.header_warning.name = header_warning +################################################# +######## Deprecation - old style pattern ####### +appender.deprecation_rolling_old.type = RollingFile +appender.deprecation_rolling_old.name = deprecation_rolling_old +appender.deprecation_rolling_old.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_deprecation.log +appender.deprecation_rolling_old.layout.type = PatternLayout +appender.deprecation_rolling_old.layout.pattern = [%d{ISO8601}][%-5p][%-25c{1.}] [%node_name] [%product_origin]%marker %m%n +appender.deprecation_rolling_old.filter.rate_limit.type = RateLimitingFilter + +appender.deprecation_rolling_old.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}\ + _deprecation-%i.log.gz +appender.deprecation_rolling_old.policies.type = Policies +appender.deprecation_rolling_old.policies.size.type = SizeBasedTriggeringPolicy +appender.deprecation_rolling_old.policies.size.size = 1GB +appender.deprecation_rolling_old.strategy.type = DefaultRolloverStrategy +appender.deprecation_rolling_old.strategy.max = 4 +################################################# +logger.deprecation.name = org.elasticsearch.deprecation +logger.deprecation.level = WARN +logger.deprecation.appenderRef.deprecation_rolling.ref = deprecation_rolling +logger.deprecation.appenderRef.deprecation_rolling_old.ref = deprecation_rolling_old +logger.deprecation.appenderRef.header_warning.ref = header_warning +logger.deprecation.additivity = false + +######## Search slowlog JSON #################### +appender.index_search_slowlog_rolling.type = RollingFile +appender.index_search_slowlog_rolling.name = index_search_slowlog_rolling +appender.index_search_slowlog_rolling.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs\ + .cluster_name}_index_search_slowlog.json +appender.index_search_slowlog_rolling.layout.type = ESJsonLayout +appender.index_search_slowlog_rolling.layout.type_name = index_search_slowlog +appender.index_search_slowlog_rolling.layout.esmessagefields=message,took,took_millis,total_hits,types,stats,search_type,total_shards,source,id + +appender.index_search_slowlog_rolling.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs\ + .cluster_name}_index_search_slowlog-%i.json.gz +appender.index_search_slowlog_rolling.policies.type = Policies +appender.index_search_slowlog_rolling.policies.size.type = SizeBasedTriggeringPolicy +appender.index_search_slowlog_rolling.policies.size.size = 1GB +appender.index_search_slowlog_rolling.strategy.type = DefaultRolloverStrategy +appender.index_search_slowlog_rolling.strategy.max = 4 +################################################# +######## Search slowlog - old style pattern #### +appender.index_search_slowlog_rolling_old.type = RollingFile +appender.index_search_slowlog_rolling_old.name = index_search_slowlog_rolling_old +appender.index_search_slowlog_rolling_old.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}\ + _index_search_slowlog.log +appender.index_search_slowlog_rolling_old.layout.type = PatternLayout +appender.index_search_slowlog_rolling_old.layout.pattern = [%d{ISO8601}][%-5p][%-25c{1.}] [%node_name]%marker %m%n + +appender.index_search_slowlog_rolling_old.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}\ + _index_search_slowlog-%i.log.gz +appender.index_search_slowlog_rolling_old.policies.type = Policies +appender.index_search_slowlog_rolling_old.policies.size.type = SizeBasedTriggeringPolicy +appender.index_search_slowlog_rolling_old.policies.size.size = 1GB +appender.index_search_slowlog_rolling_old.strategy.type = DefaultRolloverStrategy +appender.index_search_slowlog_rolling_old.strategy.max = 4 +################################################# +logger.index_search_slowlog_rolling.name = index.search.slowlog +logger.index_search_slowlog_rolling.level = trace +logger.index_search_slowlog_rolling.appenderRef.index_search_slowlog_rolling.ref = index_search_slowlog_rolling +logger.index_search_slowlog_rolling.appenderRef.index_search_slowlog_rolling_old.ref = index_search_slowlog_rolling_old +logger.index_search_slowlog_rolling.additivity = false + +######## Indexing slowlog JSON ################## +appender.index_indexing_slowlog_rolling.type = RollingFile +appender.index_indexing_slowlog_rolling.name = index_indexing_slowlog_rolling +appender.index_indexing_slowlog_rolling.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}\ + _index_indexing_slowlog.json +appender.index_indexing_slowlog_rolling.layout.type = ESJsonLayout +appender.index_indexing_slowlog_rolling.layout.type_name = index_indexing_slowlog +appender.index_indexing_slowlog_rolling.layout.esmessagefields=message,took,took_millis,doc_type,id,routing,source + +appender.index_indexing_slowlog_rolling.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}\ + _index_indexing_slowlog-%i.json.gz +appender.index_indexing_slowlog_rolling.policies.type = Policies +appender.index_indexing_slowlog_rolling.policies.size.type = SizeBasedTriggeringPolicy +appender.index_indexing_slowlog_rolling.policies.size.size = 1GB +appender.index_indexing_slowlog_rolling.strategy.type = DefaultRolloverStrategy +appender.index_indexing_slowlog_rolling.strategy.max = 4 +################################################# +######## Indexing slowlog - old style pattern ## +appender.index_indexing_slowlog_rolling_old.type = RollingFile +appender.index_indexing_slowlog_rolling_old.name = index_indexing_slowlog_rolling_old +appender.index_indexing_slowlog_rolling_old.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}\ + _index_indexing_slowlog.log +appender.index_indexing_slowlog_rolling_old.layout.type = PatternLayout +appender.index_indexing_slowlog_rolling_old.layout.pattern = [%d{ISO8601}][%-5p][%-25c{1.}] [%node_name]%marker %m%n + +appender.index_indexing_slowlog_rolling_old.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}\ + _index_indexing_slowlog-%i.log.gz +appender.index_indexing_slowlog_rolling_old.policies.type = Policies +appender.index_indexing_slowlog_rolling_old.policies.size.type = SizeBasedTriggeringPolicy +appender.index_indexing_slowlog_rolling_old.policies.size.size = 1GB +appender.index_indexing_slowlog_rolling_old.strategy.type = DefaultRolloverStrategy +appender.index_indexing_slowlog_rolling_old.strategy.max = 4 +################################################# + +logger.index_indexing_slowlog.name = index.indexing.slowlog.index +logger.index_indexing_slowlog.level = trace +logger.index_indexing_slowlog.appenderRef.index_indexing_slowlog_rolling.ref = index_indexing_slowlog_rolling +logger.index_indexing_slowlog.appenderRef.index_indexing_slowlog_rolling_old.ref = index_indexing_slowlog_rolling_old +logger.index_indexing_slowlog.additivity = false + + +appender.audit_rolling.type = RollingFile +appender.audit_rolling.name = audit_rolling +appender.audit_rolling.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_audit.json +appender.audit_rolling.layout.type = PatternLayout +appender.audit_rolling.layout.pattern = {\ + "type":"audit", \ + "timestamp":"%d{yyyy-MM-dd'T'HH:mm:ss,SSSZ}"\ + %varsNotEmpty{, "node.name":"%enc{%map{node.name}}{JSON}"}\ + %varsNotEmpty{, "node.id":"%enc{%map{node.id}}{JSON}"}\ + %varsNotEmpty{, "host.name":"%enc{%map{host.name}}{JSON}"}\ + %varsNotEmpty{, "host.ip":"%enc{%map{host.ip}}{JSON}"}\ + %varsNotEmpty{, "event.type":"%enc{%map{event.type}}{JSON}"}\ + %varsNotEmpty{, "event.action":"%enc{%map{event.action}}{JSON}"}\ + %varsNotEmpty{, "authentication.type":"%enc{%map{authentication.type}}{JSON}"}\ + %varsNotEmpty{, "user.name":"%enc{%map{user.name}}{JSON}"}\ + %varsNotEmpty{, "user.run_by.name":"%enc{%map{user.run_by.name}}{JSON}"}\ + %varsNotEmpty{, "user.run_as.name":"%enc{%map{user.run_as.name}}{JSON}"}\ + %varsNotEmpty{, "user.realm":"%enc{%map{user.realm}}{JSON}"}\ + %varsNotEmpty{, "user.run_by.realm":"%enc{%map{user.run_by.realm}}{JSON}"}\ + %varsNotEmpty{, "user.run_as.realm":"%enc{%map{user.run_as.realm}}{JSON}"}\ + %varsNotEmpty{, "user.roles":%map{user.roles}}\ + %varsNotEmpty{, "apikey.id":"%enc{%map{apikey.id}}{JSON}"}\ + %varsNotEmpty{, "apikey.name":"%enc{%map{apikey.name}}{JSON}"}\ + %varsNotEmpty{, "authentication.token.name":"%enc{%map{authentication.token.name}}{JSON}"}\ + %varsNotEmpty{, "authentication.token.type":"%enc{%map{authentication.token.type}}{JSON}"}\ + %varsNotEmpty{, "origin.type":"%enc{%map{origin.type}}{JSON}"}\ + %varsNotEmpty{, "origin.address":"%enc{%map{origin.address}}{JSON}"}\ + %varsNotEmpty{, "realm":"%enc{%map{realm}}{JSON}"}\ + %varsNotEmpty{, "url.path":"%enc{%map{url.path}}{JSON}"}\ + %varsNotEmpty{, "url.query":"%enc{%map{url.query}}{JSON}"}\ + %varsNotEmpty{, "request.method":"%enc{%map{request.method}}{JSON}"}\ + %varsNotEmpty{, "request.body":"%enc{%map{request.body}}{JSON}"}\ + %varsNotEmpty{, "request.id":"%enc{%map{request.id}}{JSON}"}\ + %varsNotEmpty{, "action":"%enc{%map{action}}{JSON}"}\ + %varsNotEmpty{, "request.name":"%enc{%map{request.name}}{JSON}"}\ + %varsNotEmpty{, "indices":%map{indices}}\ + %varsNotEmpty{, "opaque_id":"%enc{%map{opaque_id}}{JSON}"}\ + %varsNotEmpty{, "trace.id":"%enc{%map{trace.id}}{JSON}"}\ + %varsNotEmpty{, "x_forwarded_for":"%enc{%map{x_forwarded_for}}{JSON}"}\ + %varsNotEmpty{, "transport.profile":"%enc{%map{transport.profile}}{JSON}"}\ + %varsNotEmpty{, "rule":"%enc{%map{rule}}{JSON}"}\ + %varsNotEmpty{, "put":%map{put}}\ + %varsNotEmpty{, "delete":%map{delete}}\ + %varsNotEmpty{, "change":%map{change}}\ + %varsNotEmpty{, "create":%map{create}}\ + %varsNotEmpty{, "invalidate":%map{invalidate}}\ + }%n +# "node.name" node name from the `elasticsearch.yml` settings +# "node.id" node id which should not change between cluster restarts +# "host.name" unresolved hostname of the local node +# "host.ip" the local bound ip (i.e. the ip listening for connections) +# "origin.type" a received REST request is translated into one or more transport requests. This indicates which processing layer generated the event "rest" or "transport" (internal) +# "event.action" the name of the audited event, eg. "authentication_failed", "access_granted", "run_as_granted", etc. +# "authentication.type" one of "realm", "api_key", "token", "anonymous" or "internal" +# "user.name" the subject name as authenticated by a realm +# "user.run_by.name" the original authenticated subject name that is impersonating another one. +# "user.run_as.name" if this "event.action" is of a run_as type, this is the subject name to be impersonated as. +# "user.realm" the name of the realm that authenticated "user.name" +# "user.run_by.realm" the realm name of the impersonating subject ("user.run_by.name") +# "user.run_as.realm" if this "event.action" is of a run_as type, this is the realm name the impersonated user is looked up from +# "user.roles" the roles array of the user; these are the roles that are granting privileges +# "apikey.id" this field is present if and only if the "authentication.type" is "api_key" +# "apikey.name" this field is present if and only if the "authentication.type" is "api_key" +# "authentication.token.name" this field is present if and only if the authenticating credential is a service account token +# "authentication.token.type" this field is present if and only if the authenticating credential is a service account token +# "event.type" informs about what internal system generated the event; possible values are "rest", "transport", "ip_filter" and "security_config_change" +# "origin.address" the remote address and port of the first network hop, i.e. a REST proxy or another cluster node +# "realm" name of a realm that has generated an "authentication_failed" or an "authentication_successful"; the subject is not yet authenticated +# "url.path" the URI component between the port and the query string; it is percent (URL) encoded +# "url.query" the URI component after the path and before the fragment; it is percent (URL) encoded +# "request.method" the method of the HTTP request, i.e. one of GET, POST, PUT, DELETE, OPTIONS, HEAD, PATCH, TRACE, CONNECT +# "request.body" the content of the request body entity, JSON escaped +# "request.id" a synthetic identifier for the incoming request, this is unique per incoming request, and consistent across all audit events generated by that request +# "action" an action is the most granular operation that is authorized and this identifies it in a namespaced way (internal) +# "request.name" if the event is in connection to a transport message this is the name of the request class, similar to how rest requests are identified by the url path (internal) +# "indices" the array of indices that the "action" is acting upon +# "opaque_id" opaque value conveyed by the "X-Opaque-Id" request header +# "trace_id" an identifier conveyed by the part of "traceparent" request header +# "x_forwarded_for" the addresses from the "X-Forwarded-For" request header, as a verbatim string value (not an array) +# "transport.profile" name of the transport profile in case this is a "connection_granted" or "connection_denied" event +# "rule" name of the applied rule if the "origin.type" is "ip_filter" +# the "put", "delete", "change", "create", "invalidate" fields are only present +# when the "event.type" is "security_config_change" and contain the security config change (as an object) taking effect + +appender.audit_rolling.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_audit-%d{yyyy-MM-dd}.json +appender.audit_rolling.policies.type = Policies +appender.audit_rolling.policies.time.type = TimeBasedTriggeringPolicy +appender.audit_rolling.policies.time.interval = 1 +appender.audit_rolling.policies.time.modulate = true + +logger.xpack_security_audit_logfile.name = org.elasticsearch.xpack.security.audit.logfile.LoggingAuditTrail +logger.xpack_security_audit_logfile.level = info +logger.xpack_security_audit_logfile.appenderRef.audit_rolling.ref = audit_rolling +logger.xpack_security_audit_logfile.additivity = false + +logger.xmlsig.name = org.apache.xml.security.signature.XMLSignature +logger.xmlsig.level = error +logger.samlxml_decrypt.name = org.opensaml.xmlsec.encryption.support.Decrypter +logger.samlxml_decrypt.level = fatal +logger.saml2_decrypt.name = org.opensaml.saml.saml2.encryption.Decrypter +logger.saml2_decrypt.level = fatal \ No newline at end of file diff --git a/docker/thirdparties/docker-compose/elasticsearch/config/es8/log4j2.properties b/docker/thirdparties/docker-compose/elasticsearch/config/es8/log4j2.properties new file mode 100755 index 00000000000000..10a7f36c9cd9be --- /dev/null +++ b/docker/thirdparties/docker-compose/elasticsearch/config/es8/log4j2.properties @@ -0,0 +1,273 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +status = error + +appender.console.type = Console +appender.console.name = console +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%d{ISO8601}][%-5p][%-25c{1.}] [%node_name]%marker %m%n + +######## Server JSON ############################ +appender.rolling.type = RollingFile +appender.rolling.name = rolling +appender.rolling.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_server.json +appender.rolling.layout.type = ECSJsonLayout +appender.rolling.layout.dataset = elasticsearch.server + +appender.rolling.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}-%d{yyyy-MM-dd}-%i.json.gz +appender.rolling.policies.type = Policies +appender.rolling.policies.time.type = TimeBasedTriggeringPolicy +appender.rolling.policies.time.interval = 1 +appender.rolling.policies.time.modulate = true +appender.rolling.policies.size.type = SizeBasedTriggeringPolicy +appender.rolling.policies.size.size = 128MB +appender.rolling.strategy.type = DefaultRolloverStrategy +appender.rolling.strategy.fileIndex = nomax +appender.rolling.strategy.action.type = Delete +appender.rolling.strategy.action.basepath = ${sys:es.logs.base_path} +appender.rolling.strategy.action.condition.type = IfFileName +appender.rolling.strategy.action.condition.glob = ${sys:es.logs.cluster_name}-* +appender.rolling.strategy.action.condition.nested_condition.type = IfAccumulatedFileSize +appender.rolling.strategy.action.condition.nested_condition.exceeds = 2GB +################################################ +######## Server - old style pattern ########### +appender.rolling_old.type = RollingFile +appender.rolling_old.name = rolling_old +appender.rolling_old.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}.log +appender.rolling_old.layout.type = PatternLayout +appender.rolling_old.layout.pattern = [%d{ISO8601}][%-5p][%-25c{1.}] [%node_name]%marker %m%n + +appender.rolling_old.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}-%d{yyyy-MM-dd}-%i.log.gz +appender.rolling_old.policies.type = Policies +appender.rolling_old.policies.time.type = TimeBasedTriggeringPolicy +appender.rolling_old.policies.time.interval = 1 +appender.rolling_old.policies.time.modulate = true +appender.rolling_old.policies.size.type = SizeBasedTriggeringPolicy +appender.rolling_old.policies.size.size = 128MB +appender.rolling_old.strategy.type = DefaultRolloverStrategy +appender.rolling_old.strategy.fileIndex = nomax +appender.rolling_old.strategy.action.type = Delete +appender.rolling_old.strategy.action.basepath = ${sys:es.logs.base_path} +appender.rolling_old.strategy.action.condition.type = IfFileName +appender.rolling_old.strategy.action.condition.glob = ${sys:es.logs.cluster_name}-* +appender.rolling_old.strategy.action.condition.nested_condition.type = IfAccumulatedFileSize +appender.rolling_old.strategy.action.condition.nested_condition.exceeds = 2GB +################################################ + +rootLogger.level = info +rootLogger.appenderRef.console.ref = console +rootLogger.appenderRef.rolling.ref = rolling +rootLogger.appenderRef.rolling_old.ref = rolling_old + +######## Deprecation JSON ####################### +appender.deprecation_rolling.type = RollingFile +appender.deprecation_rolling.name = deprecation_rolling +appender.deprecation_rolling.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_deprecation.json +appender.deprecation_rolling.layout.type = ECSJsonLayout +# Intentionally follows a different pattern to above +appender.deprecation_rolling.layout.dataset = deprecation.elasticsearch +appender.deprecation_rolling.filter.rate_limit.type = RateLimitingFilter + +appender.deprecation_rolling.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_deprecation-%i.json.gz +appender.deprecation_rolling.policies.type = Policies +appender.deprecation_rolling.policies.size.type = SizeBasedTriggeringPolicy +appender.deprecation_rolling.policies.size.size = 1GB +appender.deprecation_rolling.strategy.type = DefaultRolloverStrategy +appender.deprecation_rolling.strategy.max = 4 + +appender.header_warning.type = HeaderWarningAppender +appender.header_warning.name = header_warning +################################################# + +logger.deprecation.name = org.elasticsearch.deprecation +logger.deprecation.level = WARN +logger.deprecation.appenderRef.deprecation_rolling.ref = deprecation_rolling +logger.deprecation.appenderRef.header_warning.ref = header_warning +logger.deprecation.additivity = false + +######## Search slowlog JSON #################### +appender.index_search_slowlog_rolling.type = RollingFile +appender.index_search_slowlog_rolling.name = index_search_slowlog_rolling +appender.index_search_slowlog_rolling.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs\ + .cluster_name}_index_search_slowlog.json +appender.index_search_slowlog_rolling.layout.type = ECSJsonLayout +appender.index_search_slowlog_rolling.layout.dataset = elasticsearch.index_search_slowlog + +appender.index_search_slowlog_rolling.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs\ + .cluster_name}_index_search_slowlog-%i.json.gz +appender.index_search_slowlog_rolling.policies.type = Policies +appender.index_search_slowlog_rolling.policies.size.type = SizeBasedTriggeringPolicy +appender.index_search_slowlog_rolling.policies.size.size = 1GB +appender.index_search_slowlog_rolling.strategy.type = DefaultRolloverStrategy +appender.index_search_slowlog_rolling.strategy.max = 4 +################################################# + +################################################# +logger.index_search_slowlog_rolling.name = index.search.slowlog +logger.index_search_slowlog_rolling.level = trace +logger.index_search_slowlog_rolling.appenderRef.index_search_slowlog_rolling.ref = index_search_slowlog_rolling +logger.index_search_slowlog_rolling.additivity = false + +######## Indexing slowlog JSON ################## +appender.index_indexing_slowlog_rolling.type = RollingFile +appender.index_indexing_slowlog_rolling.name = index_indexing_slowlog_rolling +appender.index_indexing_slowlog_rolling.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}\ + _index_indexing_slowlog.json +appender.index_indexing_slowlog_rolling.layout.type = ECSJsonLayout +appender.index_indexing_slowlog_rolling.layout.dataset = elasticsearch.index_indexing_slowlog + + +appender.index_indexing_slowlog_rolling.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}\ + _index_indexing_slowlog-%i.json.gz +appender.index_indexing_slowlog_rolling.policies.type = Policies +appender.index_indexing_slowlog_rolling.policies.size.type = SizeBasedTriggeringPolicy +appender.index_indexing_slowlog_rolling.policies.size.size = 1GB +appender.index_indexing_slowlog_rolling.strategy.type = DefaultRolloverStrategy +appender.index_indexing_slowlog_rolling.strategy.max = 4 +################################################# + + +logger.index_indexing_slowlog.name = index.indexing.slowlog.index +logger.index_indexing_slowlog.level = trace +logger.index_indexing_slowlog.appenderRef.index_indexing_slowlog_rolling.ref = index_indexing_slowlog_rolling +logger.index_indexing_slowlog.additivity = false + + +logger.com_amazonaws.name = com.amazonaws +logger.com_amazonaws.level = warn + +logger.com_amazonaws_jmx_SdkMBeanRegistrySupport.name = com.amazonaws.jmx.SdkMBeanRegistrySupport +logger.com_amazonaws_jmx_SdkMBeanRegistrySupport.level = error + +logger.com_amazonaws_metrics_AwsSdkMetrics.name = com.amazonaws.metrics.AwsSdkMetrics +logger.com_amazonaws_metrics_AwsSdkMetrics.level = error + +logger.com_amazonaws_auth_profile_internal_BasicProfileConfigFileLoader.name = com.amazonaws.auth.profile.internal.BasicProfileConfigFileLoader +logger.com_amazonaws_auth_profile_internal_BasicProfileConfigFileLoader.level = error + +logger.com_amazonaws_services_s3_internal_UseArnRegionResolver.name = com.amazonaws.services.s3.internal.UseArnRegionResolver +logger.com_amazonaws_services_s3_internal_UseArnRegionResolver.level = error + + +appender.audit_rolling.type = RollingFile +appender.audit_rolling.name = audit_rolling +appender.audit_rolling.fileName = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_audit.json +appender.audit_rolling.layout.type = PatternLayout +appender.audit_rolling.layout.pattern = {\ + "type":"audit", \ + "timestamp":"%d{yyyy-MM-dd'T'HH:mm:ss,SSSZ}"\ + %varsNotEmpty{, "cluster.name":"%enc{%map{cluster.name}}{JSON}"}\ + %varsNotEmpty{, "cluster.uuid":"%enc{%map{cluster.uuid}}{JSON}"}\ + %varsNotEmpty{, "node.name":"%enc{%map{node.name}}{JSON}"}\ + %varsNotEmpty{, "node.id":"%enc{%map{node.id}}{JSON}"}\ + %varsNotEmpty{, "host.name":"%enc{%map{host.name}}{JSON}"}\ + %varsNotEmpty{, "host.ip":"%enc{%map{host.ip}}{JSON}"}\ + %varsNotEmpty{, "event.type":"%enc{%map{event.type}}{JSON}"}\ + %varsNotEmpty{, "event.action":"%enc{%map{event.action}}{JSON}"}\ + %varsNotEmpty{, "authentication.type":"%enc{%map{authentication.type}}{JSON}"}\ + %varsNotEmpty{, "user.name":"%enc{%map{user.name}}{JSON}"}\ + %varsNotEmpty{, "user.run_by.name":"%enc{%map{user.run_by.name}}{JSON}"}\ + %varsNotEmpty{, "user.run_as.name":"%enc{%map{user.run_as.name}}{JSON}"}\ + %varsNotEmpty{, "user.realm":"%enc{%map{user.realm}}{JSON}"}\ + %varsNotEmpty{, "user.run_by.realm":"%enc{%map{user.run_by.realm}}{JSON}"}\ + %varsNotEmpty{, "user.run_as.realm":"%enc{%map{user.run_as.realm}}{JSON}"}\ + %varsNotEmpty{, "user.roles":%map{user.roles}}\ + %varsNotEmpty{, "apikey.id":"%enc{%map{apikey.id}}{JSON}"}\ + %varsNotEmpty{, "apikey.name":"%enc{%map{apikey.name}}{JSON}"}\ + %varsNotEmpty{, "authentication.token.name":"%enc{%map{authentication.token.name}}{JSON}"}\ + %varsNotEmpty{, "authentication.token.type":"%enc{%map{authentication.token.type}}{JSON}"}\ + %varsNotEmpty{, "origin.type":"%enc{%map{origin.type}}{JSON}"}\ + %varsNotEmpty{, "origin.address":"%enc{%map{origin.address}}{JSON}"}\ + %varsNotEmpty{, "realm":"%enc{%map{realm}}{JSON}"}\ + %varsNotEmpty{, "url.path":"%enc{%map{url.path}}{JSON}"}\ + %varsNotEmpty{, "url.query":"%enc{%map{url.query}}{JSON}"}\ + %varsNotEmpty{, "request.method":"%enc{%map{request.method}}{JSON}"}\ + %varsNotEmpty{, "request.body":"%enc{%map{request.body}}{JSON}"}\ + %varsNotEmpty{, "request.id":"%enc{%map{request.id}}{JSON}"}\ + %varsNotEmpty{, "action":"%enc{%map{action}}{JSON}"}\ + %varsNotEmpty{, "request.name":"%enc{%map{request.name}}{JSON}"}\ + %varsNotEmpty{, "indices":%map{indices}}\ + %varsNotEmpty{, "opaque_id":"%enc{%map{opaque_id}}{JSON}"}\ + %varsNotEmpty{, "trace.id":"%enc{%map{trace.id}}{JSON}"}\ + %varsNotEmpty{, "x_forwarded_for":"%enc{%map{x_forwarded_for}}{JSON}"}\ + %varsNotEmpty{, "transport.profile":"%enc{%map{transport.profile}}{JSON}"}\ + %varsNotEmpty{, "rule":"%enc{%map{rule}}{JSON}"}\ + %varsNotEmpty{, "put":%map{put}}\ + %varsNotEmpty{, "delete":%map{delete}}\ + %varsNotEmpty{, "change":%map{change}}\ + %varsNotEmpty{, "create":%map{create}}\ + %varsNotEmpty{, "invalidate":%map{invalidate}}\ + }%n +# "node.name" node name from the `elasticsearch.yml` settings +# "node.id" node id which should not change between cluster restarts +# "host.name" unresolved hostname of the local node +# "host.ip" the local bound ip (i.e. the ip listening for connections) +# "origin.type" a received REST request is translated into one or more transport requests. This indicates which processing layer generated the event "rest" or "transport" (internal) +# "event.action" the name of the audited event, eg. "authentication_failed", "access_granted", "run_as_granted", etc. +# "authentication.type" one of "realm", "api_key", "token", "anonymous" or "internal" +# "user.name" the subject name as authenticated by a realm +# "user.run_by.name" the original authenticated subject name that is impersonating another one. +# "user.run_as.name" if this "event.action" is of a run_as type, this is the subject name to be impersonated as. +# "user.realm" the name of the realm that authenticated "user.name" +# "user.run_by.realm" the realm name of the impersonating subject ("user.run_by.name") +# "user.run_as.realm" if this "event.action" is of a run_as type, this is the realm name the impersonated user is looked up from +# "user.roles" the roles array of the user; these are the roles that are granting privileges +# "apikey.id" this field is present if and only if the "authentication.type" is "api_key" +# "apikey.name" this field is present if and only if the "authentication.type" is "api_key" +# "authentication.token.name" this field is present if and only if the authenticating credential is a service account token +# "authentication.token.type" this field is present if and only if the authenticating credential is a service account token +# "event.type" informs about what internal system generated the event; possible values are "rest", "transport", "ip_filter" and "security_config_change" +# "origin.address" the remote address and port of the first network hop, i.e. a REST proxy or another cluster node +# "realm" name of a realm that has generated an "authentication_failed" or an "authentication_successful"; the subject is not yet authenticated +# "url.path" the URI component between the port and the query string; it is percent (URL) encoded +# "url.query" the URI component after the path and before the fragment; it is percent (URL) encoded +# "request.method" the method of the HTTP request, i.e. one of GET, POST, PUT, DELETE, OPTIONS, HEAD, PATCH, TRACE, CONNECT +# "request.body" the content of the request body entity, JSON escaped +# "request.id" a synthetic identifier for the incoming request, this is unique per incoming request, and consistent across all audit events generated by that request +# "action" an action is the most granular operation that is authorized and this identifies it in a namespaced way (internal) +# "request.name" if the event is in connection to a transport message this is the name of the request class, similar to how rest requests are identified by the url path (internal) +# "indices" the array of indices that the "action" is acting upon +# "opaque_id" opaque value conveyed by the "X-Opaque-Id" request header +# "trace_id" an identifier conveyed by the part of "traceparent" request header +# "x_forwarded_for" the addresses from the "X-Forwarded-For" request header, as a verbatim string value (not an array) +# "transport.profile" name of the transport profile in case this is a "connection_granted" or "connection_denied" event +# "rule" name of the applied rule if the "origin.type" is "ip_filter" +# the "put", "delete", "change", "create", "invalidate" fields are only present +# when the "event.type" is "security_config_change" and contain the security config change (as an object) taking effect + +appender.audit_rolling.filePattern = ${sys:es.logs.base_path}${sys:file.separator}${sys:es.logs.cluster_name}_audit-%d{yyyy-MM-dd}-%i.json.gz +appender.audit_rolling.policies.type = Policies +appender.audit_rolling.policies.time.type = TimeBasedTriggeringPolicy +appender.audit_rolling.policies.time.interval = 1 +appender.audit_rolling.policies.time.modulate = true +appender.audit_rolling.policies.size.type = SizeBasedTriggeringPolicy +appender.audit_rolling.policies.size.size = 1GB +appender.audit_rolling.strategy.type = DefaultRolloverStrategy +appender.audit_rolling.strategy.fileIndex = nomax + +logger.xpack_security_audit_logfile.name = org.elasticsearch.xpack.security.audit.logfile.LoggingAuditTrail +logger.xpack_security_audit_logfile.level = info +logger.xpack_security_audit_logfile.appenderRef.audit_rolling.ref = audit_rolling +logger.xpack_security_audit_logfile.additivity = false + +logger.xmlsig.name = org.apache.xml.security.signature.XMLSignature +logger.xmlsig.level = error +logger.samlxml_decrypt.name = org.opensaml.xmlsec.encryption.support.Decrypter +logger.samlxml_decrypt.level = fatal +logger.saml2_decrypt.name = org.opensaml.saml.saml2.encryption.Decrypter +logger.saml2_decrypt.level = fatal \ No newline at end of file diff --git a/docker/thirdparties/docker-compose/elasticsearch/es.yaml.tpl b/docker/thirdparties/docker-compose/elasticsearch/es.yaml.tpl index 38167bad6df2bc..254153809065f4 100644 --- a/docker/thirdparties/docker-compose/elasticsearch/es.yaml.tpl +++ b/docker/thirdparties/docker-compose/elasticsearch/es.yaml.tpl @@ -20,6 +20,7 @@ version: "3.9" services: doris--es_6: # es official not provide 6.x image for arm/v8, use compatible image. + # https://github.com/dockhippie/elasticsearch/tree/master/v6.8 image: webhippie/elasticsearch:6.8 ports: - ${DOCKER_ES_6_EXTERNAL_PORT}:9200 @@ -27,13 +28,16 @@ services: ELASTICSEARCH_CLUSTER_NAME: "elasticsearch6" ES_JAVA_OPTS: "-Xms256m -Xmx256m" discovery.type: "single-node" - ELASTICSEARCH_XPACK_SECURITY_ENABLED: "false" + ELASTICSEARCH_SKIP_TEMPLATES: "true" volumes: - - ./data/es6/:/usr/share/elasticsearch/data + - ./data/es6/:/var/lib/elasticsearch/data + - ./logs/es6/:/var/lib/elasticsearch/logs + - ./config/es6/log4j2.properties:/etc/elasticsearch/log4j2.properties + - ./config/es6/elasticsearch.yml:/etc/elasticsearch/elasticsearch.yml networks: - doris--es healthcheck: - test: [ "CMD", "curl", "localhost:9200" ] + test: [ "CMD", "curl", "localhost:9200/_cluster/health?wait_for_status=green" ] interval: 30s timeout: 10s retries: 100 @@ -46,12 +50,19 @@ services: ES_JAVA_OPTS: "-Xms256m -Xmx256m" discovery.type: "single-node" xpack.security.enabled: "false" + cluster.routing.allocation.disk.threshold_enabled: true + cluster.routing.allocation.disk.watermark.low: 500mb + cluster.routing.allocation.disk.watermark.high: 300mb + cluster.routing.allocation.disk.watermark.flood_stage: 200mb + ES_LOG_STYLE: "file" volumes: - ./data/es7/:/usr/share/elasticsearch/data + - ./logs/es7/:/usr/share/elasticsearch/logs + - ./config/es7/log4j2.properties:/usr/share/elasticsearch/log4j2.properties networks: - doris--es healthcheck: - test: [ "CMD", "curl", "localhost:9200" ] + test: [ "CMD", "curl", "localhost:9200/_cluster/health?wait_for_status=green" ] interval: 30s timeout: 10s retries: 100 @@ -64,12 +75,19 @@ services: ES_JAVA_OPTS: "-Xms256m -Xmx256m" discovery.type: "single-node" xpack.security.enabled: "false" + cluster.routing.allocation.disk.threshold_enabled: true + cluster.routing.allocation.disk.watermark.low: 500mb + cluster.routing.allocation.disk.watermark.high: 300mb + cluster.routing.allocation.disk.watermark.flood_stage: 200mb + ES_LOG_STYLE: "file" volumes: - ./data/es8/:/usr/share/elasticsearch/data + - ./logs/es8/:/usr/share/elasticsearch/logs + - ./config/es8/log4j2.properties:/usr/share/elasticsearch/log4j2.properties networks: - doris--es healthcheck: - test: [ "CMD", "curl", "localhost:9200" ] + test: [ "CMD", "curl", "localhost:9200/_cluster/health?wait_for_status=green" ] interval: 30s timeout: 10s retries: 100 diff --git a/docker/thirdparties/docker-compose/elasticsearch/scripts/index/es6_test1.json b/docker/thirdparties/docker-compose/elasticsearch/scripts/index/es6_test1.json index 97c5f537c5b628..26dbdb98203afc 100755 --- a/docker/thirdparties/docker-compose/elasticsearch/scripts/index/es6_test1.json +++ b/docker/thirdparties/docker-compose/elasticsearch/scripts/index/es6_test1.json @@ -46,7 +46,7 @@ "type": "long" }, "c_unsigned_long": { - "type": "unsigned_long" + "type": "long" }, "c_float": { "type": "float" diff --git a/docker/thirdparties/docker-compose/elasticsearch/scripts/index/es6_test2.json b/docker/thirdparties/docker-compose/elasticsearch/scripts/index/es6_test2.json index 1dec9e7ff441f5..e1feb6664b230d 100755 --- a/docker/thirdparties/docker-compose/elasticsearch/scripts/index/es6_test2.json +++ b/docker/thirdparties/docker-compose/elasticsearch/scripts/index/es6_test2.json @@ -49,7 +49,7 @@ "type": "long" }, "c_unsigned_long": { - "type": "unsigned_long" + "type": "long" }, "c_float": { "type": "float" diff --git a/docker/thirdparties/run-thirdparties-docker.sh b/docker/thirdparties/run-thirdparties-docker.sh index 533a9afaa6b5cc..3ff8a29997662e 100755 --- a/docker/thirdparties/run-thirdparties-docker.sh +++ b/docker/thirdparties/run-thirdparties-docker.sh @@ -180,6 +180,14 @@ if [[ "${RUN_ES}" -eq 1 ]]; then sudo mkdir -p "${ROOT}"/docker-compose/elasticsearch/data/es8/ sudo rm -rf "${ROOT}"/docker-compose/elasticsearch/data/es8/* sudo chmod -R 777 "${ROOT}"/docker-compose/elasticsearch/data + sudo mkdir -p "${ROOT}"/docker-compose/elasticsearch/logs/es6/ + sudo rm -rf "${ROOT}"/docker-compose/elasticsearch/logs/es6/* + sudo mkdir -p "${ROOT}"/docker-compose/elasticsearch/logs/es7/ + sudo rm -rf "${ROOT}"/docker-compose/elasticsearch/logs/es7/* + sudo mkdir -p "${ROOT}"/docker-compose/elasticsearch/logs/es8/ + sudo rm -rf "${ROOT}"/docker-compose/elasticsearch/logs/es8/* + sudo chmod -R 777 "${ROOT}"/docker-compose/elasticsearch/logs + sudo chmod -R 777 "${ROOT}"/docker-compose/elasticsearch/config sudo docker compose -f "${ROOT}"/docker-compose/elasticsearch/es.yaml --env-file "${ROOT}"/docker-compose/elasticsearch/es.env up -d --remove-orphans fi fi diff --git a/regression-test/data/external_table_p0/es/test_es_query.out b/regression-test/data/external_table_p0/es/test_es_query.out index cc8be6d8a36699..3b995832c4d239 100644 --- a/regression-test/data/external_table_p0/es/test_es_query.out +++ b/regression-test/data/external_table_p0/es/test_es_query.out @@ -39,47 +39,47 @@ I'm not null or empty 2022-08-08 2022-08-11T12:10:10 2022-08-11T12:10:10 2022-08-11T12:10:10 2022-08-11T11:10:10 -- !sql_6_02 -- -[1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01 00:00:00", "2020-01-02 00:00:00"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] \N string1 text#1 3.14 2022-08-08T00:00 12345 2022-08-08T20:10:10 +[1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] \N string1 text#1 3.14 2022-08-08T00:00 12345 2022-08-08T20:10:10 -- !sql_6_03 -- -[1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01 00:00:00", "2020-01-02 00:00:00"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] \N string1 text#1 3.14 2022-08-08T00:00 12345 2022-08-08T20:10:10 -[1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01 00:00:00", "2020-01-02 00:00:00"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] string2 text2 4.0 2022-08-08T00:00 2222 2022-08-08T12:10:10 -[1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01 00:00:00", "2020-01-02 00:00:00"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] I'm not null or empty string3 text3_4*5 5.0 2022-08-08T00:00 3333 2022-08-08T20:10:10 +[1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] \N string1 text#1 3.14 2022-08-08T00:00 12345 2022-08-08T20:10:10 +[1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] string2 text2 4.0 2022-08-08T00:00 2222 2022-08-08T12:10:10 +[1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] I'm not null or empty string3 text3_4*5 5.0 2022-08-08T00:00 3333 2022-08-08T20:10:10 -- !sql_6_04 -- -[1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01 00:00:00", "2020-01-02 00:00:00"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] string2 text2 4.0 2022-08-08T00:00 2222 2022-08-08T12:10:10 +[1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] string2 text2 4.0 2022-08-08T00:00 2222 2022-08-08T12:10:10 -- !sql_6_05 -- -true 1 128 32768 -1 0 1.0 1 1 1 2020-01-01T00:00 2020-01-01 12:00:00 a d 192.168.0.1 {"name":"Andy","age":18} -true 1 128 32768 -1 0 1.0 1 1 1 2020-01-01T00:00 2020-01-01 12:00:00 a d 192.168.0.1 {"name":"Andy","age":18} -true 1 128 32768 -1 0 1.0 1 1 1 2020-01-01T00:00 2020-01-01 12:00:00 a d 192.168.0.1 {"name":"Andy","age":18} +true 1 128 32768 -1 0 1.0 1.0 1.0 1.0 2020-01-01 2020-01-01T12:00 a d 192.168.0.1 {"name":"Andy","age":18} +true 1 128 32768 -1 0 1.0 1.0 1.0 1.0 2020-01-01 2020-01-01T12:00 a d 192.168.0.1 {"name":"Andy","age":18} +true 1 128 32768 -1 0 1.0 1.0 1.0 1.0 2020-01-01 2020-01-01T12:00 a d 192.168.0.1 {"name":"Andy","age":18} -- !sql_6_06 -- -true 1 128 32768 -1 0 1.0 1 1 1 2020-01-01T00:00 2020-01-01 12:00:00 a d 192.168.0.1 {"name":"Andy","age":18} -true 1 128 32768 -1 0 1.0 1 1 1 2020-01-01T00:00 2020-01-01 12:00:00 a d 192.168.0.1 {"name":"Andy","age":18} -true 1 128 32768 -1 0 1.0 1 1 1 2020-01-01T00:00 2020-01-01 12:00:00 a d 192.168.0.1 {"name":"Andy","age":18} +true 1 128 32768 -1 0 1.0 1.0 1.0 1.0 2020-01-01 2020-01-01T12:00 a d 192.168.0.1 {"name":"Andy","age":18} +true 1 128 32768 -1 0 1.0 1.0 1.0 1.0 2020-01-01 2020-01-01T12:00 a d 192.168.0.1 {"name":"Andy","age":18} +true 1 128 32768 -1 0 1.0 1.0 1.0 1.0 2020-01-01 2020-01-01T12:00 a d 192.168.0.1 {"name":"Andy","age":18} -- !sql_6_07 -- -[1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01 00:00:00", "2020-01-02 00:00:00"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] \N string1 text#1 3.14 2022-08-08T00:00 12345 2022-08-08T20:10:10 +[1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] \N string1 text#1 3.14 2022-08-08T00:00 12345 2022-08-08T20:10:10 -- !sql_6_08 -- -[1, 0, 1, 1] [1, -2, -3, 4] [128, 129, -129, -130] [32768, 32769, -32769, -32770] [-1, 0, 1, 2] [0, 1, 2, 3] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [1, 2, 3, 4] [1, 2, 3, 4] ["2020-01-01 00:00:00", "2020-01-02 00:00:00"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] ["a", "b", "c"] ["d", "e", "f"] ["192.168.0.1", "127.0.0.1"] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] -[1, 0, 1, 1] [1, -2, -3, 4] [128, 129, -129, -130] [32768, 32769, -32769, -32770] [-1, 0, 1, 2] [0, 1, 2, 3] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [1, 2, 3, 4] [1, 2, 3, 4] ["2020-01-01 00:00:00", "2020-01-02 00:00:00"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] ["a", "b", "c"] ["d", "e", "f"] ["192.168.0.1", "127.0.0.1"] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] -[1, 0, 1, 1] [1, -2, -3, 4] [128, 129, -129, -130] [32768, 32769, -32769, -32770] [-1, 0, 1, 2] [0, 1, 2, 3] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [1, 2, 3, 4] [1, 2, 3, 4] ["2020-01-01 00:00:00", "2020-01-02 00:00:00"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] ["a", "b", "c"] ["d", "e", "f"] ["192.168.0.1", "127.0.0.1"] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] +[1, 0, 1, 1] [1, -2, -3, 4] [128, 129, -129, -130] [32768, 32769, -32769, -32770] [-1, 0, 1, 2] [0, 1, 2, 3] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [1, 2, 3, 4] [1, 2, 3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] ["a", "b", "c"] ["d", "e", "f"] ["192.168.0.1", "127.0.0.1"] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] +[1, 0, 1, 1] [1, -2, -3, 4] [128, 129, -129, -130] [32768, 32769, -32769, -32770] [-1, 0, 1, 2] [0, 1, 2, 3] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [1, 2, 3, 4] [1, 2, 3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] ["a", "b", "c"] ["d", "e", "f"] ["192.168.0.1", "127.0.0.1"] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] +[1, 0, 1, 1] [1, -2, -3, 4] [128, 129, -129, -130] [32768, 32769, -32769, -32770] [-1, 0, 1, 2] [0, 1, 2, 3] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [1, 2, 3, 4] [1, 2, 3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] ["a", "b", "c"] ["d", "e", "f"] ["192.168.0.1", "127.0.0.1"] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] -- !sql_6_09 -- -[1, 0, 1, 1] [1, -2, -3, 4] [128, 129, -129, -130] [32768, 32769, -32769, -32770] [-1, 0, 1, 2] [0, 1, 2, 3] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [1, 2, 3, 4] [1, 2, 3, 4] ["2020-01-01 00:00:00", "2020-01-02 00:00:00"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] ["a", "b", "c"] ["d", "e", "f"] ["192.168.0.1", "127.0.0.1"] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] -[1, 0, 1, 1] [1, -2, -3, 4] [128, 129, -129, -130] [32768, 32769, -32769, -32770] [-1, 0, 1, 2] [0, 1, 2, 3] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [1, 2, 3, 4] [1, 2, 3, 4] ["2020-01-01 00:00:00", "2020-01-02 00:00:00"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] ["a", "b", "c"] ["d", "e", "f"] ["192.168.0.1", "127.0.0.1"] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] -[1, 0, 1, 1] [1, -2, -3, 4] [128, 129, -129, -130] [32768, 32769, -32769, -32770] [-1, 0, 1, 2] [0, 1, 2, 3] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [1, 2, 3, 4] [1, 2, 3, 4] ["2020-01-01 00:00:00", "2020-01-02 00:00:00"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] ["a", "b", "c"] ["d", "e", "f"] ["192.168.0.1", "127.0.0.1"] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] +[1, 0, 1, 1] [1, -2, -3, 4] [128, 129, -129, -130] [32768, 32769, -32769, -32770] [-1, 0, 1, 2] [0, 1, 2, 3] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [1, 2, 3, 4] [1, 2, 3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] ["a", "b", "c"] ["d", "e", "f"] ["192.168.0.1", "127.0.0.1"] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] +[1, 0, 1, 1] [1, -2, -3, 4] [128, 129, -129, -130] [32768, 32769, -32769, -32770] [-1, 0, 1, 2] [0, 1, 2, 3] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [1, 2, 3, 4] [1, 2, 3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] ["a", "b", "c"] ["d", "e", "f"] ["192.168.0.1", "127.0.0.1"] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] +[1, 0, 1, 1] [1, -2, -3, 4] [128, 129, -129, -130] [32768, 32769, -32769, -32770] [-1, 0, 1, 2] [0, 1, 2, 3] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [1, 2, 3, 4] [1, 2, 3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] ["a", "b", "c"] ["d", "e", "f"] ["192.168.0.1", "127.0.0.1"] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] -- !sql_6_10 -- -[1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01 00:00:00", "2020-01-02 00:00:00"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] \N string1 text#1 3.14 2022-08-08T00:00 12345 2022-08-08T20:10:10 +[1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] \N string1 text#1 3.14 2022-08-08T00:00 12345 2022-08-08T20:10:10 -- !sql_6_11 -- -[1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01 00:00:00", "2020-01-02 00:00:00"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] string2 text2 4.0 2022-08-08T00:00 2222 2022-08-08T12:10:10 +[1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] string2 text2 4.0 2022-08-08T00:00 2222 2022-08-08T12:10:10 -- !sql_6_12 -- -[1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01 00:00:00", "2020-01-02 00:00:00"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] I'm not null or empty string3 text3_4*5 5.0 2022-08-08T00:00 3333 2022-08-08T20:10:10 +[1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] I'm not null or empty string3 text3_4*5 5.0 2022-08-08T00:00 3333 2022-08-08T20:10:10 -- !sql_6_13 -- 2022-08-08T20:10:10 diff --git a/regression-test/data/external_table_p0/es/test_es_query_no_http_url.out b/regression-test/data/external_table_p0/es/test_es_query_no_http_url.out index ceedce57b18da7..edab82a4056655 100644 --- a/regression-test/data/external_table_p0/es/test_es_query_no_http_url.out +++ b/regression-test/data/external_table_p0/es/test_es_query_no_http_url.out @@ -6,7 +6,7 @@ ["2020-01-01", "2020-01-02"] [-1, 0, 1, 2] [0, 1, 2, 3] ["d", "e", "f"] [128, 129, -129, -130] ["192.168.0.1", "127.0.0.1"] string1 [1, 2, 3, 4] 2022-08-08 2022-08-08T12:10:10 text#1 ["2020-01-01", "2020-01-02"] 3.14 [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] ["a", "b", "c"] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] 2022-08-08T12:10:10 2022-08-08T12:10:10 2022-08-08T20:10:10 [1, -2, -3, 4] [1, 0, 1, 1] [32768, 32769, -32769, -32770] -- !sql61 -- -[1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01 00:00:00", "2020-01-02 00:00:00"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] \N string1 text#1 3.14 2022-08-08T00:00 12345 2022-08-08T20:10:10 +[1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] \N string1 text#1 3.14 2022-08-08T00:00 12345 2022-08-08T20:10:10 -- !sql71 -- [1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] ["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] \N string1 2022-08-08T20:10:10 text#1 3.14 2022-08-08T00:00 2022-08-08T12:10:10 1659931810000 2022-08-08T12:10:10 2022-08-08T20:10:10 12345 From a38890023b5a45aa263cb9dd25c60f55b13fce70 Mon Sep 17 00:00:00 2001 From: Kaijie Chen Date: Fri, 19 Jan 2024 10:44:03 +0800 Subject: [PATCH 089/200] [fix](injection) don't disturb CLOSE_LOAD message in LoadStream (#30097) --- be/src/runtime/load_stream.cpp | 22 +++++++++++++--------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/be/src/runtime/load_stream.cpp b/be/src/runtime/load_stream.cpp index efffa23e650981..7d494eba88a79f 100644 --- a/be/src/runtime/load_stream.cpp +++ b/be/src/runtime/load_stream.cpp @@ -521,15 +521,19 @@ int LoadStream::on_received_messages(StreamId id, butil::IOBuf* const messages[] void LoadStream::_dispatch(StreamId id, const PStreamHeader& hdr, butil::IOBuf* data) { VLOG_DEBUG << PStreamHeader_Opcode_Name(hdr.opcode()) << " from " << hdr.src_id() << " with tablet " << hdr.tablet_id(); - DBUG_EXECUTE_IF("LoadStream._dispatch.unknown_loadid", { - PUniqueId& load_id = const_cast(hdr.load_id()); - load_id.set_hi(UNKNOWN_ID_FOR_TEST); - load_id.set_lo(UNKNOWN_ID_FOR_TEST); - }); - DBUG_EXECUTE_IF("LoadStream._dispatch.unknown_srcid", { - PStreamHeader& t_hdr = const_cast(hdr); - t_hdr.set_src_id(UNKNOWN_ID_FOR_TEST); - }); + // CLOSE_LOAD message should not be fault injected, + // otherwise the message will be ignored and causing close wait timeout + if (hdr.opcode() != PStreamHeader::CLOSE_LOAD) { + DBUG_EXECUTE_IF("LoadStream._dispatch.unknown_loadid", { + PUniqueId& load_id = const_cast(hdr.load_id()); + load_id.set_hi(UNKNOWN_ID_FOR_TEST); + load_id.set_lo(UNKNOWN_ID_FOR_TEST); + }); + DBUG_EXECUTE_IF("LoadStream._dispatch.unknown_srcid", { + PStreamHeader& t_hdr = const_cast(hdr); + t_hdr.set_src_id(UNKNOWN_ID_FOR_TEST); + }); + } if (UniqueId(hdr.load_id()) != UniqueId(_load_id)) { Status st = Status::Error( "invalid load id {}, expected {}", print_id(hdr.load_id()), print_id(_load_id)); From b2a9c5b7d431f13fa28b0dc993f46a88430cfa46 Mon Sep 17 00:00:00 2001 From: Jibing-Li <64681310+Jibing-Li@users.noreply.github.com> Date: Fri, 19 Jan 2024 10:47:15 +0800 Subject: [PATCH 090/200] [fix](statistics)Reanalyze olapTable if getRowCount is not 0 and last time row count is 0 (#30096) Sample analyze may write 0 result if getRowCount is not updated while analyzing. So we need to reanalyze the table if getRowCount > 0 and previous analyze row count is 0. Otherwise the stats for this table may stay 0 for ever before user load new data to this table. --- .../src/main/java/org/apache/doris/common/Config.java | 2 +- .../src/main/java/org/apache/doris/catalog/OlapTable.java | 3 +++ .../java/org/apache/doris/statistics/AnalysisManager.java | 2 +- .../apache/doris/statistics/StatisticsAutoCollector.java | 2 +- .../java/org/apache/doris/statistics/TableStatsMeta.java | 2 +- .../org/apache/doris/statistics/AnalysisManagerTest.java | 7 ++++++- 6 files changed, 13 insertions(+), 5 deletions(-) diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index 87181e637bf434..a93745b4844161 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -1565,7 +1565,7 @@ public class Config extends ConfigBase { "This parameter controls the time interval for automatic collection jobs to check the health of table" + "statistics and trigger automatic collection" }) - public static int auto_check_statistics_in_minutes = 10; + public static int auto_check_statistics_in_minutes = 5; /** * If set to TRUE, the compaction slower replica will be skipped when select get queryable replicas diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index 5e36886e56a11e..2c221a45500524 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -1201,6 +1201,9 @@ public boolean needReAnalyzeTable(TableStatsMeta tblStats) { return true; } long rowCount = getRowCount(); + if (rowCount > 0 && tblStats.rowCount == 0) { + return true; + } long updateRows = tblStats.updatedRows.get(); int tblHealth = StatisticsUtil.getTableHealth(rowCount, updateRows); return tblHealth < StatisticsUtil.getTableStatsHealthThreshold(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java index 2e12b4433900d2..8f927694dc79a5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java @@ -545,7 +545,7 @@ public void updateTableStats(AnalysisInfo jobInfo) { } TableStatsMeta tableStats = findTableStatsStatus(tbl.getId()); if (tableStats == null) { - updateTableStatsStatus(new TableStatsMeta(tbl.estimatedRowCount(), jobInfo, tbl)); + updateTableStatsStatus(new TableStatsMeta(jobInfo.emptyJob ? 0 : tbl.estimatedRowCount(), jobInfo, tbl)); } else { tableStats.update(jobInfo, tbl); logCreateTableStats(tableStats); diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java index e2eeb21aad45bb..a09b56acfcc4c4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java @@ -209,7 +209,7 @@ protected void createAnalyzeJobForTbl(DatabaseIf db, @VisibleForTesting protected AnalysisInfo getReAnalyzeRequiredPart(AnalysisInfo jobInfo) { TableIf table = StatisticsUtil.findTable(jobInfo.catalogId, jobInfo.dbId, jobInfo.tblId); - // Skip tables that are too width. + // Skip tables that are too wide. if (table.getBaseSchema().size() > StatisticsUtil.getAutoAnalyzeTableWidthThreshold()) { return null; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/TableStatsMeta.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/TableStatsMeta.java index 926194a7258684..9231c6a2bc7cd1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/TableStatsMeta.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/TableStatsMeta.java @@ -157,7 +157,7 @@ public void update(AnalysisInfo analyzedJob, TableIf tableIf) { jobType = analyzedJob.jobType; if (tableIf != null) { if (tableIf instanceof OlapTable) { - rowCount = tableIf.getRowCount(); + rowCount = analyzedJob.emptyJob ? 0 : tableIf.getRowCount(); } if (!analyzedJob.emptyJob && analyzedJob.colToPartitions.keySet() .containsAll(tableIf.getBaseSchema().stream() diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisManagerTest.java index 2fc6d24e305b30..f8a77fe06db754 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisManagerTest.java @@ -275,7 +275,7 @@ public void testReAnalyze() { new MockUp() { int count = 0; - int[] rowCount = new int[]{100, 100, 200, 200}; + int[] rowCount = new int[]{100, 100, 200, 200, 1, 1}; final Column c = new Column("col1", PrimitiveType.INT); @Mock @@ -304,6 +304,11 @@ public List getBaseSchema() { .setColToPartitions(new HashMap<>()).setColName("col1").build(), olapTable); stats2.updatedRows.addAndGet(20); Assertions.assertFalse(olapTable.needReAnalyzeTable(stats2)); + + TableStatsMeta stats3 = new TableStatsMeta(0, new AnalysisInfoBuilder() + .setColToPartitions(new HashMap<>()).setEmptyJob(true).setColName("col1").build(), olapTable); + Assertions.assertTrue(olapTable.needReAnalyzeTable(stats3)); + } @Test From 52ae3a92e8c8797f736cdcab510ddc8a6b21f4b9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=A2=E5=81=A5?= Date: Fri, 19 Jan 2024 11:18:52 +0800 Subject: [PATCH 091/200] [fix](Nereids): fix eliminate join test for pk-fk constraint (#30094) --- .../nereids_rules_p0/pkfk/eliminate_inner.out | 251 +++++++----------- .../pkfk/eliminate_inner.groovy | 16 +- 2 files changed, 102 insertions(+), 165 deletions(-) diff --git a/regression-test/data/nereids_rules_p0/pkfk/eliminate_inner.out b/regression-test/data/nereids_rules_p0/pkfk/eliminate_inner.out index bb2b48ac37d917..d1e060e7dd0607 100644 --- a/regression-test/data/nereids_rules_p0/pkfk/eliminate_inner.out +++ b/regression-test/data/nereids_rules_p0/pkfk/eliminate_inner.out @@ -4,152 +4,129 @@ simple_case -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----hashJoin[INNER_JOIN] hashCondition=((pkt.pk = fkt_not_null.fk)) otherCondition=() -------PhysicalOlapScan[pkt] -------PhysicalDistribute[DistributionSpecHash] ---------PhysicalOlapScan[fkt_not_null] +--PhysicalOlapScan[fkt_not_null] -- !res -- +1 John +2 Alice +3 Bob -- !name -- with_pk_col -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----hashJoin[INNER_JOIN] hashCondition=((pkt.pk = fkt_not_null.fk)) otherCondition=() -------PhysicalOlapScan[pkt] -------PhysicalDistribute[DistributionSpecHash] ---------PhysicalOlapScan[fkt_not_null] +--PhysicalOlapScan[fkt_not_null] -- !res -- +1 John 1 +2 Alice 2 +3 Bob 3 -- !name -- with_pk_col -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----hashJoin[INNER_JOIN] hashCondition=((pkt.pk = fkt_not_null.fk)) otherCondition=() -------PhysicalOlapScan[pkt] -------PhysicalDistribute[DistributionSpecHash] ---------hashJoin[INNER_JOIN] hashCondition=((fkt_not_null1.fk = fkt_not_null2.fk)) otherCondition=() -----------PhysicalOlapScan[fkt_not_null] -----------PhysicalOlapScan[fkt_not_null] +--hashJoin[INNER_JOIN] hashCondition=((pk = fkt_not_null2.fk)) otherCondition=() +----PhysicalOlapScan[fkt_not_null] +----PhysicalOlapScan[fkt_not_null] -- !res -- +1 John 1 +2 Alice 2 +3 Bob 3 -- !name -- with_pk_col -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----hashJoin[INNER_JOIN] hashCondition=((pkt.pk = fkt_not_null.fk)) otherCondition=() -------filter((pkt.pk > 1)) ---------PhysicalOlapScan[pkt] -------PhysicalDistribute[DistributionSpecHash] ---------hashJoin[INNER_JOIN] hashCondition=((fkt_not_null1.fk = fkt_not_null2.fk)) otherCondition=() -----------filter((fkt_not_null1.fk > 1)) -------------PhysicalOlapScan[fkt_not_null] -----------filter((fkt_not_null2.fk > 1)) -------------PhysicalOlapScan[fkt_not_null] +--hashJoin[INNER_JOIN] hashCondition=((pk = fkt_not_null2.fk)) otherCondition=() +----filter((fkt_not_null1.fk > 1)) +------PhysicalOlapScan[fkt_not_null] +----filter((fkt_not_null2.fk > 1)) +------PhysicalOlapScan[fkt_not_null] -- !res -- +2 Alice 2 +3 Bob 3 -- !name -- with_pk_col -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----hashJoin[INNER_JOIN] hashCondition=((pkt.pk = fkt_not_null.fk)) otherCondition=() -------PhysicalOlapScan[pkt] -------PhysicalDistribute[DistributionSpecHash] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[fkt_not_null] +--hashAgg[LOCAL] +----PhysicalOlapScan[fkt_not_null] -- !res -- +1 1 +2 2 +3 3 -- !name -- with_pk_col -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----hashJoin[INNER_JOIN] hashCondition=((pkt.pk = fkt_not_null.fk)) otherCondition=() -------PhysicalOlapScan[pkt] -------PhysicalDistribute[DistributionSpecHash] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalOlapScan[fkt_not_null] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalOlapScan[fkt_not_null] +--hashJoin[INNER_JOIN] hashCondition=((pkt.pk = fkt_not_null.fk)) otherCondition=() +----PhysicalOlapScan[pkt] +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalUnion +----------PhysicalOlapScan[fkt_not_null] +----------PhysicalOlapScan[fkt_not_null] -- !res -- +1 John 1 +2 Alice 2 +3 Bob 3 -- !name -- fk with window -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----hashJoin[INNER_JOIN] hashCondition=((pkt.pk = fkt_not_null.fk)) otherCondition=() -------PhysicalOlapScan[pkt] -------PhysicalDistribute[DistributionSpecHash] ---------PhysicalWindow -----------PhysicalQuickSort[LOCAL_SORT] -------------PhysicalOlapScan[fkt_not_null] +--PhysicalWindow +----PhysicalQuickSort[LOCAL_SORT] +------PhysicalOlapScan[fkt_not_null] -- !res -- +1 1 1 +2 1 2 +3 1 3 -- !name -- fk with limit -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----hashJoin[INNER_JOIN] hashCondition=((pkt.pk = fkt_not_null.fk)) otherCondition=() -------PhysicalOlapScan[pkt] -------PhysicalDistribute[DistributionSpecHash] ---------PhysicalLimit[GLOBAL] -----------PhysicalDistribute[DistributionSpecGather] -------------PhysicalLimit[LOCAL] ---------------PhysicalOlapScan[fkt_not_null] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------PhysicalOlapScan[fkt_not_null] -- !res -- +1 1 -- !name -- pk with filter that same as fk -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----hashJoin[INNER_JOIN] hashCondition=((pkt.pk = fkt_not_null.fk)) otherCondition=() -------filter((pkt.pk = 1)) ---------PhysicalOlapScan[pkt] -------PhysicalDistribute[DistributionSpecHash] ---------filter((fkt_not_null.fk = 1)) -----------PhysicalOlapScan[fkt_not_null] +--filter((fkt_not_null.fk = 1)) +----PhysicalOlapScan[fkt_not_null] -- !res -- +1 John 1 -- !name -- pk with filter that included same as fk -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----hashJoin[INNER_JOIN] hashCondition=((pkt.pk = fkt_not_null.fk)) otherCondition=() -------filter((pkt.pk = 1)) ---------PhysicalOlapScan[pkt] -------PhysicalDistribute[DistributionSpecHash] ---------filter((cast(f as DOUBLE) = 1.0) and (fkt_not_null.fk = 1)) -----------PhysicalOlapScan[fkt_not_null] +--filter((cast(f as DOUBLE) = 1.0) and (fkt_not_null.fk = 1)) +----PhysicalOlapScan[fkt_not_null] -- !res -- @@ -158,13 +135,11 @@ pk with filter that not same as fk -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----hashJoin[INNER_JOIN] hashCondition=((pkt.pk = fkt_not_null.fk)) otherCondition=() -------filter((cast(p as DOUBLE) = 1.0) and (pkt.pk = 1)) ---------PhysicalOlapScan[pkt] -------PhysicalDistribute[DistributionSpecHash] ---------filter((cast(f as DOUBLE) = 1.0) and (fkt_not_null.fk = 1)) -----------PhysicalOlapScan[fkt_not_null] +--hashJoin[INNER_JOIN] hashCondition=((pkt.pk = fkt_not_null.fk)) otherCondition=() +----filter((cast(p as DOUBLE) = 1.0) and (pkt.pk = 1)) +------PhysicalOlapScan[pkt] +----filter((cast(f as DOUBLE) = 1.0) and (fkt_not_null.fk = 1)) +------PhysicalOlapScan[fkt_not_null] -- !res -- @@ -173,129 +148,96 @@ simple_case -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----filter(( not fk IS NULL)) -------PhysicalOlapScan[fkt] +--filter(( not fk IS NULL)) +----PhysicalOlapScan[fkt] -- !res -- 1 John -1 John -2 Alice 2 Alice -3 Bob -- !name -- with_pk_col -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----filter(( not fk IS NULL)) -------PhysicalOlapScan[fkt] +--filter(( not fk IS NULL)) +----PhysicalOlapScan[fkt] -- !res -- 1 John 1 -1 John 1 2 Alice 2 -2 Alice 2 -3 Bob 3 -- !name -- with_pk_col -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----hashJoin[INNER_JOIN] hashCondition=((pk = fkt2.fk)) otherCondition=() -------filter(( not fk IS NULL)) ---------PhysicalOlapScan[fkt] +--hashJoin[INNER_JOIN] hashCondition=((pk = fkt2.fk)) otherCondition=() +----filter(( not fk IS NULL)) ------PhysicalOlapScan[fkt] +----PhysicalOlapScan[fkt] -- !res -- 1 John 1 -1 John 1 -1 John 1 -1 John 1 -2 Alice 2 2 Alice 2 -2 Alice 2 -2 Alice 2 -3 Bob 3 -- !name -- with_pk_col -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----hashJoin[INNER_JOIN] hashCondition=((pk = fkt2.fk)) otherCondition=() -------filter(( not fk IS NULL) and (fkt1.fk > 1)) ---------PhysicalOlapScan[fkt] -------filter((fkt2.fk > 1)) ---------PhysicalOlapScan[fkt] +--hashJoin[INNER_JOIN] hashCondition=((pk = fkt2.fk)) otherCondition=() +----filter(( not fk IS NULL) and (fkt1.fk > 1)) +------PhysicalOlapScan[fkt] +----filter((fkt2.fk > 1)) +------PhysicalOlapScan[fkt] -- !res -- 2 Alice 2 -2 Alice 2 -2 Alice 2 -2 Alice 2 -3 Bob 3 -- !name -- with_pk_col -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----hashAgg[LOCAL] -------filter(( not fk IS NULL)) ---------PhysicalOlapScan[fkt] +--hashAgg[LOCAL] +----filter(( not fk IS NULL)) +------PhysicalOlapScan[fkt] -- !res -- 1 1 2 2 -3 3 -- !name -- with_pk_col -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----hashJoin[INNER_JOIN] hashCondition=((pkt.pk = fkt.fk)) otherCondition=() -------PhysicalOlapScan[pkt] -------PhysicalDistribute[DistributionSpecHash] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalOlapScan[fkt] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalOlapScan[fkt] +--hashJoin[INNER_JOIN] hashCondition=((pkt.pk = fkt.fk)) otherCondition=() +----PhysicalOlapScan[pkt] +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalUnion +----------PhysicalOlapScan[fkt] +----------PhysicalOlapScan[fkt] -- !res -- 1 John 1 2 Alice 2 -3 Bob 3 -- !name -- fk with window -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----PhysicalWindow -------PhysicalQuickSort[LOCAL_SORT] ---------filter(( not fk IS NULL)) -----------PhysicalOlapScan[fkt] +--PhysicalWindow +----PhysicalQuickSort[LOCAL_SORT] +------filter(( not fk IS NULL)) +--------PhysicalOlapScan[fkt] -- !res -- 1 1 1 -1 2 1 2 1 2 -2 2 2 -3 1 3 -- !name -- fk with limit @@ -304,34 +246,29 @@ fk with limit PhysicalResultSink --filter(( not fk IS NULL)) ----PhysicalLimit[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalLimit[LOCAL] -----------PhysicalOlapScan[fkt] +------PhysicalLimit[LOCAL] +--------PhysicalOlapScan[fkt] -- !res -- -1 1 -- !name -- pk with filter that same as fk -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----filter(( not fk IS NULL) and (fkt.fk = 1)) -------PhysicalOlapScan[fkt] +--filter(( not fk IS NULL) and (fkt.fk = 1)) +----PhysicalOlapScan[fkt] -- !res -- 1 John 1 -1 John 1 -- !name -- pk with filter that included same as fk -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----filter(( not fk IS NULL) and (cast(f as DOUBLE) = 1.0) and (fkt.fk = 1)) -------PhysicalOlapScan[fkt] +--filter(( not fk IS NULL) and (cast(f as DOUBLE) = 1.0) and (fkt.fk = 1)) +----PhysicalOlapScan[fkt] -- !res -- @@ -340,13 +277,11 @@ pk with filter that not same as fk -- !shape -- PhysicalResultSink ---PhysicalDistribute[DistributionSpecGather] -----hashJoin[INNER_JOIN] hashCondition=((pkt.pk = fkt.fk)) otherCondition=() -------filter((cast(p as DOUBLE) = 1.0) and (pkt.pk = 1)) ---------PhysicalOlapScan[pkt] -------PhysicalDistribute[DistributionSpecHash] ---------filter((cast(f as DOUBLE) = 1.0) and (fkt.fk = 1)) -----------PhysicalOlapScan[fkt] +--hashJoin[INNER_JOIN] hashCondition=((pkt.pk = fkt.fk)) otherCondition=() +----filter((cast(p as DOUBLE) = 1.0) and (pkt.pk = 1)) +------PhysicalOlapScan[pkt] +----filter((cast(f as DOUBLE) = 1.0) and (fkt.fk = 1)) +------PhysicalOlapScan[fkt] -- !res -- diff --git a/regression-test/suites/nereids_rules_p0/pkfk/eliminate_inner.groovy b/regression-test/suites/nereids_rules_p0/pkfk/eliminate_inner.groovy index 5280847a53a8ef..6819c06d608773 100644 --- a/regression-test/suites/nereids_rules_p0/pkfk/eliminate_inner.groovy +++ b/regression-test/suites/nereids_rules_p0/pkfk/eliminate_inner.groovy @@ -20,7 +20,6 @@ suite("eliminate_inner") { sql "set runtime_filter_mode=OFF"; sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" sql "SET enable_fallback_to_original_planner=false" - sql "SET ignore_shape_nodes='PhysicalDistribute[DistributionSpecGather], PhysicalDistribute[DistributionSpecHash],PhysicalDistribute[DistributionSpecExecutionAny],PhysicalProject'" sql "SET disable_join_reorder=true" sql """ @@ -30,6 +29,9 @@ suite("eliminate_inner") { sql """ DROP TABLE IF EXISTS fkt """ + sql """ + DROP TABLE IF EXISTS fkt_not_null + """ sql """ CREATE TABLE IF NOT EXISTS pkt( @@ -70,7 +72,7 @@ suite("eliminate_inner") { INSERT INTO fkt VALUES (1, 'John'), (2, 'Alice'), (null, 'Bob'); """ sql """ - INSERT INTO fkt VALUES (1, 'John'), (2, 'Alice'), (3, 'Bob'); + INSERT INTO fkt_not_null VALUES (1, 'John'), (2, 'Alice'), (3, 'Bob'); """ sql """ alter table pkt add constraint pk primary key (pk) @@ -78,15 +80,15 @@ suite("eliminate_inner") { sql """ alter table fkt add constraint fk foreign key (fk) references pkt(pk) """ + sql """ + alter table fkt_not_null add constraint fk foreign key (fk) references pkt(pk) + """ def check_shape_res = { sql, name -> qt_name "select \"${name}\"" qt_shape "explain shape plan ${sql}" order_qt_res "${sql}" } - def simple_case = """ - select * from pkt inner join fkt on pkt.pk = fkt.fk; - """ - // nullable + // not nullable check_shape_res("select fkt_not_null.* from pkt inner join fkt_not_null on pkt.pk = fkt_not_null.fk;", "simple_case") check_shape_res("select fkt_not_null.*, pkt.pk from pkt inner join fkt_not_null on pkt.pk = fkt_not_null.fk;", "with_pk_col") check_shape_res("select fkt_not_null.*, pkt.pk from pkt inner join (select fkt_not_null1.* from fkt_not_null as fkt_not_null1 join fkt_not_null as fkt_not_null2 on fkt_not_null1.fk = fkt_not_null2.fk) fkt_not_null on pkt.pk = fkt_not_null.fk;", "with_pk_col") @@ -99,7 +101,7 @@ suite("eliminate_inner") { check_shape_res("select fkt_not_null.*, pkt.pk from pkt inner join fkt_not_null on pkt.pk = fkt_not_null.fk where pkt.pk = 1 and fkt_not_null.fk = 1 and fkt_not_null.f = 1;", "pk with filter that included same as fk") check_shape_res("select fkt_not_null.*, pkt.pk from pkt inner join fkt_not_null on pkt.pk = fkt_not_null.fk where pkt.p = 1 and fkt_not_null.fk = 1 and fkt_not_null.f = 1;;", "pk with filter that not same as fk") - // not nullable + // nullable check_shape_res("select fkt.* from pkt inner join fkt on pkt.pk = fkt.fk;", "simple_case") check_shape_res("select fkt.*, pkt.pk from pkt inner join fkt on pkt.pk = fkt.fk;", "with_pk_col") check_shape_res("select fkt.*, pkt.pk from pkt inner join (select fkt1.* from fkt as fkt1 join fkt as fkt2 on fkt1.fk = fkt2.fk) fkt on pkt.pk = fkt.fk;", "with_pk_col") From 6b9d7c00940f2bde9ea5412ad43d2b27087c02a3 Mon Sep 17 00:00:00 2001 From: yangshijie Date: Fri, 19 Jan 2024 11:49:20 +0800 Subject: [PATCH 092/200] [improvement](ip function) refactor some ip functions and remove dirty codes (#30080) --- be/src/vec/functions/function_ip.cpp | 4 +- be/src/vec/functions/function_ip.h | 228 ++++-------------- .../doris/catalog/BuiltinScalarFunctions.java | 4 +- ...6CIDRToRange.java => Ipv6CIDRToRange.java} | 12 +- .../functions/scalar/IsIpv4String.java | 4 +- .../functions/scalar/IsIpv6String.java | 4 +- .../visitor/ScalarFunctionVisitor.java | 4 +- gensrc/script/doris_builtins_functions.py | 12 +- .../test_ipv6_cidr_to_range_function.out | 4 + .../test_ipv6_cidr_to_range_function.groovy | 10 +- 10 files changed, 79 insertions(+), 207 deletions(-) rename fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/{IPv6CIDRToRange.java => Ipv6CIDRToRange.java} (89%) diff --git a/be/src/vec/functions/function_ip.cpp b/be/src/vec/functions/function_ip.cpp index dce648bfaaf518..3faa6a42d8c115 100644 --- a/be/src/vec/functions/function_ip.cpp +++ b/be/src/vec/functions/function_ip.cpp @@ -34,8 +34,8 @@ void register_function_ip(SimpleFunctionFactory& factory) { factory.register_function>(); factory.register_alias(FunctionIPv6StringToNum::name, "inet6_aton"); - factory.register_function(); - factory.register_function(); + factory.register_function>(); + factory.register_function>(); factory.register_function(); factory.register_function(); } diff --git a/be/src/vec/functions/function_ip.h b/be/src/vec/functions/function_ip.h index 2997153049a6b0..313ad0fdd65b1b 100644 --- a/be/src/vec/functions/function_ip.h +++ b/be/src/vec/functions/function_ip.h @@ -93,8 +93,6 @@ class FunctionIPv4NumToString : public IFunction { return make_nullable(std::make_shared()); } - bool use_default_implementation_for_nulls() const override { return true; } - Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, size_t result, size_t input_rows_count) const override { ColumnWithTypeAndName& argument = block.get_by_position(arguments[0]); @@ -309,8 +307,6 @@ class FunctionIPv6NumToString : public IFunction { return make_nullable(std::make_shared()); } - bool use_default_implementation_for_nulls() const override { return true; } - Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, size_t result, size_t input_rows_count) const override { const ColumnPtr& column = block.get_by_position(arguments[0]).column; @@ -603,143 +599,53 @@ class FunctionIPv6StringToNum : public IFunction { } }; -class FunctionIsIPv4String : public IFunction { -private: - Status execute_type(Block& block, const ColumnWithTypeAndName& argument, size_t result) const { - const ColumnPtr& column = argument.column; - - if (const auto* nullable_src = typeid_cast(column.get())) { - size_t col_size = nullable_src->size(); - auto col_res = ColumnUInt8::create(col_size, 0); - auto null_map = ColumnUInt8::create(col_size, 0); - auto& col_res_data = col_res->get_data(); - auto& null_map_data = null_map->get_data(); - - for (size_t i = 0; i < col_size; ++i) { - if (nullable_src->is_null_at(i)) { - null_map_data[i] = 1; - } else { - StringRef ipv4_str = nullable_src->get_data_at(i); - if (IPv4Value::is_valid_string(ipv4_str.data, ipv4_str.size)) { - col_res_data[i] = 1; - } - } - } - - block.replace_by_position( - result, ColumnNullable::create(std::move(col_res), std::move(null_map))); - return Status::OK(); - } else if (const auto* col_src = typeid_cast(column.get())) { - size_t col_size = col_src->size(); - auto col_res = ColumnUInt8::create(col_size, 0); - auto null_map = ColumnUInt8::create(col_size, 0); - auto& col_res_data = col_res->get_data(); - - for (size_t i = 0; i < col_size; ++i) { - StringRef ipv4_str = col_src->get_data_at(i); - if (IPv4Value::is_valid_string(ipv4_str.data, ipv4_str.size)) { - col_res_data[i] = 1; - } - } - - block.replace_by_position( - result, ColumnNullable::create(std::move(col_res), std::move(null_map))); - return Status::OK(); - } else { - return Status::RuntimeError("Illegal column {} of argument of function {}", - argument.column->get_name(), get_name()); - } - } +template +class FunctionIsIPString : public IFunction { + static_assert(std::is_same_v || std::is_same_v); public: - static constexpr auto name = "is_ipv4_string"; - static FunctionPtr create() { return std::make_shared(); } + static constexpr auto name = std::is_same_v ? "is_ipv4_string" : "is_ipv6_string"; + static FunctionPtr create() { return std::make_shared>(); } String get_name() const override { return name; } size_t get_number_of_arguments() const override { return 1; } DataTypePtr get_return_type_impl(const DataTypes& arguments) const override { - return make_nullable(std::make_shared()); + const auto& addr_type = arguments[0]; + if (!is_string(remove_nullable(addr_type))) { + throw Exception(ErrorCode::INVALID_ARGUMENT, + "Illegal type {} of first argument of function {}, expected String", + addr_type->get_name(), get_name()); + } + return std::make_shared(); } - bool use_default_implementation_for_nulls() const override { return true; } - Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, size_t result, size_t input_rows_count) const override { - ColumnWithTypeAndName& argument = block.get_by_position(arguments[0]); - DCHECK(argument.type->get_type_id() == TypeIndex::String); - return execute_type(block, argument, result); - } -}; - -class FunctionIsIPv6String : public IFunction { -private: - Status execute_type(Block& block, const ColumnWithTypeAndName& argument, size_t result) const { - const ColumnPtr& column = argument.column; - - if (const auto* nullable_src = typeid_cast(column.get())) { - size_t col_size = nullable_src->size(); - auto col_res = ColumnUInt8::create(col_size, 0); - auto null_map = ColumnUInt8::create(col_size, 0); - auto& col_res_data = col_res->get_data(); - auto& null_map_data = null_map->get_data(); + const auto& addr_column_with_type_and_name = block.get_by_position(arguments[0]); + WhichDataType addr_type(addr_column_with_type_and_name.type); + const ColumnPtr& addr_column = addr_column_with_type_and_name.column; + const auto* str_addr_column = assert_cast(addr_column.get()); + auto col_res = ColumnUInt8::create(input_rows_count, 0); + auto& col_res_data = col_res->get_data(); - for (size_t i = 0; i < col_size; ++i) { - if (nullable_src->is_null_at(i)) { - null_map_data[i] = 1; - } else { - StringRef ipv6_str = nullable_src->get_data_at(i); - if (IPv6Value::is_valid_string(ipv6_str.data, ipv6_str.size)) { - col_res_data[i] = 1; - } + for (size_t i = 0; i < input_rows_count; ++i) { + if constexpr (std::is_same_v) { + StringRef ipv4_str = str_addr_column->get_data_at(i); + if (IPv4Value::is_valid_string(ipv4_str.data, ipv4_str.size)) { + col_res_data[i] = 1; } - } - - block.replace_by_position( - result, ColumnNullable::create(std::move(col_res), std::move(null_map))); - return Status::OK(); - } else if (const auto* col_src = typeid_cast(column.get())) { - size_t col_size = col_src->size(); - auto col_res = ColumnUInt8::create(col_size, 0); - auto null_map = ColumnUInt8::create(col_size, 0); - auto& col_res_data = col_res->get_data(); - - for (size_t i = 0; i < col_size; ++i) { - StringRef ipv6_str = col_src->get_data_at(i); + } else { + StringRef ipv6_str = str_addr_column->get_data_at(i); if (IPv6Value::is_valid_string(ipv6_str.data, ipv6_str.size)) { col_res_data[i] = 1; } } - - block.replace_by_position( - result, ColumnNullable::create(std::move(col_res), std::move(null_map))); - return Status::OK(); - } else { - return Status::RuntimeError("Illegal column {} of argument of function {}", - argument.column->get_name(), get_name()); } - } - -public: - static constexpr auto name = "is_ipv6_string"; - static FunctionPtr create() { return std::make_shared(); } - - String get_name() const override { return name; } - size_t get_number_of_arguments() const override { return 1; } - - DataTypePtr get_return_type_impl(const DataTypes& arguments) const override { - return make_nullable(std::make_shared()); - } - - bool use_default_implementation_for_nulls() const override { return true; } - - Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, - size_t result, size_t input_rows_count) const override { - ColumnWithTypeAndName& argument = block.get_by_position(arguments[0]); - DCHECK(argument.type->get_type_id() == TypeIndex::String); - return execute_type(block, argument, result); + block.replace_by_position(result, std::move(col_res)); + return Status::OK(); } }; @@ -859,12 +765,10 @@ class FunctionIPv6CIDRToRange : public IFunction { cidr_type->get_name(), get_name()); } DataTypePtr element = std::make_shared(); - return make_nullable(std::make_shared(DataTypes {element, element}, - Strings {"min", "max"})); + return std::make_shared(DataTypes {element, element}, + Strings {"min", "max"}); } - bool use_default_implementation_for_nulls() const override { return true; } - Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, size_t result, size_t input_rows_count) const override { const auto& addr_column_with_type_and_name = block.get_by_position(arguments[0]); @@ -873,54 +777,19 @@ class FunctionIPv6CIDRToRange : public IFunction { WhichDataType cidr_type(cidr_column_with_type_and_name.type); const ColumnPtr& addr_column = addr_column_with_type_and_name.column; const ColumnPtr& cidr_column = cidr_column_with_type_and_name.column; - const ColumnInt16* cidr_col = nullptr; - const NullMap* cidr_null_map = nullptr; + const auto* cidr_col = assert_cast(cidr_column.get()); ColumnPtr col_res = nullptr; - if (cidr_type.is_nullable()) { - const auto* cidr_column_nullable = - assert_cast(cidr_column.get()); - cidr_col = assert_cast(&cidr_column_nullable->get_nested_column()); - cidr_null_map = &cidr_column_nullable->get_null_map_data(); + if (addr_type.is_ipv6()) { + const auto* ipv6_addr_column = check_and_get_column(addr_column.get()); + col_res = execute_impl(*ipv6_addr_column, *cidr_col, input_rows_count); + } else if (addr_type.is_string()) { + const auto* str_addr_column = check_and_get_column(addr_column.get()); + col_res = execute_impl(*str_addr_column, *cidr_col, input_rows_count); } else { - cidr_col = assert_cast(cidr_column.get()); - } - - if (addr_type.is_nullable()) { - const auto* addr_column_nullable = - assert_cast(addr_column.get()); - const NullMap* addr_null_map = &addr_column_nullable->get_null_map_data(); - WhichDataType sub_addr_type(remove_nullable(addr_column_with_type_and_name.type)); - - if (sub_addr_type.is_ipv6()) { - const auto* ipv6_addr_column = - check_and_get_column(addr_column_nullable->get_nested_column()); - col_res = execute_impl(*ipv6_addr_column, addr_null_map, *cidr_col, - cidr_null_map, input_rows_count); - } else if (sub_addr_type.is_string()) { - const auto* str_addr_column = check_and_get_column( - addr_column_nullable->get_nested_column()); - col_res = execute_impl(*str_addr_column, addr_null_map, *cidr_col, - cidr_null_map, input_rows_count); - } else { - return Status::RuntimeError( - "Illegal column {} of argument of function {}, Expected IPv6 or String", - addr_column->get_name(), get_name()); - } - } else { - if (addr_type.is_ipv6()) { - const auto* ipv6_addr_column = check_and_get_column(addr_column.get()); - col_res = execute_impl(*ipv6_addr_column, nullptr, *cidr_col, nullptr, - input_rows_count); - } else if (addr_type.is_string()) { - const auto* str_addr_column = check_and_get_column(addr_column.get()); - col_res = execute_impl(*str_addr_column, nullptr, *cidr_col, nullptr, - input_rows_count); - } else { - return Status::RuntimeError( - "Illegal column {} of argument of function {}, Expected IPv6 or String", - addr_column->get_name(), get_name()); - } + return Status::RuntimeError( + "Illegal column {} of argument of function {}, Expected IPv6 or String", + addr_column->get_name(), get_name()); } block.replace_by_position(result, std::move(col_res)); @@ -928,33 +797,28 @@ class FunctionIPv6CIDRToRange : public IFunction { } template - static ColumnPtr execute_impl(const FromColumn& from_column, const NullMap* from_null_map, - const ColumnInt16& cidr_column, const NullMap* cidr_null_map, + static ColumnPtr execute_impl(const FromColumn& from_column, const ColumnInt16& cidr_column, size_t input_rows_count) { auto col_res_lower_range = ColumnIPv6::create(input_rows_count, 0); auto col_res_upper_range = ColumnIPv6::create(input_rows_count, 0); - auto col_res_null_map = ColumnUInt8::create(input_rows_count, 0); auto& vec_res_lower_range = col_res_lower_range->get_data(); auto& vec_res_upper_range = col_res_upper_range->get_data(); - auto& vec_res_null_map = col_res_null_map->get_data(); - static constexpr Int64 max_cidr_mask = IPV6_BINARY_LENGTH * 8; + static constexpr UInt8 max_cidr_mask = IPV6_BINARY_LENGTH * 8; for (size_t i = 0; i < input_rows_count; ++i) { - if ((from_null_map && (*from_null_map)[i]) || (cidr_null_map && (*cidr_null_map)[i])) { - vec_res_null_map[i] = 1; - continue; + auto cidr = cidr_column.get_int(i); + if (cidr < 0 || cidr > max_cidr_mask) { + throw Exception(ErrorCode::INVALID_ARGUMENT, "Illegal cidr value '{}'", + std::to_string(cidr)); } - Int64 cidr = cidr_column.get_int(i); - cidr = std::min(cidr, max_cidr_mask); apply_cidr_mask(from_column.get_data_at(i).data, reinterpret_cast(&vec_res_lower_range[i]), reinterpret_cast(&vec_res_upper_range[i]), cidr); } - auto col_res_struct = ColumnStruct::create( + return ColumnStruct::create( Columns {std::move(col_res_lower_range), std::move(col_res_upper_range)}); - return ColumnNullable::create(std::move(col_res_struct), std::move(col_res_null_map)); } private: diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java index 903a77b4494e7b..f4f7af5cf387eb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java @@ -188,7 +188,6 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursAdd; import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursDiff; import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursSub; -import org.apache.doris.nereids.trees.expressions.functions.scalar.IPv6CIDRToRange; import org.apache.doris.nereids.trees.expressions.functions.scalar.If; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ignore; import org.apache.doris.nereids.trees.expressions.functions.scalar.Initcap; @@ -198,6 +197,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4StringToNum; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4StringToNumOrDefault; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4StringToNumOrNull; +import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv6CIDRToRange; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv6NumToString; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv6StringToNum; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv6StringToNumOrDefault; @@ -612,7 +612,7 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(IsIpv4String.class, "is_ipv4_string"), scalar(IsIpv6String.class, "is_ipv6_string"), scalar(IsIpAddressInRange.class, "is_ip_address_in_range"), - scalar(IPv6CIDRToRange.class, "ipv6_cidr_to_range"), + scalar(Ipv6CIDRToRange.class, "ipv6_cidr_to_range"), scalar(JsonArray.class, "json_array"), scalar(JsonObject.class, "json_object"), scalar(JsonQuote.class, "json_quote"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IPv6CIDRToRange.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Ipv6CIDRToRange.java similarity index 89% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IPv6CIDRToRange.java rename to fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Ipv6CIDRToRange.java index 2ada5b0e095e1e..7d0720f230204d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IPv6CIDRToRange.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Ipv6CIDRToRange.java @@ -19,8 +19,8 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.IPv6Type; @@ -38,8 +38,8 @@ /** * scalar function `ipv6_cidr_to_range` */ -public class IPv6CIDRToRange extends ScalarFunction - implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable { +public class Ipv6CIDRToRange extends ScalarFunction + implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullable { public static final List SIGNATURES; @@ -54,17 +54,17 @@ public class IPv6CIDRToRange extends ScalarFunction FunctionSignature.ret(retType).args(StringType.INSTANCE, SmallIntType.INSTANCE)); } - public IPv6CIDRToRange(Expression arg0, Expression arg1) { + public Ipv6CIDRToRange(Expression arg0, Expression arg1) { super("ipv6_cidr_to_range", arg0, arg1); } @Override - public IPv6CIDRToRange withChildren(List children) { + public Ipv6CIDRToRange withChildren(List children) { Preconditions.checkArgument(children.size() == 2, "ipv6_cidr_to_range accept 2 args, but got %s (%s)", children.size(), children); - return new IPv6CIDRToRange(children.get(0), children.get(1)); + return new Ipv6CIDRToRange(children.get(0), children.get(1)); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IsIpv4String.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IsIpv4String.java index b1ea28d7518811..cb0f47d257933f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IsIpv4String.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IsIpv4String.java @@ -19,8 +19,8 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.BooleanType; @@ -36,7 +36,7 @@ * scalar function is_ipv4_string */ public class IsIpv4String extends ScalarFunction - implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable { + implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullable { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BooleanType.INSTANCE).args(VarcharType.SYSTEM_DEFAULT), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IsIpv6String.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IsIpv6String.java index 850e35cf3d8e01..0a41c3104b3e68 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IsIpv6String.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IsIpv6String.java @@ -19,8 +19,8 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.BooleanType; @@ -36,7 +36,7 @@ * scalar function is_ipv6_string */ public class IsIpv6String extends ScalarFunction - implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable { + implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullable { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BooleanType.INSTANCE).args(VarcharType.SYSTEM_DEFAULT), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java index 6f43a45dd7990f..899a31abbd7d01 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java @@ -184,7 +184,6 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursAdd; import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursDiff; import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursSub; -import org.apache.doris.nereids.trees.expressions.functions.scalar.IPv6CIDRToRange; import org.apache.doris.nereids.trees.expressions.functions.scalar.If; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ignore; import org.apache.doris.nereids.trees.expressions.functions.scalar.Initcap; @@ -194,6 +193,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4StringToNum; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4StringToNumOrDefault; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4StringToNumOrNull; +import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv6CIDRToRange; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv6NumToString; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv6StringToNum; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv6StringToNumOrDefault; @@ -1167,7 +1167,7 @@ default R visitIsIPAddressInRange(IsIpAddressInRange isIpAddressInRange, C conte return visitScalarFunction(isIpAddressInRange, context); } - default R visitIpv6CIDRToRange(IPv6CIDRToRange ipv6CIDRToRange, C context) { + default R visitIpv6CIDRToRange(Ipv6CIDRToRange ipv6CIDRToRange, C context) { return visitScalarFunction(ipv6CIDRToRange, context); } diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py index 0ca8dfad354efa..973a4675621547 100644 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -2014,14 +2014,14 @@ [['ipv6_string_to_num_or_default'], 'STRING', ['STRING'], 'ALWAYS_NOT_NULLABLE'], [['ipv6_string_to_num_or_null'], 'VARCHAR', ['VARCHAR'], 'ALWAYS_NULLABLE'], [['ipv6_string_to_num_or_null'], 'STRING', ['STRING'], 'ALWAYS_NULLABLE'], - [['is_ipv4_string'], 'BOOLEAN', ['VARCHAR'], 'ALWAYS_NULLABLE'], - [['is_ipv4_string'], 'BOOLEAN', ['STRING'], 'ALWAYS_NULLABLE'], - [['is_ipv6_string'], 'BOOLEAN', ['VARCHAR'], 'ALWAYS_NULLABLE'], - [['is_ipv6_string'], 'BOOLEAN', ['STRING'], 'ALWAYS_NULLABLE'], + [['is_ipv4_string'], 'BOOLEAN', ['VARCHAR'], ''], + [['is_ipv4_string'], 'BOOLEAN', ['STRING'], ''], + [['is_ipv6_string'], 'BOOLEAN', ['VARCHAR'], ''], + [['is_ipv6_string'], 'BOOLEAN', ['STRING'], ''], [['is_ip_address_in_range'], 'BOOLEAN', ['VARCHAR', 'VARCHAR'], 'ALWAYS_NOT_NULLABLE'], [['is_ip_address_in_range'], 'BOOLEAN', ['STRING', 'STRING'], 'ALWAYS_NOT_NULLABLE'], - [['ipv6_cidr_to_range'], 'STRUCT', ['VARCHAR', 'SMALLINT'], 'ALWAYS_NULLABLE'], - [['ipv6_cidr_to_range'], 'STRUCT', ['STRING', 'SMALLINT'], 'ALWAYS_NULLABLE'], + [['ipv6_cidr_to_range'], 'STRUCT', ['VARCHAR', 'SMALLINT'], ''], + [['ipv6_cidr_to_range'], 'STRUCT', ['STRING', 'SMALLINT'], ''], ], "NonNullalbe": [ diff --git a/regression-test/data/query_p0/sql_functions/ip_functions/test_ipv6_cidr_to_range_function.out b/regression-test/data/query_p0/sql_functions/ip_functions/test_ipv6_cidr_to_range_function.out index 411010046cc4c4..6ffd4ee56f7c2a 100644 --- a/regression-test/data/query_p0/sql_functions/ip_functions/test_ipv6_cidr_to_range_function.out +++ b/regression-test/data/query_p0/sql_functions/ip_functions/test_ipv6_cidr_to_range_function.out @@ -6,6 +6,8 @@ 5 2001:: 2001:ffff:ffff:ffff:ffff:ffff:ffff:ffff 6 :: ff:ffff:ffff:ffff:ffff:ffff:ffff:ffff 7 f000:: ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff +8 \N \N +9 \N \N -- !sql -- 1 :: ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff @@ -15,6 +17,8 @@ 5 2001:: 2001:ffff:ffff:ffff:ffff:ffff:ffff:ffff 6 :: ff:ffff:ffff:ffff:ffff:ffff:ffff:ffff 7 f000:: ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff +8 \N \N +9 \N \N -- !sql -- {"min": "::", "max": "ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff"} diff --git a/regression-test/suites/query_p0/sql_functions/ip_functions/test_ipv6_cidr_to_range_function.groovy b/regression-test/suites/query_p0/sql_functions/ip_functions/test_ipv6_cidr_to_range_function.groovy index f606460377a463..7b3c5107fa5974 100644 --- a/regression-test/suites/query_p0/sql_functions/ip_functions/test_ipv6_cidr_to_range_function.groovy +++ b/regression-test/suites/query_p0/sql_functions/ip_functions/test_ipv6_cidr_to_range_function.groovy @@ -42,7 +42,9 @@ suite("test_ipv6_cidr_to_range_function") { (4, '2001:df8:0:85a3::ac1f:8001', 32), (5, '2001:0db8:85a3:85a3:0000:0000:ac1f:8001', 16), (6, '0000:0000:0000:0000:0000:0000:0000:0000', 8), - (7, 'ffff:0000:0000:0000:0000:0000:0000:0000', 4) + (7, 'ffff:0000:0000:0000:0000:0000:0000:0000', 4), + (8, NULL, 8), + (9, 'ffff:0000:0000:0000:0000:0000:0000:0000', NULL) """ qt_sql "select id, struct_element(ipv6_cidr_to_range(addr, cidr), 'min') as min_range, struct_element(ipv6_cidr_to_range(addr, cidr), 'max') as max_range from test_ipv6_cidr_to_range_function order by id" @@ -70,10 +72,12 @@ suite("test_ipv6_cidr_to_range_function") { (4, '2001:df8:0:85a3::ac1f:8001', 32), (5, '2001:0db8:85a3:85a3:0000:0000:ac1f:8001', 16), (6, '0000:0000:0000:0000:0000:0000:0000:0000', 8), - (7, 'ffff:0000:0000:0000:0000:0000:0000:0000', 4) + (7, 'ffff:0000:0000:0000:0000:0000:0000:0000', 4), + (8, NULL, 8), + (9, 'ffff:0000:0000:0000:0000:0000:0000:0000', NULL) """ - qt_sql "select id, struct_element(ipv6_cidr_to_range(ipv6_string_to_num(addr), cidr), 'min') as min_range, struct_element(ipv6_cidr_to_range(ipv6_string_to_num(addr), cidr), 'max') as max_range from test_str_cidr_to_range_function order by id" + qt_sql "select id, struct_element(ipv6_cidr_to_range(ipv6_string_to_num_or_null(addr), cidr), 'min') as min_range, struct_element(ipv6_cidr_to_range(ipv6_string_to_num_or_null(addr), cidr), 'max') as max_range from test_str_cidr_to_range_function order by id" sql """ DROP TABLE IF EXISTS test_str_cidr_to_range_function """ From bcce5a932c666bb5b51153c319b32d530e2c9a87 Mon Sep 17 00:00:00 2001 From: Gabriel Date: Fri, 19 Jan 2024 11:52:12 +0800 Subject: [PATCH 093/200] [pipelineX](DOC) Add pipelineX doc (#30065) --- .../pipeline-x-execution-engine.md | 76 +++++++++++++++++++ docs/sidebars.json | 1 + .../pipeline-x-execution-engine.md | 75 ++++++++++++++++++ 3 files changed, 152 insertions(+) create mode 100644 docs/en/docs/query-acceleration/pipeline-x-execution-engine.md create mode 100644 docs/zh-CN/docs/query-acceleration/pipeline-x-execution-engine.md diff --git a/docs/en/docs/query-acceleration/pipeline-x-execution-engine.md b/docs/en/docs/query-acceleration/pipeline-x-execution-engine.md new file mode 100644 index 00000000000000..2682b0e7ed6fa9 --- /dev/null +++ b/docs/en/docs/query-acceleration/pipeline-x-execution-engine.md @@ -0,0 +1,76 @@ +--- +{ + "title": "PipelineX Execution Engine", + "language": "en", + "toc_min_heading_level": 2, + "toc_max_heading_level": 4 +} +--- + + + +# PipelineX Execution Engine + + + +## Background + +The PipelineX execution engine is an experimental feature in Doris 2.1.0, expected to address the four major issues of the Doris pipeline engine: +1. In terms of execution concurrency, Doris is currently constrained by two factors: one is the parameters set by FE, and the other is limited by the number of buckets. This concurrent strategy prevents the execution engine from fully utilizing machine resources. +2. In terms of execution logic, Doris currently has some fixed additional overhead. For example, the common expression for all instances will be initialized multiple times due to independence between all instances. +3. In terms of scheduling logic, the scheduler of the current pipeline will put all blocking tasks into a blocking queue, and a blocking queue scheduler will be responsible for polling and extracting executable tasks from the blocking queue and placing them in the runnable queue. Therefore, during the query execution process, a CPU core will always be occupied to do scheduling instead of execution. +4. In terms of profile, currently the pipeline cannot provide users concise and clear metrics. + +Its specific design and implementation can be found in [DSIP-035]([DSIP-035: PipelineX Execution Engine - DORIS - Apache Software Foundation](https://cwiki.apache.org/confluence/display/DORIS/DSIP-035%3A+PipelineX+Execution+Engine)). + +## Goals + +1. In terms of execution concurrency, pipelineX introduces local exchange optimization to fully utilize CPU resources, and distribute data evenly across different tasks to minimize data skewing. In addition, pipelineX will no longer be constrained by the number of tablets. +2. Logically, multiple pipeline tasks share all shared states of the same pipeline and eliminate additional initialization overhead, such as expressions and some const variables. +3. In terms of scheduling logic, the blocking conditions of all pipeline tasks are encapsulated using Dependency, and the execution logic of the tasks is triggered by external events (such as rpc completion) to enter the runnable queue, thereby eliminating the overhead of blocking polling threads. +4. Profile: Provide users with simple and easy to understand metrics. + +## User Interface changes + +### Set session variable + +#### enable_pipeline_x_engine + +Set `enable_pipeline_x_engine ` to `true`, BE will use PipelineX to execute by default. + +``` +set enable_pipeline_x_engine = true; +``` + +#### enable_local_shuffle + +Set `enable_local_shuffle` to true will enable local shuffle optimization. Local shuffle will try to evenly distribute data among different pipeline tasks to avoid data skewing as much as possible. + +``` +set enable_local_shuffle = true; +``` + +#### ignore_storage_data_distribution + +Settings `ignore_storage_data_distribution` is true, it means ignoring the data distribution of the storage layer. When used in conjunction with local shuffle, the concurrency capability of the pipelineX engine will no longer be constrained by the number of storage layer tables, thus fully utilizing machine resources. + +``` +set ignore_storage_data_distribution = true; +``` diff --git a/docs/sidebars.json b/docs/sidebars.json index c6b5efc203f106..6b94989d1808be 100644 --- a/docs/sidebars.json +++ b/docs/sidebars.json @@ -192,6 +192,7 @@ "label": "Query Acceleration", "items": [ "query-acceleration/pipeline-execution-engine", + "query-acceleration/pipeline-x-execution-engine", "query-acceleration/nereids", "query-acceleration/hight-concurrent-point-query", "query-acceleration/materialized-view", diff --git a/docs/zh-CN/docs/query-acceleration/pipeline-x-execution-engine.md b/docs/zh-CN/docs/query-acceleration/pipeline-x-execution-engine.md new file mode 100644 index 00000000000000..3ce942a11515ce --- /dev/null +++ b/docs/zh-CN/docs/query-acceleration/pipeline-x-execution-engine.md @@ -0,0 +1,75 @@ +--- +{ + "title": "PipelineX 执行引擎", + "language": "zh-CN", + "toc_min_heading_level": 2, + "toc_max_heading_level": 4 +} +--- + + + +# PipelineX 执行引擎 + + + +## 背景 +PipelineX 执行引擎 是 Doris 在 2.1 版本加入的实验性功能。目标是为了解决Doris pipeline引擎的四大问题: +1. 执行并发上,当前Doris执行并发收到两个因素的制约,一个是fe设置的参数,另一个是受存储层bucket数量的限制,这样的静态并发使得执行引擎无法充分利用机器资源。 +2. 执行逻辑上,当前Doris有一些固定的额外开销,例如表达式部分各个instance彼此独立,而instance的初始化参数有很多公共部分,所以需要额外进行很多重复的初始化步骤。 +3. 调度逻辑上,当前pipeline的调度器会把阻塞task全部放入一个阻塞队列中,由一个线程负责轮询并从阻塞队列中取出可执行task放入runnable队列,所以在有查询执行的过程中,会固定有一个核的资源作为调度的开销。 +4. profile方面,目前pipeline无法为用户提供简单易懂的指标。 + +它的具体设计、实现和效果可以参阅 [DSIP-035]([DSIP-035: PipelineX Execution Engine - DORIS - Apache Software Foundation](https://cwiki.apache.org/confluence/display/DORIS/DSIP-035%3A+PipelineX+Execution+Engine))。 + +## 预期效果 + +1. 执行并发上,依赖local exchange使pipelinex充分并发,可以让数据被均匀分布到不同的task中,尽可能减少数据倾斜,此外,pipelineX也将不再受存储层tablet数量的制约。 +2. 执行逻辑上,多个pipeline task共享同一个pipeline的全部共享状态,例如表达式和一些const变量,消除了额外的初始化开销。 +3. 调度逻辑上,所有pipeline task的阻塞条件都使用Dependency进行了封装,通过外部事件(例如rpc完成)触发task的执行逻辑进入runnable队列,从而消除了阻塞轮询线程的开销。 +4. profile:为用户提供简单易懂的指标。 + +## 用户接口变更 + +### 设置Session变量 + +#### enable_pipeline_x_engine + +将session变量`enable_pipeline_x_engine `设置为`true`,则 BE 在进行查询执行时就会默认将 SQL 的执行模型转变 PipelineX 的执行方式。 + +``` +set enable_pipeline_x_engine = true; +``` + +#### enable_local_shuffle + +设置`enable_local_shuffle`为true则打开local shuffle优化。local shuffle将尽可能将数据均匀分布给不同的pipeline task从而尽可能避免数据倾斜。 + +``` +set enable_local_shuffle = true; +``` + +#### ignore_storage_data_distribution + +设置`ignore_storage_data_distribution`为true则表示忽略存储层的数据分布。结合local shuffle一起使用,则pipelineX引擎的并发能力将不再受到存储层tablet数量的制约,从而充分利用机器资源。 + +``` +set ignore_storage_data_distribution = true; +``` From a949e1a761edb01e2b96469545b5fa9b7bee9496 Mon Sep 17 00:00:00 2001 From: shuke <37901441+shuke987@users.noreply.github.com> Date: Fri, 19 Jan 2024 14:17:47 +0800 Subject: [PATCH 094/200] [test](regression-test) fix case bugs (#30115) fix case bugs --- regression-test/suites/export/test_struct_export.groovy | 2 +- .../test_index_compaction_fault_injection.groovy | 8 ++++++++ .../suites/show_p0/test_show_statistic_proc.groovy | 1 + 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/regression-test/suites/export/test_struct_export.groovy b/regression-test/suites/export/test_struct_export.groovy index f9f2b1fc63c0f5..8df3a40906e67c 100644 --- a/regression-test/suites/export/test_struct_export.groovy +++ b/regression-test/suites/export/test_struct_export.groovy @@ -98,7 +98,7 @@ suite("test_struct_export", "export") { } else { throw new IllegalStateException("""${outFilePath} already exists! """) } - result = sql """ + def result = sql """ SELECT * FROM ${testTable} ORDER BY k1 INTO OUTFILE "file://${outFile}/"; """ url = result[0][3] diff --git a/regression-test/suites/fault_injection_p0/test_index_compaction_fault_injection.groovy b/regression-test/suites/fault_injection_p0/test_index_compaction_fault_injection.groovy index c3fd4f70272076..b61f6e9c0b6b49 100644 --- a/regression-test/suites/fault_injection_p0/test_index_compaction_fault_injection.groovy +++ b/regression-test/suites/fault_injection_p0/test_index_compaction_fault_injection.groovy @@ -127,6 +127,14 @@ suite("test_index_compaction_failure_injection", "nonConcurrent") { run_sql.call() int replicaNum = 1 + String[][] dedup_tablets = deduplicate_tablets(tablets) + if (dedup_tablets.size() > 0) { + replicaNum = Math.round(tablets.size() / dedup_tablets.size()) + if (replicaNum != 1 && replicaNum != 3) { + assert(false) + } + } + // before full compaction, there are 7 rowsets. int rowsetCount = get_rowset_count.call(tablets); assert (rowsetCount == 7 * replicaNum) diff --git a/regression-test/suites/show_p0/test_show_statistic_proc.groovy b/regression-test/suites/show_p0/test_show_statistic_proc.groovy index 067dc4c418101f..50825868789bb4 100644 --- a/regression-test/suites/show_p0/test_show_statistic_proc.groovy +++ b/regression-test/suites/show_p0/test_show_statistic_proc.groovy @@ -23,6 +23,7 @@ suite("test_show_statistic_proc", "nonConcurrent") { sql """grant ADMIN_PRIV on *.*.* to test_show_statistic_proc_user1""" + sql """drop database if exists test_statistic_proc_db""" sql """create database test_statistic_proc_db""" def result1 = connect(user = 'test_show_statistic_proc_user1', password = '12345', url = context.config.jdbcUrl) { From 6360ccf23d18d710c859ac18914b75dc7298affb Mon Sep 17 00:00:00 2001 From: zhangstar333 <87313068+zhangstar333@users.noreply.github.com> Date: Fri, 19 Jan 2024 14:36:01 +0800 Subject: [PATCH 095/200] [improve](profile) add init probe side timer in join node (#30079) --- .../pipeline/exec/hashjoin_probe_operator.cpp | 1 + be/src/pipeline/exec/hashjoin_probe_operator.h | 1 + .../vec/exec/join/process_hash_table_probe.h | 1 + .../exec/join/process_hash_table_probe_impl.h | 18 +++++++++++------- be/src/vec/exec/join/vhash_join_node.cpp | 1 + be/src/vec/exec/join/vhash_join_node.h | 1 + 6 files changed, 16 insertions(+), 7 deletions(-) diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.cpp b/be/src/pipeline/exec/hashjoin_probe_operator.cpp index f852d3c44405f8..c7029614c50753 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.cpp +++ b/be/src/pipeline/exec/hashjoin_probe_operator.cpp @@ -56,6 +56,7 @@ Status HashJoinProbeLocalState::init(RuntimeState* state, LocalStateInfo& info) _probe_side_output_timer = ADD_TIMER(profile(), "ProbeWhenProbeSideOutputTime"); _probe_process_hashtable_timer = ADD_TIMER(profile(), "ProbeWhenProcessHashTableTime"); _process_other_join_conjunct_timer = ADD_TIMER(profile(), "OtherJoinConjunctTime"); + _init_probe_side_timer = ADD_TIMER(profile(), "InitProbeSideTime"); return Status::OK(); } diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.h b/be/src/pipeline/exec/hashjoin_probe_operator.h index ac7954af13b5e3..18db6acc67f32c 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.h +++ b/be/src/pipeline/exec/hashjoin_probe_operator.h @@ -141,6 +141,7 @@ class HashJoinProbeLocalState final RuntimeProfile::Counter* _probe_process_hashtable_timer = nullptr; RuntimeProfile::HighWaterMarkCounter* _probe_arena_memory_usage = nullptr; RuntimeProfile::Counter* _search_hashtable_timer = nullptr; + RuntimeProfile::Counter* _init_probe_side_timer = nullptr; RuntimeProfile::Counter* _build_side_output_timer = nullptr; RuntimeProfile::Counter* _process_other_join_conjunct_timer = nullptr; }; diff --git a/be/src/vec/exec/join/process_hash_table_probe.h b/be/src/vec/exec/join/process_hash_table_probe.h index 4b3140b7b594e4..803cc34bd7877f 100644 --- a/be/src/vec/exec/join/process_hash_table_probe.h +++ b/be/src/vec/exec/join/process_hash_table_probe.h @@ -109,6 +109,7 @@ struct ProcessHashTableProbe { RuntimeProfile::Counter* _rows_returned_counter = nullptr; RuntimeProfile::Counter* _search_hashtable_timer = nullptr; + RuntimeProfile::Counter* _init_probe_side_timer = nullptr; RuntimeProfile::Counter* _build_side_output_timer = nullptr; RuntimeProfile::Counter* _probe_side_output_timer = nullptr; RuntimeProfile::Counter* _probe_process_hashtable_timer = nullptr; diff --git a/be/src/vec/exec/join/process_hash_table_probe_impl.h b/be/src/vec/exec/join/process_hash_table_probe_impl.h index 4f950f876cf0b5..c9f92c59e08dc6 100644 --- a/be/src/vec/exec/join/process_hash_table_probe_impl.h +++ b/be/src/vec/exec/join/process_hash_table_probe_impl.h @@ -50,6 +50,7 @@ ProcessHashTableProbe::ProcessHashTableProbe(Parent* parent, _has_null_in_build_side(parent->has_null_in_build_side()), _rows_returned_counter(parent->_rows_returned_counter), _search_hashtable_timer(parent->_search_hashtable_timer), + _init_probe_side_timer(parent->_init_probe_side_timer), _build_side_output_timer(parent->_build_side_output_timer), _probe_side_output_timer(parent->_probe_side_output_timer), _probe_process_hashtable_timer(parent->_probe_process_hashtable_timer), @@ -156,13 +157,16 @@ Status ProcessHashTableProbe::do_process(HashTableType& hash auto& build_index = _parent->_build_index; auto last_probe_index = probe_index; - _init_probe_side( - hash_table_ctx, probe_rows, with_other_conjuncts, - need_null_map_for_probe ? null_map->data() : nullptr, - need_null_map_for_probe && ignore_null && - (JoinOpType == doris::TJoinOp::LEFT_ANTI_JOIN || - JoinOpType == doris::TJoinOp::LEFT_SEMI_JOIN || - JoinOpType == doris::TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN || is_mark_join)); + { + SCOPED_TIMER(_init_probe_side_timer); + _init_probe_side( + hash_table_ctx, probe_rows, with_other_conjuncts, + need_null_map_for_probe ? null_map->data() : nullptr, + need_null_map_for_probe && ignore_null && + (JoinOpType == doris::TJoinOp::LEFT_ANTI_JOIN || + JoinOpType == doris::TJoinOp::LEFT_SEMI_JOIN || + JoinOpType == doris::TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN || is_mark_join)); + } auto& mcol = mutable_block.mutable_columns(); diff --git a/be/src/vec/exec/join/vhash_join_node.cpp b/be/src/vec/exec/join/vhash_join_node.cpp index 94cb5be876f941..39e05936397726 100644 --- a/be/src/vec/exec/join/vhash_join_node.cpp +++ b/be/src/vec/exec/join/vhash_join_node.cpp @@ -241,6 +241,7 @@ Status HashJoinNode::prepare(RuntimeState* state) { _open_timer = ADD_TIMER(runtime_profile(), "OpenTime"); _allocate_resource_timer = ADD_TIMER(runtime_profile(), "AllocateResourceTime"); _process_other_join_conjunct_timer = ADD_TIMER(runtime_profile(), "OtherJoinConjunctTime"); + _init_probe_side_timer = ADD_CHILD_TIMER(probe_phase_profile, "InitProbeSideTime", "ProbeTime"); RETURN_IF_ERROR(VExpr::prepare(_build_expr_ctxs, state, child(1)->row_desc())); RETURN_IF_ERROR(VExpr::prepare(_probe_expr_ctxs, state, child(0)->row_desc())); diff --git a/be/src/vec/exec/join/vhash_join_node.h b/be/src/vec/exec/join/vhash_join_node.h index b9b3d18dff7198..7fdb103d1f2c8d 100644 --- a/be/src/vec/exec/join/vhash_join_node.h +++ b/be/src/vec/exec/join/vhash_join_node.h @@ -319,6 +319,7 @@ class HashJoinNode final : public VJoinNodeBase { RuntimeProfile::Counter* _probe_process_hashtable_timer = nullptr; RuntimeProfile::Counter* _build_side_compute_hash_timer = nullptr; RuntimeProfile::Counter* _build_side_merge_block_timer = nullptr; + RuntimeProfile::Counter* _init_probe_side_timer = nullptr; RuntimeProfile::Counter* _open_timer = nullptr; RuntimeProfile::Counter* _allocate_resource_timer = nullptr; From ae0220217ba4f3492f6a594a6a0131c768fc3c99 Mon Sep 17 00:00:00 2001 From: Kaijie Chen Date: Fri, 19 Jan 2024 14:52:03 +0800 Subject: [PATCH 096/200] [fix](move-memtable) only do close wait on the last sink (#30098) --- be/src/vec/sink/load_stream_stub.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/be/src/vec/sink/load_stream_stub.cpp b/be/src/vec/sink/load_stream_stub.cpp index 347acb1b6f6f0f..c4b1016229924a 100644 --- a/be/src/vec/sink/load_stream_stub.cpp +++ b/be/src/vec/sink/load_stream_stub.cpp @@ -316,6 +316,10 @@ Status LoadStreamStub::close_wait(int64_t timeout_ms) { if (_is_closed.load()) { return _check_cancel(); } + // if there are other sinks remaining, let the last sink handle close wait + if (_use_cnt > 0) { + return Status::OK(); + } if (timeout_ms <= 0) { timeout_ms = config::close_load_stream_timeout_ms; } From acbd78c56b68b0c007d79cf765647136d31a5273 Mon Sep 17 00:00:00 2001 From: minghong Date: Fri, 19 Jan 2024 15:09:33 +0800 Subject: [PATCH 097/200] [opt](Nereids)make orToIn rule appliable to in-pred (#29990) make orToIn rule appliable to in-pred --- .../rules/expression/rules/OrToIn.java | 51 +++++++-------- .../nereids/rules/rewrite/OrToInTest.java | 15 +++++ .../shape/query13.out | 54 +++++++-------- .../shape/query15.out | 2 +- .../shape/query48.out | 2 +- .../shape/query85.out | 65 +++++++++---------- .../noStatsRfPrune/query13.out | 2 +- .../noStatsRfPrune/query15.out | 2 +- .../noStatsRfPrune/query48.out | 2 +- .../noStatsRfPrune/query85.out | 2 +- .../no_stats_shape/query13.out | 2 +- .../no_stats_shape/query15.out | 2 +- .../no_stats_shape/query48.out | 2 +- .../no_stats_shape/query85.out | 2 +- .../rf_prune/query13.out | 2 +- .../rf_prune/query15.out | 2 +- .../rf_prune/query48.out | 2 +- .../rf_prune/query85.out | 58 +++++++++-------- .../shape/query13.out | 2 +- .../shape/query15.out | 2 +- .../shape/query48.out | 2 +- .../shape/query85.out | 58 +++++++++-------- 22 files changed, 175 insertions(+), 158 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OrToIn.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OrToIn.java index aaa077d1994f1c..6df68e47a04e9c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OrToIn.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OrToIn.java @@ -29,9 +29,9 @@ import org.apache.doris.nereids.util.ExpressionUtils; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -71,9 +71,12 @@ public Expression rewrite(Expression expr, ExpressionRewriteContext ctx) { public Expression visitOr(Or or, ExpressionRewriteContext ctx) { Map> slotNameToLiteral = new HashMap<>(); List expressions = ExpressionUtils.extractDisjunction(or); + Map disConjunctToSlot = Maps.newHashMap(); for (Expression expression : expressions) { if (expression instanceof EqualTo) { - addSlotToLiteralMap((EqualTo) expression, slotNameToLiteral); + handleEqualTo((EqualTo) expression, slotNameToLiteral, disConjunctToSlot); + } else if (expression instanceof InPredicate) { + handleInPredicate((InPredicate) expression, slotNameToLiteral, disConjunctToSlot); } } List rewrittenOr = new ArrayList<>(); @@ -85,42 +88,42 @@ public Expression visitOr(Or or, ExpressionRewriteContext ctx) { } } for (Expression expression : expressions) { - if (!ableToConvertToIn(expression, slotNameToLiteral)) { + if (disConjunctToSlot.get(expression) == null) { rewrittenOr.add(expression.accept(this, null)); + } else { + Set literals = slotNameToLiteral.get(disConjunctToSlot.get(expression)); + if (literals.size() < REWRITE_OR_TO_IN_PREDICATE_THRESHOLD) { + rewrittenOr.add(expression); + } } } return ExpressionUtils.or(rewrittenOr); } - private void addSlotToLiteralMap(EqualTo equal, Map> slotNameToLiteral) { + private void handleEqualTo(EqualTo equal, Map> slotNameToLiteral, + Map disConjunctToSlot) { Expression left = equal.left(); Expression right = equal.right(); if (left instanceof NamedExpression && right instanceof Literal) { addSlotToLiteral((NamedExpression) left, (Literal) right, slotNameToLiteral); - } - if (right instanceof NamedExpression && left instanceof Literal) { + disConjunctToSlot.put(equal, (NamedExpression) left); + } else if (right instanceof NamedExpression && left instanceof Literal) { addSlotToLiteral((NamedExpression) right, (Literal) left, slotNameToLiteral); + disConjunctToSlot.put(equal, (NamedExpression) right); } } - private boolean ableToConvertToIn(Expression expression, Map> slotNameToLiteral) { - if (!(expression instanceof EqualTo)) { - return false; - } - EqualTo equalTo = (EqualTo) expression; - Expression left = equalTo.left(); - Expression right = equalTo.right(); - NamedExpression namedExpression = null; - if (left instanceof NamedExpression && right instanceof Literal) { - namedExpression = (NamedExpression) left; - } - if (right instanceof NamedExpression && left instanceof Literal) { - namedExpression = (NamedExpression) right; + private void handleInPredicate(InPredicate inPredicate, Map> slotNameToLiteral, + Map disConjunctToSlot) { + // TODO a+b in (1,2,3...) is not supported now + if (inPredicate.getCompareExpr() instanceof NamedExpression + && inPredicate.getOptions().stream().allMatch(opt -> opt instanceof Literal)) { + for (Expression opt : inPredicate.getOptions()) { + addSlotToLiteral((NamedExpression) inPredicate.getCompareExpr(), (Literal) opt, slotNameToLiteral); + } + disConjunctToSlot.put(inPredicate, (NamedExpression) inPredicate.getCompareExpr()); } - return namedExpression != null - && findSizeOfLiteralThatEqualToSameSlotInOr(namedExpression, slotNameToLiteral) - >= REWRITE_OR_TO_IN_PREDICATE_THRESHOLD; } public void addSlotToLiteral(NamedExpression namedExpression, Literal literal, @@ -129,8 +132,4 @@ public void addSlotToLiteral(NamedExpression namedExpression, Literal literal, literals.add(literal); } - public int findSizeOfLiteralThatEqualToSameSlotInOr(NamedExpression namedExpression, - Map> slotNameToLiteral) { - return slotNameToLiteral.getOrDefault(namedExpression, Collections.emptySet()).size(); - } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/OrToInTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/OrToInTest.java index f77a66dd88691f..b4cf4451cbe244 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/OrToInTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/OrToInTest.java @@ -109,4 +109,19 @@ void test5() { rewritten.toSql()); } + @Test + void test6() { + String expr = "col = 1 or col = 2 or col in (1, 2, 3)"; + Expression expression = PARSER.parseExpression(expr); + Expression rewritten = new OrToIn().rewrite(expression, new ExpressionRewriteContext(null)); + Assertions.assertEquals("col IN (1, 2, 3)", rewritten.toSql()); + } + + @Test + void test7() { + String expr = "A = 1 or A = 2 or abs(A)=5 or A in (1, 2, 3) or B = 1 or B = 2 or B in (1, 2, 3) or B+1 in (4, 5, 7)"; + Expression expression = PARSER.parseExpression(expr); + Expression rewritten = new OrToIn().rewrite(expression, new ExpressionRewriteContext(null)); + Assertions.assertEquals("(((A IN (1, 2, 3) OR B IN (1, 2, 3)) OR (abs(A) = 5)) OR (B + 1) IN (4, 5, 7))", rewritten.toSql()); + } } diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query13.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query13.out index 07764ae74a0eba..92c09059cd5078 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query13.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query13.out @@ -5,35 +5,35 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] --------PhysicalProject -----------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] +----------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 ss_store_sk->[s_store_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=(((((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)) OR ((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Primary')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) AND (household_demographics.hd_dep_count = 1))) OR ((((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))) AND (household_demographics.hd_dep_count = 1)))) build RFs:RF3 ss_cdemo_sk->[cd_demo_sk] -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------filter(((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'College')) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Primary'))) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = '2 yr Degree')))) -----------------------PhysicalOlapScan[customer_demographics] apply RFs: RF3 -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] -----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('IL', 'TN', 'TX') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('ID', 'OH', 'WY') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('IA', 'MS', 'SC') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00))))) build RFs:RF0 ca_address_sk->[ss_addr_sk] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------filter((store_sales.ss_net_profit <= 300.00) and (store_sales.ss_net_profit >= 50.00) and (store_sales.ss_sales_price <= 200.00) and (store_sales.ss_sales_price >= 50.00)) ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF4 ---------------------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalOlapScan[store] apply RFs: RF4 +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=(((((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)) OR ((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Primary')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) AND (household_demographics.hd_dep_count = 1))) OR ((((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))) AND (household_demographics.hd_dep_count = 1)))) build RFs:RF3 ss_cdemo_sk->[cd_demo_sk] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------filter(((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'College')) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Primary'))) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = '2 yr Degree')))) +------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF3 +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] +------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('IL', 'TN', 'TX') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('ID', 'OH', 'WY') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('IA', 'MS', 'SC') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00))))) build RFs:RF0 ca_address_sk->[ss_addr_sk] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------filter((store_sales.ss_net_profit <= 300.00) and (store_sales.ss_net_profit >= 50.00) and (store_sales.ss_sales_price <= 200.00) and (store_sales.ss_sales_price >= 50.00)) +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('IA', 'ID', 'IL', 'MS', 'OH', 'SC', 'TN', 'TX', 'WY')) +----------------------------------PhysicalOlapScan[customer_address] +--------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject -------------------------------filter(((ca_state IN ('IL', 'TN', 'TX') OR ca_state IN ('ID', 'OH', 'WY')) OR ca_state IN ('IA', 'MS', 'SC')) and (customer_address.ca_country = 'United States')) ---------------------------------PhysicalOlapScan[customer_address] +------------------------------filter((date_dim.d_year = 2001)) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------PhysicalProject ---------------------------filter(hd_dep_count IN (1, 3)) -----------------------------PhysicalOlapScan[household_demographics] -------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalProject -----------------PhysicalOlapScan[store] +----------------------------filter(hd_dep_count IN (1, 3)) +------------------------------PhysicalOlapScan[household_demographics] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query15.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query15.out index 76dbaeb392cf2b..6d388f54994014 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query15.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] +----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((ca_state IN ('CA', 'GA', 'WA') OR substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ----------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query48.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query48.out index 2fcf8104b4456f..4069222cacbfa9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query48.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query48.out @@ -21,7 +21,7 @@ PhysicalResultSink ------------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalDistribute[DistributionSpecHash] ----------------------PhysicalProject -------------------------filter(((ca_state IN ('ND', 'NY', 'SD') OR ca_state IN ('GA', 'KS', 'MD')) OR ca_state IN ('CO', 'MN', 'NC')) and (customer_address.ca_country = 'United States')) +------------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('CO', 'GA', 'KS', 'MD', 'MN', 'NC', 'ND', 'NY', 'SD')) --------------------------PhysicalOlapScan[customer_address] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query85.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query85.out index b120d3ff5e0ccf..547042a9a33744 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query85.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query85.out @@ -9,43 +9,42 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN] hashCondition=((reason.r_reason_sk = web_returns.wr_reason_sk)) otherCondition=() build RFs:RF9 r_reason_sk->[wr_reason_sk] +------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_page_sk = web_page.wp_web_page_sk)) otherCondition=() build RFs:RF9 ws_web_page_sk->[wp_web_page_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=((((ca_state IN ('IA', 'NC', 'TX') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('GA', 'WI', 'WV') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('KY', 'OK', 'VA') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00))))) build RFs:RF8 wr_refunded_addr_sk->[ca_address_sk] -------------------------PhysicalProject ---------------------------filter(((ca_state IN ('IA', 'NC', 'TX') OR ca_state IN ('GA', 'WI', 'WV')) OR ca_state IN ('KY', 'OK', 'VA')) and (customer_address.ca_country = 'United States')) -----------------------------PhysicalOlapScan[customer_address] apply RFs: RF8 -------------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalOlapScan[web_page] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN] hashCondition=((reason.r_reason_sk = web_returns.wr_reason_sk)) otherCondition=() build RFs:RF8 r_reason_sk->[wr_reason_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_page_sk = web_page.wp_web_page_sk)) otherCondition=() build RFs:RF7 ws_web_page_sk->[wp_web_page_sk] +----------------------------hashJoin[INNER_JOIN] hashCondition=((cd1.cd_education_status = cd2.cd_education_status) and (cd1.cd_marital_status = cd2.cd_marital_status) and (cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk)) otherCondition=() build RFs:RF5 cd_marital_status->[cd_marital_status];RF6 cd_education_status->[cd_education_status];RF7 wr_returning_cdemo_sk->[cd_demo_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_page] apply RFs: RF7 -------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN] hashCondition=((cd1.cd_education_status = cd2.cd_education_status) and (cd1.cd_marital_status = cd2.cd_marital_status) and (cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk)) otherCondition=() build RFs:RF4 cd_marital_status->[cd_marital_status];RF5 cd_education_status->[cd_education_status];RF6 wr_returning_cdemo_sk->[cd_demo_sk] +--------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF5 RF6 RF7 +------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=((((ca_state IN ('IA', 'NC', 'TX') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('GA', 'WI', 'WV') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('KY', 'OK', 'VA') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00))))) build RFs:RF4 wr_refunded_addr_sk->[ca_address_sk] +----------------------------------PhysicalProject +------------------------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('GA', 'IA', 'KY', 'NC', 'OK', 'TX', 'VA', 'WI', 'WV')) +--------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF4 +----------------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF4 RF5 RF6 -------------------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=((((((cd1.cd_marital_status = 'D') AND (cd1.cd_education_status = 'Primary')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'College')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cd1.cd_marital_status = 'U') AND (cd1.cd_education_status = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00))))) build RFs:RF3 cd_demo_sk->[wr_refunded_cdemo_sk] -------------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF1 ws_item_sk->[wr_item_sk];RF2 ws_order_number->[wr_order_number] +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=((((((cd1.cd_marital_status = 'D') AND (cd1.cd_education_status = 'Primary')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'College')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cd1.cd_marital_status = 'U') AND (cd1.cd_education_status = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00))))) build RFs:RF3 cd_demo_sk->[wr_refunded_cdemo_sk] +----------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------PhysicalProject +--------------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF1 ws_item_sk->[wr_item_sk];RF2 ws_order_number->[wr_order_number] +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 RF3 RF8 +----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 RF3 RF9 -------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------filter((web_sales.ws_net_profit <= 300.00) and (web_sales.ws_net_profit >= 50.00) and (web_sales.ws_sales_price <= 200.00) and (web_sales.ws_sales_price >= 50.00)) +----------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------------------------PhysicalProject -----------------------------------------------------filter((web_sales.ws_net_profit <= 300.00) and (web_sales.ws_net_profit >= 50.00) and (web_sales.ws_sales_price <= 200.00) and (web_sales.ws_sales_price >= 50.00)) -------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------------------------PhysicalProject -------------------------------------------------------filter((date_dim.d_year = 1998)) ---------------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------------PhysicalProject -----------------------------------------------filter(((((cd1.cd_marital_status = 'D') AND (cd1.cd_education_status = 'Primary')) OR ((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'College'))) OR ((cd1.cd_marital_status = 'U') AND (cd1.cd_education_status = 'Advanced Degree')))) -------------------------------------------------PhysicalOlapScan[customer_demographics] ---------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------PhysicalProject -------------------------PhysicalOlapScan[reason] +----------------------------------------------------filter((date_dim.d_year = 1998)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------PhysicalProject +--------------------------------------------filter(((((cd1.cd_marital_status = 'D') AND (cd1.cd_education_status = 'Primary')) OR ((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'College'))) OR ((cd1.cd_marital_status = 'U') AND (cd1.cd_education_status = 'Advanced Degree')))) +----------------------------------------------PhysicalOlapScan[customer_demographics] +--------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[reason] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query13.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query13.out index 9689a10568213b..09eea0486ff63e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query13.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query13.out @@ -25,7 +25,7 @@ PhysicalResultSink --------------------------PhysicalOlapScan[store] ------------------PhysicalDistribute[DistributionSpecReplicated] --------------------PhysicalProject -----------------------filter(((ca_state IN ('KS', 'MI', 'SD') OR ca_state IN ('CO', 'MO', 'ND')) OR ca_state IN ('NH', 'OH', 'TX')) and (customer_address.ca_country = 'United States')) +----------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('CO', 'KS', 'MI', 'MO', 'ND', 'NH', 'OH', 'SD', 'TX')) ------------------------PhysicalOlapScan[customer_address] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query15.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query15.out index d1254304e057e0..7ab12ff23f2844 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query15.out @@ -10,7 +10,7 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) +--------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(((ca_state IN ('CA', 'GA', 'WA') OR substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274')) OR (catalog_sales.cs_sales_price > 500.00))) ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query48.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query48.out index 59167dad86db85..25e8c3965aa3b4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query48.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query48.out @@ -24,7 +24,7 @@ PhysicalResultSink ------------------------PhysicalOlapScan[store] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject ---------------------filter(((ca_state IN ('IA', 'MD', 'MN') OR ca_state IN ('IL', 'TX', 'VA')) OR ca_state IN ('IN', 'MI', 'WI')) and (customer_address.ca_country = 'United States')) +--------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('IA', 'IL', 'IN', 'MD', 'MI', 'MN', 'TX', 'VA', 'WI')) ----------------------PhysicalOlapScan[customer_address] ------------PhysicalDistribute[DistributionSpecReplicated] --------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query85.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query85.out index 251ba0bd5a51ae..a381fb5be441ab 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query85.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query85.out @@ -40,7 +40,7 @@ PhysicalResultSink ------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------PhysicalProject ---------------------------------filter(((ca_state IN ('DE', 'FL', 'TX') OR ca_state IN ('ID', 'IN', 'ND')) OR ca_state IN ('IL', 'MT', 'OH')) and (customer_address.ca_country = 'United States')) +--------------------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('DE', 'FL', 'ID', 'IL', 'IN', 'MT', 'ND', 'OH', 'TX')) ----------------------------------PhysicalOlapScan[customer_address] ------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query13.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query13.out index bb8d755b7de91f..55f477930971cf 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query13.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query13.out @@ -25,7 +25,7 @@ PhysicalResultSink --------------------------PhysicalOlapScan[store] ------------------PhysicalDistribute[DistributionSpecReplicated] --------------------PhysicalProject -----------------------filter(((ca_state IN ('KS', 'MI', 'SD') OR ca_state IN ('CO', 'MO', 'ND')) OR ca_state IN ('NH', 'OH', 'TX')) and (customer_address.ca_country = 'United States')) +----------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('CO', 'KS', 'MI', 'MO', 'ND', 'NH', 'OH', 'SD', 'TX')) ------------------------PhysicalOlapScan[customer_address] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query15.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query15.out index 8e929d24b0c6a2..3ab21912425a72 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query15.out @@ -10,7 +10,7 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF1 ca_address_sk->[c_current_addr_sk] +--------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(((ca_state IN ('CA', 'GA', 'WA') OR substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF1 ca_address_sk->[c_current_addr_sk] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[cs_bill_customer_sk] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query48.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query48.out index 496160afe53473..463c25b9c7dcab 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query48.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query48.out @@ -24,7 +24,7 @@ PhysicalResultSink ------------------------PhysicalOlapScan[store] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject ---------------------filter(((ca_state IN ('IA', 'MD', 'MN') OR ca_state IN ('IL', 'TX', 'VA')) OR ca_state IN ('IN', 'MI', 'WI')) and (customer_address.ca_country = 'United States')) +--------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('IA', 'IL', 'IN', 'MD', 'MI', 'MN', 'TX', 'VA', 'WI')) ----------------------PhysicalOlapScan[customer_address] ------------PhysicalDistribute[DistributionSpecReplicated] --------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query85.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query85.out index 29433f93b42ab4..fdc9d7548628b1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query85.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query85.out @@ -40,7 +40,7 @@ PhysicalResultSink ------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------PhysicalProject ---------------------------------filter(((ca_state IN ('DE', 'FL', 'TX') OR ca_state IN ('ID', 'IN', 'ND')) OR ca_state IN ('IL', 'MT', 'OH')) and (customer_address.ca_country = 'United States')) +--------------------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('DE', 'FL', 'ID', 'IL', 'IN', 'MT', 'ND', 'OH', 'TX')) ----------------------------------PhysicalOlapScan[customer_address] ------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query13.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query13.out index 5e8f8d4e9b2978..441b3d76382391 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query13.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query13.out @@ -25,7 +25,7 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------PhysicalProject ---------------------------------filter(((ca_state IN ('KS', 'MI', 'SD') OR ca_state IN ('CO', 'MO', 'ND')) OR ca_state IN ('NH', 'OH', 'TX')) and (customer_address.ca_country = 'United States')) +--------------------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('CO', 'KS', 'MI', 'MO', 'ND', 'NH', 'OH', 'SD', 'TX')) ----------------------------------PhysicalOlapScan[customer_address] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query15.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query15.out index 7b670f886ae948..f2d4abf7f789eb 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query15.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) +----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((ca_state IN ('CA', 'GA', 'WA') OR substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274')) OR (catalog_sales.cs_sales_price > 500.00))) ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ----------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query48.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query48.out index a4f0b1a09f9d12..ad7e5318fda922 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query48.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query48.out @@ -21,7 +21,7 @@ PhysicalResultSink ------------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalDistribute[DistributionSpecHash] ----------------------PhysicalProject -------------------------filter(((ca_state IN ('IA', 'MD', 'MN') OR ca_state IN ('IL', 'TX', 'VA')) OR ca_state IN ('IN', 'MI', 'WI')) and (customer_address.ca_country = 'United States')) +------------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('IA', 'IL', 'IN', 'MD', 'MI', 'MN', 'TX', 'VA', 'WI')) --------------------------PhysicalOlapScan[customer_address] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query85.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query85.out index c6f76a0e6b4e41..2e35ec10486d9c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query85.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query85.out @@ -11,38 +11,40 @@ PhysicalResultSink ----------------PhysicalProject ------------------hashJoin[INNER_JOIN] hashCondition=((reason.r_reason_sk = web_returns.wr_reason_sk)) otherCondition=() --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_page_sk = web_page.wp_web_page_sk)) otherCondition=() +----------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=((((ca_state IN ('DE', 'FL', 'TX') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('ID', 'IN', 'ND') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('IL', 'MT', 'OH') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00))))) build RFs:RF8 wr_refunded_addr_sk->[ca_address_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN] hashCondition=((cd1.cd_education_status = cd2.cd_education_status) and (cd1.cd_marital_status = cd2.cd_marital_status) and (cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk)) otherCondition=() build RFs:RF5 cd_marital_status->[cd_marital_status];RF6 cd_education_status->[cd_education_status];RF7 wr_returning_cdemo_sk->[cd_demo_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF5 RF6 RF7 -----------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('DE', 'FL', 'ID', 'IL', 'IN', 'MT', 'ND', 'OH', 'TX')) +----------------------------PhysicalOlapScan[customer_address] apply RFs: RF8 +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_page_sk = web_page.wp_web_page_sk)) otherCondition=() build RFs:RF7 ws_web_page_sk->[wp_web_page_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=((((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00))))) build RFs:RF4 wr_refunded_cdemo_sk->[cd_demo_sk] -----------------------------------PhysicalProject -------------------------------------filter(((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) OR ((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary'))) OR ((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree')))) ---------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF4 -----------------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=((((ca_state IN ('DE', 'FL', 'TX') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('ID', 'IN', 'ND') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('IL', 'MT', 'OH') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00))))) build RFs:RF3 ca_address_sk->[wr_refunded_addr_sk] +--------------------------------PhysicalOlapScan[web_page] apply RFs: RF7 +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN] hashCondition=((cd1.cd_education_status = cd2.cd_education_status) and (cd1.cd_marital_status = cd2.cd_marital_status) and (cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk)) otherCondition=() build RFs:RF4 cd_marital_status->[cd_marital_status];RF5 cd_education_status->[cd_education_status];RF6 wr_returning_cdemo_sk->[cd_demo_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF4 RF5 RF6 +------------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF1 ws_item_sk->[wr_item_sk];RF2 ws_order_number->[wr_order_number] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 RF3 -------------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=((((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00))))) build RFs:RF3 wr_refunded_cdemo_sk->[cd_demo_sk] +------------------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------------------PhysicalProject -----------------------------------------------filter((web_sales.ws_net_profit <= 300.00) and (web_sales.ws_net_profit >= 50.00) and (web_sales.ws_sales_price <= 200.00) and (web_sales.ws_sales_price >= 50.00)) -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_year = 2000)) ---------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------------PhysicalProject -------------------------------------------filter(((ca_state IN ('DE', 'FL', 'TX') OR ca_state IN ('ID', 'IN', 'ND')) OR ca_state IN ('IL', 'MT', 'OH')) and (customer_address.ca_country = 'United States')) ---------------------------------------------PhysicalOlapScan[customer_address] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_page] +----------------------------------------------filter(((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) OR ((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary'))) OR ((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree')))) +------------------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF3 +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------PhysicalProject +----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF1 ws_item_sk->[wr_item_sk];RF2 ws_order_number->[wr_order_number] +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 +------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((web_sales.ws_net_profit <= 300.00) and (web_sales.ws_net_profit >= 50.00) and (web_sales.ws_sales_price <= 200.00) and (web_sales.ws_sales_price >= 50.00)) +------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +--------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------------------PhysicalProject +------------------------------------------------------filter((date_dim.d_year = 2000)) +--------------------------------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------PhysicalProject ------------------------PhysicalOlapScan[reason] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query13.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query13.out index 5e8f8d4e9b2978..441b3d76382391 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query13.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query13.out @@ -25,7 +25,7 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------PhysicalProject ---------------------------------filter(((ca_state IN ('KS', 'MI', 'SD') OR ca_state IN ('CO', 'MO', 'ND')) OR ca_state IN ('NH', 'OH', 'TX')) and (customer_address.ca_country = 'United States')) +--------------------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('CO', 'KS', 'MI', 'MO', 'ND', 'NH', 'OH', 'SD', 'TX')) ----------------------------------PhysicalOlapScan[customer_address] --------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out index 763ac71df87090..e89101870b893d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] +----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((ca_state IN ('CA', 'GA', 'WA') OR substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ----------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query48.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query48.out index 15f1054ab46844..fc7edfee4de422 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query48.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query48.out @@ -21,7 +21,7 @@ PhysicalResultSink ------------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalDistribute[DistributionSpecHash] ----------------------PhysicalProject -------------------------filter(((ca_state IN ('IA', 'MD', 'MN') OR ca_state IN ('IL', 'TX', 'VA')) OR ca_state IN ('IN', 'MI', 'WI')) and (customer_address.ca_country = 'United States')) +------------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('IA', 'IL', 'IN', 'MD', 'MI', 'MN', 'TX', 'VA', 'WI')) --------------------------PhysicalOlapScan[customer_address] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query85.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query85.out index dc257985c6b046..77aa3c3b857dd0 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query85.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query85.out @@ -11,38 +11,40 @@ PhysicalResultSink ----------------PhysicalProject ------------------hashJoin[INNER_JOIN] hashCondition=((reason.r_reason_sk = web_returns.wr_reason_sk)) otherCondition=() build RFs:RF9 r_reason_sk->[wr_reason_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_page_sk = web_page.wp_web_page_sk)) otherCondition=() build RFs:RF8 wp_web_page_sk->[ws_web_page_sk] +----------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=((((ca_state IN ('DE', 'FL', 'TX') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('ID', 'IN', 'ND') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('IL', 'MT', 'OH') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00))))) build RFs:RF8 wr_refunded_addr_sk->[ca_address_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN] hashCondition=((cd1.cd_education_status = cd2.cd_education_status) and (cd1.cd_marital_status = cd2.cd_marital_status) and (cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk)) otherCondition=() build RFs:RF5 cd_marital_status->[cd_marital_status];RF6 cd_education_status->[cd_education_status];RF7 wr_returning_cdemo_sk->[cd_demo_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF5 RF6 RF7 -----------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('DE', 'FL', 'ID', 'IL', 'IN', 'MT', 'ND', 'OH', 'TX')) +----------------------------PhysicalOlapScan[customer_address] apply RFs: RF8 +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_page_sk = web_page.wp_web_page_sk)) otherCondition=() build RFs:RF7 ws_web_page_sk->[wp_web_page_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=((((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00))))) build RFs:RF4 wr_refunded_cdemo_sk->[cd_demo_sk] -----------------------------------PhysicalProject -------------------------------------filter(((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) OR ((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary'))) OR ((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree')))) ---------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF4 -----------------------------------PhysicalDistribute[DistributionSpecReplicated] -------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)) otherCondition=((((ca_state IN ('DE', 'FL', 'TX') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('ID', 'IN', 'ND') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('IL', 'MT', 'OH') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00))))) build RFs:RF3 ca_address_sk->[wr_refunded_addr_sk] +--------------------------------PhysicalOlapScan[web_page] apply RFs: RF7 +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN] hashCondition=((cd1.cd_education_status = cd2.cd_education_status) and (cd1.cd_marital_status = cd2.cd_marital_status) and (cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk)) otherCondition=() build RFs:RF4 cd_marital_status->[cd_marital_status];RF5 cd_education_status->[cd_education_status];RF6 wr_returning_cdemo_sk->[cd_demo_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF4 RF5 RF6 +------------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF1 ws_item_sk->[wr_item_sk];RF2 ws_order_number->[wr_order_number] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 RF3 RF9 -------------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)) otherCondition=((((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00))))) build RFs:RF3 wr_refunded_cdemo_sk->[cd_demo_sk] +------------------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------------------PhysicalProject -----------------------------------------------filter((web_sales.ws_net_profit <= 300.00) and (web_sales.ws_net_profit >= 50.00) and (web_sales.ws_sales_price <= 200.00) and (web_sales.ws_sales_price >= 50.00)) -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF8 ---------------------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_year = 2000)) ---------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------------PhysicalProject -------------------------------------------filter(((ca_state IN ('DE', 'FL', 'TX') OR ca_state IN ('ID', 'IN', 'ND')) OR ca_state IN ('IL', 'MT', 'OH')) and (customer_address.ca_country = 'United States')) ---------------------------------------------PhysicalOlapScan[customer_address] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_page] +----------------------------------------------filter(((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) OR ((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary'))) OR ((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree')))) +------------------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF3 +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------PhysicalProject +----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF1 ws_item_sk->[wr_item_sk];RF2 ws_order_number->[wr_order_number] +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 RF9 +------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((web_sales.ws_net_profit <= 300.00) and (web_sales.ws_net_profit >= 50.00) and (web_sales.ws_sales_price <= 200.00) and (web_sales.ws_sales_price >= 50.00)) +------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +--------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------------------------PhysicalProject +------------------------------------------------------filter((date_dim.d_year = 2000)) +--------------------------------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------PhysicalProject ------------------------PhysicalOlapScan[reason] From 4c2fb951f6bb4af49fe611f9ac87a019b75c3f5c Mon Sep 17 00:00:00 2001 From: yiguolei <676222867@qq.com> Date: Fri, 19 Jan 2024 15:23:27 +0800 Subject: [PATCH 098/200] [refactor](close) refactor ispendingfinish logic and close logic to do close more quickly (#30021) --- be/src/exec/data_sink.h | 2 +- .../pipeline/exec/jdbc_table_sink_operator.h | 1 + .../pipeline/exec/olap_table_sink_operator.h | 1 + .../exec/olap_table_sink_v2_operator.h | 1 + be/src/pipeline/exec/operator.h | 2 +- .../exec/partition_sort_sink_operator.cpp | 2 -- .../exec/result_file_sink_operator.cpp | 6 ++-- .../pipeline/exec/result_file_sink_operator.h | 1 + be/src/pipeline/exec/sort_sink_operator.cpp | 1 - .../pipeline/exec/table_function_operator.cpp | 1 - be/src/pipeline/pipeline_x/operator.cpp | 14 ++++------ be/src/pipeline/pipeline_x/operator.h | 5 ++++ be/src/runtime/result_writer.h | 2 ++ be/src/runtime/runtime_state.cpp | 18 ------------ be/src/runtime/runtime_state.h | 5 ---- be/src/vec/exec/vpartition_sort_node.cpp | 3 -- be/src/vec/exec/vsort_node.cpp | 2 -- be/src/vec/exec/vtable_function_node.cpp | 1 - be/src/vec/sink/async_writer_sink.h | 23 +++++++++------ be/src/vec/sink/vresult_file_sink.cpp | 19 +++++++++---- .../vec/sink/writer/async_result_writer.cpp | 23 +++++++++------ be/src/vec/sink/writer/async_result_writer.h | 7 ----- be/src/vec/sink/writer/vjdbc_table_writer.h | 6 ++-- be/src/vec/sink/writer/vodbc_table_writer.h | 6 ++-- be/src/vec/sink/writer/vtablet_writer.cpp | 28 +++---------------- be/src/vec/sink/writer/vtablet_writer.h | 8 ++---- 26 files changed, 75 insertions(+), 113 deletions(-) diff --git a/be/src/exec/data_sink.h b/be/src/exec/data_sink.h index 3bf72ae545071a..be6cfe236b7b1c 100644 --- a/be/src/exec/data_sink.h +++ b/be/src/exec/data_sink.h @@ -74,7 +74,7 @@ class DataSink { return Status::OK(); } - virtual bool is_close_done() { return true; } + [[nodiscard]] virtual bool is_pending_finish() const { return false; } // Releases all resources that were allocated in prepare()/send(). // Further send() calls are illegal after calling close(). diff --git a/be/src/pipeline/exec/jdbc_table_sink_operator.h b/be/src/pipeline/exec/jdbc_table_sink_operator.h index 41348ccaccf25f..33018f69da5540 100644 --- a/be/src/pipeline/exec/jdbc_table_sink_operator.h +++ b/be/src/pipeline/exec/jdbc_table_sink_operator.h @@ -58,6 +58,7 @@ class JdbcTableSinkOperatorX final : public DataSinkOperatorX + requires(std::is_base_of_v) friend class AsyncWriterSink; const RowDescriptor& _row_desc; diff --git a/be/src/pipeline/exec/olap_table_sink_operator.h b/be/src/pipeline/exec/olap_table_sink_operator.h index 6aa80776a3aab0..6707ddd86e66dd 100644 --- a/be/src/pipeline/exec/olap_table_sink_operator.h +++ b/be/src/pipeline/exec/olap_table_sink_operator.h @@ -102,6 +102,7 @@ class OlapTableSinkOperatorX final : public DataSinkOperatorX + requires(std::is_base_of_v) friend class AsyncWriterSink; const RowDescriptor& _row_desc; vectorized::VExprContextSPtrs _output_vexpr_ctxs; diff --git a/be/src/pipeline/exec/olap_table_sink_v2_operator.h b/be/src/pipeline/exec/olap_table_sink_v2_operator.h index 1e68f9e98d3b21..f1b5d6e9061bfc 100644 --- a/be/src/pipeline/exec/olap_table_sink_v2_operator.h +++ b/be/src/pipeline/exec/olap_table_sink_v2_operator.h @@ -104,6 +104,7 @@ class OlapTableSinkV2OperatorX final : public DataSinkOperatorX + requires(std::is_base_of_v) friend class AsyncWriterSink; const RowDescriptor& _row_desc; vectorized::VExprContextSPtrs _output_vexpr_ctxs; diff --git a/be/src/pipeline/exec/operator.h b/be/src/pipeline/exec/operator.h index 80184374b77eaf..cd5fba5fee3f9d 100644 --- a/be/src/pipeline/exec/operator.h +++ b/be/src/pipeline/exec/operator.h @@ -297,7 +297,7 @@ class DataSinkOperator : public OperatorBase { return _sink->try_close(state, state->query_status()); } - [[nodiscard]] bool is_pending_finish() const override { return !_sink->is_close_done(); } + [[nodiscard]] bool is_pending_finish() const override { return _sink->is_pending_finish(); } Status close(RuntimeState* state) override { if (is_closed()) { diff --git a/be/src/pipeline/exec/partition_sort_sink_operator.cpp b/be/src/pipeline/exec/partition_sort_sink_operator.cpp index 14d917d62e3ab3..c09a6a90b95b0d 100644 --- a/be/src/pipeline/exec/partition_sort_sink_operator.cpp +++ b/be/src/pipeline/exec/partition_sort_sink_operator.cpp @@ -120,8 +120,6 @@ Status PartitionSortSinkOperatorX::sink(RuntimeState* state, vectorized::Block* RETURN_IF_ERROR( _split_block_by_partition(input_block, state->batch_size(), local_state)); RETURN_IF_CANCELLED(state); - RETURN_IF_ERROR( - state->check_query_state("VPartitionSortNode, while split input block.")); input_block->clear_column_data(); } } diff --git a/be/src/pipeline/exec/result_file_sink_operator.cpp b/be/src/pipeline/exec/result_file_sink_operator.cpp index 2b095748b15e81..71a2d539c95842 100644 --- a/be/src/pipeline/exec/result_file_sink_operator.cpp +++ b/be/src/pipeline/exec/result_file_sink_operator.cpp @@ -173,9 +173,9 @@ Status ResultFileSinkLocalState::close(RuntimeState* state, Status exec_status) } Status final_status = exec_status; - // close the writer - if (_writer && _writer->need_normal_close()) { - Status st = _writer->close(); + // For pipelinex engine, the writer is closed in async thread process_block + if (_writer) { + Status st = _writer->get_writer_status(); if (!st.ok() && exec_status.ok()) { // close file writer failed, should return this error to client final_status = st; diff --git a/be/src/pipeline/exec/result_file_sink_operator.h b/be/src/pipeline/exec/result_file_sink_operator.h index 57e1e8c9147acd..e196401991a981 100644 --- a/be/src/pipeline/exec/result_file_sink_operator.h +++ b/be/src/pipeline/exec/result_file_sink_operator.h @@ -106,6 +106,7 @@ class ResultFileSinkOperatorX final : public DataSinkOperatorX + requires(std::is_base_of_v) friend class AsyncWriterSink; const RowDescriptor& _row_desc; diff --git a/be/src/pipeline/exec/sort_sink_operator.cpp b/be/src/pipeline/exec/sort_sink_operator.cpp index e2c851f758f1f9..56a81422484bc9 100644 --- a/be/src/pipeline/exec/sort_sink_operator.cpp +++ b/be/src/pipeline/exec/sort_sink_operator.cpp @@ -150,7 +150,6 @@ Status SortSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block* in if (in_block->rows() > 0) { RETURN_IF_ERROR(local_state._shared_state->sorter->append_block(in_block)); RETURN_IF_CANCELLED(state); - RETURN_IF_ERROR(state->check_query_state("vsort, while sorting input.")); // update runtime predicate if (_use_topn_opt) { diff --git a/be/src/pipeline/exec/table_function_operator.cpp b/be/src/pipeline/exec/table_function_operator.cpp index cf14b3ec5e3c1c..960fccff2b4604 100644 --- a/be/src/pipeline/exec/table_function_operator.cpp +++ b/be/src/pipeline/exec/table_function_operator.cpp @@ -159,7 +159,6 @@ Status TableFunctionLocalState::get_expanded_block(RuntimeState* state, while (columns[p._child_slots.size()]->size() < state->batch_size()) { RETURN_IF_CANCELLED(state); - RETURN_IF_ERROR(state->check_query_state("VTableFunctionNode, while getting next batch.")); if (_child_block->rows() == 0) { break; diff --git a/be/src/pipeline/pipeline_x/operator.cpp b/be/src/pipeline/pipeline_x/operator.cpp index e38e7b39d95b1b..bbb7473f86878b 100644 --- a/be/src/pipeline/pipeline_x/operator.cpp +++ b/be/src/pipeline/pipeline_x/operator.cpp @@ -497,6 +497,7 @@ Status StatefulOperatorX::get_block(RuntimeState* state, vectori } template + requires(std::is_base_of_v) Status AsyncWriterSink::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); _output_vexpr_ctxs.resize(_parent->cast()._output_vexpr_ctxs.size()); @@ -516,6 +517,7 @@ Status AsyncWriterSink::init(RuntimeState* state, LocalSinkState } template + requires(std::is_base_of_v) Status AsyncWriterSink::open(RuntimeState* state) { RETURN_IF_ERROR(Base::open(state)); _writer->start_writer(state, _profile); @@ -523,12 +525,14 @@ Status AsyncWriterSink::open(RuntimeState* state) { } template + requires(std::is_base_of_v) Status AsyncWriterSink::sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) { return _writer->sink(block, source_state == SourceState::FINISHED); } template + requires(std::is_base_of_v) Status AsyncWriterSink::close(RuntimeState* state, Status exec_status) { if (_closed) { return Status::OK(); @@ -537,19 +541,13 @@ Status AsyncWriterSink::close(RuntimeState* state, Status exec_s COUNTER_SET(_wait_for_finish_dependency_timer, _finish_dependency->watcher_elapse_time()); // if the init failed, the _writer may be nullptr. so here need check if (_writer) { - if (_writer->need_normal_close()) { - if (exec_status.ok() && !state->is_cancelled()) { - RETURN_IF_ERROR(_writer->commit_trans()); - } - RETURN_IF_ERROR(_writer->close(exec_status)); - } else { - RETURN_IF_ERROR(_writer->get_writer_status()); - } + RETURN_IF_ERROR(_writer->get_writer_status()); } return Base::close(state, exec_status); } template + requires(std::is_base_of_v) Status AsyncWriterSink::try_close(RuntimeState* state, Status exec_status) { if (state->is_cancelled() || !exec_status.ok()) { _writer->force_close(!exec_status.ok() ? exec_status : Status::Cancelled("Cancelled")); diff --git a/be/src/pipeline/pipeline_x/operator.h b/be/src/pipeline/pipeline_x/operator.h index d46dc859b0c162..ca1b224c5c525e 100644 --- a/be/src/pipeline/pipeline_x/operator.h +++ b/be/src/pipeline/pipeline_x/operator.h @@ -22,6 +22,9 @@ #include "pipeline/pipeline_x/dependency.h" #include "pipeline/pipeline_x/local_exchange/local_exchanger.h" +namespace doris::vectorized { +class AsyncResultWriter; +} namespace doris::pipeline { struct LocalExchangeSinkDependency; @@ -569,6 +572,7 @@ class DataSinkOperatorXBase : public OperatorBase { protected: template + requires(std::is_base_of_v) friend class AsyncWriterSink; // _operator_id : the current Operator's ID, which is not visible to the user. // _node_id : the plan node ID corresponding to the Operator, which is visible on the profile. @@ -680,6 +684,7 @@ class StatefulOperatorX : public OperatorX { }; template + requires(std::is_base_of_v) class AsyncWriterSink : public PipelineXSinkLocalState { public: using Base = PipelineXSinkLocalState; diff --git a/be/src/runtime/result_writer.h b/be/src/runtime/result_writer.h index f65f06399b86cf..5c49954ef2f9e5 100644 --- a/be/src/runtime/result_writer.h +++ b/be/src/runtime/result_writer.h @@ -38,6 +38,8 @@ class ResultWriter { virtual Status init(RuntimeState* state) = 0; + virtual Status finish(RuntimeState* state) { return Status::OK(); } + virtual Status close(Status s = Status::OK()) = 0; [[nodiscard]] virtual int64_t get_written_rows() const { return _written_rows; } diff --git a/be/src/runtime/runtime_state.cpp b/be/src/runtime/runtime_state.cpp index 1a6f8a2661ffd1..d84b86f229084d 100644 --- a/be/src/runtime/runtime_state.cpp +++ b/be/src/runtime/runtime_state.cpp @@ -365,24 +365,6 @@ Status RuntimeState::set_mem_limit_exceeded(const std::string& msg) { return _process_status; } -Status RuntimeState::check_query_state(const std::string& msg) { - // TODO: it would be nice if this also checked for cancellation, but doing so breaks - // cases where we use Status::Cancelled("Cancelled") to indicate that the limit was reached. - // - // If the thread MemTrackerLimiter exceeds the limit, an error status is returned. - // Usually used after SCOPED_ATTACH_TASK, during query execution. - if (is_thread_context_init() && thread_context()->thread_mem_tracker()->limit_exceeded() && - !config::enable_query_memory_overcommit) { - auto failed_msg = - fmt::format("{}, {}", msg, - thread_context()->thread_mem_tracker()->tracker_limit_exceeded_str()); - thread_context()->thread_mem_tracker()->print_log_usage(failed_msg); - log_error(failed_msg); - return Status::MemoryLimitExceeded(failed_msg); - } - return query_status(); -} - const int64_t MAX_ERROR_NUM = 50; Status RuntimeState::create_error_log_file() { diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index ede451cd92aa88..91443ef9492f3f 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -236,11 +236,6 @@ class RuntimeState { // generic "Memory limit exceeded" error. Status set_mem_limit_exceeded(const std::string& msg = "Memory limit exceeded"); - // Returns a non-OK status if query execution should stop (e.g., the query was cancelled - // or a mem limit was exceeded). Exec nodes should check this periodically so execution - // doesn't continue if the query terminates abnormally. - Status check_query_state(const std::string& msg); - std::vector& output_files() { return _output_files; } void set_import_label(const std::string& import_label) { _import_label = import_label; } diff --git a/be/src/vec/exec/vpartition_sort_node.cpp b/be/src/vec/exec/vpartition_sort_node.cpp index 6a7ebcffd1e5b2..95c0abd72a8288 100644 --- a/be/src/vec/exec/vpartition_sort_node.cpp +++ b/be/src/vec/exec/vpartition_sort_node.cpp @@ -168,8 +168,6 @@ Status VPartitionSortNode::sink(RuntimeState* state, vectorized::Block* input_bl } else { RETURN_IF_ERROR(_split_block_by_partition(input_block, state->batch_size())); RETURN_IF_CANCELLED(state); - RETURN_IF_ERROR( - state->check_query_state("VPartitionSortNode, while split input block.")); input_block->clear_column_data(); } } @@ -237,7 +235,6 @@ Status VPartitionSortNode::alloc_resource(RuntimeState* state) { RETURN_IF_ERROR(VExpr::open(_partition_expr_ctxs, state)); RETURN_IF_ERROR(_vsort_exec_exprs.open(state)); RETURN_IF_CANCELLED(state); - RETURN_IF_ERROR(state->check_query_state("VPartitionSortNode, while open.")); return Status::OK(); } diff --git a/be/src/vec/exec/vsort_node.cpp b/be/src/vec/exec/vsort_node.cpp index 189c628ef0be3b..e313e3f74aceee 100644 --- a/be/src/vec/exec/vsort_node.cpp +++ b/be/src/vec/exec/vsort_node.cpp @@ -130,7 +130,6 @@ Status VSortNode::alloc_resource(doris::RuntimeState* state) { RETURN_IF_ERROR(ExecNode::alloc_resource(state)); RETURN_IF_ERROR(_vsort_exec_exprs.open(state)); RETURN_IF_CANCELLED(state); - RETURN_IF_ERROR(state->check_query_state("vsort, while open.")); return Status::OK(); } @@ -140,7 +139,6 @@ Status VSortNode::sink(RuntimeState* state, vectorized::Block* input_block, bool if (input_block->rows() > 0) { RETURN_IF_ERROR(_sorter->append_block(input_block)); RETURN_IF_CANCELLED(state); - RETURN_IF_ERROR(state->check_query_state("vsort, while sorting input.")); // update runtime predicate if (_use_topn_opt) { diff --git a/be/src/vec/exec/vtable_function_node.cpp b/be/src/vec/exec/vtable_function_node.cpp index 8affd4cbe7e265..0c35fae806ea93 100644 --- a/be/src/vec/exec/vtable_function_node.cpp +++ b/be/src/vec/exec/vtable_function_node.cpp @@ -170,7 +170,6 @@ Status VTableFunctionNode::_get_expanded_block(RuntimeState* state, Block* outpu while (columns[_child_slots.size()]->size() < state->batch_size()) { RETURN_IF_CANCELLED(state); - RETURN_IF_ERROR(state->check_query_state("VTableFunctionNode, while getting next batch.")); if (_child_block->rows() == 0) { break; diff --git a/be/src/vec/sink/async_writer_sink.h b/be/src/vec/sink/async_writer_sink.h index 600eb60928150e..1260382d6fa3a7 100644 --- a/be/src/vec/sink/async_writer_sink.h +++ b/be/src/vec/sink/async_writer_sink.h @@ -42,6 +42,7 @@ namespace vectorized { class Block; template + requires(std::is_base_of_v) class AsyncWriterSink : public DataSink { public: AsyncWriterSink(const RowDescriptor& row_desc, const std::vector& t_exprs) @@ -78,13 +79,21 @@ class AsyncWriterSink : public DataSink { return Status::OK(); } + // Non-pipeline engine will call this api to send data to sink destination Status send(RuntimeState* state, vectorized::Block* block, bool eos = false) override { SCOPED_TIMER(_exec_timer); COUNTER_UPDATE(_blocks_sent_counter, 1); COUNTER_UPDATE(_output_rows_counter, block->rows()); - return _writer->write(*block); + Status st = _writer->write(*block); + // Should also check !state->is_cancelled()???, do not know which scenario? + if (st.ok() && eos) { + // If this is the last block, then call finish to flush the buffer or commit transctions. + st = _writer->finish(state); + } + return st; } + // Pipeline engine will call this api to send data to destination. This is an async API. Status sink(RuntimeState* state, vectorized::Block* block, bool eos = false) override { return _writer->sink(block, eos); } @@ -94,13 +103,11 @@ class AsyncWriterSink : public DataSink { Status close(RuntimeState* state, Status exec_status) override { // if the init failed, the _writer may be nullptr. so here need check if (_writer) { - if (_writer->need_normal_close()) { - if (exec_status.ok() && !state->is_cancelled()) { - RETURN_IF_ERROR(_writer->commit_trans()); - } - RETURN_IF_ERROR(_writer->close(exec_status)); - } else { + // For pipeline engine, the writer is always closed in async thread process_block + if (state->enable_pipeline_exec()) { RETURN_IF_ERROR(_writer->get_writer_status()); + } else { + RETURN_IF_ERROR(_writer->close(exec_status)); } } return DataSink::close(state, exec_status); @@ -113,7 +120,7 @@ class AsyncWriterSink : public DataSink { return Status::OK(); } - bool is_close_done() override { return !_writer->is_pending_finish(); } + [[nodiscard]] bool is_pending_finish() const override { return _writer->is_pending_finish(); } protected: const std::vector& _t_output_expr; diff --git a/be/src/vec/sink/vresult_file_sink.cpp b/be/src/vec/sink/vresult_file_sink.cpp index 02d77fa6d42924..08dd881bf4bc87 100644 --- a/be/src/vec/sink/vresult_file_sink.cpp +++ b/be/src/vec/sink/vresult_file_sink.cpp @@ -113,14 +113,21 @@ Status VResultFileSink::close(RuntimeState* state, Status exec_status) { } Status final_status = exec_status; - // close the writer - if (_writer && _writer->need_normal_close()) { - Status st = _writer->close(); - if (!st.ok() && exec_status.ok()) { - // close file writer failed, should return this error to client - final_status = st; + Status writer_st = Status::OK(); + if (_writer) { + // For pipeline engine, the writer is always closed in async thread process_block + if (state->enable_pipeline_exec()) { + writer_st = _writer->get_writer_status(); + } else { + writer_st = _writer->close(exec_status); } } + + if (!writer_st.ok() && exec_status.ok()) { + // close file writer failed, should return this error to client + final_status = writer_st; + } + if (_is_top_sink) { // close sender, this is normal path end if (_sender) { diff --git a/be/src/vec/sink/writer/async_result_writer.cpp b/be/src/vec/sink/writer/async_result_writer.cpp index 0ce3196e3a3cad..35d94fe3c473a2 100644 --- a/be/src/vec/sink/writer/async_result_writer.cpp +++ b/be/src/vec/sink/writer/async_result_writer.cpp @@ -125,15 +125,22 @@ void AsyncResultWriter::process_block(RuntimeState* state, RuntimeProfile* profi } } - // if not in transaction or status is in error or force close we can do close in - // async IO thread - if (!_writer_status.ok() || !in_transaction()) { - std::lock_guard l(_m); - // Using lock to make sure the writer status is not modified - // There is a unique ptr err_msg in Status, if it is modified, the unique ptr - // maybe released. And it will core because use after free. + // If the last block is sent successfuly, then call finish to clear the buffer or commit + // transactions. + // Using lock to make sure the writer status is not modified + // There is a unique ptr err_msg in Status, if it is modified, the unique ptr + // maybe released. And it will core because use after free. + std::lock_guard l(_m); + if (_writer_status.ok() && _eos) { + _writer_status = finish(state); + } + + if (_writer_status.ok()) { _writer_status = close(_writer_status); - _need_normal_close = false; + } else { + // If it is already failed before, then not update the write status so that we could get + // the real reason. + static_cast(close(_writer_status)); } _writer_thread_closed = true; if (_finish_dependency) { diff --git a/be/src/vec/sink/writer/async_result_writer.h b/be/src/vec/sink/writer/async_result_writer.h index ced123267c5292..5fbcca98af3f06 100644 --- a/be/src/vec/sink/writer/async_result_writer.h +++ b/be/src/vec/sink/writer/async_result_writer.h @@ -61,12 +61,6 @@ class AsyncResultWriter : public ResultWriter { void force_close(Status s); - virtual bool in_transaction() { return false; } - - virtual Status commit_trans() { return Status::OK(); } - - bool need_normal_close() const { return _need_normal_close; } - Status init(RuntimeState* state) override { return Status::OK(); } virtual Status open(RuntimeState* state, RuntimeProfile* profile) = 0; @@ -110,7 +104,6 @@ class AsyncResultWriter : public ResultWriter { std::deque> _data_queue; Status _writer_status = Status::OK(); bool _eos = false; - bool _need_normal_close = true; bool _writer_thread_closed = false; // Used by pipelineX diff --git a/be/src/vec/sink/writer/vjdbc_table_writer.h b/be/src/vec/sink/writer/vjdbc_table_writer.h index 735c023fce57ed..a683259c992485 100644 --- a/be/src/vec/sink/writer/vjdbc_table_writer.h +++ b/be/src/vec/sink/writer/vjdbc_table_writer.h @@ -46,11 +46,9 @@ class VJdbcTableWriter final : public AsyncResultWriter, public JdbcConnector { Status write(vectorized::Block& block) override; - Status close(Status s) override { return JdbcConnector::close(s); } - - bool in_transaction() override { return TableConnector::_is_in_transaction; } + Status finish(RuntimeState* state) override { return JdbcConnector::finish_trans(); } - Status commit_trans() override { return JdbcConnector::finish_trans(); } + Status close(Status s) override { return JdbcConnector::close(s); } private: JdbcConnectorParam _param; diff --git a/be/src/vec/sink/writer/vodbc_table_writer.h b/be/src/vec/sink/writer/vodbc_table_writer.h index a28947355e7ca6..687b5106a8babf 100644 --- a/be/src/vec/sink/writer/vodbc_table_writer.h +++ b/be/src/vec/sink/writer/vodbc_table_writer.h @@ -46,11 +46,9 @@ class VOdbcTableWriter final : public AsyncResultWriter, public ODBCConnector { Status write(vectorized::Block& block) override; - Status close(Status s) override { return ODBCConnector::close(s); } - - bool in_transaction() override { return TableConnector::_is_in_transaction; } + Status finish(RuntimeState* state) override { return ODBCConnector::finish_trans(); } - Status commit_trans() override { return ODBCConnector::finish_trans(); } + Status close(Status s) override { return ODBCConnector::close(s); } }; } // namespace vectorized } // namespace doris \ No newline at end of file diff --git a/be/src/vec/sink/writer/vtablet_writer.cpp b/be/src/vec/sink/writer/vtablet_writer.cpp index 7c51b80a0156ea..7af7b115c43aec 100644 --- a/be/src/vec/sink/writer/vtablet_writer.cpp +++ b/be/src/vec/sink/writer/vtablet_writer.cpp @@ -865,10 +865,6 @@ void VNodeChannel::cancel(const std::string& cancel_msg) { static_cast(request->release_id()); } -bool VNodeChannel::is_send_data_rpc_done() const { - return _add_batches_finished || _cancelled; -} - Status VNodeChannel::close_wait(RuntimeState* state) { DBUG_EXECUTE_IF("VNodeChannel.close_wait_full_gc", { MemInfo::process_full_gc(); }); SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker.get()); @@ -891,7 +887,8 @@ Status VNodeChannel::close_wait(RuntimeState* state) { } // waiting for finished, it may take a long time, so we couldn't set a timeout - // In pipeline, is_close_done() is false at this time, will not block. + // For pipeline engine, the close is called in async writer's process block method, + // so that it will not block pipeline thread. while (!_add_batches_finished && !_cancelled && !state->is_cancelled()) { bthread_usleep(1000); } @@ -1358,7 +1355,7 @@ Status VTabletWriter::_send_new_partition_batch() { return Status::OK(); } -Status VTabletWriter::try_close(RuntimeState* state, Status exec_status) { +void VTabletWriter::_do_try_close(RuntimeState* state, const Status& exec_status) { SCOPED_TIMER(_close_timer); Status status = exec_status; @@ -1400,23 +1397,6 @@ Status VTabletWriter::try_close(RuntimeState* state, Status exec_status) { _close_status = status; _close_wait = true; } - - return Status::OK(); -} - -bool VTabletWriter::is_close_done() { - // Only after try_close, need to wait rpc end. - if (!_close_wait) { - return true; - } - bool close_done = true; - for (const auto& index_channel : _channels) { - index_channel->for_each_node_channel( - [&close_done](const std::shared_ptr& ch) { - close_done &= ch->is_send_data_rpc_done(); - }); - } - return close_done; } Status VTabletWriter::close(Status exec_status) { @@ -1431,7 +1411,7 @@ Status VTabletWriter::close(Status exec_status) { SCOPED_TIMER(_profile->total_time_counter()); // will make the last batch of request-> close_wait will wait this finished. - static_cast(try_close(_state, exec_status)); + _do_try_close(_state, exec_status); // If _close_status is not ok, all nodes have been canceled in try_close. if (_close_status.ok()) { diff --git a/be/src/vec/sink/writer/vtablet_writer.h b/be/src/vec/sink/writer/vtablet_writer.h index 58ddbfd8869e97..cd2eafb1f24b19 100644 --- a/be/src/vec/sink/writer/vtablet_writer.h +++ b/be/src/vec/sink/writer/vtablet_writer.h @@ -259,8 +259,6 @@ class VNodeChannel { // 2. just cancel() void mark_close(); - bool is_send_data_rpc_done() const; - bool is_closed() const { return _is_closed; } bool is_cancelled() const { return _cancelled; } std::string get_cancel_msg() { @@ -527,15 +525,11 @@ class VTabletWriter final : public AsyncResultWriter { Status open(RuntimeState* state, RuntimeProfile* profile) override; - Status try_close(RuntimeState* state, Status exec_status); - // the consumer func of sending pending batches in every NodeChannel. // use polling & NodeChannel::try_send_and_fetch_status() to achieve nonblocking sending. // only focus on pending batches and channel status, the internal errors of NodeChannels will be handled by the producer void _send_batch_process(); - bool is_close_done(); - Status on_partitions_created(TCreatePartitionResult* result); Status _send_new_partition_batch(); @@ -562,6 +556,8 @@ class VTabletWriter final : public AsyncResultWriter { Status _incremental_open_node_channel(const std::vector& partitions); + void _do_try_close(RuntimeState* state, const Status& exec_status); + TDataSink _t_sink; std::shared_ptr _mem_tracker; From a261c2788e03472e4ee8f46831ad4e2f4762559a Mon Sep 17 00:00:00 2001 From: Xiangyu Wang Date: Fri, 19 Jan 2024 15:24:12 +0800 Subject: [PATCH 099/200] [enhance-wip](multi-catalog) Speed up consume rate of hms events. (#27666) ## Proposed changes The current implement will persist all catalogs/databases of external catalogs, and only the master FE can handle hms events and make all slave nodes replay these events, this will bring some problems: - The hms event processor ( `MetastoreEventsProcessor` ) can not consume the events in time. (Add journal log is a synchronized method, we can not speed up the consume rate by using concurrent processing, and each add-journal-log operation costs about tens of milliseconds) So the meta info of hive maybe out of date. - Slave FE nodes maybe crashed if FE replays the journal logs of hms events failed. (In fact we have fixed some issues about this, but we can not make sure all the issues have been resolved) - There are many journal logs which are produced by hms events, but in fact these logs are not used anymore after FE restart. It makes the start time of all FE nodes very long. Now doris try to persis all databases/tables of external catalogs just to make sure that the dbId/tableId of databases/tables are the same through all FE nodes, it will be used by analysis jobs. In this pr, we use a meta id manager called `ExternalMetaIdMgr` to manage these meta ids. On every loop when master fetches a batch of hms events, it handles the meta ids first and produce only one meta id mappings log, slave FE nodes will replay this log to sync the changes about these meta ids. `MetastoreEventsProcessor` will start on every FE nodes and try to consume these hms events as soon as possible. ## Further comments I've submitted two prs ( #22869 #21589 ) to speed up the consume rate of hms events before, it works fine when there are many `AlterTableEvent` / `DropTableEvent` on hive cluster. But the improvement is not that significant when most of hms events are partition-events. Unfortunately, we performed a cluster upgrade (upgrade spark 2.x to spark 3.x), maybe this is the reason that resulting in the majority of Hive Metastore events became partition-events. This is also the reason for the existence of this pull request. Based on our observation, after merging this pull request, Doris is now capable of processing thousands of Hive Metastore events per second, compared to the previous capability of handling only a few dozen events. ```java 2023-12-07 05:17:03,518 INFO (replayer|105) [Env.replayJournal():2614] replayed journal id is 18287902, replay to journal id is 18287903 2023-12-07 05:17:03,735 INFO (org.apache.doris.datasource.hive.event.MetastoreEventsProcessor|37) [MetastoreEventFactory.mergeEvents():188] Event size on catalog [xxx] before merge is [1947], after merge is [1849] 2023-12-07 05:17:03,735 INFO (org.apache.doris.datasource.hive.event.MetastoreEventsProcessor|37) [MetastoreEvent.infoLog():193] EventId: 357955309 EventType: ALTER_PARTITION catalogName:[xxx],dbName:[xxx],tableName:[xxx],partitionNameBefore:[partitions=2022-05-27],partitionNameAfter:[partitions=2022-05-27] 2023-12-07 05:17:03,735 INFO (org.apache.doris.datasource.hive.event.MetastoreEventsProcessor|37) [MetastoreEvent.infoLog():193] EventId: 357955310 EventType: ALTER_PARTITION catalogName:[xxx],dbName:[xxx],tableName:[xxx],partitionNameBefore:[pday=20230318],partitionNameAfter:[pday=20230318] 2023-12-07 05:17:03,735 INFO (org.apache.doris.datasource.hive.event.MetastoreEventsProcessor|37) [MetastoreEvent.infoLog():193] EventId: 357955311 EventType: ALTER_PARTITION catalogName:[xxx],dbName:[xxx],tableName:[xxx],partitionNameBefore:[pday=20190826],partitionNameAfter:[pday=20190826] 2023-12-07 05:17:03,735 INFO (org.apache.doris.datasource.hive.event.MetastoreEventsProcessor|37) [MetastoreEvent.infoLog():193] EventId: 357955312 EventType: ALTER_PARTITION catalogName:[xxx],dbName:[xxx],tableName:[xxx],partitionNameBefore:[partitions=2021-09-16],partitionNameAfter:[partitions=2021-09-16] 2023-12-07 05:17:03,735 INFO (org.apache.doris.datasource.hive.event.MetastoreEventsProcessor|37) [MetastoreEvent.infoLog():193] EventId: 357955314 EventType: ALTER_PARTITION catalogName:[xxx],dbName:[xxx],tableName:[xxx],partitionNameBefore:[partitions=2020-04-26],partitionNameAfter:[partitions=2020-04-26] 2023-12-07 05:17:03,735 INFO (org.apache.doris.datasource.hive.event.MetastoreEventsProcessor|37) [MetastoreEvent.infoLog():193] EventId: 357955315 EventType: ALTER_PARTITION catalogName:[xxx],dbName:[xxx],tableName:[xxx],partitionNameBefore:[pday=20230702],partitionNameAfter:[pday=20230702] 2023-12-07 05:17:03,735 INFO (org.apache.doris.datasource.hive.event.MetastoreEventsProcessor|37) [MetastoreEvent.infoLog():193] EventId: 357955317 EventType: ALTER_PARTITION catalogName:[xxx],dbName:[xxx],tableName:[xxx],partitionNameBefore:[pday=20211019],partitionNameAfter:[pday=20211019] ... 2023-12-07 05:17:03,989 INFO (org.apache.doris.datasource.hive.event.MetastoreEventsProcessor|37) [MetastoreEvent.infoLog():193] EventId: 357957252 EventType: ALTER_PARTITION catalogName:[xxx],dbName:[xxx],tableName:[xxx],partitionNameBefore:[partitions=2021-08-27],partitionNameAfter:[partitions=2021-08-27] 2023-12-07 05:17:03,989 INFO (org.apache.doris.datasource.hive.event.MetastoreEventsProcessor|37) [MetastoreEvent.infoLog():193] EventId: 357957253 EventType: ALTER_PARTITION catalogName:[xxx],dbName:[xxx],tableName:[xxx],partitionNameBefore:[partitions=2022-02-05],partitionNameAfter:[partitions=2022-02-05] 2023-12-07 05:17:04,661 INFO (replayer|105) [Env.replayJournal():2614] replayed journal id is 18287903, replay to journal id is 18287904 2023-12-07 05:17:05,028 INFO (org.apache.doris.datasource.hive.event.MetastoreEventsProcessor|37) [MetastoreEventsProcessor.realRun():116] Events size are 587 on catalog [xxx] 2023-12-07 05:17:05,662 INFO (org.apache.doris.datasource.hive.event.MetastoreEventsProcessor|37) [MetastoreEventFactory.mergeEvents():188] Event size on catalog [xxx] before merge is [587], after merge is [587] ``` --- .../java/org/apache/doris/catalog/Env.java | 17 +- .../catalog/external/ExternalDatabase.java | 20 +- .../catalog/external/HMSExternalDatabase.java | 13 +- .../external/IcebergExternalDatabase.java | 4 +- .../external/PaimonExternalDatabase.java | 4 +- .../apache/doris/datasource/CatalogMgr.java | 257 ++-------------- .../doris/datasource/ExternalCatalog.java | 14 +- .../doris/datasource/ExternalMetaIdMgr.java | 263 +++++++++++++++++ .../doris/datasource/HMSExternalCatalog.java | 107 ++----- .../apache/doris/datasource/InitTableLog.java | 67 ----- .../doris/datasource/MetaIdMappingsLog.java | 274 ++++++++++++++++++ .../hive/event/AddPartitionEvent.java | 16 + .../hive/event/AlterTableEvent.java | 6 +- .../hive/event/CreateDatabaseEvent.java | 12 + .../hive/event/CreateTableEvent.java | 14 +- .../hive/event/DropDatabaseEvent.java | 11 + .../hive/event/DropPartitionEvent.java | 17 +- .../datasource/hive/event/DropTableEvent.java | 15 +- .../datasource/hive/event/MetastoreEvent.java | 9 + .../hive/event/MetastoreEventFactory.java | 26 +- .../hive/event/MetastoreEventsProcessor.java | 225 +++++++++++--- .../apache/doris/journal/JournalEntity.java | 20 +- .../org/apache/doris/persist/EditLog.java | 29 +- .../apache/doris/persist/OperationType.java | 9 + .../datasource/ExternalMetaIdMgrTest.java | 76 +++++ .../datasource/MetaIdMappingsLogTest.java | 97 +++++++ .../hms/MetastoreEventFactoryTest.java | 2 +- 27 files changed, 1138 insertions(+), 486 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalMetaIdMgr.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/InitTableLog.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/MetaIdMappingsLog.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/ExternalMetaIdMgrTest.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/MetaIdMappingsLogTest.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index 222eb195b3480c..37930708cc8cbf 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -132,6 +132,7 @@ import org.apache.doris.datasource.CatalogMgr; import org.apache.doris.datasource.EsExternalCatalog; import org.apache.doris.datasource.ExternalMetaCacheMgr; +import org.apache.doris.datasource.ExternalMetaIdMgr; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.datasource.hive.HiveTransactionMgr; import org.apache.doris.datasource.hive.event.MetastoreEventsProcessor; @@ -362,6 +363,7 @@ public class Env { private DbUsedDataQuotaInfoCollector dbUsedDataQuotaInfoCollector; private PartitionInMemoryInfoCollector partitionInMemoryInfoCollector; private CooldownConfHandler cooldownConfHandler; + private ExternalMetaIdMgr externalMetaIdMgr; private MetastoreEventsProcessor metastoreEventsProcessor; private ExportTaskRegister exportTaskRegister; @@ -649,6 +651,7 @@ public Env(boolean isCheckpointCatalog) { if (Config.enable_storage_policy) { this.cooldownConfHandler = new CooldownConfHandler(); } + this.externalMetaIdMgr = new ExternalMetaIdMgr(); this.metastoreEventsProcessor = new MetastoreEventsProcessor(); this.jobManager = new JobManager<>(); this.labelProcessor = new LabelProcessor(); @@ -844,6 +847,14 @@ public WorkloadRuntimeStatusMgr getWorkloadRuntimeStatusMgr() { return workloadRuntimeStatusMgr; } + public ExternalMetaIdMgr getExternalMetaIdMgr() { + return externalMetaIdMgr; + } + + public MetastoreEventsProcessor getMetastoreEventsProcessor() { + return metastoreEventsProcessor; + } + // use this to get correct ClusterInfoService instance public static SystemInfoService getCurrentSystemInfo() { return getCurrentEnv().getClusterInfo(); @@ -1638,9 +1649,6 @@ private void startMasterOnlyDaemonThreads() { streamLoadRecordMgr.start(); tabletLoadIndexRecorderMgr.start(); new InternalSchemaInitializer().start(); - if (Config.enable_hms_events_incremental_sync) { - metastoreEventsProcessor.start(); - } getRefreshManager().start(); // binlog gcer @@ -1662,6 +1670,9 @@ private void startNonMasterDaemonThreads() { domainResolver.start(); // fe disk updater feDiskUpdater.start(); + if (Config.enable_hms_events_incremental_sync) { + metastoreEventsProcessor.start(); + } } private void transferToNonMaster(FrontendNodeType newType) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalDatabase.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalDatabase.java index e6584d4a0b6acb..4af08e9b384317 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalDatabase.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalDatabase.java @@ -146,8 +146,15 @@ public void replayInitDb(InitDatabaseLog log, ExternalCatalog catalog) { Map tmpIdToTbl = Maps.newConcurrentMap(); for (int i = 0; i < log.getRefreshCount(); i++) { T table = getTableForReplay(log.getRefreshTableIds().get(i)); - tmpTableNameToId.put(table.getName(), table.getId()); - tmpIdToTbl.put(table.getId(), table); + // When upgrade cluster with this pr: https://github.com/apache/doris/pull/27666 + // Maybe there are some create table events will be skipped + // if the cluster has any hms catalog(s) with hms event listener enabled. + // So we need add a validation here to avoid table(s) not found, this is just a temporary solution + // because later we will remove all the logics about InitCatalogLog/InitDatabaseLog. + if (table != null) { + tmpTableNameToId.put(table.getName(), table.getId()); + tmpIdToTbl.put(table.getId(), table); + } } for (int i = 0; i < log.getCreateCount(); i++) { T table = getExternalTable(log.getCreateTableNames().get(i), log.getCreateTableIds().get(i), catalog); @@ -195,8 +202,7 @@ protected void init() { idToTbl = tmpIdToTbl; } - long currentTime = System.currentTimeMillis(); - lastUpdateTime = currentTime; + lastUpdateTime = System.currentTimeMillis(); initDatabaseLog.setLastUpdateTime(lastUpdateTime); initialized = true; Env.getCurrentEnv().getEditLog().logInitExternalDb(initDatabaseLog); @@ -370,17 +376,13 @@ public void dropTable(String tableName) { throw new NotImplementedException("dropTable() is not implemented"); } - public void dropTableForReplay(String tableName) { - throw new NotImplementedException("replayDropTableFromEvent() is not implemented"); - } - @Override public CatalogIf getCatalog() { return extCatalog; } // Only used for sync hive metastore event - public void createTableForReplay(String tableName, long tableId) { + public void createTable(String tableName, long tableId) { throw new NotImplementedException("createTable() is not implemented"); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java index d75f86bd0883da..f586ea7ed8a97d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java @@ -64,17 +64,6 @@ public void addTableForTest(HMSExternalTable tbl) { @Override public void dropTable(String tableName) { - LOG.debug("drop table [{}]", tableName); - makeSureInitialized(); - Long tableId = tableNameToId.remove(tableName); - if (tableId == null) { - LOG.warn("drop table [{}] failed", tableName); - } - idToTbl.remove(tableId); - } - - @Override - public void dropTableForReplay(String tableName) { LOG.debug("replayDropTableFromEvent [{}]", tableName); Long tableId = tableNameToId.remove(tableName); if (tableId == null) { @@ -85,7 +74,7 @@ public void dropTableForReplay(String tableName) { } @Override - public void createTableForReplay(String tableName, long tableId) { + public void createTable(String tableName, long tableId) { LOG.debug("create table [{}]", tableName); tableNameToId.put(tableName, tableId); HMSExternalTable table = getExternalTable(tableName, tableId, extCatalog); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/IcebergExternalDatabase.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/IcebergExternalDatabase.java index a915b3b2418936..1b5cd805a0f86f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/IcebergExternalDatabase.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/IcebergExternalDatabase.java @@ -49,7 +49,7 @@ public List getTablesOnIdOrder() { } @Override - public void dropTableForReplay(String tableName) { + public void dropTable(String tableName) { LOG.debug("drop table [{}]", tableName); Long tableId = tableNameToId.remove(tableName); if (tableId == null) { @@ -59,7 +59,7 @@ public void dropTableForReplay(String tableName) { } @Override - public void createTableForReplay(String tableName, long tableId) { + public void createTable(String tableName, long tableId) { LOG.debug("create table [{}]", tableName); tableNameToId.put(tableName, tableId); IcebergExternalTable table = new IcebergExternalTable(tableId, tableName, name, diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/PaimonExternalDatabase.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/PaimonExternalDatabase.java index a839b31298f198..0e6ee7332d02fc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/PaimonExternalDatabase.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/PaimonExternalDatabase.java @@ -49,7 +49,7 @@ public List getTablesOnIdOrder() { } @Override - public void dropTableForReplay(String tableName) { + public void dropTable(String tableName) { LOG.debug("drop table [{}]", tableName); Long tableId = tableNameToId.remove(tableName); if (tableId == null) { @@ -59,7 +59,7 @@ public void dropTableForReplay(String tableName) { } @Override - public void createTableForReplay(String tableName, long tableId) { + public void createTable(String tableName, long tableId) { LOG.debug("create table [{}]", tableName); tableNameToId.put(tableName, tableId); PaimonExternalTable table = new PaimonExternalTable(tableId, tableName, name, diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java index 2ad27c87f227e4..76d7702fef426b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java @@ -33,6 +33,7 @@ import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.external.ExternalDatabase; import org.apache.doris.catalog.external.ExternalTable; +import org.apache.doris.catalog.external.HMSExternalDatabase; import org.apache.doris.catalog.external.HMSExternalTable; import org.apache.doris.cluster.ClusterNamespace; import org.apache.doris.common.AnalysisException; @@ -345,10 +346,10 @@ public void alterCatalogProps(AlterCatalogPropertyStmt stmt) throws UserExceptio writeLock(); try { CatalogIf catalog = nameToCatalog.get(stmt.getCatalogName()); - Map oldProperties = catalog.getProperties(); if (catalog == null) { throw new DdlException("No catalog found with name: " + stmt.getCatalogName()); } + Map oldProperties = catalog.getProperties(); if (stmt.getNewProperties().containsKey("type") && !catalog.getType() .equalsIgnoreCase(stmt.getNewProperties().get("type"))) { throw new DdlException("Can't modify the type of catalog property with name: " + stmt.getCatalogName()); @@ -682,12 +683,6 @@ public void refreshExternalTableFromEvent(String dbName, String tableName, Strin ((HMSExternalTable) table).unsetObjectCreated(); ((HMSExternalTable) table).setEventUpdateTime(updateTime); Env.getCurrentEnv().getExtMetaCacheMgr().invalidateTableCache(catalog.getId(), dbName, tableName); - ExternalObjectLog log = new ExternalObjectLog(); - log.setCatalogId(catalog.getId()); - log.setDbId(db.getId()); - log.setTableId(table.getId()); - log.setLastUpdateTime(updateTime); - Env.getCurrentEnv().getEditLog().logRefreshExternalTable(log); } public void refreshExternalTable(String dbName, String tableName, String catalogName, boolean ignoreIfNotExists) @@ -773,43 +768,16 @@ public void dropExternalTable(String dbName, String tableName, String catalogNam } return; } - ExternalObjectLog log = new ExternalObjectLog(); - log.setCatalogId(catalog.getId()); - log.setDbId(db.getId()); - log.setTableId(table.getId()); - log.setLastUpdateTime(System.currentTimeMillis()); - replayDropExternalTable(log); - Env.getCurrentEnv().getEditLog().logDropExternalTable(log); - } - public void replayDropExternalTable(ExternalObjectLog log) { - LOG.debug("ReplayDropExternalTable,catalogId:[{}],dbId:[{}],tableId:[{}]", log.getCatalogId(), log.getDbId(), - log.getTableId()); - ExternalCatalog catalog = (ExternalCatalog) idToCatalog.get(log.getCatalogId()); - if (catalog == null) { - LOG.warn("No catalog found with id:[{}], it may have been dropped.", log.getCatalogId()); - return; - } - ExternalDatabase db = catalog.getDbForReplay(log.getDbId()); - if (db == null) { - LOG.warn("No db found with id:[{}], it may have been dropped.", log.getDbId()); - return; - } - ExternalTable table = db.getTableForReplay(log.getTableId()); - if (table == null) { - LOG.warn("No table found with id:[{}], it may have been dropped.", log.getTableId()); - return; - } db.writeLock(); try { - db.dropTableForReplay(table.getName()); - db.setLastUpdateTime(log.getLastUpdateTime()); + db.dropTable(table.getName()); + Env.getCurrentEnv().getExtMetaCacheMgr().invalidateTableCache( + catalog.getId(), db.getFullName(), table.getName()); + ((HMSExternalDatabase) db).setLastUpdateTime(System.currentTimeMillis()); } finally { db.writeUnlock(); } - - Env.getCurrentEnv().getExtMetaCacheMgr() - .invalidateTableCache(catalog.getId(), db.getFullName(), table.getName()); } public boolean externalTableExistInLocal(String dbName, String tableName, String catalogName) throws DdlException { @@ -823,9 +791,9 @@ public boolean externalTableExistInLocal(String dbName, String tableName, String return ((ExternalCatalog) catalog).tableExistInLocal(dbName, tableName); } - public void createExternalTableFromEvent(String dbName, String tableName, String catalogName, - boolean ignoreIfExists) - throws DdlException { + public void createExternalTableFromEvent(String dbName, String tableName, + String catalogName, long updateTime, + boolean ignoreIfExists) throws DdlException { CatalogIf catalog = nameToCatalog.get(catalogName); if (catalog == null) { throw new DdlException("No catalog found with name: " + catalogName); @@ -848,33 +816,21 @@ public void createExternalTableFromEvent(String dbName, String tableName, String } return; } - ExternalObjectLog log = new ExternalObjectLog(); - log.setCatalogId(catalog.getId()); - log.setDbId(db.getId()); - log.setTableName(tableName); - log.setTableId(Env.getCurrentEnv().getNextId()); - log.setLastUpdateTime(System.currentTimeMillis()); - replayCreateExternalTableFromEvent(log); - Env.getCurrentEnv().getEditLog().logCreateExternalTable(log); - } - public void replayCreateExternalTableFromEvent(ExternalObjectLog log) { - LOG.debug("ReplayCreateExternalTable,catalogId:[{}],dbId:[{}],tableId:[{}],tableName:[{}]", log.getCatalogId(), - log.getDbId(), log.getTableId(), log.getTableName()); - ExternalCatalog catalog = (ExternalCatalog) idToCatalog.get(log.getCatalogId()); - if (catalog == null) { - LOG.warn("No catalog found with id:[{}], it may have been dropped.", log.getCatalogId()); - return; - } - ExternalDatabase db = catalog.getDbForReplay(log.getDbId()); - if (db == null) { - LOG.warn("No db found with id:[{}], it may have been dropped.", log.getDbId()); + long tblId = Env.getCurrentEnv().getExternalMetaIdMgr().getTblId(catalog.getId(), dbName, tableName); + // -1L means it will be dropped later, ignore + if (tblId == ExternalMetaIdMgr.META_ID_FOR_NOT_EXISTS) { return; } + db.writeLock(); try { - db.createTableForReplay(log.getTableName(), log.getTableId()); - db.setLastUpdateTime(log.getLastUpdateTime()); + ((HMSExternalDatabase) db).createTable(tableName, tblId); + ((HMSExternalDatabase) db).setLastUpdateTime(System.currentTimeMillis()); + table = db.getTableNullable(tableName); + if (table != null) { + ((HMSExternalTable) table).setEventUpdateTime(updateTime); + } } finally { db.writeUnlock(); } @@ -896,34 +852,8 @@ public void dropExternalDatabase(String dbName, String catalogName, boolean igno return; } - ExternalObjectLog log = new ExternalObjectLog(); - log.setCatalogId(catalog.getId()); - log.setDbId(db.getId()); - log.setInvalidCache(true); - replayDropExternalDatabase(log); - Env.getCurrentEnv().getEditLog().logDropExternalDatabase(log); - } - - public void replayDropExternalDatabase(ExternalObjectLog log) { - writeLock(); - try { - LOG.debug("ReplayDropExternalTable,catalogId:[{}],dbId:[{}],tableId:[{}]", log.getCatalogId(), - log.getDbId(), log.getTableId()); - ExternalCatalog catalog = (ExternalCatalog) idToCatalog.get(log.getCatalogId()); - if (catalog == null) { - LOG.warn("No catalog found with id:[{}], it may have been dropped.", log.getCatalogId()); - return; - } - ExternalDatabase db = catalog.getDbForReplay(log.getDbId()); - if (db == null) { - LOG.warn("No db found with id:[{}], it may have been dropped.", log.getDbId()); - return; - } - catalog.dropDatabaseForReplay(db.getFullName()); - Env.getCurrentEnv().getExtMetaCacheMgr().invalidateDbCache(catalog.getId(), db.getFullName()); - } finally { - writeUnlock(); - } + ((HMSExternalCatalog) catalog).dropDatabase(dbName); + Env.getCurrentEnv().getExtMetaCacheMgr().invalidateDbCache(catalog.getId(), dbName); } public void createExternalDatabase(String dbName, String catalogName, boolean ignoreIfExists) throws DdlException { @@ -942,28 +872,13 @@ public void createExternalDatabase(String dbName, String catalogName, boolean ig return; } - ExternalObjectLog log = new ExternalObjectLog(); - log.setCatalogId(catalog.getId()); - log.setDbId(Env.getCurrentEnv().getNextId()); - log.setDbName(dbName); - replayCreateExternalDatabase(log); - Env.getCurrentEnv().getEditLog().logCreateExternalDatabase(log); - } - - public void replayCreateExternalDatabase(ExternalObjectLog log) { - writeLock(); - try { - LOG.debug("ReplayCreateExternalDatabase,catalogId:[{}],dbId:[{}],dbName:[{}]", log.getCatalogId(), - log.getDbId(), log.getDbName()); - ExternalCatalog catalog = (ExternalCatalog) idToCatalog.get(log.getCatalogId()); - if (catalog == null) { - LOG.warn("No catalog found with id:[{}], it may have been dropped.", log.getCatalogId()); - return; - } - catalog.createDatabaseForReplay(log.getDbId(), log.getDbName()); - } finally { - writeUnlock(); + long dbId = Env.getCurrentEnv().getExternalMetaIdMgr().getDbId(catalog.getId(), dbName); + // -1L means it will be dropped later, ignore + if (dbId == ExternalMetaIdMgr.META_ID_FOR_NOT_EXISTS) { + return; } + + ((HMSExternalCatalog) catalog).createDatabase(dbId, dbName); } public void addExternalPartitions(String catalogName, String dbName, String tableName, @@ -999,48 +914,6 @@ public void addExternalPartitions(String catalogName, String dbName, String tabl HMSExternalTable hmsTable = (HMSExternalTable) table; Env.getCurrentEnv().getExtMetaCacheMgr().addPartitionsCache(catalog.getId(), hmsTable, partitionNames); hmsTable.setEventUpdateTime(updateTime); - ExternalObjectLog log = new ExternalObjectLog(); - log.setCatalogId(catalog.getId()); - log.setDbId(db.getId()); - log.setTableId(table.getId()); - log.setPartitionNames(partitionNames); - log.setLastUpdateTime(updateTime); - Env.getCurrentEnv().getEditLog().logAddExternalPartitions(log); - } - - public void replayAddExternalPartitions(ExternalObjectLog log) { - LOG.debug("ReplayAddExternalPartitions,catalogId:[{}],dbId:[{}],tableId:[{}]", log.getCatalogId(), - log.getDbId(), log.getTableId()); - ExternalCatalog catalog = (ExternalCatalog) idToCatalog.get(log.getCatalogId()); - if (catalog == null) { - LOG.warn("No catalog found with id:[{}], it may have been dropped.", log.getCatalogId()); - return; - } - ExternalDatabase db = catalog.getDbForReplay(log.getDbId()); - if (db == null) { - LOG.warn("No db found with id:[{}], it may have been dropped.", log.getDbId()); - return; - } - ExternalTable table = db.getTableForReplay(log.getTableId()); - if (table == null) { - LOG.warn("No table found with id:[{}], it may have been dropped.", log.getTableId()); - return; - } - if (!(table instanceof HMSExternalTable)) { - LOG.warn("only support HMSTable"); - return; - } - - HMSExternalTable hmsTable = (HMSExternalTable) table; - try { - Env.getCurrentEnv().getExtMetaCacheMgr() - .addPartitionsCache(catalog.getId(), hmsTable, log.getPartitionNames()); - hmsTable.setEventUpdateTime(log.getLastUpdateTime()); - } catch (HMSClientException e) { - LOG.warn("Network problem occurs or hms table has been deleted, fallback to invalidate table cache", e); - Env.getCurrentEnv().getExtMetaCacheMgr().invalidateTableCache(catalog.getId(), - db.getFullName(), table.getName()); - } } public void dropExternalPartitions(String catalogName, String dbName, String tableName, @@ -1069,42 +942,9 @@ public void dropExternalPartitions(String catalogName, String dbName, String tab return; } - ExternalObjectLog log = new ExternalObjectLog(); - log.setCatalogId(catalog.getId()); - log.setDbId(db.getId()); - log.setTableId(table.getId()); - log.setPartitionNames(partitionNames); - log.setLastUpdateTime(updateTime); - replayDropExternalPartitions(log); - Env.getCurrentEnv().getEditLog().logDropExternalPartitions(log); - } - - public void replayDropExternalPartitions(ExternalObjectLog log) { - LOG.debug("ReplayDropExternalPartitions,catalogId:[{}],dbId:[{}],tableId:[{}]", log.getCatalogId(), - log.getDbId(), log.getTableId()); - ExternalCatalog catalog = (ExternalCatalog) idToCatalog.get(log.getCatalogId()); - if (catalog == null) { - LOG.warn("No catalog found with id:[{}], it may have been dropped.", log.getCatalogId()); - return; - } - ExternalDatabase db = catalog.getDbForReplay(log.getDbId()); - if (db == null) { - LOG.warn("No db found with id:[{}], it may have been dropped.", log.getDbId()); - return; - } - ExternalTable table = db.getTableForReplay(log.getTableId()); - if (table == null) { - LOG.warn("No table found with id:[{}], it may have been dropped.", log.getTableId()); - return; - } - if (!(table instanceof HMSExternalTable)) { - LOG.warn("only support HMSTable"); - return; - } HMSExternalTable hmsTable = (HMSExternalTable) table; - Env.getCurrentEnv().getExtMetaCacheMgr() - .dropPartitionsCache(catalog.getId(), hmsTable, log.getPartitionNames()); - hmsTable.setEventUpdateTime(log.getLastUpdateTime()); + Env.getCurrentEnv().getExtMetaCacheMgr().dropPartitionsCache(catalog.getId(), hmsTable, partitionNames); + hmsTable.setEventUpdateTime(updateTime); } public void refreshExternalPartitions(String catalogName, String dbName, String tableName, @@ -1136,42 +976,9 @@ public void refreshExternalPartitions(String catalogName, String dbName, String return; } - ExternalObjectLog log = new ExternalObjectLog(); - log.setCatalogId(catalog.getId()); - log.setDbId(db.getId()); - log.setTableId(table.getId()); - log.setPartitionNames(partitionNames); - log.setLastUpdateTime(updateTime); - replayRefreshExternalPartitions(log); - Env.getCurrentEnv().getEditLog().logInvalidateExternalPartitions(log); - } - - public void replayRefreshExternalPartitions(ExternalObjectLog log) { - LOG.debug("replayRefreshExternalPartitions,catalogId:[{}],dbId:[{}],tableId:[{}]", log.getCatalogId(), - log.getDbId(), log.getTableId()); - ExternalCatalog catalog = (ExternalCatalog) idToCatalog.get(log.getCatalogId()); - if (catalog == null) { - LOG.warn("No catalog found with id:[{}], it may have been dropped.", log.getCatalogId()); - return; - } - ExternalDatabase db = catalog.getDbForReplay(log.getDbId()); - if (db == null) { - LOG.warn("No db found with id:[{}], it may have been dropped.", log.getDbId()); - return; - } - ExternalTable table = db.getTableForReplay(log.getTableId()); - if (table == null) { - LOG.warn("No table found with id:[{}], it may have been dropped.", log.getTableId()); - return; - } - if (!(table instanceof HMSExternalTable)) { - LOG.warn("only support HMSTable"); - return; - } - Env.getCurrentEnv().getExtMetaCacheMgr() - .invalidatePartitionsCache(catalog.getId(), db.getFullName(), table.getName(), - log.getPartitionNames()); - ((HMSExternalTable) table).setEventUpdateTime(log.getLastUpdateTime()); + Env.getCurrentEnv().getExtMetaCacheMgr().invalidatePartitionsCache( + catalog.getId(), db.getFullName(), table.getName(), partitionNames); + ((HMSExternalTable) table).setEventUpdateTime(updateTime); } public void registerCatalogRefreshListener(Env env) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java index 3ff599d0ab476f..6fcd495b67f3b1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java @@ -521,13 +521,6 @@ protected ExternalDatabase getDbForInit(String dbName, return null; } - /** - * External catalog has no cluster semantics. - */ - protected static String getRealTableName(String tableName) { - return ClusterNamespace.getNameFromFullName(tableName); - } - public static ExternalCatalog read(DataInput in) throws IOException { String json = Text.readString(in); return GsonUtils.GSON.fromJson(json, ExternalCatalog.class); @@ -546,9 +539,6 @@ public void gsonPostProcess() throws IOException { db.setTableExtCatalog(this); } objectCreated = false; - if (this instanceof HMSExternalCatalog) { - ((HMSExternalCatalog) this).setLastSyncedEventId(-1L); - } // TODO: This code is to compatible with older version of metadata. // Could only remove after all users upgrate to the new version. if (logType == null) { @@ -569,11 +559,11 @@ public void addDatabaseForTest(ExternalDatabase db) { dbNameToId.put(ClusterNamespace.getNameFromFullName(db.getFullName()), db.getId()); } - public void dropDatabaseForReplay(String dbName) { + public void dropDatabase(String dbName) { throw new NotImplementedException("dropDatabase not implemented"); } - public void createDatabaseForReplay(long dbId, String dbName) { + public void createDatabase(long dbId, String dbName) { throw new NotImplementedException("createDatabase not implemented"); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalMetaIdMgr.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalMetaIdMgr.java new file mode 100644 index 00000000000000..621c25b3698b30 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalMetaIdMgr.java @@ -0,0 +1,263 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.datasource; + +import org.apache.doris.catalog.Env; +import org.apache.doris.datasource.hive.event.MetastoreEventsProcessor; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Maps; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Map; +import javax.annotation.Nullable; +import javax.validation.constraints.NotNull; + +/** + *
+ * ExternalMetaIdMgr is responsible for managing external meta ids.
+ * Now it just manages the external meta ids of hms events,
+ * but it will be extended to manage other external meta ids in the future.
+ * 
+ * TODO: remove InitCatalogLog and InitDatabaseLog, manage external meta ids at ExternalMetaIdMgr + */ +public class ExternalMetaIdMgr { + + private static final Logger LOG = LogManager.getLogger(ExternalMetaIdMgr.class); + + public static final long META_ID_FOR_NOT_EXISTS = -1L; + + private final Map idToCtlMgr = Maps.newConcurrentMap(); + + public ExternalMetaIdMgr() { + } + + // invoke this method only on master + public static long nextMetaId() { + return Env.getCurrentEnv().getNextId(); + } + + // return the db id of the specified db, -1 means not exists + public long getDbId(long catalogId, String dbName) { + DbMetaIdMgr dbMetaIdMgr = getDbMetaIdMgr(catalogId, dbName); + if (dbMetaIdMgr == null) { + return META_ID_FOR_NOT_EXISTS; + } + return dbMetaIdMgr.dbId; + } + + // return the tbl id of the specified tbl, -1 means not exists + public long getTblId(long catalogId, String dbName, String tblName) { + TblMetaIdMgr tblMetaIdMgr = getTblMetaIdMgr(catalogId, dbName, tblName); + if (tblMetaIdMgr == null) { + return META_ID_FOR_NOT_EXISTS; + } + return tblMetaIdMgr.tblId; + } + + // return the partition id of the specified partition, -1 means not exists + public long getPartitionId(long catalogId, String dbName, + String tblName, String partitionName) { + PartitionMetaIdMgr partitionMetaIdMgr = getPartitionMetaIdMgr(catalogId, dbName, tblName, partitionName); + if (partitionMetaIdMgr == null) { + return META_ID_FOR_NOT_EXISTS; + } + return partitionMetaIdMgr.partitionId; + } + + private @Nullable DbMetaIdMgr getDbMetaIdMgr(long catalogId, String dbName) { + CtlMetaIdMgr ctlMetaIdMgr = idToCtlMgr.get(catalogId); + if (ctlMetaIdMgr == null) { + return null; + } + return ctlMetaIdMgr.dbNameToMgr.get(dbName); + } + + private @Nullable TblMetaIdMgr getTblMetaIdMgr(long catalogId, String dbName, String tblName) { + DbMetaIdMgr dbMetaIdMgr = getDbMetaIdMgr(catalogId, dbName); + if (dbMetaIdMgr == null) { + return null; + } + return dbMetaIdMgr.tblNameToMgr.get(tblName); + } + + private PartitionMetaIdMgr getPartitionMetaIdMgr(long catalogId, String dbName, + String tblName, String partitionName) { + TblMetaIdMgr tblMetaIdMgr = getTblMetaIdMgr(catalogId, dbName, tblName); + if (tblMetaIdMgr == null) { + return null; + } + return tblMetaIdMgr.partitionNameToMgr.get(partitionName); + } + + public void replayMetaIdMappingsLog(@NotNull MetaIdMappingsLog log) { + Preconditions.checkNotNull(log); + long catalogId = log.getCatalogId(); + CtlMetaIdMgr ctlMetaIdMgr = idToCtlMgr.computeIfAbsent(catalogId, CtlMetaIdMgr::new); + for (MetaIdMappingsLog.MetaIdMapping mapping : log.getMetaIdMappings()) { + handleMetaIdMapping(mapping, ctlMetaIdMgr); + } + if (log.isFromHmsEvent()) { + CatalogIf catalogIf = Env.getCurrentEnv().getCatalogMgr().getCatalog(log.getCatalogId()); + if (catalogIf != null) { + MetastoreEventsProcessor metastoreEventsProcessor = Env.getCurrentEnv().getMetastoreEventsProcessor(); + metastoreEventsProcessor.updateMasterLastSyncedEventId( + (HMSExternalCatalog) catalogIf, log.getLastSyncedEventId()); + } + } + } + + // no lock because the operations is serialized currently + private void handleMetaIdMapping(MetaIdMappingsLog.MetaIdMapping mapping, CtlMetaIdMgr ctlMetaIdMgr) { + MetaIdMappingsLog.OperationType opType = MetaIdMappingsLog.getOperationType(mapping.getOpType()); + MetaIdMappingsLog.MetaObjectType objType = MetaIdMappingsLog.getMetaObjectType(mapping.getMetaObjType()); + switch (opType) { + case ADD: + handleAddMetaIdMapping(mapping, ctlMetaIdMgr, objType); + break; + + case DELETE: + handleDelMetaIdMapping(mapping, ctlMetaIdMgr, objType); + break; + + default: + break; + } + } + + private static void handleDelMetaIdMapping(MetaIdMappingsLog.MetaIdMapping mapping, + CtlMetaIdMgr ctlMetaIdMgr, + MetaIdMappingsLog.MetaObjectType objType) { + TblMetaIdMgr tblMetaIdMgr; + DbMetaIdMgr dbMetaIdMgr; + switch (objType) { + case DATABASE: + ctlMetaIdMgr.dbNameToMgr.remove(mapping.getDbName()); + break; + + case TABLE: + dbMetaIdMgr = ctlMetaIdMgr.dbNameToMgr.get(mapping.getDbName()); + if (dbMetaIdMgr != null) { + dbMetaIdMgr.tblNameToMgr.remove(mapping.getTblName()); + } + break; + + case PARTITION: + dbMetaIdMgr = ctlMetaIdMgr.dbNameToMgr.get(mapping.getDbName()); + if (dbMetaIdMgr != null) { + tblMetaIdMgr = dbMetaIdMgr.tblNameToMgr.get(mapping.getTblName()); + if (tblMetaIdMgr != null) { + tblMetaIdMgr.partitionNameToMgr.remove(mapping.getPartitionName()); + } + } + break; + + default: + break; + } + } + + private static void handleAddMetaIdMapping(MetaIdMappingsLog.MetaIdMapping mapping, + CtlMetaIdMgr ctlMetaIdMgr, + MetaIdMappingsLog.MetaObjectType objType) { + DbMetaIdMgr dbMetaIdMgr; + TblMetaIdMgr tblMetaIdMgr; + switch (objType) { + case DATABASE: + ctlMetaIdMgr.dbNameToMgr.put(mapping.getDbName(), + new DbMetaIdMgr(mapping.getId(), mapping.getDbName())); + break; + + case TABLE: + dbMetaIdMgr = ctlMetaIdMgr.dbNameToMgr + .computeIfAbsent(mapping.getDbName(), DbMetaIdMgr::new); + dbMetaIdMgr.tblNameToMgr.put(mapping.getTblName(), + new TblMetaIdMgr(mapping.getId(), mapping.getTblName())); + break; + + case PARTITION: + dbMetaIdMgr = ctlMetaIdMgr.dbNameToMgr + .computeIfAbsent(mapping.getDbName(), DbMetaIdMgr::new); + tblMetaIdMgr = dbMetaIdMgr.tblNameToMgr + .computeIfAbsent(mapping.getTblName(), TblMetaIdMgr::new); + tblMetaIdMgr.partitionNameToMgr.put(mapping.getPartitionName(), + new PartitionMetaIdMgr(mapping.getId(), mapping.getPartitionName())); + break; + + default: + break; + } + } + + public static class CtlMetaIdMgr { + protected final long catalogId; + + protected CtlMetaIdMgr(long catalogId) { + this.catalogId = catalogId; + } + + protected Map dbNameToMgr = Maps.newConcurrentMap(); + } + + public static class DbMetaIdMgr { + protected volatile long dbId = META_ID_FOR_NOT_EXISTS; + protected final String dbName; + + protected DbMetaIdMgr(long dbId, String dbName) { + this.dbId = dbId; + this.dbName = dbName; + } + + protected DbMetaIdMgr(String dbName) { + this.dbName = dbName; + } + + protected Map tblNameToMgr = Maps.newConcurrentMap(); + } + + public static class TblMetaIdMgr { + protected volatile long tblId = META_ID_FOR_NOT_EXISTS; + protected final String tblName; + + protected TblMetaIdMgr(long tblId, String tblName) { + this.tblId = tblId; + this.tblName = tblName; + } + + protected TblMetaIdMgr(String tblName) { + this.tblName = tblName; + } + + protected Map partitionNameToMgr = Maps.newConcurrentMap(); + } + + public static class PartitionMetaIdMgr { + protected volatile long partitionId = META_ID_FOR_NOT_EXISTS; + protected final String partitionName; + + protected PartitionMetaIdMgr(long partitionId, String partitionName) { + this.partitionId = partitionId; + this.partitionName = partitionName; + } + + protected PartitionMetaIdMgr(String partitionName) { + this.partitionName = partitionName; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java index ae6b5f04738b07..dd6788ade2c968 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java @@ -23,23 +23,19 @@ import org.apache.doris.catalog.external.ExternalDatabase; import org.apache.doris.catalog.external.ExternalTable; import org.apache.doris.catalog.external.HMSExternalDatabase; +import org.apache.doris.cluster.ClusterNamespace; import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; import org.apache.doris.datasource.hive.HMSCachedClient; import org.apache.doris.datasource.hive.HMSCachedClientFactory; -import org.apache.doris.datasource.hive.event.MetastoreNotificationFetchException; import org.apache.doris.datasource.jdbc.client.JdbcClientConfig; import org.apache.doris.datasource.property.PropertyConverter; import org.apache.doris.datasource.property.constants.HMSProperties; import com.google.common.base.Strings; import com.google.common.collect.Lists; -import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.math.NumberUtils; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId; -import org.apache.hadoop.hive.metastore.api.NotificationEventResponse; import org.apache.hadoop.security.UserGroupInformation; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -57,10 +53,7 @@ public class HMSExternalCatalog extends ExternalCatalog { private static final int MIN_CLIENT_POOL_SIZE = 8; protected HMSCachedClient client; - // Record the latest synced event id when processing hive events - // Must set to -1 otherwise client.getNextNotification will throw exception - // Reference to https://github.com/apDdlache/doris/issues/18251 - private long lastSyncedEventId = -1L; + public static final String ENABLE_SELF_SPLITTER = "enable.self.splitter"; public static final String FILE_META_CACHE_TTL_SECOND = "file.meta.cache.ttl-second"; // broker name for file split and query scan. @@ -132,18 +125,6 @@ public void checkProperties() throws DdlException { } } - public String getHiveMetastoreUris() { - return catalogProperty.getOrDefault(HMSProperties.HIVE_METASTORE_URIS, ""); - } - - public String getHiveVersion() { - return catalogProperty.getOrDefault(HMSProperties.HIVE_VERSION, ""); - } - - protected List listDatabaseNames() { - return client.getAllDatabases(); - } - @Override protected void initLocalObjectsImpl() { HiveConf hiveConf = null; @@ -195,13 +176,13 @@ public List listTableNames(SessionContext ctx, String dbName) { hmsExternalDatabase.getTables().forEach(table -> names.add(table.getName())); return names; } else { - return client.getAllTables(getRealTableName(dbName)); + return client.getAllTables(ClusterNamespace.getNameFromFullName(dbName)); } } @Override public boolean tableExist(SessionContext ctx, String dbName, String tblName) { - return client.tableExists(getRealTableName(dbName), tblName); + return client.tableExists(ClusterNamespace.getNameFromFullName(dbName), tblName); } @Override @@ -211,7 +192,7 @@ public boolean tableExistInLocal(String dbName, String tblName) { if (hmsExternalDatabase == null) { return false; } - return hmsExternalDatabase.getTable(getRealTableName(tblName)).isPresent(); + return hmsExternalDatabase.getTable(ClusterNamespace.getNameFromFullName(tblName)).isPresent(); } public HMSCachedClient getClient() { @@ -219,69 +200,8 @@ public HMSCachedClient getClient() { return client; } - public void setLastSyncedEventId(long lastSyncedEventId) { - this.lastSyncedEventId = lastSyncedEventId; - } - - public NotificationEventResponse getNextEventResponse(HMSExternalCatalog hmsExternalCatalog) - throws MetastoreNotificationFetchException { - makeSureInitialized(); - long currentEventId = getCurrentEventId(); - if (lastSyncedEventId < 0) { - refreshCatalog(hmsExternalCatalog); - // invoke getCurrentEventId() and save the event id before refresh catalog to avoid missing events - // but set lastSyncedEventId to currentEventId only if there is not any problems when refreshing catalog - lastSyncedEventId = currentEventId; - LOG.info( - "First pulling events on catalog [{}],refreshCatalog and init lastSyncedEventId," - + "lastSyncedEventId is [{}]", - hmsExternalCatalog.getName(), lastSyncedEventId); - return null; - } - - LOG.debug("Catalog [{}] getNextEventResponse, currentEventId is {},lastSyncedEventId is {}", - hmsExternalCatalog.getName(), currentEventId, lastSyncedEventId); - if (currentEventId == lastSyncedEventId) { - LOG.info("Event id not updated when pulling events on catalog [{}]", hmsExternalCatalog.getName()); - return null; - } - - try { - return client.getNextNotification(lastSyncedEventId, Config.hms_events_batch_size_per_rpc, null); - } catch (MetastoreNotificationFetchException e) { - // Need a fallback to handle this because this error state can not be recovered until restarting FE - if (StringUtils.isNotEmpty(e.getMessage()) - && e.getMessage().contains(HiveMetaStoreClient.REPL_EVENTS_MISSING_IN_METASTORE)) { - refreshCatalog(hmsExternalCatalog); - // set lastSyncedEventId to currentEventId after refresh catalog successfully - lastSyncedEventId = currentEventId; - LOG.warn("Notification events are missing, maybe an event can not be handled " - + "or processing rate is too low, fallback to refresh the catalog"); - return null; - } - throw e; - } - } - - private void refreshCatalog(HMSExternalCatalog hmsExternalCatalog) { - CatalogLog log = new CatalogLog(); - log.setCatalogId(hmsExternalCatalog.getId()); - log.setInvalidCache(true); - Env.getCurrentEnv().getCatalogMgr().refreshCatalog(log); - } - - private long getCurrentEventId() { - makeSureInitialized(); - CurrentNotificationEventId currentNotificationEventId = client.getCurrentNotificationEventId(); - if (currentNotificationEventId == null) { - LOG.warn("Get currentNotificationEventId is null"); - return -1; - } - return currentNotificationEventId.getEventId(); - } - @Override - public void dropDatabaseForReplay(String dbName) { + public void dropDatabase(String dbName) { LOG.debug("drop database [{}]", dbName); Long dbId = dbNameToId.remove(dbName); if (dbId == null) { @@ -291,7 +211,7 @@ public void dropDatabaseForReplay(String dbName) { } @Override - public void createDatabaseForReplay(long dbId, String dbName) { + public void createDatabase(long dbId, String dbName) { LOG.debug("create database [{}]", dbName); dbNameToId.put(dbName, dbId); ExternalDatabase db = getDbForInit(dbName, dbId, logType); @@ -317,4 +237,17 @@ public void setDefaultPropsWhenCreating(boolean isReplay) { catalogProperty.addProperty(PROP_ALLOW_FALLBACK_TO_SIMPLE_AUTH, "true"); } } + + public String getHiveMetastoreUris() { + return catalogProperty.getOrDefault(HMSProperties.HIVE_METASTORE_URIS, ""); + } + + public String getHiveVersion() { + return catalogProperty.getOrDefault(HMSProperties.HIVE_VERSION, ""); + } + + protected List listDatabaseNames() { + return client.getAllDatabases(); + } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InitTableLog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InitTableLog.java deleted file mode 100644 index 2f462b551c7806..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InitTableLog.java +++ /dev/null @@ -1,67 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you 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 org.apache.doris.datasource; - -import org.apache.doris.catalog.Column; -import org.apache.doris.common.io.Text; -import org.apache.doris.common.io.Writable; -import org.apache.doris.persist.gson.GsonUtils; - -import com.google.gson.annotations.SerializedName; -import lombok.Data; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.util.List; - -@Data -public class InitTableLog implements Writable { - enum Type { - HMS, - ES, - UNKNOWN; - } - - @SerializedName(value = "catalogId") - private long catalogId; - - @SerializedName(value = "dbId") - private long dbId; - - @SerializedName(value = "tableId") - private long tableId; - - @SerializedName(value = "type") - private Type type; - - @SerializedName(value = "schema") - protected volatile List schema; - - public InitTableLog() {} - - @Override - public void write(DataOutput out) throws IOException { - Text.writeString(out, GsonUtils.GSON.toJson(this)); - } - - public static InitTableLog read(DataInput in) throws IOException { - String json = Text.readString(in); - return GsonUtils.GSON.fromJson(json, InitTableLog.class); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/MetaIdMappingsLog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/MetaIdMappingsLog.java new file mode 100644 index 00000000000000..629b4d13a4041b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/MetaIdMappingsLog.java @@ -0,0 +1,274 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.datasource; + +import org.apache.doris.common.io.Text; +import org.apache.doris.common.io.Writable; +import org.apache.doris.persist.gson.GsonUtils; + +import com.google.common.collect.Lists; +import com.google.gson.annotations.SerializedName; +import lombok.Data; +import lombok.Getter; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; + +@Data +public class MetaIdMappingsLog implements Writable { + + public static final short OPERATION_TYPE_IGNORE = 0; + public static final short OPERATION_TYPE_ADD = 1; + public static final short OPERATION_TYPE_DELETE = 2; + + public static final short META_OBJECT_TYPE_IGNORE = 0; + public static final short META_OBJECT_TYPE_DATABASE = 1; + public static final short META_OBJECT_TYPE_TABLE = 2; + public static final short META_OBJECT_TYPE_PARTITION = 3; + + @SerializedName(value = "ctlId") + private long catalogId = -1L; + + @SerializedName(value = "fromEvent") + private boolean fromHmsEvent = false; + + // The synced event id of master + @SerializedName(value = "lastEventId") + private long lastSyncedEventId = -1L; + + @SerializedName(value = "metaIdMappings") + private List metaIdMappings = Lists.newLinkedList(); + + public MetaIdMappingsLog() { + } + + @Override + public int hashCode() { + return Objects.hash(catalogId, lastSyncedEventId, + metaIdMappings == null ? 0 : Arrays.hashCode(metaIdMappings.toArray())); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof MetaIdMappingsLog)) { + return false; + } + return Objects.equals(this.catalogId, ((MetaIdMappingsLog) obj).catalogId) + && Objects.equals(this.fromHmsEvent, ((MetaIdMappingsLog) obj).fromHmsEvent) + && Objects.equals(this.lastSyncedEventId, ((MetaIdMappingsLog) obj).lastSyncedEventId) + && Objects.equals(this.metaIdMappings, ((MetaIdMappingsLog) obj).metaIdMappings); + } + + @Override + public void write(DataOutput out) throws IOException { + Text.writeString(out, GsonUtils.GSON.toJson(this)); + } + + public static MetaIdMappingsLog read(DataInput in) throws IOException { + String json = Text.readString(in); + return GsonUtils.GSON.fromJson(json, MetaIdMappingsLog.class); + } + + public void addMetaIdMapping(MetaIdMapping metaIdMapping) { + this.metaIdMappings.add(metaIdMapping); + } + + public void addMetaIdMappings(List metaIdMappings) { + this.metaIdMappings.addAll(metaIdMappings); + } + + public static OperationType getOperationType(short opType) { + switch (opType) { + case OPERATION_TYPE_ADD: + return OperationType.ADD; + case OPERATION_TYPE_DELETE: + return OperationType.DELETE; + default: + return OperationType.IGNORE; + } + } + + public static MetaObjectType getMetaObjectType(short metaObjType) { + switch (metaObjType) { + case META_OBJECT_TYPE_DATABASE: + return MetaObjectType.DATABASE; + case META_OBJECT_TYPE_TABLE: + return MetaObjectType.TABLE; + case META_OBJECT_TYPE_PARTITION: + return MetaObjectType.PARTITION; + default: + return MetaObjectType.IGNORE; + } + } + + @Getter + public static class MetaIdMapping { + + @SerializedName(value = "opType") + private short opType; + @SerializedName(value = "metaObjType") + private short metaObjType; + // name of Database + @SerializedName(value = "dbName") + private String dbName; + // name of Table + @SerializedName(value = "tblName") + private String tblName; + // name of Partition + @SerializedName(value = "pName") + private String partitionName; + // id of Database/Table/Partition + @SerializedName(value = "id") + private long id; + + public MetaIdMapping() {} + + public MetaIdMapping(short opType, + short metaObjType, + String dbName, + String tblName, + String partitionName, + long id) { + this.opType = opType; + this.metaObjType = metaObjType; + this.dbName = dbName; + this.tblName = tblName; + this.partitionName = partitionName; + this.id = id; + } + + public MetaIdMapping(short opType, + short metaObjType, + String dbName, + String tblName, + String partitionName) { + this.opType = opType; + this.metaObjType = metaObjType; + this.dbName = dbName; + this.tblName = tblName; + this.partitionName = partitionName; + this.id = -1L; + } + + public MetaIdMapping(short opType, + short metaObjType, + String dbName, + String tblName, + long id) { + this.opType = opType; + this.metaObjType = metaObjType; + this.dbName = dbName; + this.tblName = tblName; + this.partitionName = null; + this.id = id; + } + + public MetaIdMapping(short opType, + short metaObjType, + String dbName, + String tblName) { + this.opType = opType; + this.metaObjType = metaObjType; + this.dbName = dbName; + this.tblName = tblName; + this.partitionName = null; + this.id = -1L; + } + + public MetaIdMapping(short opType, + short metaObjType, + String dbName, + long id) { + this.opType = opType; + this.metaObjType = metaObjType; + this.dbName = dbName; + this.tblName = null; + this.partitionName = null; + this.id = id; + } + + public MetaIdMapping(short opType, + short metaObjType, + String dbName) { + this.opType = opType; + this.metaObjType = metaObjType; + this.dbName = dbName; + this.tblName = null; + this.partitionName = null; + this.id = -1L; + } + + @Override + public int hashCode() { + return Objects.hash(opType, metaObjType, dbName, tblName, partitionName, id); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof MetaIdMapping)) { + return false; + } + return Objects.equals(this.opType, ((MetaIdMapping) obj).opType) + && Objects.equals(this.metaObjType, ((MetaIdMapping) obj).metaObjType) + && Objects.equals(this.dbName, ((MetaIdMapping) obj).dbName) + && Objects.equals(this.tblName, ((MetaIdMapping) obj).tblName) + && Objects.equals(this.partitionName, ((MetaIdMapping) obj).partitionName) + && Objects.equals(this.id, ((MetaIdMapping) obj).id); + } + + } + + public enum OperationType { + IGNORE(OPERATION_TYPE_IGNORE), + // Add a Database/Table/Partition + ADD(OPERATION_TYPE_ADD), + // Delete Database/Table/Partition + DELETE(OPERATION_TYPE_DELETE); + + private final short opType; + + OperationType(short opType) { + this.opType = opType; + } + + public short getOperationType() { + return opType; + } + } + + public enum MetaObjectType { + IGNORE(META_OBJECT_TYPE_IGNORE), + DATABASE(META_OBJECT_TYPE_DATABASE), + TABLE(META_OBJECT_TYPE_TABLE), + PARTITION(META_OBJECT_TYPE_PARTITION); + + private final short metaObjType; + + MetaObjectType(short metaObjType) { + this.metaObjType = metaObjType; + } + + public short getMetaObjectType() { + return metaObjType; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AddPartitionEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AddPartitionEvent.java index e1dacbd0b295d0..ffc7b95ff59aad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AddPartitionEvent.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AddPartitionEvent.java @@ -20,8 +20,11 @@ import org.apache.doris.catalog.Env; import org.apache.doris.common.DdlException; +import org.apache.doris.datasource.ExternalMetaIdMgr; +import org.apache.doris.datasource.MetaIdMappingsLog; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import org.apache.hadoop.hive.common.FileUtils; @@ -109,4 +112,17 @@ protected void process() throws MetastoreNotificationException { debugString("Failed to process event"), e); } } + + @Override + protected List transferToMetaIdMappings() { + List metaIdMappings = Lists.newArrayList(); + for (String partitionName : this.getAllPartitionNames()) { + MetaIdMappingsLog.MetaIdMapping metaIdMapping = new MetaIdMappingsLog.MetaIdMapping( + MetaIdMappingsLog.OPERATION_TYPE_ADD, + MetaIdMappingsLog.META_OBJECT_TYPE_PARTITION, + dbName, tblName, partitionName, ExternalMetaIdMgr.nextMetaId()); + metaIdMappings.add(metaIdMapping); + } + return ImmutableList.copyOf(metaIdMappings); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AlterTableEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AlterTableEvent.java index 6de71fbbc592a0..c69812a22b30db 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AlterTableEvent.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AlterTableEvent.java @@ -100,7 +100,8 @@ private void processRecreateTable() throws DdlException { Env.getCurrentEnv().getCatalogMgr() .dropExternalTable(tableBefore.getDbName(), tableBefore.getTableName(), catalogName, true); Env.getCurrentEnv().getCatalogMgr() - .createExternalTableFromEvent(tableAfter.getDbName(), tableAfter.getTableName(), catalogName, true); + .createExternalTableFromEvent( + tableAfter.getDbName(), tableAfter.getTableName(), catalogName, eventTime, true); } private void processRename() throws DdlException { @@ -118,7 +119,8 @@ private void processRename() throws DdlException { Env.getCurrentEnv().getCatalogMgr() .dropExternalTable(tableBefore.getDbName(), tableBefore.getTableName(), catalogName, true); Env.getCurrentEnv().getCatalogMgr() - .createExternalTableFromEvent(tableAfter.getDbName(), tableAfter.getTableName(), catalogName, true); + .createExternalTableFromEvent( + tableAfter.getDbName(), tableAfter.getTableName(), catalogName, eventTime, true); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/CreateDatabaseEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/CreateDatabaseEvent.java index 42d813319cc5f1..d79d23824ab163 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/CreateDatabaseEvent.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/CreateDatabaseEvent.java @@ -20,8 +20,11 @@ import org.apache.doris.catalog.Env; import org.apache.doris.common.DdlException; +import org.apache.doris.datasource.ExternalMetaIdMgr; +import org.apache.doris.datasource.MetaIdMappingsLog; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import org.apache.hadoop.hive.metastore.api.NotificationEvent; @@ -59,4 +62,13 @@ protected void process() throws MetastoreNotificationException { debugString("Failed to process event"), e); } } + + @Override + protected List transferToMetaIdMappings() { + MetaIdMappingsLog.MetaIdMapping metaIdMapping = new MetaIdMappingsLog.MetaIdMapping( + MetaIdMappingsLog.OPERATION_TYPE_ADD, + MetaIdMappingsLog.META_OBJECT_TYPE_DATABASE, + dbName, ExternalMetaIdMgr.nextMetaId()); + return ImmutableList.of(metaIdMapping); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/CreateTableEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/CreateTableEvent.java index 4c3615fbda8915..246ce8626f4aca 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/CreateTableEvent.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/CreateTableEvent.java @@ -19,8 +19,11 @@ import org.apache.doris.catalog.Env; import org.apache.doris.common.DdlException; +import org.apache.doris.datasource.ExternalMetaIdMgr; +import org.apache.doris.datasource.MetaIdMappingsLog; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import org.apache.hadoop.hive.metastore.api.NotificationEvent; import org.apache.hadoop.hive.metastore.api.Table; @@ -77,10 +80,19 @@ protected void process() throws MetastoreNotificationException { try { infoLog("catalogName:[{}],dbName:[{}],tableName:[{}]", catalogName, dbName, tblName); Env.getCurrentEnv().getCatalogMgr() - .createExternalTableFromEvent(dbName, hmsTbl.getTableName(), catalogName, true); + .createExternalTableFromEvent(dbName, hmsTbl.getTableName(), catalogName, eventTime, true); } catch (DdlException e) { throw new MetastoreNotificationException( debugString("Failed to process event"), e); } } + + @Override + protected List transferToMetaIdMappings() { + MetaIdMappingsLog.MetaIdMapping metaIdMapping = new MetaIdMappingsLog.MetaIdMapping( + MetaIdMappingsLog.OPERATION_TYPE_ADD, + MetaIdMappingsLog.META_OBJECT_TYPE_TABLE, + dbName, tblName, ExternalMetaIdMgr.nextMetaId()); + return ImmutableList.of(metaIdMapping); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropDatabaseEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropDatabaseEvent.java index 3481f832fe0006..ca69e6f14d015a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropDatabaseEvent.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropDatabaseEvent.java @@ -20,8 +20,10 @@ import org.apache.doris.catalog.Env; import org.apache.doris.common.DdlException; +import org.apache.doris.datasource.MetaIdMappingsLog; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import org.apache.hadoop.hive.metastore.api.NotificationEvent; @@ -59,4 +61,13 @@ protected void process() throws MetastoreNotificationException { debugString("Failed to process event"), e); } } + + @Override + protected List transferToMetaIdMappings() { + MetaIdMappingsLog.MetaIdMapping metaIdMapping = new MetaIdMappingsLog.MetaIdMapping( + MetaIdMappingsLog.OPERATION_TYPE_DELETE, + MetaIdMappingsLog.META_OBJECT_TYPE_DATABASE, + dbName); + return ImmutableList.of(metaIdMapping); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropPartitionEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropPartitionEvent.java index f71f44cf5aba0a..dd443010289126 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropPartitionEvent.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropPartitionEvent.java @@ -20,8 +20,10 @@ import org.apache.doris.catalog.Env; import org.apache.doris.common.DdlException; +import org.apache.doris.datasource.MetaIdMappingsLog; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import org.apache.hadoop.hive.metastore.api.FieldSchema; @@ -124,7 +126,7 @@ protected boolean canBeBatched(MetastoreEvent that) { return false; } - // `that` event can be batched if this event's partitions contains all of the partitions which `that` event has + // `that` event can be batched if this event's partitions contains all the partitions which `that` event has // else just remove `that` event's relevant partitions for (String partitionName : getAllPartitionNames()) { if (thatPartitionEvent instanceof AddPartitionEvent) { @@ -136,4 +138,17 @@ protected boolean canBeBatched(MetastoreEvent that) { return getAllPartitionNames().containsAll(thatPartitionEvent.getAllPartitionNames()); } + + @Override + protected List transferToMetaIdMappings() { + List metaIdMappings = Lists.newArrayList(); + for (String partitionName : this.getAllPartitionNames()) { + MetaIdMappingsLog.MetaIdMapping metaIdMapping = new MetaIdMappingsLog.MetaIdMapping( + MetaIdMappingsLog.OPERATION_TYPE_DELETE, + MetaIdMappingsLog.META_OBJECT_TYPE_DATABASE, + dbName, tblName, partitionName); + metaIdMappings.add(metaIdMapping); + } + return ImmutableList.copyOf(metaIdMappings); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropTableEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropTableEvent.java index c333506cad2603..0f62e2460820ad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropTableEvent.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropTableEvent.java @@ -20,8 +20,10 @@ import org.apache.doris.catalog.Env; import org.apache.doris.common.DdlException; +import org.apache.doris.datasource.MetaIdMappingsLog; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import org.apache.hadoop.hive.metastore.api.NotificationEvent; import org.apache.hadoop.hive.metastore.messaging.json.JSONDropTableMessage; @@ -89,14 +91,23 @@ protected boolean canBeBatched(MetastoreEvent that) { return false; } - /** + /* * Check if `that` event is a rename event, a rename event can not be batched * because the process of `that` event will change the reference relation of this table, * otherwise it can be batched because this event is a drop-table event * and the process of this event will drop the whole table, * and `that` event must be a MetastoreTableEvent event otherwise `isSameTable` will return false - * */ + */ MetastoreTableEvent thatTblEvent = (MetastoreTableEvent) that; return !thatTblEvent.willChangeTableName(); } + + @Override + protected List transferToMetaIdMappings() { + MetaIdMappingsLog.MetaIdMapping metaIdMapping = new MetaIdMappingsLog.MetaIdMapping( + MetaIdMappingsLog.OPERATION_TYPE_DELETE, + MetaIdMappingsLog.META_OBJECT_TYPE_TABLE, + dbName, tblName); + return ImmutableList.of(metaIdMapping); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEvent.java index f9771562ed4bb2..a9d165b4d03384 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEvent.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEvent.java @@ -17,8 +17,10 @@ package org.apache.doris.datasource.hive.event; +import org.apache.doris.datasource.MetaIdMappingsLog; import org.apache.doris.datasource.hive.HMSCachedClient; +import com.google.common.collect.ImmutableList; import org.apache.hadoop.hive.metastore.api.NotificationEvent; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -227,6 +229,13 @@ protected String getPartitionName(Map part, List partiti return name.toString(); } + /** + * Create a MetaIdMapping list from the event if the event is a create/add/drop event + */ + protected List transferToMetaIdMappings() { + return ImmutableList.of(); + } + @Override public String toString() { return String.format(STR_FORMAT_EVENT_ID_TYPE, eventId, eventType); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventFactory.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventFactory.java index aabc562dba126c..a3ba092703b1bd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventFactory.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventFactory.java @@ -18,7 +18,9 @@ package org.apache.doris.datasource.hive.event; +import org.apache.doris.catalog.Env; import org.apache.doris.datasource.HMSExternalCatalog; +import org.apache.doris.datasource.MetaIdMappingsLog; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; @@ -77,23 +79,39 @@ List getMetastoreEvents(List events, HMSExter for (NotificationEvent event : events) { metastoreEvents.addAll(transferNotificationEventToMetastoreEvents(event, catalogName)); } - return createBatchEvents(catalogName, metastoreEvents); + List mergedEvents = mergeEvents(catalogName, metastoreEvents); + if (Env.getCurrentEnv().isMaster()) { + logMetaIdMappings(hmsExternalCatalog.getId(), events.get(events.size() - 1).getEventId(), mergedEvents); + } + return mergedEvents; + } + + private void logMetaIdMappings(long catalogId, long lastSyncedEventId, List mergedEvents) { + MetaIdMappingsLog log = new MetaIdMappingsLog(); + log.setCatalogId(catalogId); + log.setFromHmsEvent(true); + log.setLastSyncedEventId(lastSyncedEventId); + for (MetastoreEvent event : mergedEvents) { + log.addMetaIdMappings(event.transferToMetaIdMappings()); + } + Env.getCurrentEnv().getExternalMetaIdMgr().replayMetaIdMappingsLog(log); + Env.getCurrentEnv().getEditLog().logMetaIdMappingsLog(log); } /** * Merge events to reduce the cost time on event processing, currently mainly handles MetastoreTableEvent * because merge MetastoreTableEvent is simple and cost-effective. * For example, consider there are some events as following: - * + *
      *    event1: alter table db1.t1 add partition p1;
      *    event2: alter table db1.t1 drop partition p2;
      *    event3: alter table db1.t2 add partition p3;
      *    event4: alter table db2.t3 rename to t4;
      *    event5: drop table db1.t1;
-     *
+     * 
* Only `event3 event4 event5` will be reserved and other events will be skipped. * */ - public List createBatchEvents(String catalogName, List events) { + public List mergeEvents(String catalogName, List events) { List eventsCopy = Lists.newArrayList(events); Map> indexMap = Maps.newLinkedHashMap(); for (int i = 0; i < events.size(); i++) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java index 622d84428fa77a..28793aecf21c31 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java @@ -18,13 +18,22 @@ package org.apache.doris.datasource.hive.event; +import org.apache.doris.analysis.RedirectStatus; import org.apache.doris.catalog.Env; import org.apache.doris.common.Config; import org.apache.doris.common.util.MasterDaemon; import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.CatalogLog; import org.apache.doris.datasource.HMSClientException; import org.apache.doris.datasource.HMSExternalCatalog; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.MasterOpExecutor; +import org.apache.doris.qe.OriginStatement; +import com.google.common.collect.Maps; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.NotificationEvent; import org.apache.hadoop.hive.metastore.api.NotificationEventResponse; @@ -35,6 +44,7 @@ import java.util.Collections; import java.util.List; +import java.util.Map; /** * A metastore event is a instance of the class @@ -68,6 +78,13 @@ public class MetastoreEventsProcessor extends MasterDaemon { // event factory which is used to get or create MetastoreEvents private final MetastoreEventFactory metastoreEventFactory; + // manager the lastSyncedEventId of hms catalogs + // use HashMap is fine because all operations are in one thread + private final Map lastSyncedEventIdMap = Maps.newHashMap(); + + // manager the masterLastSyncedEventId of hms catalogs + private final Map masterLastSyncedEventIdMap = Maps.newHashMap(); + private boolean isRunning; public MetastoreEventsProcessor() { @@ -76,13 +93,56 @@ public MetastoreEventsProcessor() { this.isRunning = false; } + @Override + protected void runAfterCatalogReady() { + if (isRunning) { + LOG.warn("Last task not finished,ignore current task."); + return; + } + isRunning = true; + try { + realRun(); + } catch (Exception ex) { + LOG.warn("Task failed", ex); + } + isRunning = false; + } + + private void realRun() { + List catalogIds = Env.getCurrentEnv().getCatalogMgr().getCatalogIds(); + for (Long catalogId : catalogIds) { + CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(catalogId); + if (catalog instanceof HMSExternalCatalog) { + HMSExternalCatalog hmsExternalCatalog = (HMSExternalCatalog) catalog; + try { + List events = getNextHMSEvents(hmsExternalCatalog); + if (!events.isEmpty()) { + LOG.info("Events size are {} on catalog [{}]", events.size(), + hmsExternalCatalog.getName()); + processEvents(events, hmsExternalCatalog); + } + } catch (MetastoreNotificationFetchException e) { + LOG.warn("Failed to fetch hms events on {}. msg: ", hmsExternalCatalog.getName(), e); + } catch (Exception ex) { + LOG.warn("Failed to process hive metastore [{}] events .", + hmsExternalCatalog.getName(), ex); + } + } + } + } + /** * Fetch the next batch of NotificationEvents from metastore. The default batch size is * {@link Config#hms_events_batch_size_per_rpc} */ - private List getNextHMSEvents(HMSExternalCatalog hmsExternalCatalog) { + private List getNextHMSEvents(HMSExternalCatalog hmsExternalCatalog) throws Exception { LOG.debug("Start to pull events on catalog [{}]", hmsExternalCatalog.getName()); - NotificationEventResponse response = hmsExternalCatalog.getNextEventResponse(hmsExternalCatalog); + NotificationEventResponse response; + if (Env.getCurrentEnv().isMaster()) { + response = getNextEventResponseForMaster(hmsExternalCatalog); + } else { + response = getNextEventResponseForSlave(hmsExternalCatalog); + } if (response == null) { return Collections.emptyList(); @@ -99,11 +159,11 @@ private void doExecute(List events, HMSExternalCatalog hmsExtern && hmsClientException.getCause() instanceof NoSuchObjectException) { LOG.warn(event.debugString("Failed to process event and skip"), hmsClientException); } else { - hmsExternalCatalog.setLastSyncedEventId(event.getEventId() - 1); + updateLastSyncedEventId(hmsExternalCatalog, event.getEventId() - 1); throw hmsClientException; } } catch (Exception e) { - hmsExternalCatalog.setLastSyncedEventId(event.getEventId() - 1); + updateLastSyncedEventId(hmsExternalCatalog, event.getEventId() - 1); throw e; } } @@ -116,45 +176,142 @@ private void processEvents(List events, HMSExternalCatalog hm //transfer List metastoreEvents = metastoreEventFactory.getMetastoreEvents(events, hmsExternalCatalog); doExecute(metastoreEvents, hmsExternalCatalog); - hmsExternalCatalog.setLastSyncedEventId(events.get(events.size() - 1).getEventId()); + updateLastSyncedEventId(hmsExternalCatalog, events.get(events.size() - 1).getEventId()); } - @Override - protected void runAfterCatalogReady() { - if (isRunning) { - LOG.warn("Last task not finished,ignore current task."); - return; + private NotificationEventResponse getNextEventResponseForMaster(HMSExternalCatalog hmsExternalCatalog) + throws MetastoreNotificationFetchException { + long lastSyncedEventId = getLastSyncedEventId(hmsExternalCatalog); + long currentEventId = getCurrentHmsEventId(hmsExternalCatalog); + if (lastSyncedEventId < 0) { + refreshCatalogForMaster(hmsExternalCatalog); + // invoke getCurrentEventId() and save the event id before refresh catalog to avoid missing events + // but set lastSyncedEventId to currentEventId only if there is not any problems when refreshing catalog + updateLastSyncedEventId(hmsExternalCatalog, currentEventId); + LOG.info( + "First pulling events on catalog [{}],refreshCatalog and init lastSyncedEventId," + + "lastSyncedEventId is [{}]", + hmsExternalCatalog.getName(), lastSyncedEventId); + return null; } - isRunning = true; + + LOG.debug("Catalog [{}] getNextEventResponse, currentEventId is {}, lastSyncedEventId is {}", + hmsExternalCatalog.getName(), currentEventId, lastSyncedEventId); + if (currentEventId == lastSyncedEventId) { + LOG.info("Event id not updated when pulling events on catalog [{}]", hmsExternalCatalog.getName()); + return null; + } + try { - realRun(); - } catch (Exception ex) { - LOG.warn("Task failed", ex); + return hmsExternalCatalog.getClient().getNextNotification(lastSyncedEventId, + Config.hms_events_batch_size_per_rpc, null); + } catch (MetastoreNotificationFetchException e) { + // Need a fallback to handle this because this error state can not be recovered until restarting FE + if (StringUtils.isNotEmpty(e.getMessage()) + && e.getMessage().contains(HiveMetaStoreClient.REPL_EVENTS_MISSING_IN_METASTORE)) { + refreshCatalogForMaster(hmsExternalCatalog); + // set lastSyncedEventId to currentEventId after refresh catalog successfully + updateLastSyncedEventId(hmsExternalCatalog, currentEventId); + LOG.warn("Notification events are missing, maybe an event can not be handled " + + "or processing rate is too low, fallback to refresh the catalog"); + return null; + } + throw e; } - isRunning = false; } - private void realRun() { - List catalogIds = Env.getCurrentEnv().getCatalogMgr().getCatalogIds(); - for (Long catalogId : catalogIds) { - CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(catalogId); - if (catalog instanceof HMSExternalCatalog) { - HMSExternalCatalog hmsExternalCatalog = (HMSExternalCatalog) catalog; - try { - List events = getNextHMSEvents(hmsExternalCatalog); - if (!events.isEmpty()) { - LOG.info("Events size are {} on catalog [{}]", events.size(), - hmsExternalCatalog.getName()); - processEvents(events, hmsExternalCatalog); - } - } catch (MetastoreNotificationFetchException e) { - LOG.warn("Failed to fetch hms events on {}. msg: ", hmsExternalCatalog.getName(), e); - } catch (Exception ex) { - LOG.warn("Failed to process hive metastore [{}] events .", - hmsExternalCatalog.getName(), ex); - } + private NotificationEventResponse getNextEventResponseForSlave(HMSExternalCatalog hmsExternalCatalog) + throws Exception { + long lastSyncedEventId = getLastSyncedEventId(hmsExternalCatalog); + long masterLastSyncedEventId = getMasterLastSyncedEventId(hmsExternalCatalog); + // do nothing if masterLastSyncedEventId has not been synced + if (masterLastSyncedEventId == -1L) { + LOG.info("LastSyncedEventId of master has not been synced on catalog [{}]", hmsExternalCatalog.getName()); + return null; + } + // do nothing if lastSyncedEventId is equals to masterLastSyncedEventId + if (lastSyncedEventId == masterLastSyncedEventId) { + LOG.info("Event id not updated when pulling events on catalog [{}]", hmsExternalCatalog.getName()); + return null; + } + + if (lastSyncedEventId < 0) { + refreshCatalogForSlave(hmsExternalCatalog); + // Use masterLastSyncedEventId to avoid missing events + updateLastSyncedEventId(hmsExternalCatalog, masterLastSyncedEventId); + LOG.info( + "First pulling events on catalog [{}],refreshCatalog and init lastSyncedEventId," + + "lastSyncedEventId is [{}]", + hmsExternalCatalog.getName(), lastSyncedEventId); + return null; + } + + LOG.debug("Catalog [{}] getNextEventResponse, masterLastSyncedEventId is {}, lastSyncedEventId is {}", + hmsExternalCatalog.getName(), masterLastSyncedEventId, lastSyncedEventId); + + // For slave FE nodes, only fetch events which id is lower than masterLastSyncedEventId + int maxEventSize = Math.min((int) (masterLastSyncedEventId - lastSyncedEventId), + Config.hms_events_batch_size_per_rpc); + try { + return hmsExternalCatalog.getClient().getNextNotification(lastSyncedEventId, maxEventSize, null); + } catch (MetastoreNotificationFetchException e) { + // Need a fallback to handle this because this error state can not be recovered until restarting FE + if (StringUtils.isNotEmpty(e.getMessage()) + && e.getMessage().contains(HiveMetaStoreClient.REPL_EVENTS_MISSING_IN_METASTORE)) { + refreshCatalogForMaster(hmsExternalCatalog); + // set masterLastSyncedEventId to lastSyncedEventId after refresh catalog successfully + updateLastSyncedEventId(hmsExternalCatalog, masterLastSyncedEventId); + LOG.warn("Notification events are missing, maybe an event can not be handled " + + "or processing rate is too low, fallback to refresh the catalog"); + return null; } + throw e; + } + } + + private long getCurrentHmsEventId(HMSExternalCatalog hmsExternalCatalog) { + CurrentNotificationEventId currentNotificationEventId = hmsExternalCatalog.getClient() + .getCurrentNotificationEventId(); + if (currentNotificationEventId == null) { + LOG.warn("Get currentNotificationEventId is null"); + return -1L; } + return currentNotificationEventId.getEventId(); + } + + private long getLastSyncedEventId(HMSExternalCatalog hmsExternalCatalog) { + // Returns to -1 if not exists, otherwise client.getNextNotification will throw exception + // Reference to https://github.com/apDdlache/doris/issues/18251 + return lastSyncedEventIdMap.getOrDefault(hmsExternalCatalog.getId(), -1L); + } + + private void updateLastSyncedEventId(HMSExternalCatalog hmsExternalCatalog, long eventId) { + lastSyncedEventIdMap.put(hmsExternalCatalog.getId(), eventId); + } + + private long getMasterLastSyncedEventId(HMSExternalCatalog hmsExternalCatalog) { + return masterLastSyncedEventIdMap.getOrDefault(hmsExternalCatalog.getId(), -1L); + } + + public void updateMasterLastSyncedEventId(HMSExternalCatalog hmsExternalCatalog, long eventId) { + masterLastSyncedEventIdMap.put(hmsExternalCatalog.getId(), eventId); + } + + private void refreshCatalogForMaster(HMSExternalCatalog hmsExternalCatalog) { + CatalogLog log = new CatalogLog(); + log.setCatalogId(hmsExternalCatalog.getId()); + log.setInvalidCache(true); + Env.getCurrentEnv().getCatalogMgr().replayRefreshCatalog(log); + } + + private void refreshCatalogForSlave(HMSExternalCatalog hmsExternalCatalog) throws Exception { + // Transfer to master to refresh catalog + String sql = "REFRESH CATALOG " + hmsExternalCatalog.getName(); + OriginStatement originStmt = new OriginStatement(sql, 0); + MasterOpExecutor masterOpExecutor = new MasterOpExecutor(originStmt, new ConnectContext(), + RedirectStatus.FORWARD_WITH_SYNC, false); + LOG.debug("Transfer to master to refresh catalog, stmt: {}", sql); + masterOpExecutor.execute(); } public static MessageDeserializer getMessageDeserializer(String messageFormat) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java index 1bedda0a7e9d7e..d74c519407f303 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java +++ b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java @@ -42,7 +42,7 @@ import org.apache.doris.datasource.ExternalObjectLog; import org.apache.doris.datasource.InitCatalogLog; import org.apache.doris.datasource.InitDatabaseLog; -import org.apache.doris.datasource.InitTableLog; +import org.apache.doris.datasource.MetaIdMappingsLog; import org.apache.doris.ha.MasterInfo; import org.apache.doris.insertoverwrite.InsertOverwriteLog; import org.apache.doris.job.base.AbstractJob; @@ -746,19 +746,18 @@ public void readFields(DataInput in) throws IOException { isRead = true; break; } - case OperationType.OP_INIT_EXTERNAL_TABLE: { - data = InitTableLog.read(in); - isRead = true; - break; - } - case OperationType.OP_REFRESH_EXTERNAL_DB: + case OperationType.OP_INIT_EXTERNAL_TABLE: case OperationType.OP_DROP_EXTERNAL_TABLE: case OperationType.OP_CREATE_EXTERNAL_TABLE: case OperationType.OP_DROP_EXTERNAL_DB: case OperationType.OP_CREATE_EXTERNAL_DB: case OperationType.OP_ADD_EXTERNAL_PARTITIONS: case OperationType.OP_DROP_EXTERNAL_PARTITIONS: - case OperationType.OP_REFRESH_EXTERNAL_PARTITIONS: + case OperationType.OP_REFRESH_EXTERNAL_PARTITIONS: { + isRead = true; + break; + } + case OperationType.OP_REFRESH_EXTERNAL_DB: case OperationType.OP_REFRESH_EXTERNAL_TABLE: { data = ExternalObjectLog.read(in); isRead = true; @@ -906,6 +905,11 @@ public void readFields(DataInput in) throws IOException { isRead = true; break; } + case OperationType.OP_ADD_META_ID_MAPPINGS: { + data = MetaIdMappingsLog.read(in); + isRead = true; + break; + } default: { IOException e = new IOException(); LOG.error("UNKNOWN Operation Type {}", opCode, e); diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index 8c2424d48373ea..958277dd6b7492 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -51,6 +51,7 @@ import org.apache.doris.datasource.ExternalObjectLog; import org.apache.doris.datasource.InitCatalogLog; import org.apache.doris.datasource.InitDatabaseLog; +import org.apache.doris.datasource.MetaIdMappingsLog; import org.apache.doris.ha.MasterInfo; import org.apache.doris.insertoverwrite.InsertOverwriteLog; import org.apache.doris.job.base.AbstractJob; @@ -1011,38 +1012,24 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { break; } case OperationType.OP_DROP_EXTERNAL_TABLE: { - final ExternalObjectLog log = (ExternalObjectLog) journal.getData(); - env.getCatalogMgr().replayDropExternalTable(log); break; } case OperationType.OP_CREATE_EXTERNAL_TABLE: { - final ExternalObjectLog log = (ExternalObjectLog) journal.getData(); - env.getCatalogMgr().replayCreateExternalTableFromEvent(log); break; } case OperationType.OP_DROP_EXTERNAL_DB: { - final ExternalObjectLog log = (ExternalObjectLog) journal.getData(); - env.getCatalogMgr().replayDropExternalDatabase(log); break; } case OperationType.OP_CREATE_EXTERNAL_DB: { - final ExternalObjectLog log = (ExternalObjectLog) journal.getData(); - env.getCatalogMgr().replayCreateExternalDatabase(log); break; } case OperationType.OP_ADD_EXTERNAL_PARTITIONS: { - final ExternalObjectLog log = (ExternalObjectLog) journal.getData(); - env.getCatalogMgr().replayAddExternalPartitions(log); break; } case OperationType.OP_DROP_EXTERNAL_PARTITIONS: { - final ExternalObjectLog log = (ExternalObjectLog) journal.getData(); - env.getCatalogMgr().replayDropExternalPartitions(log); break; } case OperationType.OP_REFRESH_EXTERNAL_PARTITIONS: { - final ExternalObjectLog log = (ExternalObjectLog) journal.getData(); - env.getCatalogMgr().replayRefreshExternalPartitions(log); break; } case OperationType.OP_CREATE_WORKLOAD_GROUP: { @@ -1165,6 +1152,10 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { env.getBackupHandler().getRepoMgr().alterRepo(repository, true); break; } + case OperationType.OP_ADD_META_ID_MAPPINGS: { + env.getExternalMetaIdMgr().replayMetaIdMappingsLog((MetaIdMappingsLog) journal.getData()); + break; + } case OperationType.OP_LOG_UPDATE_ROWS: case OperationType.OP_LOG_NEW_PARTITION_LOADED: case OperationType.OP_LOG_ALTER_COLUMN_STATS: { @@ -1886,26 +1877,32 @@ public void logRefreshExternalTable(ExternalObjectLog log) { logEdit(OperationType.OP_REFRESH_EXTERNAL_TABLE, log); } + @Deprecated public void logDropExternalTable(ExternalObjectLog log) { logEdit(OperationType.OP_DROP_EXTERNAL_TABLE, log); } + @Deprecated public void logCreateExternalTable(ExternalObjectLog log) { logEdit(OperationType.OP_CREATE_EXTERNAL_TABLE, log); } + @Deprecated public void logDropExternalDatabase(ExternalObjectLog log) { logEdit(OperationType.OP_DROP_EXTERNAL_DB, log); } + @Deprecated public void logCreateExternalDatabase(ExternalObjectLog log) { logEdit(OperationType.OP_CREATE_EXTERNAL_DB, log); } + @Deprecated public void logAddExternalPartitions(ExternalObjectLog log) { logEdit(OperationType.OP_ADD_EXTERNAL_PARTITIONS, log); } + @Deprecated public void logDropExternalPartitions(ExternalObjectLog log) { logEdit(OperationType.OP_DROP_EXTERNAL_PARTITIONS, log); } @@ -2005,6 +2002,10 @@ public void logInsertOverwrite(InsertOverwriteLog log) { logEdit(OperationType.OP_INSERT_OVERWRITE, log); } + public void logMetaIdMappingsLog(MetaIdMappingsLog log) { + logEdit(OperationType.OP_ADD_META_ID_MAPPINGS, log); + } + public String getNotReadyReason() { if (journal == null) { return "journal is null"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java index 0945dc0f1510e3..0868d7f371bd3e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java @@ -288,12 +288,19 @@ public class OperationType { public static final short OP_ADD_CONSTRAINT = 346; public static final short OP_DROP_CONSTRAINT = 347; + @Deprecated public static final short OP_DROP_EXTERNAL_TABLE = 350; + @Deprecated public static final short OP_DROP_EXTERNAL_DB = 351; + @Deprecated public static final short OP_CREATE_EXTERNAL_TABLE = 352; + @Deprecated public static final short OP_CREATE_EXTERNAL_DB = 353; + @Deprecated public static final short OP_ADD_EXTERNAL_PARTITIONS = 354; + @Deprecated public static final short OP_DROP_EXTERNAL_PARTITIONS = 355; + @Deprecated public static final short OP_REFRESH_EXTERNAL_PARTITIONS = 356; public static final short OP_ALTER_USER = 400; @@ -364,6 +371,8 @@ public class OperationType { public static final short OP_LOG_ALTER_COLUMN_STATS = 464; + public static final short OP_ADD_META_ID_MAPPINGS = 470; + /** * Get opcode name by op code. **/ diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/ExternalMetaIdMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/ExternalMetaIdMgrTest.java new file mode 100644 index 00000000000000..12e018a4cffaaf --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/ExternalMetaIdMgrTest.java @@ -0,0 +1,76 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.datasource; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class ExternalMetaIdMgrTest { + + @Test + public void testReplayMetaIdMappingsLog() { + ExternalMetaIdMgr mgr = new ExternalMetaIdMgr(); + MetaIdMappingsLog log1 = new MetaIdMappingsLog(); + log1.setCatalogId(1L); + log1.setFromHmsEvent(false); + log1.addMetaIdMapping(new MetaIdMappingsLog.MetaIdMapping( + MetaIdMappingsLog.OPERATION_TYPE_ADD, + MetaIdMappingsLog.META_OBJECT_TYPE_DATABASE, + "db1", ExternalMetaIdMgr.nextMetaId())); + mgr.replayMetaIdMappingsLog(log1); + Assertions.assertNotEquals(-1L, mgr.getDbId(1L, "db1")); + + MetaIdMappingsLog log2 = new MetaIdMappingsLog(); + log2.setCatalogId(1L); + log2.setFromHmsEvent(false); + log2.addMetaIdMapping(new MetaIdMappingsLog.MetaIdMapping( + MetaIdMappingsLog.OPERATION_TYPE_DELETE, + MetaIdMappingsLog.META_OBJECT_TYPE_DATABASE, + "db1")); + mgr.replayMetaIdMappingsLog(log2); + Assertions.assertEquals(-1L, mgr.getDbId(1L, "db1")); + + MetaIdMappingsLog log3 = new MetaIdMappingsLog(); + log3.setCatalogId(1L); + log3.setFromHmsEvent(false); + log3.addMetaIdMapping(new MetaIdMappingsLog.MetaIdMapping( + MetaIdMappingsLog.OPERATION_TYPE_ADD, + MetaIdMappingsLog.META_OBJECT_TYPE_TABLE, + "db1", "tbl1", ExternalMetaIdMgr.nextMetaId())); + mgr.replayMetaIdMappingsLog(log3); + Assertions.assertEquals(-1L, mgr.getDbId(1L, "db1")); + Assertions.assertNotEquals(-1L, mgr.getTblId(1L, "db1", "tbl1")); + + MetaIdMappingsLog log4 = new MetaIdMappingsLog(); + log4.setCatalogId(1L); + log4.setFromHmsEvent(false); + log4.addMetaIdMapping(new MetaIdMappingsLog.MetaIdMapping( + MetaIdMappingsLog.OPERATION_TYPE_DELETE, + MetaIdMappingsLog.META_OBJECT_TYPE_TABLE, + "db1", "tbl1")); + log4.addMetaIdMapping(new MetaIdMappingsLog.MetaIdMapping( + MetaIdMappingsLog.OPERATION_TYPE_ADD, + MetaIdMappingsLog.META_OBJECT_TYPE_PARTITION, + "db1", "tbl1", "p1", ExternalMetaIdMgr.nextMetaId())); + mgr.replayMetaIdMappingsLog(log4); + Assertions.assertEquals(-1L, mgr.getDbId(1L, "db1")); + Assertions.assertEquals(-1L, mgr.getTblId(1L, "db1", "tbl1")); + Assertions.assertNotEquals(-1L, mgr.getPartitionId(1L, "db1", "tbl1", "p1")); + } + +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/MetaIdMappingsLogTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/MetaIdMappingsLogTest.java new file mode 100644 index 00000000000000..fec57c29eda880 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/MetaIdMappingsLogTest.java @@ -0,0 +1,97 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.datasource; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; + +public class MetaIdMappingsLogTest { + + @Test + public void testSerialization() throws Exception { + // 1. Write objects to file + MetaIdMappingsLog log1 = new MetaIdMappingsLog(); + Path path = Files.createFile(Paths.get("./metaIdMappingsLogTest.txt")); + try (DataOutputStream dos = new DataOutputStream(Files.newOutputStream(path))) { + log1.setFromHmsEvent(true); + log1.setLastSyncedEventId(-1L); + log1.setCatalogId(1L); + log1.addMetaIdMapping(new MetaIdMappingsLog.MetaIdMapping( + MetaIdMappingsLog.OPERATION_TYPE_ADD, + MetaIdMappingsLog.META_OBJECT_TYPE_DATABASE, + "db1", ExternalMetaIdMgr.nextMetaId())); + log1.addMetaIdMapping(new MetaIdMappingsLog.MetaIdMapping( + MetaIdMappingsLog.OPERATION_TYPE_DELETE, + MetaIdMappingsLog.META_OBJECT_TYPE_TABLE, + "db1", "tbl1", ExternalMetaIdMgr.nextMetaId())); + log1.addMetaIdMapping(new MetaIdMappingsLog.MetaIdMapping( + MetaIdMappingsLog.OPERATION_TYPE_DELETE, + MetaIdMappingsLog.META_OBJECT_TYPE_TABLE, + "db1", "tbl2", ExternalMetaIdMgr.nextMetaId())); + log1.addMetaIdMapping(new MetaIdMappingsLog.MetaIdMapping( + MetaIdMappingsLog.OPERATION_TYPE_DELETE, + MetaIdMappingsLog.META_OBJECT_TYPE_DATABASE, + "db2")); + log1.addMetaIdMapping(new MetaIdMappingsLog.MetaIdMapping( + MetaIdMappingsLog.OPERATION_TYPE_ADD, + MetaIdMappingsLog.META_OBJECT_TYPE_PARTITION, + "db1", "tbl1", "p1", ExternalMetaIdMgr.nextMetaId())); + log1.addMetaIdMapping(new MetaIdMappingsLog.MetaIdMapping( + MetaIdMappingsLog.OPERATION_TYPE_ADD, + MetaIdMappingsLog.META_OBJECT_TYPE_PARTITION, + "db1", "tbl1", "p2", ExternalMetaIdMgr.nextMetaId())); + log1.addMetaIdMapping(new MetaIdMappingsLog.MetaIdMapping( + MetaIdMappingsLog.OPERATION_TYPE_DELETE, + MetaIdMappingsLog.META_OBJECT_TYPE_DATABASE, + "db2")); + log1.addMetaIdMapping(new MetaIdMappingsLog.MetaIdMapping( + MetaIdMappingsLog.OPERATION_TYPE_DELETE, + MetaIdMappingsLog.META_OBJECT_TYPE_TABLE, + "db1", "tbl1", ExternalMetaIdMgr.nextMetaId())); + log1.addMetaIdMapping(new MetaIdMappingsLog.MetaIdMapping( + MetaIdMappingsLog.OPERATION_TYPE_DELETE, + MetaIdMappingsLog.META_OBJECT_TYPE_PARTITION, + "db1", "tbl1", "p2", ExternalMetaIdMgr.nextMetaId())); + log1.write(dos); + dos.flush(); + } catch (Throwable throwable) { + throwable.printStackTrace(); + Files.deleteIfExists(path); + Assertions.fail(); + } + + // 2. Read objects from file + MetaIdMappingsLog log2; + try (DataInputStream dis = new DataInputStream(Files.newInputStream(path))) { + log2 = MetaIdMappingsLog.read(dis); + Assertions.assertEquals(log1, log2); + } catch (Throwable throwable) { + throwable.printStackTrace(); + Assertions.fail(); + } finally { + Files.deleteIfExists(path); + } + } + +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/external/hms/MetastoreEventFactoryTest.java b/fe/fe-core/src/test/java/org/apache/doris/external/hms/MetastoreEventFactoryTest.java index c9e566dc9d8d6b..136cac6b71200f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/external/hms/MetastoreEventFactoryTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/external/hms/MetastoreEventFactoryTest.java @@ -454,7 +454,7 @@ public void testCreateBatchEvents() { for (int j = 0; j < 1000; j++) { events.add(producer.produceOneEvent(j)); } - List mergedEvents = factory.createBatchEvents(testCtl, events); + List mergedEvents = factory.mergeEvents(testCtl, events); for (MetastoreEvent event : events) { processEvent(validateCatalog, event); From 6d2c7c08defaaf702c24c6afb6e6257fb9f4ff8c Mon Sep 17 00:00:00 2001 From: zhannngchen <48427519+zhannngchen@users.noreply.github.com> Date: Fri, 19 Jan 2024 15:59:01 +0800 Subject: [PATCH 100/200] [fix](tablet meta) stale rowset is not loaded while BE start (#30077) --- be/src/olap/tablet_meta.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/be/src/olap/tablet_meta.cpp b/be/src/olap/tablet_meta.cpp index f14622081d91f8..64bf7d4c198ced 100644 --- a/be/src/olap/tablet_meta.cpp +++ b/be/src/olap/tablet_meta.cpp @@ -550,6 +550,13 @@ void TabletMeta::init_from_pb(const TabletMetaPB& tablet_meta_pb) { _rs_metas.push_back(std::move(rs_meta)); } + // init _stale_rs_metas + for (auto& it : tablet_meta_pb.stale_rs_metas()) { + RowsetMetaSharedPtr rs_meta(new RowsetMeta()); + rs_meta->init_from_pb(it); + _stale_rs_metas.push_back(std::move(rs_meta)); + } + // For mow table, delete bitmap of stale rowsets has not been persisted. // When be restart, query should not read the stale rowset, otherwise duplicate keys // will be read out. Therefore, we don't add them to _stale_rs_meta for mow table. From 1321c9224b4105c9865c427fc6b024cf2f2fd6e8 Mon Sep 17 00:00:00 2001 From: walter Date: Fri, 19 Jan 2024 16:05:25 +0800 Subject: [PATCH 101/200] [fix](cloud) Fix cloud start.sh (#30135) --- cloud/script/start.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cloud/script/start.sh b/cloud/script/start.sh index d1bfb773675f12..ecd738938f898c 100644 --- a/cloud/script/start.sh +++ b/cloud/script/start.sh @@ -56,7 +56,7 @@ fi lib_path="${DORIS_HOME}/lib" bin="${DORIS_HOME}/lib/doris_cloud" -if ! ldd "${bin}" | grep -Ei 'libfdb_c.*not found' &>/dev/null; then +if ldd "${bin}" | grep -Ei 'libfdb_c.*not found' &>/dev/null; then if ! command -v patchelf &>/dev/null; then echo "patchelf is needed to launch meta_service" exit 1 From ff9c1dfd15bc1dc3db8cd1dae2a7a8f2f41a8d6a Mon Sep 17 00:00:00 2001 From: Gabriel Date: Fri, 19 Jan 2024 16:12:35 +0800 Subject: [PATCH 102/200] [pipelineX](filescan) Support parallel executing for external table scanning (#30121) --- .../java/org/apache/doris/planner/OlapScanNode.java | 5 +++++ .../main/java/org/apache/doris/planner/ScanNode.java | 7 +++++-- .../doris/planner/external/FileQueryScanNode.java | 12 ++++++++++++ 3 files changed, 22 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java index e5381a2018fd84..be2c9a24c91d9f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java @@ -1736,6 +1736,11 @@ public boolean pushDownAggNoGroupingCheckCol(FunctionCallExpr aggExpr, Column co return true; } + + @Override + public int getScanRangeNum() { + return getScanTabletIds().size(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java index 7ddf2ae1d4c043..9881fe6f6e3449 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java @@ -725,8 +725,11 @@ public boolean ignoreStorageDataDistribution(ConnectContext context) { && context.getSessionVariable().isIgnoreStorageDataDistribution() && context.getSessionVariable().getEnablePipelineXEngine() && !fragment.isHasNullAwareLeftAntiJoin() - && ((this instanceof OlapScanNode) && ((OlapScanNode) this).getScanTabletIds().size() - < ConnectContext.get().getSessionVariable().getParallelExecInstanceNum()); + && getScanRangeNum() < ConnectContext.get().getSessionVariable().getParallelExecInstanceNum(); + } + + public int getScanRangeNum() { + return Integer.MAX_VALUE; } public boolean haveLimitAndConjunts() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/FileQueryScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/FileQueryScanNode.java index 71c731498a643e..4a3c3d2ff9ab85 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/FileQueryScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/FileQueryScanNode.java @@ -485,6 +485,18 @@ protected void genSlotToSchemaIdMapForOrc() { params.setSlotNameToSchemaPos(columnNameToPosition); } + @Override + public int getScanRangeNum() { + Preconditions.checkNotNull(scanRangeLocations); + int i = 0; + for (TScanRangeLocations tScanRangeLocations : scanRangeLocations) { + TScanRange tScanRange = tScanRangeLocations.getScanRange(); + TFileScanRange tFileScanRange = tScanRange.getExtScanRange().getFileScanRange(); + i += tFileScanRange.getRangesSize(); + } + return i; + } + protected abstract TFileType getLocationType() throws UserException; protected abstract TFileType getLocationType(String location) throws UserException; From 0fd46fb76511af274004cfd9a26ee4b69f74819b Mon Sep 17 00:00:00 2001 From: yujun Date: Fri, 19 Jan 2024 18:07:04 +0800 Subject: [PATCH 103/200] [feature](merge-cloud) rewrite cloud table property (#30069) --- .../doris/alter/MaterializedViewHandler.java | 2 +- .../doris/alter/SchemaChangeHandler.java | 2 +- .../doris/analysis/CreateTableStmt.java | 4 +- .../org/apache/doris/backup/RestoreJob.java | 2 +- .../catalog/DynamicPartitionProperty.java | 50 +++++---- .../java/org/apache/doris/catalog/Env.java | 9 +- .../org/apache/doris/catalog/EnvFactory.java | 102 +++++++++--------- .../org/apache/doris/catalog/OlapTable.java | 2 +- .../org/apache/doris/catalog/Partition.java | 2 +- .../org/apache/doris/catalog/Replica.java | 2 +- .../doris/catalog/ReplicaAllocation.java | 2 +- .../apache/doris/catalog/TableProperty.java | 5 +- .../java/org/apache/doris/catalog/Tablet.java | 4 +- .../CloudDynamicPartitionProperty.java | 44 ++++++++ .../doris/cloud/catalog/CloudEnvFactory.java | 102 ++++++++++++++++++ .../common/util/CloudPropertyAnalyzer.java | 50 +++++++++ .../datasource/CloudInternalCatalog.java | 2 +- .../cloud/system/CloudSystemInfoService.java | 43 ++++++++ .../doris/common/util/PropertyAnalyzer.java | 45 +++++++- .../apache/doris/datasource/CatalogMgr.java | 2 +- .../doris/datasource/InternalCatalog.java | 2 +- .../plans/commands/info/CreateTableInfo.java | 13 ++- .../doris/analysis/CreateTableStmtTest.java | 12 ++- .../doris/binlog/BinlogManagerTest.java | 2 +- .../apache/doris/catalog/EnvFactoryTest.java | 31 +++--- .../cloud/catalog/CloudEnvFactoryTest.java | 52 +++++++++ .../StatisticsAutoCollectorTest.java | 4 +- 27 files changed, 464 insertions(+), 128 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudDynamicPartitionProperty.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnvFactory.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/cloud/common/util/CloudPropertyAnalyzer.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/cloud/system/CloudSystemInfoService.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/cloud/catalog/CloudEnvFactoryTest.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java index 3f7122f36505e9..347da5f1a497d3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java @@ -398,7 +398,7 @@ private RollupJobV2 createMaterializedViewJob(String rawSql, String mvName, Stri long baseTabletId = baseTablet.getId(); long mvTabletId = idGeneratorBuffer.getNextId(); - Tablet newTablet = EnvFactory.createTablet(mvTabletId); + Tablet newTablet = EnvFactory.getInstance().createTablet(mvTabletId); mvIndex.addTablet(newTablet, mvTabletMeta); addedTablets.add(newTablet); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index b5895ead65360b..dbb9ee1466d0c8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -1548,7 +1548,7 @@ private void createJob(String rawSql, long dbId, OlapTable olapTable, Map properties) { } } + protected boolean supportProperty(String property) { + return true; + } + private ReplicaAllocation analyzeReplicaAllocation(Map properties) { try { return PropertyAnalyzer.analyzeReplicaAllocation(properties, "dynamic_partition"); @@ -217,33 +221,35 @@ public String getSortedReservedHistoryPeriods(String reservedHistoryPeriods, Str * use table replication_num as dynamic_partition.replication_num default value */ public String getProperties(ReplicaAllocation tableReplicaAlloc) { + ReplicaAllocation tmpAlloc = this.replicaAlloc.isNotSet() ? tableReplicaAlloc : this.replicaAlloc; StringBuilder sb = new StringBuilder(); - sb.append(",\n\"" + ENABLE + "\" = \"" + enable + "\""); - sb.append(",\n\"" + TIME_UNIT + "\" = \"" + timeUnit + "\""); - sb.append(",\n\"" + TIME_ZONE + "\" = \"" + tz.getID() + "\""); - sb.append(",\n\"" + START + "\" = \"" + start + "\""); - sb.append(",\n\"" + END + "\" = \"" + end + "\""); - sb.append(",\n\"" + PREFIX + "\" = \"" + prefix + "\""); - if (Config.isNotCloudMode()) { - ReplicaAllocation tmpAlloc = this.replicaAlloc.isNotSet() ? tableReplicaAlloc : this.replicaAlloc; - sb.append(",\n\"" + REPLICATION_ALLOCATION + "\" = \"" + tmpAlloc.toCreateStmt() + "\""); - } - sb.append(",\n\"" + BUCKETS + "\" = \"" + buckets + "\""); - sb.append(",\n\"" + CREATE_HISTORY_PARTITION + "\" = \"" + createHistoryPartition + "\""); - sb.append(",\n\"" + HISTORY_PARTITION_NUM + "\" = \"" + historyPartitionNum + "\""); - sb.append(",\n\"" + HOT_PARTITION_NUM + "\" = \"" + hotPartitionNum + "\""); - sb.append(",\n\"" + RESERVED_HISTORY_PERIODS + "\" = \"" + reservedHistoryPeriods + "\""); - if (Config.isNotCloudMode()) { - sb.append(",\n\"" + STORAGE_POLICY + "\" = \"" + storagePolicy + "\""); - if (!Strings.isNullOrEmpty(storageMedium)) { - sb.append(",\n\"" + STORAGE_MEDIUM + "\" = \"" + storageMedium + "\""); + BiConsumer addProperty = (property, value) -> { + if (supportProperty(property)) { + sb.append(",\n\"" + property + "\" = \"" + value + "\""); } + }; + addProperty.accept(ENABLE, enable); + addProperty.accept(TIME_UNIT, timeUnit); + addProperty.accept(TIME_ZONE, tz.getID()); + addProperty.accept(START, start); + addProperty.accept(END, end); + addProperty.accept(PREFIX, prefix); + addProperty.accept(REPLICATION_ALLOCATION, tmpAlloc.toCreateStmt()); + addProperty.accept(BUCKETS, buckets); + addProperty.accept(CREATE_HISTORY_PARTITION, createHistoryPartition); + addProperty.accept(HISTORY_PARTITION_NUM, historyPartitionNum); + addProperty.accept(HOT_PARTITION_NUM, hotPartitionNum); + addProperty.accept(RESERVED_HISTORY_PERIODS, reservedHistoryPeriods); + addProperty.accept(STORAGE_POLICY, storagePolicy); + if (!Strings.isNullOrEmpty(storageMedium)) { + addProperty.accept(STORAGE_MEDIUM, storageMedium); } if (getTimeUnit().equalsIgnoreCase(TimeUnit.WEEK.toString())) { - sb.append(",\n\"" + START_DAY_OF_WEEK + "\" = \"" + startOfWeek.dayOfWeek + "\""); + addProperty.accept(START_DAY_OF_WEEK, startOfWeek.dayOfWeek); } else if (getTimeUnit().equalsIgnoreCase(TimeUnit.MONTH.toString())) { - sb.append(",\n\"" + START_DAY_OF_MONTH + "\" = \"" + startOfMonth.day + "\""); + addProperty.accept(START_DAY_OF_MONTH, startOfMonth.day); } + return sb.toString(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index 37930708cc8cbf..2c0ff9f692eb7f 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -623,7 +623,7 @@ public BinlogManager getBinlogManager() { } private static class SingletonHolder { - private static final Env INSTANCE = EnvFactory.createEnv(false); + private static final Env INSTANCE = EnvFactory.getInstance().createEnv(false); } private Env() { @@ -673,7 +673,7 @@ public Env(boolean isCheckpointCatalog) { this.journalObservable = new JournalObservable(); this.masterInfo = new MasterInfo(); - this.systemInfo = new SystemInfoService(); + this.systemInfo = EnvFactory.getInstance().createSystemInfoService(); this.heartbeatMgr = new HeartbeatMgr(systemInfo, !isCheckpointCatalog); this.tabletInvertedIndex = new TabletInvertedIndex(); this.colocateTableIndex = new ColocateTableIndex(); @@ -773,7 +773,7 @@ public static Env getCurrentEnv() { // only checkpoint thread it self will goes here. // so no need to care about the thread safe. if (CHECKPOINT == null) { - CHECKPOINT = EnvFactory.createEnv(true); + CHECKPOINT = EnvFactory.getInstance().createEnv(true); } return CHECKPOINT; } else { @@ -3253,7 +3253,6 @@ public static void getDdlStmt(DdlStmt ddlStmt, String dbName, TableIf table, Lis // replicationNum ReplicaAllocation replicaAlloc = olapTable.getDefaultReplicaAllocation(); - if (Config.isCloudMode()) { sb.append("\"").append(PropertyAnalyzer.PROPERTIES_FILE_CACHE_TTL_SECONDS).append("\" = \""); sb.append(olapTable.getTTLSeconds()).append("\""); @@ -3312,7 +3311,7 @@ public static void getDdlStmt(DdlStmt ddlStmt, String dbName, TableIf table, Lis sb.append(olapTable.getDataSortInfo().toSql()); } - if (Config.isCloudMode() && olapTable.getTTLSeconds() != 0) { + if (olapTable.getTTLSeconds() != 0) { sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_FILE_CACHE_TTL_SECONDS).append("\" = \""); sb.append(olapTable.getTTLSeconds()).append("\""); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/EnvFactory.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/EnvFactory.java index c653159d7b6820..9ecee918322141 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/EnvFactory.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/EnvFactory.java @@ -17,79 +17,75 @@ package org.apache.doris.catalog; -import org.apache.doris.cloud.catalog.CloudEnv; -import org.apache.doris.cloud.catalog.CloudPartition; -import org.apache.doris.cloud.catalog.CloudReplica; -import org.apache.doris.cloud.catalog.CloudTablet; -import org.apache.doris.cloud.datasource.CloudInternalCatalog; +import org.apache.doris.cloud.catalog.CloudEnvFactory; import org.apache.doris.common.Config; +import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.system.SystemInfoService; import java.lang.reflect.Type; +import java.util.Map; public class EnvFactory { - public static Env createEnv(boolean isCheckpointCatalog) { - if (Config.isCloudMode()) { - return new CloudEnv(isCheckpointCatalog); - } else { - return new Env(isCheckpointCatalog); - } + public EnvFactory() { } - public static InternalCatalog createInternalCatalog() { - if (Config.isCloudMode()) { - return new CloudInternalCatalog(); - } else { - return new InternalCatalog(); - } + private static class SingletonHolder { + private static final EnvFactory INSTANCE = + Config.isCloudMode() ? new CloudEnvFactory() : new EnvFactory(); } - public static Type getPartitionClass() { - if (Config.isCloudMode()) { - return CloudPartition.class; - } else { - return Partition.class; - } + public static EnvFactory getInstance() { + return SingletonHolder.INSTANCE; } - public static Partition createPartition() { - if (Config.isCloudMode()) { - return new CloudPartition(); - } else { - return new Partition(); - } + public Env createEnv(boolean isCheckpointCatalog) { + return new Env(isCheckpointCatalog); } - public static Type getTabletClass() { - if (Config.isCloudMode()) { - return CloudTablet.class; - } else { - return Tablet.class; - } + public InternalCatalog createInternalCatalog() { + return new InternalCatalog(); } - public static Tablet createTablet() { - if (Config.isCloudMode()) { - return new CloudTablet(); - } else { - return new Tablet(); - } + public SystemInfoService createSystemInfoService() { + return new SystemInfoService(); } - public static Tablet createTablet(long tabletId) { - if (Config.isCloudMode()) { - return new CloudTablet(tabletId); - } else { - return new Tablet(tabletId); - } + public Type getPartitionClass() { + return Partition.class; } - public static Replica createReplica() { - if (Config.isCloudMode()) { - return new CloudReplica(); - } else { - return new Replica(); - } + public Partition createPartition() { + return new Partition(); } + + public Type getTabletClass() { + return Tablet.class; + } + + public Tablet createTablet() { + return new Tablet(); + } + + public Tablet createTablet(long tabletId) { + return new Tablet(tabletId); + } + + public Replica createReplica() { + return new Replica(); + } + + public ReplicaAllocation createDefReplicaAllocation() { + return new ReplicaAllocation((short) 3); + } + + public PropertyAnalyzer createPropertyAnalyzer() { + return new PropertyAnalyzer(); + } + + public DynamicPartitionProperty createDynamicPartitionProperty(Map properties) { + return new DynamicPartitionProperty(properties); + } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index 2c221a45500524..1d06b0305a02b8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -609,7 +609,7 @@ public Status resetIdsForRestore(Env env, Database db, ReplicaAllocation restore idx.clearTabletsForRestore(); for (int i = 0; i < tabletNum; i++) { long newTabletId = env.getNextId(); - Tablet newTablet = EnvFactory.createTablet(newTabletId); + Tablet newTablet = EnvFactory.getInstance().createTablet(newTabletId); idx.addTablet(newTablet, null /* tablet meta */, true /* is restore */); // replicas diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Partition.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Partition.java index cf4ac871c40911..e1da0019c5a681 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Partition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Partition.java @@ -331,7 +331,7 @@ public boolean visualiseShadowIndex(long shadowIndexId, boolean isBaseIndex) { } public static Partition read(DataInput in) throws IOException { - Partition partition = EnvFactory.createPartition(); + Partition partition = EnvFactory.getInstance().createPartition(); partition.readFields(in); return partition; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java index e6b9066af78628..f40396e23b8a03 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java @@ -671,7 +671,7 @@ public void readFields(DataInput in) throws IOException { } public static Replica read(DataInput in) throws IOException { - Replica replica = EnvFactory.createReplica(); + Replica replica = EnvFactory.getInstance().createReplica(); replica.readFields(in); return replica; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/ReplicaAllocation.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/ReplicaAllocation.java index e81864402f5531..ecb536e47d067c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/ReplicaAllocation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/ReplicaAllocation.java @@ -46,7 +46,7 @@ public class ReplicaAllocation implements Writable { public static final ReplicaAllocation NOT_SET; static { - DEFAULT_ALLOCATION = new ReplicaAllocation((short) 3); + DEFAULT_ALLOCATION = EnvFactory.getInstance().createDefReplicaAllocation(); NOT_SET = new ReplicaAllocation(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java index 5e857acf884a96..23a844538a4c72 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java @@ -56,7 +56,8 @@ public class TableProperty implements Writable { private Map properties; // the follower variables are built from "properties" - private DynamicPartitionProperty dynamicPartitionProperty = new DynamicPartitionProperty(Maps.newHashMap()); + private DynamicPartitionProperty dynamicPartitionProperty = + EnvFactory.getInstance().createDynamicPartitionProperty(Maps.newHashMap()); private ReplicaAllocation replicaAlloc = ReplicaAllocation.DEFAULT_ALLOCATION; private boolean isInMemory = false; private short minLoadReplicaNum = -1; @@ -180,7 +181,7 @@ private TableProperty executeBuildDynamicProperty() { dynamicPartitionProperties.put(entry.getKey(), entry.getValue()); } } - dynamicPartitionProperty = new DynamicPartitionProperty(dynamicPartitionProperties); + dynamicPartitionProperty = EnvFactory.getInstance().createDynamicPartitionProperty(dynamicPartitionProperties); return this; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java index badeb992465ce9..984444a4f812f1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java @@ -404,10 +404,10 @@ public void readFields(DataInput in) throws IOException { public static Tablet read(DataInput in) throws IOException { if (Env.getCurrentEnvJournalVersion() >= FeMetaVersion.VERSION_115) { String json = Text.readString(in); - return GsonUtils.GSON.fromJson(json, EnvFactory.getTabletClass()); + return GsonUtils.GSON.fromJson(json, EnvFactory.getInstance().getTabletClass()); } - Tablet tablet = EnvFactory.createTablet(); + Tablet tablet = EnvFactory.getInstance().createTablet(); tablet.readFields(in); return tablet; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudDynamicPartitionProperty.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudDynamicPartitionProperty.java new file mode 100644 index 00000000000000..4f871cf6bac216 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudDynamicPartitionProperty.java @@ -0,0 +1,44 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.cloud.catalog; + +import org.apache.doris.catalog.DynamicPartitionProperty; + +import com.google.common.collect.Sets; + +import java.util.Map; +import java.util.Set; + +public class CloudDynamicPartitionProperty extends DynamicPartitionProperty { + + private static Set unsupportedProperties = Sets.newHashSet( + DynamicPartitionProperty.REPLICATION_NUM, + DynamicPartitionProperty.REPLICATION_ALLOCATION, + DynamicPartitionProperty.STORAGE_POLICY); + + public CloudDynamicPartitionProperty(Map properties) { + super(properties); + } + + @Override + protected boolean supportProperty(String property) { + return !unsupportedProperties.contains(property); + } + +} + diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnvFactory.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnvFactory.java new file mode 100644 index 00000000000000..46b916b9dbb856 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnvFactory.java @@ -0,0 +1,102 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.cloud.catalog; + +import org.apache.doris.catalog.DynamicPartitionProperty; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.EnvFactory; +import org.apache.doris.catalog.Partition; +import org.apache.doris.catalog.Replica; +import org.apache.doris.catalog.ReplicaAllocation; +import org.apache.doris.catalog.Tablet; +import org.apache.doris.cloud.common.util.CloudPropertyAnalyzer; +import org.apache.doris.cloud.datasource.CloudInternalCatalog; +import org.apache.doris.cloud.system.CloudSystemInfoService; +import org.apache.doris.common.util.PropertyAnalyzer; +import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.system.SystemInfoService; + +import java.lang.reflect.Type; +import java.util.Map; + +public class CloudEnvFactory extends EnvFactory { + + public CloudEnvFactory() { + } + + @Override + public Env createEnv(boolean isCheckpointCatalog) { + return new CloudEnv(isCheckpointCatalog); + } + + @Override + public InternalCatalog createInternalCatalog() { + return new CloudInternalCatalog(); + } + + @Override + public SystemInfoService createSystemInfoService() { + return new CloudSystemInfoService(); + } + + @Override + public Type getPartitionClass() { + return CloudPartition.class; + } + + @Override + public Partition createPartition() { + return new CloudPartition(); + } + + @Override + public Type getTabletClass() { + return CloudTablet.class; + } + + @Override + public Tablet createTablet() { + return new CloudTablet(); + } + + @Override + public Tablet createTablet(long tabletId) { + return new CloudTablet(tabletId); + } + + @Override + public Replica createReplica() { + return new CloudReplica(); + } + + @Override + public ReplicaAllocation createDefReplicaAllocation() { + return new ReplicaAllocation((short) 1); + } + + @Override + public PropertyAnalyzer createPropertyAnalyzer() { + return new CloudPropertyAnalyzer(); + } + + @Override + public DynamicPartitionProperty createDynamicPartitionProperty(Map properties) { + return new CloudDynamicPartitionProperty(properties); + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/common/util/CloudPropertyAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/common/util/CloudPropertyAnalyzer.java new file mode 100644 index 00000000000000..0e6f0f7051fa82 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/common/util/CloudPropertyAnalyzer.java @@ -0,0 +1,50 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.cloud.common.util; + +import org.apache.doris.catalog.DynamicPartitionProperty; +import org.apache.doris.catalog.ReplicaAllocation; +import org.apache.doris.common.util.PropertyAnalyzer; + +import com.google.common.collect.ImmutableMap; + +public class CloudPropertyAnalyzer extends PropertyAnalyzer { + + public CloudPropertyAnalyzer() { + forceProperties = ImmutableMap.builder() + .put(PropertyAnalyzer.PROPERTIES_INMEMORY, "true") + //.put(PropertyAnalyzer.PROPERTIES_STORAGE_MEDIUM, null) + .put(PropertyAnalyzer.PROPERTIES_STORAGE_FORMAT, "") + .put(PropertyAnalyzer.PROPERTIES_STORAGE_POLICY, "") + .put(PropertyAnalyzer.PROPERTIES_STORAGE_COOLDOWN_TIME, "") + .put(PropertyAnalyzer.PROPERTIES_MIN_LOAD_REPLICA_NUM, "-1") + .put(PropertyAnalyzer.PROPERTIES_DISABLE_AUTO_COMPACTION, "false") + .put(PropertyAnalyzer.PROPERTIES_ENABLE_LIGHT_SCHEMA_CHANGE, "true") + .put(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM, + String.valueOf(ReplicaAllocation.DEFAULT_ALLOCATION.getTotalReplicaNum())) + .put(PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION, + ReplicaAllocation.DEFAULT_ALLOCATION.toCreateStmt()) + //.put(DynamicPartitionProperty.PROPERTIES_STORAGE_MEDIUM, "") + .put(DynamicPartitionProperty.REPLICATION_NUM, + String.valueOf(ReplicaAllocation.DEFAULT_ALLOCATION.getTotalReplicaNum())) + .put(DynamicPartitionProperty.REPLICATION_ALLOCATION, + ReplicaAllocation.DEFAULT_ALLOCATION.toCreateStmt()) + .build(); + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java index f2e3f10c2800fe..d7efdefe775d0c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java @@ -318,7 +318,7 @@ private void createCloudTablets(MaterializedIndex index, ReplicaState replicaSta DistributionInfo distributionInfo, long version, ReplicaAllocation replicaAlloc, TabletMeta tabletMeta, Set tabletIdSet) throws DdlException { for (int i = 0; i < distributionInfo.getBucketNum(); ++i) { - Tablet tablet = EnvFactory.createTablet(Env.getCurrentEnv().getNextId()); + Tablet tablet = EnvFactory.getInstance().createTablet(Env.getCurrentEnv().getNextId()); // add tablet to inverted index first index.addTablet(tablet, tabletMeta); diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/system/CloudSystemInfoService.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/system/CloudSystemInfoService.java new file mode 100644 index 00000000000000..5ae16c9f35b1f5 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/system/CloudSystemInfoService.java @@ -0,0 +1,43 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.cloud.system; + +import org.apache.doris.catalog.ReplicaAllocation; +import org.apache.doris.common.DdlException; +import org.apache.doris.resource.Tag; +import org.apache.doris.system.SystemInfoService; +import org.apache.doris.thrift.TStorageMedium; + +import com.google.common.collect.Maps; + +import java.util.List; +import java.util.Map; + +public class CloudSystemInfoService extends SystemInfoService { + + @Override + public Map> selectBackendIdsForReplicaCreation( + ReplicaAllocation replicaAlloc, Map nextIndexs, + TStorageMedium storageMedium, boolean isStorageMediumSpecified, + boolean isOnlyForCheck) + throws DdlException { + return Maps.newHashMap(); + } + +} + diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java index f1e457336f2594..afebd26ee06a39 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java @@ -23,6 +23,7 @@ import org.apache.doris.catalog.DataProperty; import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.EnvFactory; import org.apache.doris.catalog.EsResource; import org.apache.doris.catalog.KeysType; import org.apache.doris.catalog.Partition; @@ -48,6 +49,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Strings; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import org.apache.commons.lang3.StringUtils; @@ -194,6 +196,27 @@ public class PropertyAnalyzer { public static final long TIME_SERIES_COMPACTION_TIME_THRESHOLD_SECONDS_DEFAULT_VALUE = 3600; public static final long TIME_SERIES_COMPACTION_EMPTY_ROWSETS_THRESHOLD_DEFAULT_VALUE = 5; + // Use forceProperties to rewrite olap's property. + // For a key-value pair in forceProperties, + // if the value is null, then delete this property from properties and skip check this property, + // otherwise rewrite this property into properties and check property using the force value. + // + // In most cases, specified a none-null force value is better then specified a null force value. + protected ImmutableMap forceProperties; + + public PropertyAnalyzer() { + forceProperties = ImmutableMap.builder() + .put(PROPERTIES_FILE_CACHE_TTL_SECONDS, "0") + .build(); + } + + private static class SingletonHolder { + private static final PropertyAnalyzer INSTANCE = EnvFactory.getInstance().createPropertyAnalyzer(); + } + + public static PropertyAnalyzer getInstance() { + return SingletonHolder.INSTANCE; + } /** * check and replace members of DataProperty by properties. @@ -1285,7 +1308,27 @@ public static void checkCatalogProperties(Map properties, boolea } } - public static Map rewriteReplicaAllocationProperties( + public Map rewriteOlapProperties( + String ctl, String db, Map properties) { + if (properties == null) { + properties = Maps.newHashMap(); + } + rewriteReplicaAllocationProperties(ctl, db, properties); + rewriteForceProperties(properties); + return properties; + } + + private void rewriteForceProperties(Map properties) { + forceProperties.forEach((property, value) -> { + if (value == null) { + properties.remove(property); + } else { + properties.put(property, value); + } + }); + } + + private static Map rewriteReplicaAllocationProperties( String ctl, String db, Map properties) { if (Config.force_olap_table_replication_num <= 0) { return rewriteReplicaAllocationPropertiesByDatabase(ctl, db, properties); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java index 76d7702fef426b..219f29ebec17ef 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java @@ -115,7 +115,7 @@ public static CatalogMgr read(DataInput in) throws IOException { } private void initInternalCatalog() { - internalCatalog = EnvFactory.createInternalCatalog(); + internalCatalog = EnvFactory.getInstance().createInternalCatalog(); addCatalog(internalCatalog); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java index ae32e45aa81c37..51787b6635319d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java @@ -2877,7 +2877,7 @@ public void createTablets(MaterializedIndex index, ReplicaState replicaState, for (int i = 0; i < distributionInfo.getBucketNum(); ++i) { // create a new tablet with random chosen backends - Tablet tablet = EnvFactory.createTablet(idGeneratorBuffer.getNextId()); + Tablet tablet = EnvFactory.getInstance().createTablet(idGeneratorBuffer.getNextId()); // add tablet to inverted index first index.addTablet(tablet, tabletMeta); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java index a1a5dd9836cd70..ffaba171b299fe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java @@ -284,16 +284,15 @@ public void validate(ConnectContext ctx) { } if (engineName.equalsIgnoreCase("olap")) { - properties = PropertyAnalyzer.rewriteReplicaAllocationProperties(ctlName, dbName, - properties); boolean enableDuplicateWithoutKeysByDefault = false; - if (properties != null) { - try { + properties = PropertyAnalyzer.getInstance().rewriteOlapProperties(ctlName, dbName, properties); + try { + if (properties != null) { enableDuplicateWithoutKeysByDefault = - PropertyAnalyzer.analyzeEnableDuplicateWithoutKeysByDefault(properties); - } catch (Exception e) { - throw new AnalysisException(e.getMessage(), e.getCause()); + PropertyAnalyzer.analyzeEnableDuplicateWithoutKeysByDefault(properties); } + } catch (Exception e) { + throw new AnalysisException(e.getMessage(), e.getCause()); } if (keys.isEmpty()) { boolean hasAggColumn = false; diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateTableStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateTableStmtTest.java index ccd865dd6d4493..fed3f0d677168e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateTableStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateTableStmtTest.java @@ -111,7 +111,8 @@ public void testNormal() throws UserException, AnalysisException { stmt.analyze(analyzer); Assert.assertEquals("db1", stmt.getDbName()); Assert.assertEquals("table1", stmt.getTableName()); - Assert.assertTrue(stmt.getProperties() == null || stmt.getProperties().isEmpty()); + Assert.assertEquals(PropertyAnalyzer.getInstance().rewriteOlapProperties("", "", null), + stmt.getProperties()); } @Test @@ -121,7 +122,8 @@ public void testCreateTableWithRandomDistribution() throws UserException { stmt.analyze(analyzer); Assert.assertEquals("db1", stmt.getDbName()); Assert.assertEquals("table1", stmt.getTableName()); - Assert.assertTrue(stmt.getProperties() == null || stmt.getProperties().isEmpty()); + Assert.assertEquals(PropertyAnalyzer.getInstance().rewriteOlapProperties("", "", null), + stmt.getProperties()); Assert.assertTrue(stmt.toSql().contains("DISTRIBUTED BY RANDOM\nBUCKETS 6")); } @@ -273,7 +275,8 @@ public void testCreateTableWithRollup() throws UserException { stmt.analyze(analyzer); Assert.assertEquals("db1", stmt.getDbName()); Assert.assertEquals("table1", stmt.getTableName()); - Assert.assertTrue(stmt.getProperties() == null || stmt.getProperties().isEmpty()); + Assert.assertEquals(PropertyAnalyzer.getInstance().rewriteOlapProperties("", "", null), + stmt.getProperties()); Assert.assertTrue(stmt.toSql() .contains("rollup( `index1` (`col1`, `col2`) FROM `table1`, `index2` (`col2`, `col3`) FROM `table1`)")); } @@ -287,7 +290,8 @@ public void testDefaultDbNormal() throws UserException { Assert.assertEquals("testDb", stmt.getDbName()); Assert.assertEquals("table1", stmt.getTableName()); Assert.assertNull(stmt.getPartitionDesc()); - Assert.assertTrue(stmt.getProperties() == null || stmt.getProperties().isEmpty()); + Assert.assertEquals(PropertyAnalyzer.getInstance().rewriteOlapProperties("", "", null), + stmt.getProperties()); } @Test(expected = AnalysisException.class) diff --git a/fe/fe-core/src/test/java/org/apache/doris/binlog/BinlogManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/binlog/BinlogManagerTest.java index 03f8d325d774bb..1031e3d81e3bdc 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/binlog/BinlogManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/binlog/BinlogManagerTest.java @@ -119,7 +119,7 @@ public boolean isEnable() { new MockUp() { @Mock public InternalCatalog getCurrentInternalCatalog() { - return EnvFactory.createInternalCatalog(); + return EnvFactory.getInstance().createInternalCatalog(); } }; diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/EnvFactoryTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/EnvFactoryTest.java index 1a939457f5ea74..9701989b7eee76 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/EnvFactoryTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/EnvFactoryTest.java @@ -18,6 +18,7 @@ package org.apache.doris.catalog; import org.apache.doris.cloud.catalog.CloudEnv; +import org.apache.doris.cloud.catalog.CloudEnvFactory; import org.apache.doris.cloud.catalog.CloudPartition; import org.apache.doris.cloud.catalog.CloudReplica; import org.apache.doris.cloud.catalog.CloudTablet; @@ -33,27 +34,23 @@ public class EnvFactoryTest { @Test public void testCreate() throws Exception { Config.cloud_unique_id = ""; + EnvFactory envFactory = EnvFactory.getInstance(); + Assert.assertTrue(envFactory instanceof EnvFactory); + Assert.assertFalse(envFactory instanceof CloudEnvFactory); Assert.assertTrue(Env.getCurrentEnv() instanceof Env); Assert.assertFalse(Env.getCurrentEnv() instanceof CloudEnv); Assert.assertTrue(Env.getCurrentInternalCatalog() instanceof InternalCatalog); Assert.assertFalse(Env.getCurrentInternalCatalog() instanceof CloudInternalCatalog); - Assert.assertTrue(EnvFactory.createEnv(false) instanceof Env); - Assert.assertFalse(EnvFactory.createEnv(false) instanceof CloudEnv); - Assert.assertTrue(EnvFactory.createInternalCatalog() instanceof InternalCatalog); - Assert.assertFalse(EnvFactory.createInternalCatalog() instanceof CloudInternalCatalog); - Assert.assertTrue(EnvFactory.createPartition() instanceof Partition); - Assert.assertFalse(EnvFactory.createPartition() instanceof CloudPartition); - Assert.assertTrue(EnvFactory.createTablet() instanceof Tablet); - Assert.assertFalse(EnvFactory.createTablet() instanceof CloudTablet); - Assert.assertTrue(EnvFactory.createReplica() instanceof Replica); - Assert.assertFalse(EnvFactory.createReplica() instanceof CloudReplica); - - Config.cloud_unique_id = "test_cloud"; - Assert.assertTrue(EnvFactory.createEnv(false) instanceof CloudEnv); - Assert.assertTrue(EnvFactory.createInternalCatalog() instanceof CloudInternalCatalog); - Assert.assertTrue(EnvFactory.createPartition() instanceof CloudPartition); - Assert.assertTrue(EnvFactory.createTablet() instanceof CloudTablet); - Assert.assertTrue(EnvFactory.createReplica() instanceof CloudReplica); + Assert.assertTrue(envFactory.createEnv(false) instanceof Env); + Assert.assertFalse(envFactory.createEnv(false) instanceof CloudEnv); + Assert.assertTrue(envFactory.createInternalCatalog() instanceof InternalCatalog); + Assert.assertFalse(envFactory.createInternalCatalog() instanceof CloudInternalCatalog); + Assert.assertTrue(envFactory.createPartition() instanceof Partition); + Assert.assertFalse(envFactory.createPartition() instanceof CloudPartition); + Assert.assertTrue(envFactory.createTablet() instanceof Tablet); + Assert.assertFalse(envFactory.createTablet() instanceof CloudTablet); + Assert.assertTrue(envFactory.createReplica() instanceof Replica); + Assert.assertFalse(envFactory.createReplica() instanceof CloudReplica); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/cloud/catalog/CloudEnvFactoryTest.java b/fe/fe-core/src/test/java/org/apache/doris/cloud/catalog/CloudEnvFactoryTest.java new file mode 100644 index 00000000000000..6c3d0d3ab9b514 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/cloud/catalog/CloudEnvFactoryTest.java @@ -0,0 +1,52 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.cloud.catalog; + +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.EnvFactory; +import org.apache.doris.cloud.datasource.CloudInternalCatalog; +import org.apache.doris.common.Config; +import org.apache.doris.common.util.PropertyAnalyzer; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Map; + +public class CloudEnvFactoryTest { + + @Test + public void testCreate() throws Exception { + Config.cloud_unique_id = "test_cloud"; + EnvFactory envFactory = EnvFactory.getInstance(); + Assert.assertTrue(envFactory instanceof CloudEnvFactory); + Assert.assertTrue(Env.getCurrentEnv() instanceof CloudEnv); + Assert.assertTrue(Env.getCurrentInternalCatalog() instanceof CloudInternalCatalog); + Assert.assertTrue(envFactory.createEnv(false) instanceof CloudEnv); + Assert.assertTrue(envFactory.createInternalCatalog() instanceof CloudInternalCatalog); + Assert.assertTrue(envFactory.createPartition() instanceof CloudPartition); + Assert.assertTrue(envFactory.createTablet() instanceof CloudTablet); + Assert.assertTrue(envFactory.createReplica() instanceof CloudReplica); + + Map properties = PropertyAnalyzer.getInstance().rewriteOlapProperties( + "catalog_not_exist", "db_not_exist", null); + Assert.assertEquals("1", properties.get(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)); + } + +} + diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java index bffdfbd2c68ae6..83dcdd688243e2 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java @@ -465,7 +465,7 @@ public TableIf findTable(long catalogId, long dbId, long tblId) { @Test public void testDisableAuto1() throws Exception { - InternalCatalog catalog1 = EnvFactory.createInternalCatalog(); + InternalCatalog catalog1 = EnvFactory.getInstance().createInternalCatalog(); List catalogs = Lists.newArrayList(); catalogs.add(catalog1); @@ -493,7 +493,7 @@ protected boolean canCollect() { @Test public void testDisableAuto2() throws Exception { - InternalCatalog catalog1 = EnvFactory.createInternalCatalog(); + InternalCatalog catalog1 = EnvFactory.getInstance().createInternalCatalog(); List catalogs = Lists.newArrayList(); catalogs.add(catalog1); From f9c274c338afbe14f97cc16d1327e9a548e524b3 Mon Sep 17 00:00:00 2001 From: Dongyang Li Date: Fri, 19 Jan 2024 23:53:06 +0800 Subject: [PATCH 104/200] [ci](perf) add required (#30155) Co-authored-by: stephen --- .asf.yaml | 2 + .../workflows/comment-to-trigger-teamcity.yml | 21 +-- .../pipeline/common/doris-utils.sh | 12 +- .../pipeline/common/github-utils.sh | 25 ---- .../pipeline/performance/run-tpcds.sh | 1 + .../pipeline/performance/run-tpch.sh | 1 + .../pipeline/tpch/tpch-sf100/clean.sh | 40 ------ .../pipeline/tpch/tpch-sf100/conf/be.conf | 85 ------------ .../tpch/tpch-sf100/conf/external.json | 26 ---- .../pipeline/tpch/tpch-sf100/conf/fe.conf | 78 ----------- .../tpch/tpch-sf100/conf/odbcinst.ini | 43 ------ .../pipeline/tpch/tpch-sf100/deploy.sh | 119 ---------------- .../pipeline/tpch/tpch-sf100/prepare.sh | 69 --------- .../pipeline/tpch/tpch-sf100/run.sh | 131 ------------------ 14 files changed, 11 insertions(+), 642 deletions(-) delete mode 100644 regression-test/pipeline/tpch/tpch-sf100/clean.sh delete mode 100644 regression-test/pipeline/tpch/tpch-sf100/conf/be.conf delete mode 100644 regression-test/pipeline/tpch/tpch-sf100/conf/external.json delete mode 100644 regression-test/pipeline/tpch/tpch-sf100/conf/fe.conf delete mode 100644 regression-test/pipeline/tpch/tpch-sf100/conf/odbcinst.ini delete mode 100644 regression-test/pipeline/tpch/tpch-sf100/deploy.sh delete mode 100644 regression-test/pipeline/tpch/tpch-sf100/prepare.sh delete mode 100644 regression-test/pipeline/tpch/tpch-sf100/run.sh diff --git a/.asf.yaml b/.asf.yaml index cc73010384354f..3633289db7989b 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -57,6 +57,7 @@ github: - External Regression (Doris External Regression) - FE UT (Doris FE UT) - BE UT (Doris BE UT) + - performance (Performance New) - Build Broker - Build Documents - ShellCheck @@ -92,6 +93,7 @@ github: - External Regression (Doris External Regression) - FE UT (Doris FE UT) - BE UT (Doris BE UT) + - performance (Performance New) - Build Broker - ShellCheck - Build Documents diff --git a/.github/workflows/comment-to-trigger-teamcity.yml b/.github/workflows/comment-to-trigger-teamcity.yml index 0fdd80ba28fa85..8ae0832fc9e69b 100644 --- a/.github/workflows/comment-to-trigger-teamcity.yml +++ b/.github/workflows/comment-to-trigger-teamcity.yml @@ -45,7 +45,6 @@ jobs: "${COMMENT_BODY}" == *'run p1'* || "${COMMENT_BODY}" == *'run external'* || "${COMMENT_BODY}" == *'run pipelinex_p0'* || - "${COMMENT_BODY}" == *'run clickbench'* || "${COMMENT_BODY}" == *'run arm'* || "${COMMENT_BODY}" == *'run performance'* ]]; then echo "comment_trigger=true" | tee -a "$GITHUB_OUTPUT" @@ -63,7 +62,7 @@ jobs: echo "TARGET_BRANCH='${TARGET_BRANCH}'" | tee -a "$GITHUB_OUTPUT" echo "COMMENT_BODY='${COMMENT_BODY}'" | tee -a "$GITHUB_OUTPUT" - reg="run (buildall|compile|p0|p1|feut|beut|external|clickbench|pipelinex_p0|arm|performance)( [1-9]*[0-9]+)*" + reg="run (buildall|compile|p0|p1|feut|beut|external|pipelinex_p0|arm|performance)( [1-9]*[0-9]+)*" COMMENT_TRIGGER_TYPE="$(echo -e "${COMMENT_BODY}" | xargs | grep -E "${reg}" | awk -F' ' '{print $2}' | sed -n 1p | sed 's/\r//g')" COMMENT_REPEAT_TIMES="$(echo -e "${COMMENT_BODY}" | xargs | grep -E "${reg}" | awk -F' ' '{print $3}' | sed -n 1p | sed 's/\r//g')" echo "COMMENT_TRIGGER_TYPE=${COMMENT_TRIGGER_TYPE}" | tee -a "$GITHUB_OUTPUT" @@ -90,11 +89,6 @@ jobs: else echo "changed_be_ut=false" | tee -a "$GITHUB_OUTPUT" fi - if file_changed_ckb; then - echo "changed_ckb=true" | tee -a "$GITHUB_OUTPUT" - else - echo "changed_ckb=false" | tee -a "$GITHUB_OUTPUT" - fi if file_changed_regression_p0; then echo "changed_p0=true" | tee -a "$GITHUB_OUTPUT" echo "changed_external=true" | tee -a "$GITHUB_OUTPUT" @@ -120,7 +114,6 @@ jobs: echo "INFO: failed to _get_pr_changed_files, default trigger all" echo "changed_fe_ut=true" | tee -a "$GITHUB_OUTPUT" echo "changed_be_ut=true" | tee -a "$GITHUB_OUTPUT" - echo "changed_ckb=true" | tee -a "$GITHUB_OUTPUT" echo "changed_p0=true" | tee -a "$GITHUB_OUTPUT" echo "changed_external=true" | tee -a "$GITHUB_OUTPUT" echo "changed_pipelinex_p0=true" | tee -a "$GITHUB_OUTPUT" @@ -243,18 +236,6 @@ jobs: "arm" \ "${{ steps.parse.outputs.COMMENT_REPEAT_TIMES }}" - - name: "Trigger or Skip clickbench" - if: ${{ fromJSON(steps.parse.outputs.comment_trigger) && contains(fromJSON('["clickbench", "buildall"]'), steps.parse.outputs.COMMENT_TRIGGER_TYPE) }} - run: | - source ./regression-test/pipeline/common/teamcity-utils.sh - set -x - trigger_or_skip_build \ - "${{ steps.changes.outputs.changed_ckb }}" \ - "${{ steps.parse.outputs.PULL_REQUEST_NUM }}" \ - "${{ steps.parse.outputs.COMMIT_ID_FROM_TRIGGER }}" \ - "clickbench" \ - "${{ steps.parse.outputs.COMMENT_REPEAT_TIMES }}" - - name: "Trigger or Skip performance" if: ${{ fromJSON(steps.parse.outputs.comment_trigger) && contains(fromJSON('["performance", "buildall"]'), steps.parse.outputs.COMMENT_TRIGGER_TYPE) }} run: | diff --git a/regression-test/pipeline/common/doris-utils.sh b/regression-test/pipeline/common/doris-utils.sh index 47bcedad351861..327f00beac12fe 100644 --- a/regression-test/pipeline/common/doris-utils.sh +++ b/regression-test/pipeline/common/doris-utils.sh @@ -394,18 +394,18 @@ archive_doris_logs() { print_doris_fe_log() { if [[ ! -d "${DORIS_HOME:-}" ]]; then return 1; fi - echo "WARNING: --------------------tail -n 100 ${DORIS_HOME}/fe/log/fe.out--------------------" + echo -e "\n\n\n\nWARNING: --------------------tail -n 100 ${DORIS_HOME}/fe/log/fe.out--------------------" tail -n 100 "${DORIS_HOME}"/fe/log/fe.out - echo "WARNING: --------------------tail -n 100 ${DORIS_HOME}/fe/log/fe.log--------------------" + echo -e "\n\n\n\nWARNING: --------------------tail -n 100 ${DORIS_HOME}/fe/log/fe.log--------------------" tail -n 100 "${DORIS_HOME}"/fe/log/fe.log - echo "WARNING: ----------------------------------------" + echo -e "WARNING: ----------------------------------------\n\n\n\n" } print_doris_be_log() { if [[ ! -d "${DORIS_HOME:-}" ]]; then return 1; fi - echo "WARNING: --------------------tail -n 100 ${DORIS_HOME}/be/log/be.out--------------------" + echo -e "\n\n\n\nWARNING: --------------------tail -n 100 ${DORIS_HOME}/be/log/be.out--------------------" tail -n 100 "${DORIS_HOME}"/be/log/be.out - echo "WARNING: --------------------tail -n 100 ${DORIS_HOME}/be/log/be.INFO--------------------" + echo -e "\n\n\n\nWARNING: --------------------tail -n 100 ${DORIS_HOME}/be/log/be.INFO--------------------" tail -n 100 "${DORIS_HOME}"/be/log/be.INFO - echo "WARNING: ----------------------------------------" + echo -e "WARNING: ----------------------------------------\n\n\n\n" } diff --git a/regression-test/pipeline/common/github-utils.sh b/regression-test/pipeline/common/github-utils.sh index 317667d15b2e46..5a9d2ebb7b32b1 100644 --- a/regression-test/pipeline/common/github-utils.sh +++ b/regression-test/pipeline/common/github-utils.sh @@ -304,31 +304,6 @@ file_changed_regression_p1() { file_changed_regression_p0 } -file_changed_ckb() { - local all_files - all_files=$(cat all_files) - if _only_modified_regression_conf; then echo "return no need" && return 1; fi - if [[ -z ${all_files} ]]; then echo "return need" && return 0; fi - for af in ${all_files}; do - if [[ "${af}" == 'be'* ]] || - [[ "${af}" == 'bin'* ]] || - [[ "${af}" == 'conf'* ]] || - [[ "${af}" == 'fe'* ]] || - [[ "${af}" == 'gensrc'* ]] || - [[ "${af}" == 'thirdparty'* ]] || - [[ "${af}" == 'build.sh' ]] || - [[ "${af}" == 'env.sh' ]] || - [[ "${af}" == 'regression-test/pipeline/common/github-utils.sh' ]] || - [[ "${af}" == 'regression-test/pipeline/common/doris-utils.sh' ]] || - [[ "${af}" == 'regression-test/pipeline/common/oss-utils.sh' ]] || - [[ "${af}" == 'tools/tpch-tools/bin/run-tpch-queries.sh' ]] || - [[ "${af}" == 'regression-test/pipeline/tpch/tpch-sf100/'* ]]; then - echo "clickbench performance related file changed, return need" && return 0 - fi - done - echo "return no need" && return 1 -} - file_changed_performance() { local all_files all_files=$(cat all_files) diff --git a/regression-test/pipeline/performance/run-tpcds.sh b/regression-test/pipeline/performance/run-tpcds.sh index d054b85ee4da0a..0ca31c57e74a04 100644 --- a/regression-test/pipeline/performance/run-tpcds.sh +++ b/regression-test/pipeline/performance/run-tpcds.sh @@ -125,6 +125,7 @@ exit_flag=0 echo "#### 3. run tpcds-sf${SF} query" set_session_variable runtime_filter_mode global bash "${teamcity_build_checkoutDir}"/tools/tpcds-tools/bin/run-tpcds-queries.sh -s "${SF}" | tee "${teamcity_build_checkoutDir}"/run-tpcds-queries.log + echo cold_run_time_threshold=${cold_run_time_threshold_master:-315000} # ms hot_run_time_threshold=${hot_run_time_threshold_master:-190000} # ms if [[ "${target_branch}" == "branch-2.0" ]]; then diff --git a/regression-test/pipeline/performance/run-tpch.sh b/regression-test/pipeline/performance/run-tpch.sh index 1620c5f2922d73..f1e4c8df2a2e1b 100644 --- a/regression-test/pipeline/performance/run-tpch.sh +++ b/regression-test/pipeline/performance/run-tpch.sh @@ -117,6 +117,7 @@ exit_flag=0 echo "#### 3. run tpch-sf${SF} query" set_session_variable runtime_filter_mode global bash "${teamcity_build_checkoutDir}"/tools/tpch-tools/bin/run-tpch-queries.sh -s "${SF}" | tee "${teamcity_build_checkoutDir}"/run-tpch-queries.log + echo cold_run_time_threshold=${cold_run_time_threshold_master:-120000} # ms hot_run_time_threshold=${hot_run_time_threshold_master:-42000} # ms if [[ "${target_branch}" == "branch-2.0" ]]; then diff --git a/regression-test/pipeline/tpch/tpch-sf100/clean.sh b/regression-test/pipeline/tpch/tpch-sf100/clean.sh deleted file mode 100644 index 16fa490ef9c237..00000000000000 --- a/regression-test/pipeline/tpch/tpch-sf100/clean.sh +++ /dev/null @@ -1,40 +0,0 @@ -#!/usr/bin/env bash -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you 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. - -# Build Step: Command Line -: <:', separate by ',' -# property 'medium' has a higher priority than the extension of path -# -# Default value is ${DORIS_HOME}/storage, you should create it by hand. -# storage_root_path = ${DORIS_HOME}/storage - -# Default dirs to put jdbc drivers,default value is ${DORIS_HOME}/jdbc_drivers -# jdbc_drivers_dir = ${DORIS_HOME}/jdbc_drivers - -# Advanced configurations -# sys_log_dir = ${DORIS_HOME}/log -# sys_log_roll_mode = SIZE-MB-1024 -# sys_log_roll_num = 10 -# sys_log_verbose_modules = * -# log_buffer_level = -1 -# palo_cgroups - -priority_networks=172.16.0.0/24 -storage_root_path=/mnt/datadisk0/doris-storage diff --git a/regression-test/pipeline/tpch/tpch-sf100/conf/external.json b/regression-test/pipeline/tpch/tpch-sf100/conf/external.json deleted file mode 100644 index 9461d836e6749a..00000000000000 --- a/regression-test/pipeline/tpch/tpch-sf100/conf/external.json +++ /dev/null @@ -1,26 +0,0 @@ -[ - { - "file": "docker/thirdparties/docker-compose/mysql/mysql-5.7.env", - "replacements": { - "DOCKER_MYSQL_57_EXTERNAL_PORT": 7111 - } - }, - { - "file": "docker/thirdparties/docker-compose/postgresql/postgresql-14.env", - "replacements": { - "DOCKER_PG_14_EXTERNAL_PORT": 7121 - } - }, - { - "file": "docker/thirdparties/docker-compose/hive/gen_env.sh", - "replacements": { - "FS_PORT": 7131, - "HMS_PORT": 7141 - } - }, { - "file": "docker/thirdparties/start-thirdparties-docker.sh", - "replacements": { - "CONTAINER_UID": "doris-regression-fakeid-fakecommit" - } - } -] diff --git a/regression-test/pipeline/tpch/tpch-sf100/conf/fe.conf b/regression-test/pipeline/tpch/tpch-sf100/conf/fe.conf deleted file mode 100644 index 7c02d3898dcb45..00000000000000 --- a/regression-test/pipeline/tpch/tpch-sf100/conf/fe.conf +++ /dev/null @@ -1,78 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you 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. - -##################################################################### -## The uppercase properties are read and exported by bin/start_fe.sh. -## To see all Frontend configurations, -## see fe/src/org/apache/doris/common/Config.java -##################################################################### - -CUR_DATE=`date +%Y%m%d-%H%M%S` - -# the output dir of stderr and stdout -LOG_DIR = ${DORIS_HOME}/log - -JAVA_OPTS="-Dsun.security.krb5.debug=true -Djavax.security.auth.useSubjectCredsOnly=false -Xss4m -Xmx8192m -XX:+UseMembar -XX:SurvivorRatio=8 -XX:MaxTenuringThreshold=7 -XX:+PrintGCDateStamps -XX:+PrintGCDetails -XX:+UseConcMarkSweepGC -XX:+UseParNewGC -XX:+CMSClassUnloadingEnabled -XX:-CMSParallelRemarkEnabled -XX:CMSInitiatingOccupancyFraction=80 -XX:SoftRefLRUPolicyMSPerMB=0 -Xloggc:$DORIS_HOME/log/fe.gc.log.$CUR_DATE" - -# For jdk 9+, this JAVA_OPTS will be used as default JVM options -JAVA_OPTS_FOR_JDK_9="-Dsun.security.krb5.debug=true -Djavax.security.auth.useSubjectCredsOnly=false -Xss4m -Xmx8192m -XX:SurvivorRatio=8 -XX:MaxTenuringThreshold=7 -XX:+CMSClassUnloadingEnabled -XX:-CMSParallelRemarkEnabled -XX:CMSInitiatingOccupancyFraction=80 -XX:SoftRefLRUPolicyMSPerMB=0 -Xlog:gc*:$DORIS_HOME/log/fe.gc.log.$CUR_DATE:time" - -## -## the lowercase properties are read by main program. -## - -# INFO, WARN, ERROR, FATAL -sys_log_level = INFO - -# NORMAL, BRIEF, ASYNC -sys_log_mode = NORMAL - -# store metadata, must be created before start FE. -# Default value is ${DORIS_HOME}/doris-meta -# meta_dir = ${DORIS_HOME}/doris-meta - -# Default dirs to put jdbc drivers,default value is ${DORIS_HOME}/jdbc_drivers -# jdbc_drivers_dir = ${DORIS_HOME}/jdbc_drivers - -http_port = 8030 -rpc_port = 9020 -query_port = 9030 -edit_log_port = 9010 -arrow_flight_sql_port = -1 - -# Choose one if there are more than one ip except loopback address. -# Note that there should at most one ip match this list. -# If no ip match this rule, will choose one randomly. -# use CIDR format, e.g. 10.10.10.0/24 or IP format, e.g. 10.10.10.1 -# Default value is empty. -# priority_networks = 10.10.10.0/24;192.168.0.0/16 - -# Advanced configurations -# log_roll_size_mb = 1024 -# sys_log_dir = ${DORIS_HOME}/log -# sys_log_roll_num = 10 -# sys_log_verbose_modules = org.apache.doris -# audit_log_dir = ${DORIS_HOME}/log -# audit_log_modules = slow_query, query -# audit_log_roll_num = 10 -# meta_delay_toleration_second = 10 -# qe_max_connection = 1024 -# qe_query_timeout_second = 300 -# qe_slow_log_ms = 5000 - -priority_networks=172.16.0.0/24 -meta_dir=/mnt/datadisk0/doris-meta diff --git a/regression-test/pipeline/tpch/tpch-sf100/conf/odbcinst.ini b/regression-test/pipeline/tpch/tpch-sf100/conf/odbcinst.ini deleted file mode 100644 index 41e21f92277e98..00000000000000 --- a/regression-test/pipeline/tpch/tpch-sf100/conf/odbcinst.ini +++ /dev/null @@ -1,43 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you 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. - -# Example driver definitions, you should not use the example odbc driver -# before you prepare env in your server - -# Driver from the postgresql-odbc package -# Setup from the unixODBC package -[PostgreSQL] -Description = ODBC for PostgreSQL -Driver = /usr/lib/psqlodbc.so -Setup = /usr/lib/libodbcpsqlS.so -FileUsage = 1 - - -# Driver from the mysql-connector-odbc package -# Setup from the unixODBC package -[MySQL ODBC 8.0 Unicode Driver] -Description = ODBC for MySQL -Driver = /usr/lib64/libmyodbc8w.so -FileUsage = 1 - -# Driver from the oracle-connector-odbc package -# Setup from the unixODBC package -[Oracle 19 ODBC driver] -Description=Oracle ODBC driver for Oracle 19 -Driver=/usr/lib/libsqora.so.19.1 - - diff --git a/regression-test/pipeline/tpch/tpch-sf100/deploy.sh b/regression-test/pipeline/tpch/tpch-sf100/deploy.sh deleted file mode 100644 index 64d816f0acadaf..00000000000000 --- a/regression-test/pipeline/tpch/tpch-sf100/deploy.sh +++ /dev/null @@ -1,119 +0,0 @@ -#!/usr/bin/env bash -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you 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. - -# Build Step: Command Line -: </dev/null; then sudo apt install -y pigz; fi - tar -I pigz -xf "${pull_request_id:-}_${commit_id:-}.tar.gz" - if [[ -d output && -d output/fe && -d output/be ]]; then - echo "INFO: be version: $(./output/be/lib/doris_be --version)" - rm -rf "${pull_request_id}_${commit_id}.tar.gz" - fi -else - echo "ERROR: download compiled binary failed" && exit 1 -fi - -echo "#### 3. copy conf from regression-test/pipeline/tpch/tpch-sf100/conf/" -rm -f "${DORIS_HOME}"/fe/conf/fe_custom.conf "${DORIS_HOME}"/be/conf/be_custom.conf -if [[ -f "${teamcity_build_checkoutDir}"/regression-test/pipeline/tpch/tpch-sf100/conf/fe.conf && - -f "${teamcity_build_checkoutDir}"/regression-test/pipeline/tpch/tpch-sf100/conf/be.conf ]]; then - cp -f "${teamcity_build_checkoutDir}"/regression-test/pipeline/tpch/tpch-sf100/conf/fe.conf "${DORIS_HOME}"/fe/conf/ - cp -f "${teamcity_build_checkoutDir}"/regression-test/pipeline/tpch/tpch-sf100/conf/be.conf "${DORIS_HOME}"/be/conf/ -else - echo "ERROR: doris conf file missing in ${teamcity_build_checkoutDir}/regression-test/pipeline/tpch/tpch-sf100/conf/" - exit 1 -fi - -echo "#### 4. start Doris" -meta_dir=$(get_doris_conf_value "${DORIS_HOME}"/fe/conf/fe.conf meta_dir) -storage_root_path=$(get_doris_conf_value "${DORIS_HOME}"/be/conf/be.conf storage_root_path) -mkdir -p "${meta_dir}" -mkdir -p "${storage_root_path}" -if ! start_doris_fe; then - echo "ERROR: Start doris fe failed." - print_doris_fe_log - need_backup_doris_logs=true - exit_flag=1 -fi -if ! start_doris_be; then - echo "ERROR: Start doris be failed." - print_doris_be_log - need_backup_doris_logs=true - exit_flag=1 -fi - -# wait 10s for doris totally started, otherwize may encounter the error below, -# ERROR 1105 (HY000) at line 102: errCode = 2, detailMessage = Failed to find enough backend, please check the replication num,replication tag and storage medium. -sleep 10s - -echo "#### 5. set session variables" -echo "TODO" - -echo "#### 6. check if need backup doris logs" -if ${need_backup_doris_logs}; then - print_doris_fe_log - print_doris_be_log - if file_name=$(archive_doris_logs "${pull_request_id}_${commit_id}_doris_logs.tar.gz"); then - upload_doris_log_to_oss "${file_name}" - fi -fi - -exit "${exit_flag}" diff --git a/regression-test/pipeline/tpch/tpch-sf100/prepare.sh b/regression-test/pipeline/tpch/tpch-sf100/prepare.sh deleted file mode 100644 index 07959a03d6f8e7..00000000000000 --- a/regression-test/pipeline/tpch/tpch-sf100/prepare.sh +++ /dev/null @@ -1,69 +0,0 @@ -#!/usr/bin/env bash -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you 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. - -# Build Step: Command Line -: </dev/null; then exit 1; fi - line_end=$(sed -n '/^Total hot run time/=' "${teamcity_build_checkoutDir}"/run-tpch-queries.log) - line_begin=$((line_end - 23)) - comment_body="${comment_body} - -run tpch-sf${SF} query with default conf and set session variable runtime_filter_mode=off -$(sed -n "${line_begin},${line_end}p" "${teamcity_build_checkoutDir}"/run-tpch-queries.log)" - - echo "#### 4. comment result on tpch" - comment_body=$(echo "${comment_body}" | sed -e ':a;N;$!ba;s/\t/\\t/g;s/\n/\\n/g') # 将所有的 Tab字符替换为\t 换行符替换为\n - create_an_issue_comment_tpch "${pull_request_id:-}" "${comment_body}" - - stop_doris -) -exit_flag="$?" - -echo "#### 5. check if need backup doris logs" -if [[ ${exit_flag} != "0" ]]; then - print_doris_fe_log - print_doris_be_log - if file_name=$(archive_doris_logs "${pull_request_id}_${commit_id}_doris_logs.tar.gz"); then - upload_doris_log_to_oss "${file_name}" - fi -fi - -exit "${exit_flag}" From 0af3b1ea80bd27676f9e35fa15bff03e2857bce5 Mon Sep 17 00:00:00 2001 From: zhangdong <493738387@qq.com> Date: Sat, 20 Jan 2024 22:39:37 +0800 Subject: [PATCH 105/200] [doc](ranger) update ranger doc #30101 add some descriptions for ranger --- docs/en/docs/lakehouse/multi-catalog/hive.md | 13 +++++++++++++ docs/zh-CN/docs/lakehouse/multi-catalog/hive.md | 13 +++++++++++++ 2 files changed, 26 insertions(+) diff --git a/docs/en/docs/lakehouse/multi-catalog/hive.md b/docs/en/docs/lakehouse/multi-catalog/hive.md index 90db7fe3543171..e5fc390a05d70f 100644 --- a/docs/en/docs/lakehouse/multi-catalog/hive.md +++ b/docs/en/docs/lakehouse/multi-catalog/hive.md @@ -415,6 +415,19 @@ To connect to the Hive Metastore with Ranger permission verification enabled, yo "access_controller.class" = "org.apache.doris.catalog.authorizer.RangerHiveAccessControllerFactory", ``` +>Note: +> +>"access_controller.properties.ranger.service.name" It refers to the type of service, such as hive, hdfs, etc. +> If the configuration of `ranger-hive-security.xml`is as follows, "access_controller.properties.ranger.service.name" should be set to `hive` instead of `testative` +> + +```sql + + ranger.plugin.hive.service.name + testhive + +``` + 2. Configure all FE environments: 1. Copy the configuration files ranger-hive-audit.xml, ranger-hive-security.xml, and ranger-policymgr-ssl.xml under the HMS conf directory to the FE conf directory. diff --git a/docs/zh-CN/docs/lakehouse/multi-catalog/hive.md b/docs/zh-CN/docs/lakehouse/multi-catalog/hive.md index 52e40fc1d0b729..b5eedd627477f6 100644 --- a/docs/zh-CN/docs/lakehouse/multi-catalog/hive.md +++ b/docs/zh-CN/docs/lakehouse/multi-catalog/hive.md @@ -396,6 +396,19 @@ Apache Ranger是一个用来在Hadoop平台上进行监控,启用服务,以 "access_controller.class" = "org.apache.doris.catalog.authorizer.RangerHiveAccessControllerFactory", ``` +>注意: +> +>"access_controller.properties.ranger.service.name"指的是service的类型,例如hive,hdfs等 +> 如果`ranger-hive-security.xml`配置如下,"access_controller.properties.ranger.service.name"应设置为`hive`而不是`testhive` +> + +```sql + + ranger.plugin.hive.service.name + testhive + +``` + 2. 配置所有 FE 环境: 1. 将 HMS conf 目录下的配置文件ranger-hive-audit.xml,ranger-hive-security.xml,ranger-policymgr-ssl.xml复制到 FE 的 conf 目录下。 From decc6e2585fcd0bf1852df0207e02d25369a212c Mon Sep 17 00:00:00 2001 From: Dongyang Li Date: Sat, 20 Jan 2024 23:07:15 +0800 Subject: [PATCH 106/200] [fix](ci) remove required of performance (#30185) encounter this issue: https://docs.github.com/en/pull-requests/collaborating-with-pull-requests/collaborating-on-repositories-with-code-quality-features/troubleshooting-required-status-checks#required-status-checks-from-unexpected-sources --- .asf.yaml | 2 -- 1 file changed, 2 deletions(-) diff --git a/.asf.yaml b/.asf.yaml index 3633289db7989b..cc73010384354f 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -57,7 +57,6 @@ github: - External Regression (Doris External Regression) - FE UT (Doris FE UT) - BE UT (Doris BE UT) - - performance (Performance New) - Build Broker - Build Documents - ShellCheck @@ -93,7 +92,6 @@ github: - External Regression (Doris External Regression) - FE UT (Doris FE UT) - BE UT (Doris BE UT) - - performance (Performance New) - Build Broker - ShellCheck - Build Documents From 0f3f3b75b6f5c8826387068f5dcc55de1b268318 Mon Sep 17 00:00:00 2001 From: shuke <37901441+shuke987@users.noreply.github.com> Date: Sun, 21 Jan 2024 09:04:23 +0800 Subject: [PATCH 107/200] Revert "[ci](perf) add required (#30155)" (#30165) This reverts commit f9c274c338afbe14f97cc16d1327e9a548e524b3. --- .../workflows/comment-to-trigger-teamcity.yml | 21 ++- .../pipeline/common/doris-utils.sh | 12 +- .../pipeline/common/github-utils.sh | 25 ++++ .../pipeline/performance/run-tpcds.sh | 1 - .../pipeline/performance/run-tpch.sh | 1 - .../pipeline/tpch/tpch-sf100/clean.sh | 40 ++++++ .../pipeline/tpch/tpch-sf100/conf/be.conf | 85 ++++++++++++ .../tpch/tpch-sf100/conf/external.json | 26 ++++ .../pipeline/tpch/tpch-sf100/conf/fe.conf | 78 +++++++++++ .../tpch/tpch-sf100/conf/odbcinst.ini | 43 ++++++ .../pipeline/tpch/tpch-sf100/deploy.sh | 119 ++++++++++++++++ .../pipeline/tpch/tpch-sf100/prepare.sh | 69 +++++++++ .../pipeline/tpch/tpch-sf100/run.sh | 131 ++++++++++++++++++ 13 files changed, 642 insertions(+), 9 deletions(-) create mode 100644 regression-test/pipeline/tpch/tpch-sf100/clean.sh create mode 100644 regression-test/pipeline/tpch/tpch-sf100/conf/be.conf create mode 100644 regression-test/pipeline/tpch/tpch-sf100/conf/external.json create mode 100644 regression-test/pipeline/tpch/tpch-sf100/conf/fe.conf create mode 100644 regression-test/pipeline/tpch/tpch-sf100/conf/odbcinst.ini create mode 100644 regression-test/pipeline/tpch/tpch-sf100/deploy.sh create mode 100644 regression-test/pipeline/tpch/tpch-sf100/prepare.sh create mode 100644 regression-test/pipeline/tpch/tpch-sf100/run.sh diff --git a/.github/workflows/comment-to-trigger-teamcity.yml b/.github/workflows/comment-to-trigger-teamcity.yml index 8ae0832fc9e69b..0fdd80ba28fa85 100644 --- a/.github/workflows/comment-to-trigger-teamcity.yml +++ b/.github/workflows/comment-to-trigger-teamcity.yml @@ -45,6 +45,7 @@ jobs: "${COMMENT_BODY}" == *'run p1'* || "${COMMENT_BODY}" == *'run external'* || "${COMMENT_BODY}" == *'run pipelinex_p0'* || + "${COMMENT_BODY}" == *'run clickbench'* || "${COMMENT_BODY}" == *'run arm'* || "${COMMENT_BODY}" == *'run performance'* ]]; then echo "comment_trigger=true" | tee -a "$GITHUB_OUTPUT" @@ -62,7 +63,7 @@ jobs: echo "TARGET_BRANCH='${TARGET_BRANCH}'" | tee -a "$GITHUB_OUTPUT" echo "COMMENT_BODY='${COMMENT_BODY}'" | tee -a "$GITHUB_OUTPUT" - reg="run (buildall|compile|p0|p1|feut|beut|external|pipelinex_p0|arm|performance)( [1-9]*[0-9]+)*" + reg="run (buildall|compile|p0|p1|feut|beut|external|clickbench|pipelinex_p0|arm|performance)( [1-9]*[0-9]+)*" COMMENT_TRIGGER_TYPE="$(echo -e "${COMMENT_BODY}" | xargs | grep -E "${reg}" | awk -F' ' '{print $2}' | sed -n 1p | sed 's/\r//g')" COMMENT_REPEAT_TIMES="$(echo -e "${COMMENT_BODY}" | xargs | grep -E "${reg}" | awk -F' ' '{print $3}' | sed -n 1p | sed 's/\r//g')" echo "COMMENT_TRIGGER_TYPE=${COMMENT_TRIGGER_TYPE}" | tee -a "$GITHUB_OUTPUT" @@ -89,6 +90,11 @@ jobs: else echo "changed_be_ut=false" | tee -a "$GITHUB_OUTPUT" fi + if file_changed_ckb; then + echo "changed_ckb=true" | tee -a "$GITHUB_OUTPUT" + else + echo "changed_ckb=false" | tee -a "$GITHUB_OUTPUT" + fi if file_changed_regression_p0; then echo "changed_p0=true" | tee -a "$GITHUB_OUTPUT" echo "changed_external=true" | tee -a "$GITHUB_OUTPUT" @@ -114,6 +120,7 @@ jobs: echo "INFO: failed to _get_pr_changed_files, default trigger all" echo "changed_fe_ut=true" | tee -a "$GITHUB_OUTPUT" echo "changed_be_ut=true" | tee -a "$GITHUB_OUTPUT" + echo "changed_ckb=true" | tee -a "$GITHUB_OUTPUT" echo "changed_p0=true" | tee -a "$GITHUB_OUTPUT" echo "changed_external=true" | tee -a "$GITHUB_OUTPUT" echo "changed_pipelinex_p0=true" | tee -a "$GITHUB_OUTPUT" @@ -236,6 +243,18 @@ jobs: "arm" \ "${{ steps.parse.outputs.COMMENT_REPEAT_TIMES }}" + - name: "Trigger or Skip clickbench" + if: ${{ fromJSON(steps.parse.outputs.comment_trigger) && contains(fromJSON('["clickbench", "buildall"]'), steps.parse.outputs.COMMENT_TRIGGER_TYPE) }} + run: | + source ./regression-test/pipeline/common/teamcity-utils.sh + set -x + trigger_or_skip_build \ + "${{ steps.changes.outputs.changed_ckb }}" \ + "${{ steps.parse.outputs.PULL_REQUEST_NUM }}" \ + "${{ steps.parse.outputs.COMMIT_ID_FROM_TRIGGER }}" \ + "clickbench" \ + "${{ steps.parse.outputs.COMMENT_REPEAT_TIMES }}" + - name: "Trigger or Skip performance" if: ${{ fromJSON(steps.parse.outputs.comment_trigger) && contains(fromJSON('["performance", "buildall"]'), steps.parse.outputs.COMMENT_TRIGGER_TYPE) }} run: | diff --git a/regression-test/pipeline/common/doris-utils.sh b/regression-test/pipeline/common/doris-utils.sh index 327f00beac12fe..47bcedad351861 100644 --- a/regression-test/pipeline/common/doris-utils.sh +++ b/regression-test/pipeline/common/doris-utils.sh @@ -394,18 +394,18 @@ archive_doris_logs() { print_doris_fe_log() { if [[ ! -d "${DORIS_HOME:-}" ]]; then return 1; fi - echo -e "\n\n\n\nWARNING: --------------------tail -n 100 ${DORIS_HOME}/fe/log/fe.out--------------------" + echo "WARNING: --------------------tail -n 100 ${DORIS_HOME}/fe/log/fe.out--------------------" tail -n 100 "${DORIS_HOME}"/fe/log/fe.out - echo -e "\n\n\n\nWARNING: --------------------tail -n 100 ${DORIS_HOME}/fe/log/fe.log--------------------" + echo "WARNING: --------------------tail -n 100 ${DORIS_HOME}/fe/log/fe.log--------------------" tail -n 100 "${DORIS_HOME}"/fe/log/fe.log - echo -e "WARNING: ----------------------------------------\n\n\n\n" + echo "WARNING: ----------------------------------------" } print_doris_be_log() { if [[ ! -d "${DORIS_HOME:-}" ]]; then return 1; fi - echo -e "\n\n\n\nWARNING: --------------------tail -n 100 ${DORIS_HOME}/be/log/be.out--------------------" + echo "WARNING: --------------------tail -n 100 ${DORIS_HOME}/be/log/be.out--------------------" tail -n 100 "${DORIS_HOME}"/be/log/be.out - echo -e "\n\n\n\nWARNING: --------------------tail -n 100 ${DORIS_HOME}/be/log/be.INFO--------------------" + echo "WARNING: --------------------tail -n 100 ${DORIS_HOME}/be/log/be.INFO--------------------" tail -n 100 "${DORIS_HOME}"/be/log/be.INFO - echo -e "WARNING: ----------------------------------------\n\n\n\n" + echo "WARNING: ----------------------------------------" } diff --git a/regression-test/pipeline/common/github-utils.sh b/regression-test/pipeline/common/github-utils.sh index 5a9d2ebb7b32b1..317667d15b2e46 100644 --- a/regression-test/pipeline/common/github-utils.sh +++ b/regression-test/pipeline/common/github-utils.sh @@ -304,6 +304,31 @@ file_changed_regression_p1() { file_changed_regression_p0 } +file_changed_ckb() { + local all_files + all_files=$(cat all_files) + if _only_modified_regression_conf; then echo "return no need" && return 1; fi + if [[ -z ${all_files} ]]; then echo "return need" && return 0; fi + for af in ${all_files}; do + if [[ "${af}" == 'be'* ]] || + [[ "${af}" == 'bin'* ]] || + [[ "${af}" == 'conf'* ]] || + [[ "${af}" == 'fe'* ]] || + [[ "${af}" == 'gensrc'* ]] || + [[ "${af}" == 'thirdparty'* ]] || + [[ "${af}" == 'build.sh' ]] || + [[ "${af}" == 'env.sh' ]] || + [[ "${af}" == 'regression-test/pipeline/common/github-utils.sh' ]] || + [[ "${af}" == 'regression-test/pipeline/common/doris-utils.sh' ]] || + [[ "${af}" == 'regression-test/pipeline/common/oss-utils.sh' ]] || + [[ "${af}" == 'tools/tpch-tools/bin/run-tpch-queries.sh' ]] || + [[ "${af}" == 'regression-test/pipeline/tpch/tpch-sf100/'* ]]; then + echo "clickbench performance related file changed, return need" && return 0 + fi + done + echo "return no need" && return 1 +} + file_changed_performance() { local all_files all_files=$(cat all_files) diff --git a/regression-test/pipeline/performance/run-tpcds.sh b/regression-test/pipeline/performance/run-tpcds.sh index 0ca31c57e74a04..d054b85ee4da0a 100644 --- a/regression-test/pipeline/performance/run-tpcds.sh +++ b/regression-test/pipeline/performance/run-tpcds.sh @@ -125,7 +125,6 @@ exit_flag=0 echo "#### 3. run tpcds-sf${SF} query" set_session_variable runtime_filter_mode global bash "${teamcity_build_checkoutDir}"/tools/tpcds-tools/bin/run-tpcds-queries.sh -s "${SF}" | tee "${teamcity_build_checkoutDir}"/run-tpcds-queries.log - echo cold_run_time_threshold=${cold_run_time_threshold_master:-315000} # ms hot_run_time_threshold=${hot_run_time_threshold_master:-190000} # ms if [[ "${target_branch}" == "branch-2.0" ]]; then diff --git a/regression-test/pipeline/performance/run-tpch.sh b/regression-test/pipeline/performance/run-tpch.sh index f1e4c8df2a2e1b..1620c5f2922d73 100644 --- a/regression-test/pipeline/performance/run-tpch.sh +++ b/regression-test/pipeline/performance/run-tpch.sh @@ -117,7 +117,6 @@ exit_flag=0 echo "#### 3. run tpch-sf${SF} query" set_session_variable runtime_filter_mode global bash "${teamcity_build_checkoutDir}"/tools/tpch-tools/bin/run-tpch-queries.sh -s "${SF}" | tee "${teamcity_build_checkoutDir}"/run-tpch-queries.log - echo cold_run_time_threshold=${cold_run_time_threshold_master:-120000} # ms hot_run_time_threshold=${hot_run_time_threshold_master:-42000} # ms if [[ "${target_branch}" == "branch-2.0" ]]; then diff --git a/regression-test/pipeline/tpch/tpch-sf100/clean.sh b/regression-test/pipeline/tpch/tpch-sf100/clean.sh new file mode 100644 index 00000000000000..16fa490ef9c237 --- /dev/null +++ b/regression-test/pipeline/tpch/tpch-sf100/clean.sh @@ -0,0 +1,40 @@ +#!/usr/bin/env bash +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +# Build Step: Command Line +: <:', separate by ',' +# property 'medium' has a higher priority than the extension of path +# +# Default value is ${DORIS_HOME}/storage, you should create it by hand. +# storage_root_path = ${DORIS_HOME}/storage + +# Default dirs to put jdbc drivers,default value is ${DORIS_HOME}/jdbc_drivers +# jdbc_drivers_dir = ${DORIS_HOME}/jdbc_drivers + +# Advanced configurations +# sys_log_dir = ${DORIS_HOME}/log +# sys_log_roll_mode = SIZE-MB-1024 +# sys_log_roll_num = 10 +# sys_log_verbose_modules = * +# log_buffer_level = -1 +# palo_cgroups + +priority_networks=172.16.0.0/24 +storage_root_path=/mnt/datadisk0/doris-storage diff --git a/regression-test/pipeline/tpch/tpch-sf100/conf/external.json b/regression-test/pipeline/tpch/tpch-sf100/conf/external.json new file mode 100644 index 00000000000000..9461d836e6749a --- /dev/null +++ b/regression-test/pipeline/tpch/tpch-sf100/conf/external.json @@ -0,0 +1,26 @@ +[ + { + "file": "docker/thirdparties/docker-compose/mysql/mysql-5.7.env", + "replacements": { + "DOCKER_MYSQL_57_EXTERNAL_PORT": 7111 + } + }, + { + "file": "docker/thirdparties/docker-compose/postgresql/postgresql-14.env", + "replacements": { + "DOCKER_PG_14_EXTERNAL_PORT": 7121 + } + }, + { + "file": "docker/thirdparties/docker-compose/hive/gen_env.sh", + "replacements": { + "FS_PORT": 7131, + "HMS_PORT": 7141 + } + }, { + "file": "docker/thirdparties/start-thirdparties-docker.sh", + "replacements": { + "CONTAINER_UID": "doris-regression-fakeid-fakecommit" + } + } +] diff --git a/regression-test/pipeline/tpch/tpch-sf100/conf/fe.conf b/regression-test/pipeline/tpch/tpch-sf100/conf/fe.conf new file mode 100644 index 00000000000000..7c02d3898dcb45 --- /dev/null +++ b/regression-test/pipeline/tpch/tpch-sf100/conf/fe.conf @@ -0,0 +1,78 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +##################################################################### +## The uppercase properties are read and exported by bin/start_fe.sh. +## To see all Frontend configurations, +## see fe/src/org/apache/doris/common/Config.java +##################################################################### + +CUR_DATE=`date +%Y%m%d-%H%M%S` + +# the output dir of stderr and stdout +LOG_DIR = ${DORIS_HOME}/log + +JAVA_OPTS="-Dsun.security.krb5.debug=true -Djavax.security.auth.useSubjectCredsOnly=false -Xss4m -Xmx8192m -XX:+UseMembar -XX:SurvivorRatio=8 -XX:MaxTenuringThreshold=7 -XX:+PrintGCDateStamps -XX:+PrintGCDetails -XX:+UseConcMarkSweepGC -XX:+UseParNewGC -XX:+CMSClassUnloadingEnabled -XX:-CMSParallelRemarkEnabled -XX:CMSInitiatingOccupancyFraction=80 -XX:SoftRefLRUPolicyMSPerMB=0 -Xloggc:$DORIS_HOME/log/fe.gc.log.$CUR_DATE" + +# For jdk 9+, this JAVA_OPTS will be used as default JVM options +JAVA_OPTS_FOR_JDK_9="-Dsun.security.krb5.debug=true -Djavax.security.auth.useSubjectCredsOnly=false -Xss4m -Xmx8192m -XX:SurvivorRatio=8 -XX:MaxTenuringThreshold=7 -XX:+CMSClassUnloadingEnabled -XX:-CMSParallelRemarkEnabled -XX:CMSInitiatingOccupancyFraction=80 -XX:SoftRefLRUPolicyMSPerMB=0 -Xlog:gc*:$DORIS_HOME/log/fe.gc.log.$CUR_DATE:time" + +## +## the lowercase properties are read by main program. +## + +# INFO, WARN, ERROR, FATAL +sys_log_level = INFO + +# NORMAL, BRIEF, ASYNC +sys_log_mode = NORMAL + +# store metadata, must be created before start FE. +# Default value is ${DORIS_HOME}/doris-meta +# meta_dir = ${DORIS_HOME}/doris-meta + +# Default dirs to put jdbc drivers,default value is ${DORIS_HOME}/jdbc_drivers +# jdbc_drivers_dir = ${DORIS_HOME}/jdbc_drivers + +http_port = 8030 +rpc_port = 9020 +query_port = 9030 +edit_log_port = 9010 +arrow_flight_sql_port = -1 + +# Choose one if there are more than one ip except loopback address. +# Note that there should at most one ip match this list. +# If no ip match this rule, will choose one randomly. +# use CIDR format, e.g. 10.10.10.0/24 or IP format, e.g. 10.10.10.1 +# Default value is empty. +# priority_networks = 10.10.10.0/24;192.168.0.0/16 + +# Advanced configurations +# log_roll_size_mb = 1024 +# sys_log_dir = ${DORIS_HOME}/log +# sys_log_roll_num = 10 +# sys_log_verbose_modules = org.apache.doris +# audit_log_dir = ${DORIS_HOME}/log +# audit_log_modules = slow_query, query +# audit_log_roll_num = 10 +# meta_delay_toleration_second = 10 +# qe_max_connection = 1024 +# qe_query_timeout_second = 300 +# qe_slow_log_ms = 5000 + +priority_networks=172.16.0.0/24 +meta_dir=/mnt/datadisk0/doris-meta diff --git a/regression-test/pipeline/tpch/tpch-sf100/conf/odbcinst.ini b/regression-test/pipeline/tpch/tpch-sf100/conf/odbcinst.ini new file mode 100644 index 00000000000000..41e21f92277e98 --- /dev/null +++ b/regression-test/pipeline/tpch/tpch-sf100/conf/odbcinst.ini @@ -0,0 +1,43 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +# Example driver definitions, you should not use the example odbc driver +# before you prepare env in your server + +# Driver from the postgresql-odbc package +# Setup from the unixODBC package +[PostgreSQL] +Description = ODBC for PostgreSQL +Driver = /usr/lib/psqlodbc.so +Setup = /usr/lib/libodbcpsqlS.so +FileUsage = 1 + + +# Driver from the mysql-connector-odbc package +# Setup from the unixODBC package +[MySQL ODBC 8.0 Unicode Driver] +Description = ODBC for MySQL +Driver = /usr/lib64/libmyodbc8w.so +FileUsage = 1 + +# Driver from the oracle-connector-odbc package +# Setup from the unixODBC package +[Oracle 19 ODBC driver] +Description=Oracle ODBC driver for Oracle 19 +Driver=/usr/lib/libsqora.so.19.1 + + diff --git a/regression-test/pipeline/tpch/tpch-sf100/deploy.sh b/regression-test/pipeline/tpch/tpch-sf100/deploy.sh new file mode 100644 index 00000000000000..64d816f0acadaf --- /dev/null +++ b/regression-test/pipeline/tpch/tpch-sf100/deploy.sh @@ -0,0 +1,119 @@ +#!/usr/bin/env bash +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +# Build Step: Command Line +: </dev/null; then sudo apt install -y pigz; fi + tar -I pigz -xf "${pull_request_id:-}_${commit_id:-}.tar.gz" + if [[ -d output && -d output/fe && -d output/be ]]; then + echo "INFO: be version: $(./output/be/lib/doris_be --version)" + rm -rf "${pull_request_id}_${commit_id}.tar.gz" + fi +else + echo "ERROR: download compiled binary failed" && exit 1 +fi + +echo "#### 3. copy conf from regression-test/pipeline/tpch/tpch-sf100/conf/" +rm -f "${DORIS_HOME}"/fe/conf/fe_custom.conf "${DORIS_HOME}"/be/conf/be_custom.conf +if [[ -f "${teamcity_build_checkoutDir}"/regression-test/pipeline/tpch/tpch-sf100/conf/fe.conf && + -f "${teamcity_build_checkoutDir}"/regression-test/pipeline/tpch/tpch-sf100/conf/be.conf ]]; then + cp -f "${teamcity_build_checkoutDir}"/regression-test/pipeline/tpch/tpch-sf100/conf/fe.conf "${DORIS_HOME}"/fe/conf/ + cp -f "${teamcity_build_checkoutDir}"/regression-test/pipeline/tpch/tpch-sf100/conf/be.conf "${DORIS_HOME}"/be/conf/ +else + echo "ERROR: doris conf file missing in ${teamcity_build_checkoutDir}/regression-test/pipeline/tpch/tpch-sf100/conf/" + exit 1 +fi + +echo "#### 4. start Doris" +meta_dir=$(get_doris_conf_value "${DORIS_HOME}"/fe/conf/fe.conf meta_dir) +storage_root_path=$(get_doris_conf_value "${DORIS_HOME}"/be/conf/be.conf storage_root_path) +mkdir -p "${meta_dir}" +mkdir -p "${storage_root_path}" +if ! start_doris_fe; then + echo "ERROR: Start doris fe failed." + print_doris_fe_log + need_backup_doris_logs=true + exit_flag=1 +fi +if ! start_doris_be; then + echo "ERROR: Start doris be failed." + print_doris_be_log + need_backup_doris_logs=true + exit_flag=1 +fi + +# wait 10s for doris totally started, otherwize may encounter the error below, +# ERROR 1105 (HY000) at line 102: errCode = 2, detailMessage = Failed to find enough backend, please check the replication num,replication tag and storage medium. +sleep 10s + +echo "#### 5. set session variables" +echo "TODO" + +echo "#### 6. check if need backup doris logs" +if ${need_backup_doris_logs}; then + print_doris_fe_log + print_doris_be_log + if file_name=$(archive_doris_logs "${pull_request_id}_${commit_id}_doris_logs.tar.gz"); then + upload_doris_log_to_oss "${file_name}" + fi +fi + +exit "${exit_flag}" diff --git a/regression-test/pipeline/tpch/tpch-sf100/prepare.sh b/regression-test/pipeline/tpch/tpch-sf100/prepare.sh new file mode 100644 index 00000000000000..07959a03d6f8e7 --- /dev/null +++ b/regression-test/pipeline/tpch/tpch-sf100/prepare.sh @@ -0,0 +1,69 @@ +#!/usr/bin/env bash +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +# Build Step: Command Line +: </dev/null; then exit 1; fi + line_end=$(sed -n '/^Total hot run time/=' "${teamcity_build_checkoutDir}"/run-tpch-queries.log) + line_begin=$((line_end - 23)) + comment_body="${comment_body} + +run tpch-sf${SF} query with default conf and set session variable runtime_filter_mode=off +$(sed -n "${line_begin},${line_end}p" "${teamcity_build_checkoutDir}"/run-tpch-queries.log)" + + echo "#### 4. comment result on tpch" + comment_body=$(echo "${comment_body}" | sed -e ':a;N;$!ba;s/\t/\\t/g;s/\n/\\n/g') # 将所有的 Tab字符替换为\t 换行符替换为\n + create_an_issue_comment_tpch "${pull_request_id:-}" "${comment_body}" + + stop_doris +) +exit_flag="$?" + +echo "#### 5. check if need backup doris logs" +if [[ ${exit_flag} != "0" ]]; then + print_doris_fe_log + print_doris_be_log + if file_name=$(archive_doris_logs "${pull_request_id}_${commit_id}_doris_logs.tar.gz"); then + upload_doris_log_to_oss "${file_name}" + fi +fi + +exit "${exit_flag}" From d6ed4149f5b2b15e97a5a8861387141749c0339e Mon Sep 17 00:00:00 2001 From: Chester <42577861+superdiaodiao@users.noreply.github.com> Date: Sun, 21 Jan 2024 10:51:58 +0800 Subject: [PATCH 108/200] [feature](function) support ip function is_ipv4_compat, is_ipv4_mapped (#29954) --- be/src/vec/common/unaligned.h | 33 ++++- be/src/vec/functions/function_ip.cpp | 4 + be/src/vec/functions/function_ip.h | 130 ++++++++++++------ .../ip-functions/is-ipv4-compat.md | 65 +++++++++ .../ip-functions/is-ipv4-mapped.md | 69 ++++++++++ docs/sidebars.json | 7 +- .../ip-functions/is-ipv4-compat.md | 65 +++++++++ .../ip-functions/is-ipv4-mapped.md | 68 +++++++++ .../doris/catalog/BuiltinScalarFunctions.java | 4 + .../functions/scalar/IsIpv4Compat.java | 67 +++++++++ .../functions/scalar/IsIpv4Mapped.java | 67 +++++++++ .../visitor/ScalarFunctionVisitor.java | 10 ++ gensrc/script/doris_builtins_functions.py | 4 + .../data/nereids_function_p0/ip_functions.out | 32 ++++- .../ip_functions/test_ip_functions.out | 32 ++++- .../nereids_function_p0/ip_functions.groovy | 12 ++ .../ip_functions/test_ip_functions.groovy | 12 ++ 17 files changed, 635 insertions(+), 46 deletions(-) create mode 100644 docs/en/docs/sql-manual/sql-functions/ip-functions/is-ipv4-compat.md create mode 100644 docs/en/docs/sql-manual/sql-functions/ip-functions/is-ipv4-mapped.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/is-ipv4-compat.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/is-ipv4-mapped.md create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IsIpv4Compat.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IsIpv4Mapped.java diff --git a/be/src/vec/common/unaligned.h b/be/src/vec/common/unaligned.h index d56eca59b23f39..ee807c606e53ea 100644 --- a/be/src/vec/common/unaligned.h +++ b/be/src/vec/common/unaligned.h @@ -20,8 +20,9 @@ #pragma once -#include - +#include +#include +#include #include template @@ -40,3 +41,31 @@ void unaligned_store(void* address, const typename std::enable_if::type static_assert(std::is_trivially_copyable_v); memcpy(address, &src, sizeof(src)); } + +inline void reverse_memcpy(void* dst, const void* src, size_t size) { + uint8_t* uint_dst = reinterpret_cast(dst) + size; // Perform addition here + const uint8_t* uint_src = reinterpret_cast(src); + + while (size) { + --uint_dst; + *uint_dst = *uint_src; + ++uint_src; + --size; + } +} + +template +inline T unaligned_load_endian(const void* address) { + T res {}; + if constexpr (std::endian::native == endian) { + memcpy(&res, address, sizeof(res)); + } else { + reverse_memcpy(&res, address, sizeof(res)); + } + return res; +} + +template +inline T unaligned_load_little_endian(const void* address) { + return unaligned_load_endian(address); +} \ No newline at end of file diff --git a/be/src/vec/functions/function_ip.cpp b/be/src/vec/functions/function_ip.cpp index 3faa6a42d8c115..140100d22b3230 100644 --- a/be/src/vec/functions/function_ip.cpp +++ b/be/src/vec/functions/function_ip.cpp @@ -27,6 +27,7 @@ void register_function_ip(SimpleFunctionFactory& factory) { factory.register_function>(); factory.register_alias(FunctionIPv4StringToNum::name, "inet_aton"); + factory.register_function(); factory.register_alias(FunctionIPv6NumToString::name, "inet6_ntoa"); factory.register_function>(); @@ -34,6 +35,9 @@ void register_function_ip(SimpleFunctionFactory& factory) { factory.register_function>(); factory.register_alias(FunctionIPv6StringToNum::name, "inet6_aton"); + + factory.register_function(); + factory.register_function(); factory.register_function>(); factory.register_function>(); factory.register_function(); diff --git a/be/src/vec/functions/function_ip.h b/be/src/vec/functions/function_ip.h index 313ad0fdd65b1b..8a260b8fd82397 100644 --- a/be/src/vec/functions/function_ip.h +++ b/be/src/vec/functions/function_ip.h @@ -364,44 +364,6 @@ ColumnPtr convertToIPv6(const StringColumnType& string_column, vec_null_map_to = &col_null_map_to->get_data(); } - /// This is a special treatment for source column of type String - /// to preserve previous behavior when IPv6 was a domain type of String - if constexpr (std::is_same_v) { - if (string_column.get_offsets()[0] - 1 == IPV6_BINARY_LENGTH) { - if constexpr (std::is_same_v) { - auto col_res = ColumnString::create(); - - if constexpr (exception_mode == IPStringToNumExceptionMode::Null) { - col_null_map_to = ColumnUInt8::create(column_size, false); - if (null_map) { - memcpy(col_null_map_to->get_data().data(), null_map->data(), column_size); - } - - return ColumnNullable::create(std::move(col_res), std::move(col_null_map_to)); - } - - return col_res; - } else { - auto col_res = ColumnIPv6::create(); - auto& vec_res = col_res->get_data(); - - vec_res.resize(column_size); - memcpy(vec_res.data(), string_column.get_chars().data(), - column_size * IPV6_BINARY_LENGTH); - - if constexpr (exception_mode == IPStringToNumExceptionMode::Null) { - col_null_map_to = ColumnUInt8::create(column_size, false); - if (null_map) { - memcpy(col_null_map_to->get_data().data(), null_map->data(), column_size); - } - return ColumnNullable::create(std::move(col_res), std::move(col_null_map_to)); - } - - return col_res; - } - } - } - auto column_create = [](size_t column_size) -> typename ToColumn::MutablePtr { if constexpr (std::is_same_v) { auto column_string = ColumnString::create(); @@ -833,4 +795,94 @@ class FunctionIPv6CIDRToRange : public IFunction { } }; -} // namespace doris::vectorized \ No newline at end of file +class FunctionIsIPv4Compat : public IFunction { +public: + static constexpr auto name = "is_ipv4_compat"; + static FunctionPtr create() { return std::make_shared(); } + + String get_name() const override { return name; } + + size_t get_number_of_arguments() const override { return 1; } + + DataTypePtr get_return_type_impl(const DataTypes& arguments) const override { + return std::make_shared(); + } + + Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, + size_t result, size_t input_rows_count) const override { + const ColumnPtr& column = block.get_by_position(arguments[0]).column; + const auto* col_in = check_and_get_column(column.get()); + + if (!col_in) + throw Exception(ErrorCode::INVALID_ARGUMENT, + "Illegal column {} of argument of function {}, expected String", + column->get_name(), get_name()); + size_t col_size = col_in->size(); + auto col_res = ColumnUInt8::create(col_size, 0); + auto& col_res_data = col_res->get_data(); + + for (size_t i = 0; i < col_size; ++i) { + auto ipv4_in = col_in->get_data_at(i); + if (is_ipv4_compat(reinterpret_cast(ipv4_in.data))) { + col_res_data[i] = 1; + } + } + + block.replace_by_position(result, std::move(col_res)); + return Status::OK(); + } + +private: + static bool is_ipv4_compat(const UInt8* address) { + return (unaligned_load_little_endian(address) == 0) && + (unaligned_load_little_endian(address + 8) == 0) && + (unaligned_load_little_endian(address + 12) != 0); + } +}; + +class FunctionIsIPv4Mapped : public IFunction { +public: + static constexpr auto name = "is_ipv4_mapped"; + static FunctionPtr create() { return std::make_shared(); } + + String get_name() const override { return name; } + + size_t get_number_of_arguments() const override { return 1; } + + DataTypePtr get_return_type_impl(const DataTypes& arguments) const override { + return std::make_shared(); + } + + Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, + size_t result, size_t input_rows_count) const override { + const ColumnPtr& column = block.get_by_position(arguments[0]).column; + const auto* col_in = check_and_get_column(column.get()); + + if (!col_in) + throw Exception(ErrorCode::INVALID_ARGUMENT, + "Illegal column {} of argument of function {}, expected String", + column->get_name(), get_name()); + size_t col_size = col_in->size(); + auto col_res = ColumnUInt8::create(col_size, 0); + auto& col_res_data = col_res->get_data(); + + for (size_t i = 0; i < col_size; ++i) { + auto ipv4_in = col_in->get_data_at(i); + if (is_ipv4_mapped(reinterpret_cast(ipv4_in.data))) { + col_res_data[i] = 1; + } + } + + block.replace_by_position(result, std::move(col_res)); + return Status::OK(); + } + +private: + static bool is_ipv4_mapped(const UInt8* address) { + return (unaligned_load_little_endian(address) == 0) && + ((unaligned_load_little_endian(address + 8) & 0x00000000FFFFFFFFULL) == + 0x00000000FFFF0000ULL); + } +}; + +} // namespace doris::vectorized diff --git a/docs/en/docs/sql-manual/sql-functions/ip-functions/is-ipv4-compat.md b/docs/en/docs/sql-manual/sql-functions/ip-functions/is-ipv4-compat.md new file mode 100644 index 00000000000000..84c5aff9b63546 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/ip-functions/is-ipv4-compat.md @@ -0,0 +1,65 @@ +--- +{ +"title": "IS_IPV4_COMPAT", +"language": "en" +} +--- + + + +## IS_IPV4_COMPAT + + + +IS_IPV4_COMPAT + + + +### description + +#### Syntax + +`VARCHAR IS_IPV4_COMPAT(INET6_ATON(VARCHAR ipv4_addr))` + +This function takes an IPv6 address represented in numeric form as a binary string, as returned by INET6_ATON(). +It returns 1 if the argument is a valid IPv4-compatible IPv6 address, 0 otherwise (unless expr is NULL, in which case the function returns NULL). +IPv4-compatible addresses have the form ::ipv4_address. + +### example + +``` +mysql> SELECT IS_IPV4_COMPAT(INET6_ATON('::ffff:10.0.5.9')) AS is_result; ++-----------+ +| is_result | ++-----------+ +| 0 | ++-----------+ +1 row in set (0.02 sec) + +mysql> SELECT IS_IPV4_COMPAT(INET6_ATON('::10.0.5.9')) AS is_result; ++-----------+ +| is_result | ++-----------+ +| 1 | ++-----------+ +1 row in set (0.03 sec) +``` + +### keywords + +IS_IPV4_COMPAT, IP \ No newline at end of file diff --git a/docs/en/docs/sql-manual/sql-functions/ip-functions/is-ipv4-mapped.md b/docs/en/docs/sql-manual/sql-functions/ip-functions/is-ipv4-mapped.md new file mode 100644 index 00000000000000..9e338eb13539ad --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/ip-functions/is-ipv4-mapped.md @@ -0,0 +1,69 @@ +--- +{ +"title": "IS_IPV4_MAPPED", +"language": "en" +} +--- + + + +## IS_IPV4_MAPPED + + + +IS_IPV4_MAPPED + + + +### description + +#### Syntax + +`VARCHAR IS_IPV4_MAPPED(INET6_ATON(VARCHAR ipv4_addr))` + +This function takes an IPv6 address represented in numeric form as a binary string, as returned by INET6_ATON(). +It returns 1 if the argument is a valid IPv4-mapped IPv6 address, 0 otherwise, unless expr is NULL, in which case the function returns NULL. +IPv4-mapped addresses have the form ::ffff:ipv4_address. + +### notice + +`When the source input doesn't have a prefix of '::ffff:', but if it's still a valid ipv4 address, this result will also be 1 for the reason that the INET6_ATON() automatically adds the prefix for it.` + +### example + +``` +mysql> SELECT IS_IPV4_MAPPED(INET6_ATON('::ffff:10.0.5.9')) AS is_result; ++-----------+ +| is_result | ++-----------+ +| 1 | ++-----------+ +1 row in set (0.02 sec) + +mysql> SELECT IS_IPV4_MAPPED(INET6_ATON('::10.0.5.9')) AS is_result; ++-----------+ +| is_result | ++-----------+ +| 0 | ++-----------+ +1 row in set (0.03 sec) +``` + +### keywords + +IS_IPV4_MAPPED, IP \ No newline at end of file diff --git a/docs/sidebars.json b/docs/sidebars.json index 6b94989d1808be..bda6b8da3105ce 100644 --- a/docs/sidebars.json +++ b/docs/sidebars.json @@ -811,11 +811,12 @@ "sql-manual/sql-functions/ip-functions/ipv4-string-to-num-or-null", "sql-manual/sql-functions/ip-functions/ipv6-num-to-string", "sql-manual/sql-functions/ip-functions/inet6-ntoa", - "sql-manual/sql-functions/ip-functions/ipv6-num-to-string", + "sql-manual/sql-functions/ip-functions/ipv6-string-to-num", "sql-manual/sql-functions/ip-functions/inet6-aton", "sql-manual/sql-functions/ip-functions/ipv6-string-to-num-or-default", - "sql-manual/sql-functions/ip-functions/ipv6-string-to-num-or-null" - + "sql-manual/sql-functions/ip-functions/ipv6-string-to-num-or-null", + "sql-manual/sql-functions/ip-functions/is-ipv4-compat", + "sql-manual/sql-functions/ip-functions/is-ipv4-mapped" ] }, { diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/is-ipv4-compat.md b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/is-ipv4-compat.md new file mode 100644 index 00000000000000..fba93b83430153 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/is-ipv4-compat.md @@ -0,0 +1,65 @@ +--- +{ +"title": "IS_IPV4_COMPAT", +"language": "zh-CN" +} +--- + + + +## IS_IPV4_COMPAT + + + +IS_IPV4_COMPAT + + + +### description + +#### Syntax + +`VARCHAR IS_IPV4_COMPAT(INET6_ATON(VARCHAR ipv4_addr))` + +该函数采用以数字形式表示的二进制字符串形式的 IPv6 地址,由 INET6_ATON() 返回。 +如果参数是有效的 IPv4 兼容 IPv6 地址,则返回 1,否则返回 0(除非 expr 为 NULL,在这种情况下该函数返回 NULL)。 +IPv4 兼容地址的格式为::ipv4_address。 + +### example + +``` +mysql> SELECT IS_IPV4_COMPAT(INET6_ATON('::ffff:10.0.5.9')) AS is_result; ++-----------+ +| is_result | ++-----------+ +| 0 | ++-----------+ +1 row in set (0.02 sec) + +mysql> SELECT IS_IPV4_COMPAT(INET6_ATON('::10.0.5.9')) AS is_result; ++-----------+ +| is_result | ++-----------+ +| 1 | ++-----------+ +1 row in set (0.03 sec) +``` + +### keywords + +IS_IPV4_COMPAT, IP \ No newline at end of file diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/is-ipv4-mapped.md b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/is-ipv4-mapped.md new file mode 100644 index 00000000000000..66de4022209340 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/is-ipv4-mapped.md @@ -0,0 +1,68 @@ +--- +{ +"title": "IS_IPV4_MAPPED", +"language": "zh-CN" +} +--- + + + +## IS_IPV4_MAPPED + + + +IS_IPV4_MAPPED + + + +### description + +#### Syntax + +`VARCHAR IS_IPV4_MAPPED(INET6_ATON(VARCHAR ipv4_addr))` + +该函数采用以数字形式表示的二进制字符串形式的lPv6地址,由INET6_ATON返回。 +如果参数是有效的IPv4映射IPv6地址,则返回1,否则返回0,除非expr为 NULL,在这种情况下该函数返回NULL。 +IPv4映射地址的格式为::ffff:ipv4_address + +### notice +`当源输入没有'::ffff:'前缀时,但如果它仍然是有效的ipv4地址,则该结果也将为1,因为INET6_ATON()会自动为有效的ipv4地址添加前缀。` + +### example + +``` +mysql> SELECT IS_IPV4_MAPPED(INET6_ATON('::ffff:10.0.5.9')) AS is_result; ++-----------+ +| is_result | ++-----------+ +| 1 | ++-----------+ +1 row in set (0.02 sec) + +mysql> SELECT IS_IPV4_MAPPED(INET6_ATON('::10.0.5.9')) AS is_result; ++-----------+ +| is_result | ++-----------+ +| 0 | ++-----------+ +1 row in set (0.03 sec) +``` + +### keywords + +IS_IPV4_MAPPED, IP \ No newline at end of file diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java index f4f7af5cf387eb..68da4df08abc03 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java @@ -203,6 +203,8 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv6StringToNumOrDefault; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv6StringToNumOrNull; import org.apache.doris.nereids.trees.expressions.functions.scalar.IsIpAddressInRange; +import org.apache.doris.nereids.trees.expressions.functions.scalar.IsIpv4Compat; +import org.apache.doris.nereids.trees.expressions.functions.scalar.IsIpv4Mapped; import org.apache.doris.nereids.trees.expressions.functions.scalar.IsIpv4String; import org.apache.doris.nereids.trees.expressions.functions.scalar.IsIpv6String; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonArray; @@ -609,6 +611,8 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(Ipv6StringToNum.class, "ipv6_string_to_num", "inet6_aton"), scalar(Ipv6StringToNumOrDefault.class, "ipv6_string_to_num_or_default"), scalar(Ipv6StringToNumOrNull.class, "ipv6_string_to_num_or_null"), + scalar(IsIpv4Compat.class, "is_ipv4_compat"), + scalar(IsIpv4Mapped.class, "is_ipv4_mapped"), scalar(IsIpv4String.class, "is_ipv4_string"), scalar(IsIpv6String.class, "is_ipv6_string"), scalar(IsIpAddressInRange.class, "is_ip_address_in_range"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IsIpv4Compat.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IsIpv4Compat.java new file mode 100644 index 00000000000000..684eee0e9c9cd9 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IsIpv4Compat.java @@ -0,0 +1,67 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; +import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.BooleanType; +import org.apache.doris.nereids.types.StringType; +import org.apache.doris.nereids.types.VarcharType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * scalar function is_ipv4_compat + */ +public class IsIpv4Compat extends ScalarFunction + implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(BooleanType.INSTANCE).args(VarcharType.SYSTEM_DEFAULT), + FunctionSignature.ret(BooleanType.INSTANCE).args(StringType.INSTANCE)); + + public IsIpv4Compat(Expression arg0) { + super("is_ipv4_compat", arg0); + } + + @Override + public IsIpv4Compat withChildren(List children) { + Preconditions.checkArgument(children.size() == 1, + "is_ipv4_compat accept 1 args, but got %s (%s)", + children.size(), + children); + return new IsIpv4Compat(children.get(0)); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitIsIpv4Compat(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IsIpv4Mapped.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IsIpv4Mapped.java new file mode 100644 index 00000000000000..c3640af5437cea --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/IsIpv4Mapped.java @@ -0,0 +1,67 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; +import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.BooleanType; +import org.apache.doris.nereids.types.StringType; +import org.apache.doris.nereids.types.VarcharType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * scalar function is_ipv4_mapped + */ +public class IsIpv4Mapped extends ScalarFunction + implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(BooleanType.INSTANCE).args(VarcharType.SYSTEM_DEFAULT), + FunctionSignature.ret(BooleanType.INSTANCE).args(StringType.INSTANCE)); + + public IsIpv4Mapped(Expression arg0) { + super("is_ipv4_mapped", arg0); + } + + @Override + public IsIpv4Mapped withChildren(List children) { + Preconditions.checkArgument(children.size() == 1, + "is_ipv4_mapped accept 1 args, but got %s (%s)", + children.size(), + children); + return new IsIpv4Mapped(children.get(0)); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitIsIpv4Mapped(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java index 899a31abbd7d01..01823f78199856 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java @@ -199,6 +199,8 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv6StringToNumOrDefault; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv6StringToNumOrNull; import org.apache.doris.nereids.trees.expressions.functions.scalar.IsIpAddressInRange; +import org.apache.doris.nereids.trees.expressions.functions.scalar.IsIpv4Compat; +import org.apache.doris.nereids.trees.expressions.functions.scalar.IsIpv4Mapped; import org.apache.doris.nereids.trees.expressions.functions.scalar.IsIpv4String; import org.apache.doris.nereids.trees.expressions.functions.scalar.IsIpv6String; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonArray; @@ -1155,6 +1157,14 @@ default R visitIpv6StringToNumOrNull(Ipv6StringToNumOrNull ipv6StringToNumOrNull return visitScalarFunction(ipv6StringToNumOrNull, context); } + default R visitIsIpv4Compat(IsIpv4Compat isIpv4Compat, C context) { + return visitScalarFunction(isIpv4Compat, context); + } + + default R visitIsIpv4Mapped(IsIpv4Mapped isIpv4Mapped, C context) { + return visitScalarFunction(isIpv4Mapped, context); + } + default R visitIsIpv4String(IsIpv4String isIpv4String, C context) { return visitScalarFunction(isIpv4String, context); } diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py index 973a4675621547..6a6ff96e83da8b 100644 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -2014,6 +2014,10 @@ [['ipv6_string_to_num_or_default'], 'STRING', ['STRING'], 'ALWAYS_NOT_NULLABLE'], [['ipv6_string_to_num_or_null'], 'VARCHAR', ['VARCHAR'], 'ALWAYS_NULLABLE'], [['ipv6_string_to_num_or_null'], 'STRING', ['STRING'], 'ALWAYS_NULLABLE'], + [['is_ipv4_compat'], 'BOOLEAN', ['VARCHAR'], ''], + [['is_ipv4_compat'], 'BOOLEAN', ['STRING'], ''], + [['is_ipv4_mapped'], 'BOOLEAN', ['VARCHAR'], ''], + [['is_ipv4_mapped'], 'BOOLEAN', ['STRING'], ''], [['is_ipv4_string'], 'BOOLEAN', ['VARCHAR'], ''], [['is_ipv4_string'], 'BOOLEAN', ['STRING'], ''], [['is_ipv6_string'], 'BOOLEAN', ['VARCHAR'], ''], diff --git a/regression-test/data/nereids_function_p0/ip_functions.out b/regression-test/data/nereids_function_p0/ip_functions.out index 2e3b749a1529d4..914b5ccd683952 100644 --- a/regression-test/data/nereids_function_p0/ip_functions.out +++ b/regression-test/data/nereids_function_p0/ip_functions.out @@ -162,4 +162,34 @@ false true -- !ip55 -- -false \ No newline at end of file +false + +-- !ip56 -- +true + +-- !ip57 -- +false + +-- !ip58 -- +false + +-- !ip59 -- +true + +-- !ip60 -- +false + +-- !ip61 -- +true + +-- !ip62 -- +false + +-- !ip63 -- +true + +-- !ip64 -- +false + +-- !ip65 -- +true \ No newline at end of file diff --git a/regression-test/data/query_p0/sql_functions/ip_functions/test_ip_functions.out b/regression-test/data/query_p0/sql_functions/ip_functions/test_ip_functions.out index 71f02449d3bd34..1837acc4bab4e9 100644 --- a/regression-test/data/query_p0/sql_functions/ip_functions/test_ip_functions.out +++ b/regression-test/data/query_p0/sql_functions/ip_functions/test_ip_functions.out @@ -137,4 +137,34 @@ AAAAAAAAFFFFFFFFFFFFFFFFAAAAAAAA 00000000000000000000FFFFC0A80001 -- !sql -- -2A0206B8000000000000000000000011 \ No newline at end of file +2A0206B8000000000000000000000011 + +-- !sql -- +true + +-- !sql -- +false + +-- !sql -- +false + +-- !sql -- +true + +-- !sql -- +false + +-- !sql -- +true + +-- !sql -- +false + +-- !sql -- +true + +-- !sql -- +false + +-- !sql -- +true \ No newline at end of file diff --git a/regression-test/suites/nereids_function_p0/ip_functions.groovy b/regression-test/suites/nereids_function_p0/ip_functions.groovy index 1ac001020943b4..b5d4d177e89fb0 100644 --- a/regression-test/suites/nereids_function_p0/ip_functions.groovy +++ b/regression-test/suites/nereids_function_p0/ip_functions.groovy @@ -77,4 +77,16 @@ suite("ip_functions") { qt_ip53 "SELECT is_ipv4_string('255.255.255.256');" qt_ip54 "SELECT is_ipv6_string('2001:5b0:23ff:fffa::113');" qt_ip55 "SELECT is_ipv6_string('2001:da8:e000:1691:2eaa:7eff:ffe7:7924e');" + + qt_ip56 "SELECT is_ipv4_compat(inet6_aton('::10.0.5.9'));" + qt_ip57 "SELECT is_ipv4_compat(inet6_aton('::ffff:10.0.5.9'));" + qt_ip58 "SELECT is_ipv4_compat(inet6_aton('::'));" + qt_ip59 "SELECT is_ipv4_compat(inet6_aton('::c0a8:0001'));" + qt_ip60 "SELECT is_ipv4_compat(inet6_aton('::0.0.0.0'));" + qt_ip61 "SELECT is_ipv4_compat(inet6_aton('::255.255.255.255'));" + + qt_ip62 "SELECT is_ipv4_mapped(inet6_aton('::10.0.5.9'));" + qt_ip63 "SELECT is_ipv4_mapped(inet6_aton('::ffff:10.0.5.9'));" + qt_ip64 "SELECT is_ipv4_mapped(inet6_aton('::'));" + qt_ip65 "SELECT is_ipv4_mapped(inet6_aton('::ffff:c0a8:0001'));" } \ No newline at end of file diff --git a/regression-test/suites/query_p0/sql_functions/ip_functions/test_ip_functions.groovy b/regression-test/suites/query_p0/sql_functions/ip_functions/test_ip_functions.groovy index 828de6cbe415ca..c5ff6d36a1359f 100644 --- a/regression-test/suites/query_p0/sql_functions/ip_functions/test_ip_functions.groovy +++ b/regression-test/suites/query_p0/sql_functions/ip_functions/test_ip_functions.groovy @@ -67,4 +67,16 @@ suite("test_ip_functions", "arrow_flight_sql") { qt_sql "SELECT hex(ipv6_string_to_num_or_null('aaaa:aaaa:ffff:ffff:ffff:ffff:aaaa:aaaa'));" qt_sql "SELECT hex(inet6_aton('192.168.0.1'));" qt_sql "SELECT hex(inet6_aton('2a02:6b8::11'));" + + qt_sql "SELECT is_ipv4_compat(inet6_aton('::10.0.5.9'));" + qt_sql "SELECT is_ipv4_compat(inet6_aton('::ffff:10.0.5.9'));" + qt_sql "SELECT is_ipv4_compat(inet6_aton('::'));" + qt_sql "SELECT is_ipv4_compat(inet6_aton('::c0a8:0001'));" + qt_sql "SELECT is_ipv4_compat(inet6_aton('::0.0.0.0'));" + qt_sql "SELECT is_ipv4_compat(inet6_aton('::255.255.255.255'));" + + qt_sql "SELECT is_ipv4_mapped(inet6_aton('::10.0.5.9'));" + qt_sql "SELECT is_ipv4_mapped(inet6_aton('::ffff:10.0.5.9'));" + qt_sql "SELECT is_ipv4_mapped(inet6_aton('::'));" + qt_sql "SELECT is_ipv4_mapped(inet6_aton('::ffff:c0a8:0001'));" } From 78236868677967d4e4d7324bdc043b569597b028 Mon Sep 17 00:00:00 2001 From: HappenLee Date: Sun, 21 Jan 2024 10:54:47 +0800 Subject: [PATCH 109/200] [Bug](fix) try to fix the coredump of streambyte decode of sse (#30190) --- be/src/vec/core/block.cpp | 3 ++- be/src/vec/data_types/data_type_decimal.cpp | 2 -- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/be/src/vec/core/block.cpp b/be/src/vec/core/block.cpp index e3ce4885cb08af..dc9c90a339745e 100644 --- a/be/src/vec/core/block.cpp +++ b/be/src/vec/core/block.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include @@ -869,7 +870,7 @@ Status Block::serialize(int be_exec_version, PBlock* pblock, *uncompressed_bytes = content_uncompressed_size; const size_t serialize_bytes = buf - column_values.data(); *compressed_bytes = serialize_bytes; - column_values.resize(serialize_bytes); + column_values.resize(serialize_bytes + STREAMVBYTE_PADDING); // compress if (compression_type != segment_v2::NO_COMPRESSION && content_uncompressed_size > 0) { diff --git a/be/src/vec/data_types/data_type_decimal.cpp b/be/src/vec/data_types/data_type_decimal.cpp index 4e4c7ddd8197ba..0573a17729ec90 100644 --- a/be/src/vec/data_types/data_type_decimal.cpp +++ b/be/src/vec/data_types/data_type_decimal.cpp @@ -25,8 +25,6 @@ #include #include -#include - #include "runtime/decimalv2_value.h" #include "util/string_parser.hpp" #include "vec/columns/column.h" From 4a8669020af171750e6c879eb5bac046be861c02 Mon Sep 17 00:00:00 2001 From: meiyi Date: Sun, 21 Jan 2024 11:42:25 +0800 Subject: [PATCH 110/200] [fix](group commit) Fix some group commit case (#30132) --- .../apache/doris/alter/SchemaChangeJobV2.java | 15 ++++++----- .../org/apache/doris/alter/SystemHandler.java | 11 ++++---- .../doris/analysis/NativeInsertStmt.java | 2 +- .../doris/httpv2/rest/CheckWalSizeAction.java | 3 +-- .../apache/doris/httpv2/rest/LoadAction.java | 23 ++++++++--------- .../apache/doris/load/GroupCommitManager.java | 25 ++++++++----------- .../doris/planner/GroupCommitPlanner.java | 3 --- gensrc/proto/internal_service.proto | 6 ++--- 8 files changed, 38 insertions(+), 50 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java index eef902dea825da..5c74164ae33c36 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java @@ -48,7 +48,6 @@ import org.apache.doris.common.io.Text; import org.apache.doris.common.util.DbUtil; import org.apache.doris.common.util.TimeUtils; -import org.apache.doris.load.GroupCommitManager.SchemaChangeStatus; import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.task.AgentBatchTask; import org.apache.doris.task.AgentTask; @@ -602,8 +601,8 @@ protected void runRunningJob() throws AlterCancelException { private void waitWalFinished() { // wait wal done here - Env.getCurrentEnv().getGroupCommitManager().setStatus(tableId, SchemaChangeStatus.BLOCK); - LOG.info("block table {}", tableId); + Env.getCurrentEnv().getGroupCommitManager().blockTable(tableId); + LOG.info("block group commit for table={} when schema change", tableId); List aliveBeIds = Env.getCurrentSystemInfo().getAllBackendIds(true); long expireTime = System.currentTimeMillis() + Config.check_wal_queue_timeout_threshold; while (true) { @@ -611,21 +610,21 @@ private void waitWalFinished() { boolean walFinished = Env.getCurrentEnv().getGroupCommitManager() .isPreviousWalFinished(tableId, aliveBeIds); if (walFinished) { - LOG.info("all wal is finished"); + LOG.info("all wal is finished for table={}", tableId); break; } else if (System.currentTimeMillis() > expireTime) { - LOG.warn("waitWalFinished time out"); + LOG.warn("waitWalFinished time out for table={}", tableId); break; } else { try { Thread.sleep(100); } catch (InterruptedException ie) { - LOG.info("schema change job sleep wait for wal InterruptedException: ", ie); + LOG.warn("failed to wait for wal for table={} when schema change", tableId, ie); } } } - Env.getCurrentEnv().getGroupCommitManager().setStatus(tableId, SchemaChangeStatus.NORMAL); - LOG.info("release table {}", tableId); + Env.getCurrentEnv().getGroupCommitManager().unblockTable(tableId); + LOG.info("unblock group commit for table={} when schema change", tableId); } private void onFinished(OlapTable tbl) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SystemHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SystemHandler.java index 935dcf3629330f..86551ba0735b0f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SystemHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SystemHandler.java @@ -83,7 +83,8 @@ private void runAlterJobV2() { } List backendTabletIds = invertedIndex.getTabletIdsByBackendId(beId); - if (Config.drop_backend_after_decommission && checkTablets(beId, backendTabletIds) && checkWal(backend)) { + boolean hasWal = checkWal(backend); + if (Config.drop_backend_after_decommission && checkTablets(beId, backendTabletIds) && hasWal) { try { systemInfoService.dropBackend(beId); LOG.info("no available tablet on decommission backend {}, drop it", beId); @@ -94,8 +95,9 @@ private void runAlterJobV2() { continue; } - LOG.info("backend {} lefts {} replicas to decommission: {}", beId, backendTabletIds.size(), - backendTabletIds.subList(0, Math.min(10, backendTabletIds.size()))); + LOG.info("backend {} lefts {} replicas to decommission: {}{}", beId, backendTabletIds.size(), + backendTabletIds.subList(0, Math.min(10, backendTabletIds.size())), + hasWal ? "; and has unfinished WALs" : ""); } } @@ -197,8 +199,7 @@ private boolean checkTablets(Long beId, List backendTabletIds) { } private boolean checkWal(Backend backend) { - return Env.getCurrentEnv().getGroupCommitManager() - .getAllWalQueueSize(backend) == 0; + return Env.getCurrentEnv().getGroupCommitManager().getAllWalQueueSize(backend) == 0; } private List checkDecommission(DecommissionBackendClause decommissionBackendClause) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java index d69ca40cecad0d..96dbb2e0edfa93 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java @@ -1140,7 +1140,7 @@ public void analyzeGroupCommit(Analyzer analyzer) throws AnalysisException { return; } boolean partialUpdate = ConnectContext.get().getSessionVariable().isEnableUniqueKeyPartialUpdate(); - if (!partialUpdate && ConnectContext.get().getSessionVariable().isEnableInsertGroupCommit() + if (!isExplain() && !partialUpdate && ConnectContext.get().getSessionVariable().isEnableInsertGroupCommit() && ConnectContext.get().getSessionVariable().getSqlMode() != SqlModeHelper.MODE_NO_BACKSLASH_ESCAPES && targetTable instanceof OlapTable && ((OlapTable) targetTable).getTableProperty().getUseSchemaLightChange() diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/CheckWalSizeAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/CheckWalSizeAction.java index f7822580fb760b..fdc39e8badd0ff 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/CheckWalSizeAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/CheckWalSizeAction.java @@ -84,8 +84,7 @@ public Object execute(HttpServletRequest request, HttpServletResponse response) List backends = getBackends(hostInfos); List backendsList = new ArrayList<>(); for (Backend backend : backends) { - long size = Env.getCurrentEnv().getGroupCommitManager() - .getAllWalQueueSize(backend); + long size = Env.getCurrentEnv().getGroupCommitManager().getAllWalQueueSize(backend); backendsList.add(backend.getHost() + ":" + backend.getHeartbeatPort() + ":" + size); } return ResponseEntityBuilder.ok(backendsList); diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java index a78a7e9fa58f0c..6952bd37b5c91b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java @@ -88,12 +88,11 @@ public Object streamLoad(HttpServletRequest request, @PathVariable(value = DB_KEY) String db, @PathVariable(value = TABLE_KEY) String table) { boolean groupCommit = false; String groupCommitStr = request.getHeader("group_commit"); - if (groupCommitStr != null && groupCommitStr.equals("async_mode")) { + if (groupCommitStr != null && groupCommitStr.equalsIgnoreCase("async_mode")) { groupCommit = true; try { - String[] pair = new String[] {db, table}; - if (isGroupCommitBlock(pair)) { - String msg = "insert table " + pair[1] + " is blocked on schema change"; + if (isGroupCommitBlock(db, table)) { + String msg = "insert table " + table + " is blocked on schema change"; return new RestBaseResult(msg); } } catch (Exception e) { @@ -122,19 +121,17 @@ public Object streamLoad(HttpServletRequest request, } } - @RequestMapping(path = "/api/_http_stream", - method = RequestMethod.PUT) - public Object streamLoadWithSql(HttpServletRequest request, - HttpServletResponse response) { + @RequestMapping(path = "/api/_http_stream", method = RequestMethod.PUT) + public Object streamLoadWithSql(HttpServletRequest request, HttpServletResponse response) { String sql = request.getHeader("sql"); LOG.info("streaming load sql={}", sql); boolean groupCommit = false; String groupCommitStr = request.getHeader("group_commit"); - if (groupCommitStr != null && groupCommitStr.equals("async_mode")) { + if (groupCommitStr != null && groupCommitStr.equalsIgnoreCase("async_mode")) { groupCommit = true; try { String[] pair = parseDbAndTb(sql); - if (isGroupCommitBlock(pair)) { + if (isGroupCommitBlock(pair[0], pair[1])) { String msg = "insert table " + pair[1] + " is blocked on schema change"; return new RestBaseResult(msg); } @@ -164,11 +161,11 @@ public Object streamLoadWithSql(HttpServletRequest request, } } - private boolean isGroupCommitBlock(String[] pair) throws TException { - String fullDbName = getFullDbName(pair[0]); + private boolean isGroupCommitBlock(String db, String table) throws TException { + String fullDbName = getFullDbName(db); Database dbObj = Env.getCurrentInternalCatalog() .getDbOrException(fullDbName, s -> new TException("database is invalid for dbName: " + s)); - Table tblObj = dbObj.getTableOrException(pair[1], s -> new TException("table is invalid: " + s)); + Table tblObj = dbObj.getTableOrException(table, s -> new TException("table is invalid: " + s)); return Env.getCurrentEnv().getGroupCommitManager().isBlock(tblObj.getId()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/GroupCommitManager.java b/fe/fe-core/src/main/java/org/apache/doris/load/GroupCommitManager.java index 3b9719b2594d72..12410945e9f87d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/GroupCommitManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/GroupCommitManager.java @@ -17,7 +17,6 @@ package org.apache.doris.load; - import org.apache.doris.catalog.Env; import org.apache.doris.common.Config; import org.apache.doris.proto.InternalService.PGetWalQueueSizeRequest; @@ -30,31 +29,27 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.HashSet; import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; +import java.util.Set; import java.util.concurrent.Future; public class GroupCommitManager { - public enum SchemaChangeStatus { - BLOCK, NORMAL - } - private static final Logger LOG = LogManager.getLogger(GroupCommitManager.class); - private final Map statusMap = new ConcurrentHashMap<>(); + private Set blockedTableIds = new HashSet<>(); public boolean isBlock(long tableId) { - if (statusMap.containsKey(tableId)) { - return statusMap.get(tableId) == SchemaChangeStatus.BLOCK; - } - return false; + return blockedTableIds.contains(tableId); + } + + public void blockTable(long tableId) { + blockedTableIds.add(tableId); } - public void setStatus(long tableId, SchemaChangeStatus status) { - LOG.debug("Setting status for tableId {}: {}", tableId, status); - statusMap.put(tableId, status); + public void unblockTable(long tableId) { + blockedTableIds.remove(tableId); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/GroupCommitPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/planner/GroupCommitPlanner.java index 8b9f6b18331971..b69ece3b9aed81 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/GroupCommitPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/GroupCommitPlanner.java @@ -153,9 +153,6 @@ public PGroupCommitInsertResponse executeGroupCommitInsert(ConnectContext ctx, } } PGroupCommitInsertRequest request = PGroupCommitInsertRequest.newBuilder() - .setDbId(db.getId()) - .setTableId(table.getId()) - .setBaseSchemaVersion(table.getBaseSchemaVersion()) .setExecPlanFragmentRequest(InternalService.PExecPlanFragmentRequest.newBuilder() .setRequest(execPlanFragmentParamsBytes) .setCompact(false).setVersion(InternalService.PFragmentRequestVersion.VERSION_2).build()) diff --git a/gensrc/proto/internal_service.proto b/gensrc/proto/internal_service.proto index 433144b304b975..cf45d0395229e7 100644 --- a/gensrc/proto/internal_service.proto +++ b/gensrc/proto/internal_service.proto @@ -766,11 +766,11 @@ message PGlobResponse { } message PGroupCommitInsertRequest { - optional int64 db_id = 1; - optional int64 table_id = 2; + optional int64 db_id = 1; // deprecated + optional int64 table_id = 2; // deprecated // Descriptors.TDescriptorTable // optional bytes desc_tbl = 3; - optional int64 base_schema_version = 4; + optional int64 base_schema_version = 4; // deprecated // TExecPlanFragmentParams -> TPlanFragment -> PlanNodes.TPlan // optional bytes plan_node = 5; From 45ca7d587ef9826e306e92f15ca3fd7c5d2b69a8 Mon Sep 17 00:00:00 2001 From: minghong Date: Mon, 22 Jan 2024 09:22:19 +0800 Subject: [PATCH 111/200] [opt](nereids) do not change RuntimeFilter Type from IN-OR_BLOOM to BLOOM on broadcast join (#30148) 1. do not change RuntimeFilter Type from IN-OR_BLOOM to BLOOM on broadcast join tpcds1T, q48 improved from 4.x sec to 1.x sec 2. skip some redunant runtime filter example: A join B on A.a1=B.b and A.a1 = A.a2 RF B.b->(A.a1, A.a2) however, RF(B.b->A.a2) is implied by RF(B.a->A.a1) and A.a1=A.a2 we skip RF(B.b->A.a2) Issue Number: close #xxx --- .../translator/RuntimeFilterTranslator.java | 6 ---- .../post/RuntimeFilterGenerator.java | 2 +- .../plans/physical/AbstractPhysicalJoin.java | 15 ++++++++++ .../plans/physical/AbstractPhysicalPlan.java | 21 +++++++++----- .../trees/plans/physical/RuntimeFilter.java | 4 +++ .../rf_prune/query64.out | 16 +++++------ .../shape/query64.out | 28 +++++++++---------- 7 files changed, 56 insertions(+), 36 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/RuntimeFilterTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/RuntimeFilterTranslator.java index 91f97906097f20..30a69ff97de69c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/RuntimeFilterTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/RuntimeFilterTranslator.java @@ -38,7 +38,6 @@ import org.apache.doris.planner.RuntimeFilter.RuntimeFilterTarget; import org.apache.doris.planner.ScanNode; import org.apache.doris.qe.ConnectContext; -import org.apache.doris.qe.SessionVariable; import org.apache.doris.statistics.StatisticalType; import org.apache.doris.thrift.TRuntimeFilterType; @@ -185,11 +184,6 @@ private org.apache.doris.planner.RuntimeFilter finalize(org.apache.doris.planner origFilter.markFinalized(); origFilter.assignToPlanNodes(); origFilter.extractTargetsPosition(); - // Number of parallel instances are large for pipeline engine, so we prefer bloom filter. - if (origFilter.hasRemoteTargets() && origFilter.getType() == TRuntimeFilterType.IN_OR_BLOOM - && SessionVariable.enablePipelineEngine()) { - origFilter.setType(TRuntimeFilterType.BLOOM); - } return origFilter; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterGenerator.java index 76d189ba63d405..8a75ad36c5a0fd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterGenerator.java @@ -567,7 +567,7 @@ private void pushDownRuntimeFilterIntoCTE(RuntimeFilterContext ctx) { PhysicalHashJoin join = innerEntry.getValue(); Preconditions.checkState(join != null); TRuntimeFilterType type = TRuntimeFilterType.IN_OR_BLOOM; - if (ctx.getSessionVariable().getEnablePipelineEngine()) { + if (ctx.getSessionVariable().getEnablePipelineEngine() && !join.isBroadCastJoin()) { type = TRuntimeFilterType.BLOOM; } EqualTo newEqualTo = ((EqualTo) JoinUtils.swapEqualToForChildrenOrder( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalJoin.java index 16ba68aac62c52..7d1c65b589947f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalJoin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalJoin.java @@ -19,6 +19,8 @@ import org.apache.doris.nereids.hint.DistributeHint; import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.DistributionSpec; +import org.apache.doris.nereids.properties.DistributionSpecReplicated; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.trees.expressions.Expression; @@ -251,4 +253,17 @@ public String toString() { return Utils.toSqlString(this.getClass().getSimpleName() + "[" + id.asInt() + "]" + getGroupIdWithPrefix(), args.toArray()); } + + /** + * true if this is a broadcast join + */ + public boolean isBroadCastJoin() { + if (child(1) instanceof PhysicalDistribute) { + DistributionSpec distSpec = ((PhysicalDistribute) child(1)).getDistributionSpec(); + if (distSpec instanceof DistributionSpecReplicated) { + return true; + } + } + return false; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalPlan.java index a2968ca8089c46..6a19abcc8fce34 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalPlan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalPlan.java @@ -112,19 +112,26 @@ public boolean pushDownRuntimeFilter(CascadesContext context, IdGenerator(A.a1, A.a2) + // however, RF(B.b->A.a2) is implied by RF(B.a->A.a1) and A.a1=A.a2 + // we skip RF(B.b->A.a2) + this.addAppliedRuntimeFilter(filter); + filter.addTargetSlot(scanSlot, scan); + filter.addTargetExpression(scanSlot); + ctx.addJoinToTargetMap(builderNode, scanSlot.getExprId()); + ctx.setTargetExprIdToFilter(scanSlot.getExprId(), filter); + ctx.setTargetsOnScanNode(ctx.getAliasTransferPair((NamedExpression) probeExpr).first, scanSlot); + } } else { filter = new RuntimeFilter(generator.getNextId(), src, ImmutableList.of(scanSlot), type, exprOrder, builderNode, buildSideNdv, scan); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/RuntimeFilter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/RuntimeFilter.java index a8f4c3cd7c80c7..3a3b01daecdab0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/RuntimeFilter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/RuntimeFilter.java @@ -148,6 +148,10 @@ public List getTargetScans() { return targetScans; } + public boolean hasTargetScan(PhysicalRelation scan) { + return targetScans.contains(scan); + } + @Override public String toString() { StringBuilder sb = new StringBuilder(); diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query64.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query64.out index e5e915d4422886..367d659e25c771 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query64.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query64.out @@ -16,22 +16,22 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) --------------------------PhysicalProject ----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() build RFs:RF16 ss_addr_sk->[ca_address_sk] +--------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() build RFs:RF15 ss_addr_sk->[ca_address_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF16 +------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF15 ----------------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF14 ss_item_sk->[sr_item_sk];RF15 ss_ticket_number->[sr_ticket_number] +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF13 ss_item_sk->[sr_item_sk];RF14 ss_ticket_number->[sr_ticket_number] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF14 RF15 +------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF13 RF14 ----------------------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF13 ss_cdemo_sk->[cd_demo_sk] +--------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF12 ss_cdemo_sk->[cd_demo_sk] ----------------------------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF13 +--------------------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF12 ----------------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk];RF12 i_item_sk->[cs_item_sk] +------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk,cs_item_sk] --------------------------------------------------PhysicalProject ----------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() ------------------------------------------------------PhysicalProject @@ -53,7 +53,7 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ----------------------------------------------------------------------------------PhysicalProject ------------------------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF4 cr_order_number->[cs_order_number];RF5 cr_item_sk->[cs_item_sk] --------------------------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF12 +----------------------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF11 --------------------------------------------------------------------------------------PhysicalProject ----------------------------------------------------------------------------------------PhysicalOlapScan[catalog_returns] ------------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query64.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query64.out index 43bdb50fcce033..d40500b774a75c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query64.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query64.out @@ -7,31 +7,31 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) --------PhysicalDistribute[DistributionSpecHash] ----------hashAgg[LOCAL] ------------PhysicalProject ---------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_shipto_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF20 d_date_sk->[c_first_shipto_date_sk] +--------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_shipto_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF19 d_date_sk->[c_first_shipto_date_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF19 d_date_sk->[c_first_sales_date_sk] +------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[c_first_sales_date_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) build RFs:RF18 c_customer_sk->[ss_customer_sk] +----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) build RFs:RF17 c_customer_sk->[ss_customer_sk] ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF17 p_promo_sk->[ss_promo_sk] +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF16 p_promo_sk->[ss_promo_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() build RFs:RF16 ss_addr_sk->[ca_address_sk] +--------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() build RFs:RF15 ss_addr_sk->[ca_address_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF16 +------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF15 ----------------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF14 ss_item_sk->[sr_item_sk];RF15 ss_ticket_number->[sr_ticket_number] +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF13 ss_item_sk->[sr_item_sk];RF14 ss_ticket_number->[sr_ticket_number] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF14 RF15 +------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF13 RF14 ----------------------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF13 ss_cdemo_sk->[cd_demo_sk] +--------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF12 ss_cdemo_sk->[cd_demo_sk] ----------------------------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF13 +--------------------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF12 ----------------------------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk];RF12 i_item_sk->[cs_item_sk] +------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk,cs_item_sk] --------------------------------------------------PhysicalProject ----------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF10 s_store_sk->[ss_store_sk] ------------------------------------------------------PhysicalProject @@ -43,7 +43,7 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ------------------------------------------------------------------PhysicalProject --------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = cs_ui.cs_item_sk)) otherCondition=() build RFs:RF6 cs_item_sk->[ss_item_sk] ----------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 RF10 RF11 RF17 RF18 +------------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 RF10 RF11 RF16 RF17 ----------------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------------------------------------------------PhysicalProject --------------------------------------------------------------------------filter((sale > (2 * refund))) @@ -53,7 +53,7 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ----------------------------------------------------------------------------------PhysicalProject ------------------------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF4 cr_order_number->[cs_order_number];RF5 cr_item_sk->[cs_item_sk] --------------------------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF12 +----------------------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF11 --------------------------------------------------------------------------------------PhysicalProject ----------------------------------------------------------------------------------------PhysicalOlapScan[catalog_returns] ------------------------------------------------------------------PhysicalDistribute[DistributionSpecReplicated] @@ -86,7 +86,7 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) --------------------------------------PhysicalProject ----------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_hdemo_sk = hd2.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[c_current_hdemo_sk] ------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 RF2 RF3 RF19 RF20 +--------------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 RF2 RF3 RF18 RF19 ------------------------------------------PhysicalDistribute[DistributionSpecReplicated] --------------------------------------------PhysicalProject ----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((hd2.hd_income_band_sk = ib2.ib_income_band_sk)) otherCondition=() build RFs:RF0 ib_income_band_sk->[hd_income_band_sk] From 7ecf4e7bc7d540d01e88c2e56d55dfa591b00b51 Mon Sep 17 00:00:00 2001 From: Kaijie Chen Date: Mon, 22 Jan 2024 09:58:49 +0800 Subject: [PATCH 112/200] [config](load) set default memtable_flush_running_count_limit to 2 (#30146) --- be/src/common/config.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 04e537cd3241bc..5468ac29a02385 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -592,7 +592,7 @@ DEFINE_mInt64(write_buffer_size, "104857600"); // max buffer size used in memtable for the aggregated table, default 400MB DEFINE_mInt64(write_buffer_size_for_agg, "419430400"); // max parallel flush task per memtable writer -DEFINE_mInt32(memtable_flush_running_count_limit, "5"); +DEFINE_mInt32(memtable_flush_running_count_limit, "2"); DEFINE_Int32(load_process_max_memory_limit_percent, "50"); // 50% From e02f2a0307197d780df67a946111ff1fd042f560 Mon Sep 17 00:00:00 2001 From: walter Date: Mon, 22 Jan 2024 10:34:10 +0800 Subject: [PATCH 113/200] [feature](merge-cloud) Add retrying service config (#30137) Co-authored-by: Lei Zhang <27994433+SWJTU-ZhangLei@users.noreply.github.com> --- .../resources/retrying_service_config.json | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) create mode 100644 fe/fe-core/src/main/resources/retrying_service_config.json diff --git a/fe/fe-core/src/main/resources/retrying_service_config.json b/fe/fe-core/src/main/resources/retrying_service_config.json new file mode 100644 index 00000000000000..1589c1fe4060c7 --- /dev/null +++ b/fe/fe-core/src/main/resources/retrying_service_config.json @@ -0,0 +1,20 @@ +{ + "methodConfig": [ + { + "name": [ + { + "service": "doris.cloud.MetaService" + } + ], + "retryPolicy": { + "maxAttempts": 10, + "initialBackoff": "0.5s", + "maxBackoff": "30s", + "backoffMultiplier": 2, + "retryableStatusCodes": [ + "UNAVAILABLE" + ] + } + } + ] +} From 37e903d5b1ec286851c7289eeeb150a6821d074f Mon Sep 17 00:00:00 2001 From: zzzxl <33418555+zzzxl1993@users.noreply.github.com> Date: Mon, 22 Jan 2024 10:34:25 +0800 Subject: [PATCH 114/200] [fix](phrase_prefix) fix match_phrase_prefix query incorrect result (#29946) --- .../query/phrase_prefix_query.cpp | 5 +- .../test_index_match_phrase_prefix_1.out | 13 ++++ .../test_index_match_phrase_prefix_1.groovy | 62 +++++++++++++++++++ 3 files changed, 79 insertions(+), 1 deletion(-) create mode 100644 regression-test/data/inverted_index_p0/test_index_match_phrase_prefix_1.out create mode 100644 regression-test/suites/inverted_index_p0/test_index_match_phrase_prefix_1.groovy diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.cpp b/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.cpp index 7920336c752765..407e515dc9212f 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.cpp @@ -17,6 +17,7 @@ #include "phrase_prefix_query.h" +#include "CLucene/util/stringUtil.h" #include "olap/rowset//segment_v2/inverted_index/query/prefix_query.h" namespace doris::segment_v2 { @@ -43,7 +44,9 @@ void PhrasePrefixQuery::add(const std::wstring& field_name, const std::vectorgetReader(), field_name, terms[i], prefix_terms, _max_expansions); if (prefix_terms.empty()) { - continue; + std::wstring ws_term = StringUtil::string_to_wstring(terms[i]); + Term* t = _CLNEW Term(field_name.c_str(), ws_term.c_str()); + prefix_terms.push_back(t); } _query->add(prefix_terms); for (auto& t : prefix_terms) { diff --git a/regression-test/data/inverted_index_p0/test_index_match_phrase_prefix_1.out b/regression-test/data/inverted_index_p0/test_index_match_phrase_prefix_1.out new file mode 100644 index 00000000000000..e7e934f394e05c --- /dev/null +++ b/regression-test/data/inverted_index_p0/test_index_match_phrase_prefix_1.out @@ -0,0 +1,13 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +7 + +-- !sql -- +7 + +-- !sql -- +6 + +-- !sql -- +6 + diff --git a/regression-test/suites/inverted_index_p0/test_index_match_phrase_prefix_1.groovy b/regression-test/suites/inverted_index_p0/test_index_match_phrase_prefix_1.groovy new file mode 100644 index 00000000000000..1e2ac0d790017e --- /dev/null +++ b/regression-test/suites/inverted_index_p0/test_index_match_phrase_prefix_1.groovy @@ -0,0 +1,62 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + + +suite("test_index_match_phrase_prefix_1", "p0"){ + def indexTbName1 = "test_index_match_phrase_prefix_1" + + sql "DROP TABLE IF EXISTS ${indexTbName1}" + + sql """ + CREATE TABLE ${indexTbName1} ( + `a` int(11) NULL COMMENT "", + `b` string NULL COMMENT "", + `c` string NULL COMMENT "", + `d` string NULL COMMENT "", + INDEX b_idx (`b`) USING INVERTED COMMENT '', + INDEX c_idx (`c`) USING INVERTED PROPERTIES("parser" = "unicode", "support_phrase" = "true") COMMENT '', + INDEX d_idx (`d`) USING INVERTED PROPERTIES("parser" = "english", "support_phrase" = "true") COMMENT '' + ) ENGINE=OLAP + DUPLICATE KEY(`a`) + COMMENT "OLAP" + DISTRIBUTED BY RANDOM BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql """ INSERT INTO ${indexTbName1} VALUES (1, "O1704361998540E2Cemx9S", "O1704361998540E2Cemx9S", "O1704361998540E2Cemx9S"); """ + sql """ INSERT INTO ${indexTbName1} VALUES (2, "O1704361998540E2Cemx9S)123456789", "O1704361998540E2Cemx9S)123456789", "O1704361998540E2Cemx9S)123456789"); """ + sql """ INSERT INTO ${indexTbName1} VALUES (3, "O1704361998540E2Cemx9S=123456789", "O1704361998540E2Cemx9S=123456789", "O1704361998540E2Cemx9S=123456789"); """ + sql """ INSERT INTO ${indexTbName1} VALUES (4, "O1704361998540E2Cemx9S+123456789", "O1704361998540E2Cemx9S+123456789", "O1704361998540E2Cemx9S+123456789"); """ + sql """ INSERT INTO ${indexTbName1} VALUES (5, "O1704361998540E2Cemx9S!123456789", "O1704361998540E2Cemx9S!123456789", "O1704361998540E2Cemx9S!123456789"); """ + sql """ INSERT INTO ${indexTbName1} VALUES (6, "O1704361998540E2Cemx9S 123456789", "O1704361998540E2Cemx9S 123456789", "O1704361998540E2Cemx9S 123456789"); """ + sql """ INSERT INTO ${indexTbName1} VALUES (7, "O1704361998540E2Cemx9S*123456789", "O1704361998540E2Cemx9S*123456789", "O1704361998540E2Cemx9S*123456789"); """ + + try { + sql "sync" + + qt_sql """ select count() from ${indexTbName1} where c match_phrase_prefix 'O1704361998540E2Cemx9S'; """ + qt_sql """ select count() from ${indexTbName1} where d match_phrase_prefix 'O1704361998540E2Cemx9S'; """ + + qt_sql """ select count() from ${indexTbName1} where c match_phrase_prefix 'O1704361998540E2Cemx9S=123456789'; """ + qt_sql """ select count() from ${indexTbName1} where d match_phrase_prefix 'O1704361998540E2Cemx9S=123456789'; """ + + } finally { + //try_sql("DROP TABLE IF EXISTS ${testTable}") + } +} \ No newline at end of file From 10032044cc8c68eeaa43730eb4c7a37435be31d4 Mon Sep 17 00:00:00 2001 From: lihangyu <15605149486@163.com> Date: Mon, 22 Jan 2024 11:02:55 +0800 Subject: [PATCH 115/200] [Improve](Variant) support implicit cast to numeric and string type (#30029) --- .../doris/analysis/BinaryPredicate.java | 8 ++ .../apache/doris/analysis/MatchPredicate.java | 10 ++- .../data/variant_p0/sql/implicit_cast.out | 80 +++++++++++++++++++ .../suites/variant_p0/sql/implicit_cast.sql | 15 ++++ 4 files changed, 111 insertions(+), 2 deletions(-) create mode 100644 regression-test/data/variant_p0/sql/implicit_cast.out create mode 100644 regression-test/suites/variant_p0/sql/implicit_cast.sql diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/BinaryPredicate.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/BinaryPredicate.java index bb30a46b3e9539..1ce647a473cc9c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/BinaryPredicate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/BinaryPredicate.java @@ -496,6 +496,14 @@ private Type getCmpType() throws AnalysisException { SessionVariable.getEnableDecimal256()); } + // Variant can be implicit cast to numeric type and string type at present + if (t1.isVariantType() && (t2.isNumericType() || t2.isStringType())) { + return Type.fromPrimitiveType(t2); + } + if (t2.isVariantType() && (t1.isNumericType() || t1.isStringType())) { + return Type.fromPrimitiveType(t1); + } + return Type.DOUBLE; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/MatchPredicate.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/MatchPredicate.java index f106aec956c72c..e284d86e2bb986 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/MatchPredicate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/MatchPredicate.java @@ -271,9 +271,10 @@ public void analyzeImpl(Analyzer analyzer) throws AnalysisException { throw new AnalysisException("right operand of " + op.toString() + " must be of type STRING: " + toSql()); } - if (!getChild(0).getType().isStringType() && !getChild(0).getType().isArrayType()) { + if (!getChild(0).getType().isStringType() && !getChild(0).getType().isArrayType() + && !getChild(0).getType().isVariantType()) { throw new AnalysisException( - "left operand of " + op.toString() + " must be of type STRING or ARRAY: " + toSql()); + "left operand of " + op.toString() + " must be of type STRING, ARRAY or VARIANT: " + toSql()); } fn = getBuiltinFunction(op.toString(), @@ -295,6 +296,11 @@ public void analyzeImpl(Analyzer analyzer) throws AnalysisException { } } + // CAST variant to right expr type + if (e1.type.isVariantType()) { + setChild(0, e1.castTo(e2.getType())); + } + if (e1 instanceof SlotRef) { SlotRef slotRef = (SlotRef) e1; SlotDescriptor slotDesc = slotRef.getDesc(); diff --git a/regression-test/data/variant_p0/sql/implicit_cast.out b/regression-test/data/variant_p0/sql/implicit_cast.out new file mode 100644 index 00000000000000..b0f5d96087b5ac --- /dev/null +++ b/regression-test/data/variant_p0/sql/implicit_cast.out @@ -0,0 +1,80 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !implicit_cast -- +0 + +-- !implicit_cast_2 -- +0 + +-- !implicit_cast_3 -- +0 + +-- !implicit_cast_4 -- +5000 + +-- !implicit_cast_5 -- +leonardomso/33-js-concepts 3 +ytdl-org/youtube-dl 3 +Bogdanp/neko 2 +bminossi/AllVideoPocsFromHackerOne 2 +disclose/diodata 2 + +-- !implicit_cast_6 -- +282 + +-- !implicit_cast_7 -- +14690758274 + +-- !implicit_cast_8 -- +0 + +-- !implicit_cast_9 -- +19829 + +-- !implicit_cast_10 -- +49390617 +64890096 +10696700 +33066637 +32271952 +2051941 +32271952 +57325392 +42386044 +73801003 + +-- !implicit_cast_11 -- +27 {"id":147350463,"name":"leonardomso/33-js-concepts","url":"https://api.github.com/repos/leonardomso/33-js-concepts"} +36 {"id":26109545,"name":"odeke-em/drive","url":"https://api.github.com/repos/odeke-em/drive"} +46 {"id":141905736,"name":"GO-LiFE/GoFIT_SDK_Android","url":"https://api.github.com/repos/GO-LiFE/GoFIT_SDK_Android"} +56 {"id":289417971,"name":"MrXujiang/h5-Dooring","url":"https://api.github.com/repos/MrXujiang/h5-Dooring"} +86 {"id":106453399,"name":"redsaph/cleartext","url":"https://api.github.com/repos/redsaph/cleartext"} +98 {"id":162998479,"name":"sherlock-project/sherlock","url":"https://api.github.com/repos/sherlock-project/sherlock"} +101 {"id":326215605,"name":"okandavut/react-spotify-nowplaying","url":"https://api.github.com/repos/okandavut/react-spotify-nowplaying"} +112 {"id":178435468,"name":"sentriz/gonic","url":"https://api.github.com/repos/sentriz/gonic"} +122 {"id":182606378,"name":"netlify-labs/react-netlify-identity-widget","url":"https://api.github.com/repos/netlify-labs/react-netlify-identity-widget"} +169 {"id":68730444,"name":"microsoft/BotBuilder-Samples","url":"https://api.github.com/repos/microsoft/BotBuilder-Samples"} + +-- !implicit_cast_12 -- +27 14690746717 WatchEvent leonardomso/33-js-concepts +36 14690746732 WatchEvent odeke-em/drive +46 14690746749 WatchEvent GO-LiFE/GoFIT_SDK_Android +56 14690746773 WatchEvent MrXujiang/h5-Dooring +86 14690746843 WatchEvent redsaph/cleartext +98 14690746866 WatchEvent sherlock-project/sherlock +101 14690746870 WatchEvent okandavut/react-spotify-nowplaying +112 14690746899 WatchEvent sentriz/gonic +122 14690746914 WatchEvent netlify-labs/react-netlify-identity-widget +169 14690747028 WatchEvent microsoft/BotBuilder-Samples + +-- !implicit_cast_13 -- +user +user +user +user +user +user +user +user +user +user + diff --git a/regression-test/suites/variant_p0/sql/implicit_cast.sql b/regression-test/suites/variant_p0/sql/implicit_cast.sql new file mode 100644 index 00000000000000..0653a52eed77ca --- /dev/null +++ b/regression-test/suites/variant_p0/sql/implicit_cast.sql @@ -0,0 +1,15 @@ +set exec_mem_limit=8G; +set enable_two_phase_read_opt = true; +set topn_opt_limit_threshold = 1024; +SELECT count() from ghdata; +SELECT cast(v["repo"]["name"] as string) as repo_name, count() AS stars FROM ghdata WHERE v["type"] = 'WatchEvent' GROUP BY repo_name ORDER BY stars DESC, repo_name LIMIT 5; +SELECT COUNT() FROM ghdata WHERE v["type"] match 'WatchEvent'; +SELECT max(cast(v["id"] as bigint)) FROM ghdata; +SELECT sum(cast(v["payload"]["pull_request"]["milestone"]["creator"]["site_admin"] as int)) FROM ghdata; +SELECT sum(length(v["payload"]["pull_request"]["base"]["repo"]["html_url"])) FROM ghdata; +SELECT v["payload"]["member"]["id"] FROM ghdata where v["payload"]["member"]["id"] is not null ORDER BY k LIMIT 10; +-- select k, v:payload.commits.author.name AS name, e FROM ghdata as t lateral view explode(cast(v:payload.commits.author.name as array)) tm1 as e order by k limit 5; +select k, json_extract(v, '$.repo') from ghdata WHERE v["type"] = 'WatchEvent' order by k limit 10; +-- SELECT v["payload"]["member"]["id"], count() FROM ghdata where v["payload"]["member"]["id"] is not null group by v["payload"]["member"]["id"] order by 1, 2 desc LIMIT 10; +select k, v["id"], v["type"], v["repo"]["name"] from ghdata WHERE v["type"] = 'WatchEvent' order by k limit 10; +SELECT v["payload"]["pusher_type"] FROM ghdata where v["payload"]["pusher_type"] is not null ORDER BY k LIMIT 10; \ No newline at end of file From a9dd715991ab008826c8cfc2039493b00ee5f8c7 Mon Sep 17 00:00:00 2001 From: Chester <42577861+superdiaodiao@users.noreply.github.com> Date: Mon, 22 Jan 2024 11:33:04 +0800 Subject: [PATCH 116/200] [improvement](function) switch inet(6)_aton alias origin function (#30196) --- be/src/vec/functions/function_ip.cpp | 4 +- .../sql-functions/ip-functions/inet-aton.md | 20 ++++++---- .../sql-functions/ip-functions/inet6-aton.md | 30 +++++++++------ .../sql-functions/ip-functions/inet-aton.md | 19 ++++++---- .../sql-functions/ip-functions/inet6-aton.md | 29 ++++++++------ .../doris/catalog/BuiltinScalarFunctions.java | 8 ++-- .../apache/doris/rewrite/FunctionAlias.java | 4 +- gensrc/script/doris_builtins_functions.py | 16 ++++---- .../data/nereids_function_p0/ip_functions.out | 38 ++++++++++++++++++- .../ip_functions/test_ip_functions.out | 38 ++++++++++++++++++- .../nereids_function_p0/ip_functions.groovy | 12 ++++++ .../ip_functions/test_ip_functions.groovy | 12 ++++++ 12 files changed, 174 insertions(+), 56 deletions(-) diff --git a/be/src/vec/functions/function_ip.cpp b/be/src/vec/functions/function_ip.cpp index 140100d22b3230..349e095b4a2c2c 100644 --- a/be/src/vec/functions/function_ip.cpp +++ b/be/src/vec/functions/function_ip.cpp @@ -25,7 +25,7 @@ void register_function_ip(SimpleFunctionFactory& factory) { factory.register_function>(); factory.register_function>(); factory.register_function>(); - factory.register_alias(FunctionIPv4StringToNum::name, + factory.register_alias(FunctionIPv4StringToNum::name, "inet_aton"); factory.register_function(); @@ -33,7 +33,7 @@ void register_function_ip(SimpleFunctionFactory& factory) { factory.register_function>(); factory.register_function>(); factory.register_function>(); - factory.register_alias(FunctionIPv6StringToNum::name, + factory.register_alias(FunctionIPv6StringToNum::name, "inet6_aton"); factory.register_function(); diff --git a/docs/en/docs/sql-manual/sql-functions/ip-functions/inet-aton.md b/docs/en/docs/sql-manual/sql-functions/ip-functions/inet-aton.md index 1120a4852f4279..56f9308f347720 100644 --- a/docs/en/docs/sql-manual/sql-functions/ip-functions/inet-aton.md +++ b/docs/en/docs/sql-manual/sql-functions/ip-functions/inet-aton.md @@ -40,20 +40,26 @@ Takes a string containing an IPv4 address in the format A.B.C.D (dot-separated n ### notice -`will return an error if the input string is not a valid IPv4 address` +`It is the alias of ipv4_string_to_num_or_null. +It will return NULL if the input string is not a valid IP address or NULL, which is the same with MySQL` ### example ``` mysql> select inet_aton('192.168.0.1'); -+--------------------------------+ -| inet_aton('192.168.0.1') | -+--------------------------------+ -| 3232235521 | -+--------------------------------+ ++-------------------------------------------+ +| ipv4_string_to_num_or_null('192.168.0.1') | ++-------------------------------------------+ +| 3232235521 | ++-------------------------------------------+ 1 row in set (0.01 sec) mysql> SELECT inet_aton('192.168'); -ERROR 1105 (HY000): errCode = 2, detailMessage = (172.17.0.2)[CANCELLED][INVALID_ARGUMENT][E33] Invalid IPv4 value ++---------------------------------------+ +| ipv4_string_to_num_or_null('192.168') | ++---------------------------------------+ +| NULL | ++---------------------------------------+ +1 row in set (0.01 sec) ``` ### keywords diff --git a/docs/en/docs/sql-manual/sql-functions/ip-functions/inet6-aton.md b/docs/en/docs/sql-manual/sql-functions/ip-functions/inet6-aton.md index 69bf4a448619ba..25f60b2cb8c646 100644 --- a/docs/en/docs/sql-manual/sql-functions/ip-functions/inet6-aton.md +++ b/docs/en/docs/sql-manual/sql-functions/ip-functions/inet6-aton.md @@ -41,28 +41,34 @@ If the input string contains a valid IPv4 address, returns its IPv6 equivalent. ### notice -`will return an error if the input string is not a valid IP address` +`It is the alias of ipv6_string_to_num_or_null. +It will return NULL if the input string is not a valid IP address or NULL, which is the same with MySQL` ### example ``` mysql> select hex(inet6_aton('1111::ffff')); -+----------------------------------+ -| hex(inet6_aton('1111::ffff')) | -+----------------------------------+ -| 1111000000000000000000000000FFFF | -+----------------------------------+ ++-----------------------------------------------+ +| hex(ipv6_string_to_num_or_null('1111::ffff')) | ++-----------------------------------------------+ +| 1111000000000000000000000000FFFF | ++-----------------------------------------------+ 1 row in set (0.02 sec) mysql> select hex(inet6_aton('192.168.0.1')); -+----------------------------------+ -| hex(inet6_aton('192.168.0.1')) | -+----------------------------------+ -| 00000000000000000000FFFFC0A80001 | -+----------------------------------+ ++------------------------------------------------+ +| hex(ipv6_string_to_num_or_null('192.168.0.1')) | ++------------------------------------------------+ +| 00000000000000000000FFFFC0A80001 | ++------------------------------------------------+ 1 row in set (0.02 sec) mysql> select hex(inet6_aton('notaaddress')); -ERROR 1105 (HY000): errCode = 2, detailMessage = (172.17.0.2)[CANCELLED][E33] Invalid IPv6 value ++--------------------------------------------------+ +| hex(ipv6_string_to_num_or_null('notaaddress')) | ++--------------------------------------------------+ +| NULL | ++--------------------------------------------------+ +1 row in set (0.02 sec) ``` ### keywords diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/inet-aton.md b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/inet-aton.md index 2260dfbc49d885..50a31f179944c1 100644 --- a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/inet-aton.md +++ b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/inet-aton.md @@ -40,20 +40,25 @@ inet_aton ### notice -`如果输入字符串不是有效的 IPv4 地址,将返回错误` +`该函数为ipv4_string_to_num_or_null的别称。如果输入字符串不是有效的 IPv4 地址或者NULL,将返回NULL,和MySQL一样` ### example ``` mysql> select inet_aton('192.168.0.1'); -+--------------------------------+ -| inet_aton('192.168.0.1') | -+--------------------------------+ -| 3232235521 | -+--------------------------------+ ++-------------------------------------------+ +| ipv4_string_to_num_or_null('192.168.0.1') | ++-------------------------------------------+ +| 3232235521 | ++-------------------------------------------+ 1 row in set (0.01 sec) mysql> SELECT inet_aton('192.168'); -ERROR 1105 (HY000): errCode = 2, detailMessage = (172.17.0.2)[CANCELLED][INVALID_ARGUMENT][E33] Invalid IPv4 value ++---------------------------------------+ +| ipv4_string_to_num_or_null('192.168') | ++---------------------------------------+ +| NULL | ++---------------------------------------+ +1 row in set (0.01 sec) ``` ### keywords diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/inet6-aton.md b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/inet6-aton.md index cc2bd6f04d2938..53eac5cb900d48 100644 --- a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/inet6-aton.md +++ b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/inet6-aton.md @@ -41,28 +41,33 @@ IPv6NumToString 的反向函数,它接受一个 IP 地址字符串并返回二 ### notice -`如果输入非法的IP地址,会抛出异常` +`该函数为ipv6_string_to_num_or_null的别称。如果输入字符串不是有效的 IPv4 地址或者NULL,将返回NULL,和MySQL一样` ### example ``` mysql> select hex(inet6_aton('1111::ffff')); -+----------------------------------+ -| hex(inet6_aton('1111::ffff')) | -+----------------------------------+ -| 1111000000000000000000000000FFFF | -+----------------------------------+ ++-----------------------------------------------+ +| hex(ipv6_string_to_num_or_null('1111::ffff')) | ++-----------------------------------------------+ +| 1111000000000000000000000000FFFF | ++-----------------------------------------------+ 1 row in set (0.02 sec) mysql> select hex(inet6_aton('192.168.0.1')); -+----------------------------------+ -| hex(inet6_aton('192.168.0.1')) | -+----------------------------------+ -| 00000000000000000000FFFFC0A80001 | -+----------------------------------+ ++------------------------------------------------+ +| hex(ipv6_string_to_num_or_null('192.168.0.1')) | ++------------------------------------------------+ +| 00000000000000000000FFFFC0A80001 | ++------------------------------------------------+ 1 row in set (0.02 sec) mysql> select hex(inet6_aton('notaaddress')); -ERROR 1105 (HY000): errCode = 2, detailMessage = (172.17.0.2)[CANCELLED][E33] Invalid IPv6 value ++--------------------------------------------------+ +| hex(ipv6_string_to_num_or_null('notaaddress')) | ++--------------------------------------------------+ +| NULL | ++--------------------------------------------------+ +1 row in set (0.02 sec) ``` ### keywords diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java index 68da4df08abc03..fcc2a351bf0e72 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java @@ -604,13 +604,13 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(InnerProduct.class, "inner_product"), scalar(Instr.class, "instr"), scalar(Ipv4NumToString.class, "ipv4_num_to_string", "inet_ntoa"), - scalar(Ipv4StringToNum.class, "ipv4_string_to_num", "inet_aton"), + scalar(Ipv4StringToNum.class, "ipv4_string_to_num"), scalar(Ipv4StringToNumOrDefault.class, "ipv4_string_to_num_or_default"), - scalar(Ipv4StringToNumOrNull.class, "ipv4_string_to_num_or_null"), + scalar(Ipv4StringToNumOrNull.class, "ipv4_string_to_num_or_null", "inet_aton"), scalar(Ipv6NumToString.class, "ipv6_num_to_string", "inet6_ntoa"), - scalar(Ipv6StringToNum.class, "ipv6_string_to_num", "inet6_aton"), + scalar(Ipv6StringToNum.class, "ipv6_string_to_num"), scalar(Ipv6StringToNumOrDefault.class, "ipv6_string_to_num_or_default"), - scalar(Ipv6StringToNumOrNull.class, "ipv6_string_to_num_or_null"), + scalar(Ipv6StringToNumOrNull.class, "ipv6_string_to_num_or_null", "inet6_aton"), scalar(IsIpv4Compat.class, "is_ipv4_compat"), scalar(IsIpv4Mapped.class, "is_ipv4_mapped"), scalar(IsIpv4String.class, "is_ipv4_string"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/rewrite/FunctionAlias.java b/fe/fe-core/src/main/java/org/apache/doris/rewrite/FunctionAlias.java index 9e9c6dc9f75a21..b32cc9919b160c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/rewrite/FunctionAlias.java +++ b/fe/fe-core/src/main/java/org/apache/doris/rewrite/FunctionAlias.java @@ -38,8 +38,8 @@ public final class FunctionAlias implements ExprRewriteRule { .put("char_length", "character_length").put("curdate", "current_date").put("curtime", "current_time") .put("schema", "database").put("day", "dayofmonth").put("date_add", "days_add").put("adddate", "days_add") .put("date_sub", "days_sub").put("subdate", "days_sub").put("inet_ntoa", "ipv4_num_to_string") - .put("inet_aton", "ipv4_string_to_num").put("inet6_ntoa", "ipv6_num_to_string") - .put("inet6_aton", "ipv6_string_to_num").put("lcase", "lower").put("add_months", "months_add") + .put("inet_aton", "ipv4_string_to_num_or_null").put("inet6_ntoa", "ipv6_num_to_string") + .put("inet6_aton", "ipv6_string_to_num_or_null").put("lcase", "lower").put("add_months", "months_add") .put("current_timestamp", "now").put("localtime", "now").put("localtimestamp", "now").put("ifnull", "nvl") .put("rand", "random").put("sha", "sha1").put("substr", "substring").put("ucase", "upper").build(); diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py index 6a6ff96e83da8b..183b38bdfdae04 100644 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -1998,22 +1998,22 @@ [['ipv4_num_to_string','inet_ntoa'], 'VARCHAR', ['SMALLINT'], 'ALWAYS_NULLABLE'], [['ipv4_num_to_string','inet_ntoa'], 'VARCHAR', ['INT'], 'ALWAYS_NULLABLE'], [['ipv4_num_to_string','inet_ntoa'], 'VARCHAR', ['BIGINT'], 'ALWAYS_NULLABLE'], - [['ipv4_string_to_num','inet_aton'], 'BIGINT', ['VARCHAR'], 'ALWAYS_NOT_NULLABLE'], - [['ipv4_string_to_num','inet_aton'], 'BIGINT', ['STRING'], 'ALWAYS_NOT_NULLABLE'], + [['ipv4_string_to_num'], 'BIGINT', ['VARCHAR'], 'ALWAYS_NOT_NULLABLE'], + [['ipv4_string_to_num'], 'BIGINT', ['STRING'], 'ALWAYS_NOT_NULLABLE'], [['ipv4_string_to_num_or_default'], 'BIGINT', ['VARCHAR'], 'ALWAYS_NOT_NULLABLE'], [['ipv4_string_to_num_or_default'], 'BIGINT', ['STRING'], 'ALWAYS_NOT_NULLABLE'], - [['ipv4_string_to_num_or_null'], 'BIGINT', ['VARCHAR'], 'ALWAYS_NULLABLE'], - [['ipv4_string_to_num_or_null'], 'BIGINT', ['STRING'], 'ALWAYS_NULLABLE'], + [['ipv4_string_to_num_or_null','inet_aton'], 'BIGINT', ['VARCHAR'], 'ALWAYS_NULLABLE'], + [['ipv4_string_to_num_or_null','inet_aton'], 'BIGINT', ['STRING'], 'ALWAYS_NULLABLE'], [['ipv6_num_to_string','inet6_ntoa'], 'VARCHAR', ['VARCHAR'], 'ALWAYS_NULLABLE'], [['ipv6_num_to_string','inet6_ntoa'], 'STRING', ['STRING'], 'ALWAYS_NULLABLE'], [['ipv6_num_to_string','inet6_ntoa'], 'VARCHAR', ['IPV6'], 'ALWAYS_NULLABLE'], [['ipv6_num_to_string','inet6_ntoa'], 'STRING', ['IPV6'], 'ALWAYS_NULLABLE'], - [['ipv6_string_to_num','inet6_aton'], 'VARCHAR', ['VARCHAR'], 'ALWAYS_NOT_NULLABLE'], - [['ipv6_string_to_num','inet6_aton'], 'STRING', ['STRING'], 'ALWAYS_NOT_NULLABLE'], + [['ipv6_string_to_num'], 'VARCHAR', ['VARCHAR'], 'ALWAYS_NOT_NULLABLE'], + [['ipv6_string_to_num'], 'STRING', ['STRING'], 'ALWAYS_NOT_NULLABLE'], [['ipv6_string_to_num_or_default'], 'VARCHAR', ['VARCHAR'], 'ALWAYS_NOT_NULLABLE'], [['ipv6_string_to_num_or_default'], 'STRING', ['STRING'], 'ALWAYS_NOT_NULLABLE'], - [['ipv6_string_to_num_or_null'], 'VARCHAR', ['VARCHAR'], 'ALWAYS_NULLABLE'], - [['ipv6_string_to_num_or_null'], 'STRING', ['STRING'], 'ALWAYS_NULLABLE'], + [['ipv6_string_to_num_or_null','inet6_aton'], 'VARCHAR', ['VARCHAR'], 'ALWAYS_NULLABLE'], + [['ipv6_string_to_num_or_null','inet6_aton'], 'STRING', ['STRING'], 'ALWAYS_NULLABLE'], [['is_ipv4_compat'], 'BOOLEAN', ['VARCHAR'], ''], [['is_ipv4_compat'], 'BOOLEAN', ['STRING'], ''], [['is_ipv4_mapped'], 'BOOLEAN', ['VARCHAR'], ''], diff --git a/regression-test/data/nereids_function_p0/ip_functions.out b/regression-test/data/nereids_function_p0/ip_functions.out index 914b5ccd683952..a89ca34fe60549 100644 --- a/regression-test/data/nereids_function_p0/ip_functions.out +++ b/regression-test/data/nereids_function_p0/ip_functions.out @@ -35,6 +35,15 @@ -- !ip12 -- 3232235521 +-- !ip12_1 -- +\N + +-- !ip12_2 -- +\N + +-- !ip12_3 -- +\N + -- !ip13 -- \N @@ -152,6 +161,15 @@ AAAAAAAAFFFFFFFFFFFFFFFFAAAAAAAA -- !ip51 -- 2A0206B8000000000000000000000011 +-- !ip51_1 -- +\N + +-- !ip51_2 -- +\N + +-- !ip51_3 -- +\N + -- !ip52 -- true @@ -182,6 +200,15 @@ false -- !ip61 -- true +-- !ip61_1 -- +\N + +-- !ip61_2 -- +\N + +-- !ip61_3 -- +\N + -- !ip62 -- false @@ -192,4 +219,13 @@ true false -- !ip65 -- -true \ No newline at end of file +true + +-- !ip65_1 -- +\N + +-- !ip65_2 -- +\N + +-- !ip65_3 -- +\N \ No newline at end of file diff --git a/regression-test/data/query_p0/sql_functions/ip_functions/test_ip_functions.out b/regression-test/data/query_p0/sql_functions/ip_functions/test_ip_functions.out index 1837acc4bab4e9..569dc78a1e05c2 100644 --- a/regression-test/data/query_p0/sql_functions/ip_functions/test_ip_functions.out +++ b/regression-test/data/query_p0/sql_functions/ip_functions/test_ip_functions.out @@ -25,6 +25,15 @@ -- !sql -- \N +-- !sql -- +\N + +-- !sql -- +\N + +-- !sql -- +\N + -- !sql -- 2a02:6b8::11 @@ -139,6 +148,15 @@ AAAAAAAAFFFFFFFFFFFFFFFFAAAAAAAA -- !sql -- 2A0206B8000000000000000000000011 +-- !sql -- +\N + +-- !sql -- +\N + +-- !sql -- +\N + -- !sql -- true @@ -157,6 +175,15 @@ false -- !sql -- true +-- !sql -- +\N + +-- !sql -- +\N + +-- !sql -- +\N + -- !sql -- false @@ -167,4 +194,13 @@ true false -- !sql -- -true \ No newline at end of file +true + +-- !sql -- +\N + +-- !sql -- +\N + +-- !sql -- +\N \ No newline at end of file diff --git a/regression-test/suites/nereids_function_p0/ip_functions.groovy b/regression-test/suites/nereids_function_p0/ip_functions.groovy index b5d4d177e89fb0..98a2574e62f943 100644 --- a/regression-test/suites/nereids_function_p0/ip_functions.groovy +++ b/regression-test/suites/nereids_function_p0/ip_functions.groovy @@ -31,6 +31,9 @@ suite("ip_functions") { qt_ip10 "SELECT ipv4_string_to_num_or_null('');" qt_ip11 "SELECT ipv4_string_to_num_or_default('');" qt_ip12 "SELECT inet_aton('192.168.0.1');" + qt_ip12_1 "SELECT inet_aton('192.168');" + qt_ip12_2 "SELECT inet_aton('');" + qt_ip12_3 "SELECT inet_aton(NULL);" qt_ip13 "SELECT ipv6_num_to_string(unhex('0A0005091'));" qt_ip14 "SELECT ipv6_num_to_string(unhex('2A0206B8000000000000000000000011'));" @@ -72,6 +75,9 @@ suite("ip_functions") { qt_ip49 "SELECT hex(ipv6_string_to_num_or_null('aaaa:aaaa:ffff:ffff:ffff:ffff:aaaa:aaaa'));" qt_ip50 "SELECT hex(inet6_aton('192.168.0.1'));" qt_ip51 "SELECT hex(inet6_aton('2a02:6b8::11'));" + qt_ip51_1 "SELECT hex(inet6_aton(''));" + qt_ip51_2 "SELECT hex(inet6_aton(NULL));" + qt_ip51_3 "SELECT hex(inet6_aton('KK'));" qt_ip52 "SELECT is_ipv4_string('255.255.255.255');" qt_ip53 "SELECT is_ipv4_string('255.255.255.256');" @@ -84,9 +90,15 @@ suite("ip_functions") { qt_ip59 "SELECT is_ipv4_compat(inet6_aton('::c0a8:0001'));" qt_ip60 "SELECT is_ipv4_compat(inet6_aton('::0.0.0.0'));" qt_ip61 "SELECT is_ipv4_compat(inet6_aton('::255.255.255.255'));" + qt_ip61_1 "SELECT is_ipv4_compat(inet6_aton(''));" + qt_ip61_2 "SELECT is_ipv4_compat(inet6_aton(NULL));" + qt_ip61_3 "SELECT is_ipv4_compat(inet6_aton('KK'));" qt_ip62 "SELECT is_ipv4_mapped(inet6_aton('::10.0.5.9'));" qt_ip63 "SELECT is_ipv4_mapped(inet6_aton('::ffff:10.0.5.9'));" qt_ip64 "SELECT is_ipv4_mapped(inet6_aton('::'));" qt_ip65 "SELECT is_ipv4_mapped(inet6_aton('::ffff:c0a8:0001'));" + qt_ip65_1 "SELECT is_ipv4_mapped(inet6_aton(''));" + qt_ip65_2 "SELECT is_ipv4_mapped(inet6_aton(NULL));" + qt_ip65_3 "SELECT is_ipv4_mapped(inet6_aton('KK'));" } \ No newline at end of file diff --git a/regression-test/suites/query_p0/sql_functions/ip_functions/test_ip_functions.groovy b/regression-test/suites/query_p0/sql_functions/ip_functions/test_ip_functions.groovy index c5ff6d36a1359f..5cbf59ac36c5df 100644 --- a/regression-test/suites/query_p0/sql_functions/ip_functions/test_ip_functions.groovy +++ b/regression-test/suites/query_p0/sql_functions/ip_functions/test_ip_functions.groovy @@ -26,6 +26,9 @@ suite("test_ip_functions", "arrow_flight_sql") { qt_sql "SELECT ipv4_string_to_num_or_null('');" qt_sql "SELECT ipv4_string_to_num_or_default('');" qt_sql "SELECT inet_aton('192.168.0.1');" + qt_sql "SELECT inet_aton('192.168');" + qt_sql "SELECT inet_aton('');" + qt_sql "SELECT inet_aton(NULL);" qt_sql "SELECT ipv6_num_to_string(unhex('0A0005091'));" qt_sql "SELECT ipv6_num_to_string(unhex('2A0206B8000000000000000000000011'));" @@ -67,6 +70,9 @@ suite("test_ip_functions", "arrow_flight_sql") { qt_sql "SELECT hex(ipv6_string_to_num_or_null('aaaa:aaaa:ffff:ffff:ffff:ffff:aaaa:aaaa'));" qt_sql "SELECT hex(inet6_aton('192.168.0.1'));" qt_sql "SELECT hex(inet6_aton('2a02:6b8::11'));" + qt_sql "SELECT hex(inet6_aton(''));" + qt_sql "SELECT hex(inet6_aton(NULL));" + qt_sql "SELECT hex(inet6_aton('KK'));" qt_sql "SELECT is_ipv4_compat(inet6_aton('::10.0.5.9'));" qt_sql "SELECT is_ipv4_compat(inet6_aton('::ffff:10.0.5.9'));" @@ -74,9 +80,15 @@ suite("test_ip_functions", "arrow_flight_sql") { qt_sql "SELECT is_ipv4_compat(inet6_aton('::c0a8:0001'));" qt_sql "SELECT is_ipv4_compat(inet6_aton('::0.0.0.0'));" qt_sql "SELECT is_ipv4_compat(inet6_aton('::255.255.255.255'));" + qt_sql "SELECT is_ipv4_compat(inet6_aton(''));" + qt_sql "SELECT is_ipv4_compat(inet6_aton(NULL));" + qt_sql "SELECT is_ipv4_compat(inet6_aton('KK'));" qt_sql "SELECT is_ipv4_mapped(inet6_aton('::10.0.5.9'));" qt_sql "SELECT is_ipv4_mapped(inet6_aton('::ffff:10.0.5.9'));" qt_sql "SELECT is_ipv4_mapped(inet6_aton('::'));" qt_sql "SELECT is_ipv4_mapped(inet6_aton('::ffff:c0a8:0001'));" + qt_sql "SELECT is_ipv4_mapped(inet6_aton(''));" + qt_sql "SELECT is_ipv4_mapped(inet6_aton(NULL));" + qt_sql "SELECT is_ipv4_mapped(inet6_aton('KK'));" } From c753b2203625de358ac24156039bc8881283c3b8 Mon Sep 17 00:00:00 2001 From: Guangdong Liu Date: Mon, 22 Jan 2024 12:16:58 +0800 Subject: [PATCH 117/200] [regression test](schema change) add boolean type check for agg (#30186) --- .../test_schema_change_agg.groovy | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/regression-test/suites/schema_change_p0/test_schema_change_agg.groovy b/regression-test/suites/schema_change_p0/test_schema_change_agg.groovy index 3b2526cfbc1a14..2534c219ca2bad 100644 --- a/regression-test/suites/schema_change_p0/test_schema_change_agg.groovy +++ b/regression-test/suites/schema_change_p0/test_schema_change_agg.groovy @@ -191,5 +191,33 @@ suite("test_schema_change_agg", "p0") { assertEquals(2, row[1]); assertEquals(3, row[2]); } + + // boolean type + sql """ alter table ${tableName3} add column v15 boolean replace NOT NULL default "0" after k13 """ + + sleep(10) + max_try_num = 6000 + while (max_try_num--) { + String res = getJobState(tableName3) + if (res == "FINISHED" || res == "CANCELLED") { + assertEquals("FINISHED", res) + sleep(3000) + break + } else { + sleep(2000) + if (max_try_num < 1){ + assertEquals(1,2) + } + } + } + + sql """ insert into ${tableName3} values (10002, 2, 3, 4, 5, 6.6, 1.7, 8.81, + 'a', 'b', 'c', '2021-10-30', '2021-10-30 00:00:00', true) """ + + test { + sql """ALTER table ${tableName3} modify COLUMN v15 int replace NOT NULL default "0" after k13""" + exception "Can not change BOOLEAN to INT" + } + } From ff5216a27a393a383fa09ca6b37ba7cefc6ee1fc Mon Sep 17 00:00:00 2001 From: jakevin Date: Mon, 22 Jan 2024 12:23:50 +0800 Subject: [PATCH 118/200] [refactor](Nereids): Eager Aggregation unify pushdown agg function (#30142) --- .../rewrite/PushDownMinMaxThroughJoin.java | 17 ++- .../rules/rewrite/PushDownSumThroughJoin.java | 4 +- .../PushDownSumThroughJoinOneSide.java | 117 +----------------- 3 files changed, 19 insertions(+), 119 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownMinMaxThroughJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownMinMaxThroughJoin.java index 48ded00defe5ee..3057f1eafc4b7e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownMinMaxThroughJoin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownMinMaxThroughJoin.java @@ -81,7 +81,7 @@ public List buildRules() { return null; } LogicalAggregate> agg = ctx.root; - return pushMinMax(agg, agg.child(), ImmutableList.of()); + return pushMinMaxSum(agg, agg.child(), ImmutableList.of()); }) .toRule(RuleType.PUSH_DOWN_MIN_MAX_THROUGH_JOIN), logicalAggregate(logicalProject(innerLogicalJoin())) @@ -102,13 +102,16 @@ public List buildRules() { return null; } LogicalAggregate>> agg = ctx.root; - return pushMinMax(agg, agg.child().child(), agg.child().getProjects()); + return pushMinMaxSum(agg, agg.child().child(), agg.child().getProjects()); }) .toRule(RuleType.PUSH_DOWN_MIN_MAX_THROUGH_JOIN) ); } - private LogicalAggregate pushMinMax(LogicalAggregate agg, + /** + * Push down Min/Max/Sum through join. + */ + public static LogicalAggregate pushMinMaxSum(LogicalAggregate agg, LogicalJoin join, List projects) { List leftOutput = join.left().getOutput(); List rightOutput = join.right().getOutput(); @@ -125,6 +128,9 @@ private LogicalAggregate pushMinMax(LogicalAggregate agg, throw new IllegalStateException("Slot " + slot + " not found in join output"); } } + if (leftFuncs.isEmpty() && rightFuncs.isEmpty()) { + return null; + } Set leftGroupBy = new HashSet<>(); Set rightGroupBy = new HashSet<>(); @@ -177,6 +183,11 @@ private LogicalAggregate pushMinMax(LogicalAggregate agg, Preconditions.checkState(left != join.left() || right != join.right()); Plan newJoin = join.withChildren(left, right); + // top agg + // replace + // min(x) -> min(min#) + // max(x) -> max(max#) + // sum(x) -> sum(sum#) List newOutputExprs = new ArrayList<>(); for (NamedExpression ne : agg.getOutputExpressions()) { if (ne instanceof Alias && ((Alias) ne).child() instanceof AggregateFunction) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownSumThroughJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownSumThroughJoin.java index 91cb2a6050b738..e8987e670a52ed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownSumThroughJoin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownSumThroughJoin.java @@ -53,12 +53,12 @@ * | * * (x) * -> - * aggregate: Sum(min1) + * aggregate: Sum(sum1) * | * join * | \ * | * - * aggregate: Sum(x) as min1 + * aggregate: Sum(x) as sum1 * */ public class PushDownSumThroughJoin implements RewriteRuleFactory { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownSumThroughJoinOneSide.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownSumThroughJoinOneSide.java index 3f4fa09cd71b08..88b13b383a36e5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownSumThroughJoinOneSide.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownSumThroughJoinOneSide.java @@ -19,9 +19,6 @@ import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; -import org.apache.doris.nereids.trees.expressions.Alias; -import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; @@ -30,15 +27,9 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; -import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableList.Builder; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; import java.util.List; -import java.util.Map; import java.util.Set; /** @@ -79,7 +70,7 @@ public List buildRules() { return null; } LogicalAggregate> agg = ctx.root; - return pushSum(agg, agg.child(), ImmutableList.of()); + return PushDownMinMaxThroughJoin.pushMinMaxSum(agg, agg.child(), ImmutableList.of()); }) .toRule(RuleType.PUSH_DOWN_SUM_THROUGH_JOIN), logicalAggregate(logicalProject(innerLogicalJoin())) @@ -98,112 +89,10 @@ public List buildRules() { return null; } LogicalAggregate>> agg = ctx.root; - return pushSum(agg, agg.child().child(), agg.child().getProjects()); + return PushDownMinMaxThroughJoin.pushMinMaxSum(agg, agg.child().child(), + agg.child().getProjects()); }) .toRule(RuleType.PUSH_DOWN_SUM_THROUGH_JOIN) ); } - - private LogicalAggregate pushSum(LogicalAggregate agg, - LogicalJoin join, List projects) { - List leftOutput = join.left().getOutput(); - List rightOutput = join.right().getOutput(); - - List leftSums = new ArrayList<>(); - List rightSums = new ArrayList<>(); - for (AggregateFunction f : agg.getAggregateFunctions()) { - Sum sum = (Sum) f; - Slot slot = (Slot) sum.child(); - if (leftOutput.contains(slot)) { - leftSums.add(sum); - } else if (rightOutput.contains(slot)) { - rightSums.add(sum); - } else { - throw new IllegalStateException("Slot " + slot + " not found in join output"); - } - } - if (leftSums.isEmpty() && rightSums.isEmpty()) { - return null; - } - - Set leftGroupBy = new HashSet<>(); - Set rightGroupBy = new HashSet<>(); - for (Expression e : agg.getGroupByExpressions()) { - Slot slot = (Slot) e; - if (leftOutput.contains(slot)) { - leftGroupBy.add(slot); - } else if (rightOutput.contains(slot)) { - rightGroupBy.add(slot); - } else { - return null; - } - } - join.getHashJoinConjuncts().forEach(e -> e.getInputSlots().forEach(slot -> { - if (leftOutput.contains(slot)) { - leftGroupBy.add(slot); - } else if (rightOutput.contains(slot)) { - rightGroupBy.add(slot); - } else { - throw new IllegalStateException("Slot " + slot + " not found in join output"); - } - })); - - Plan left = join.left(); - Plan right = join.right(); - - Map leftSumSlotToOutput = new HashMap<>(); - Map rightSumSlotToOutput = new HashMap<>(); - - // left Sum agg - if (!leftSums.isEmpty()) { - Builder leftSumAggOutputBuilder = ImmutableList.builder() - .addAll(leftGroupBy); - leftSums.forEach(func -> { - Alias alias = func.alias(func.getName()); - leftSumSlotToOutput.put((Slot) func.child(0), alias); - leftSumAggOutputBuilder.add(alias); - }); - left = new LogicalAggregate<>(ImmutableList.copyOf(leftGroupBy), leftSumAggOutputBuilder.build(), - join.left()); - } - - // right Sum agg - if (!rightSums.isEmpty()) { - Builder rightSumAggOutputBuilder = ImmutableList.builder() - .addAll(rightGroupBy); - rightSums.forEach(func -> { - Alias alias = func.alias(func.getName()); - rightSumSlotToOutput.put((Slot) func.child(0), alias); - rightSumAggOutputBuilder.add(alias); - }); - right = new LogicalAggregate<>(ImmutableList.copyOf(rightGroupBy), rightSumAggOutputBuilder.build(), - join.right()); - } - - Preconditions.checkState(left != join.left() || right != join.right()); - Plan newJoin = join.withChildren(left, right); - - // top Sum agg - // replace sum(x) -> sum(sum#) - List newOutputExprs = new ArrayList<>(); - for (NamedExpression ne : agg.getOutputExpressions()) { - if (ne instanceof Alias && ((Alias) ne).child() instanceof Sum) { - Sum oldTopSum = (Sum) ((Alias) ne).child(); - - Slot slot = (Slot) oldTopSum.child(0); - if (leftSumSlotToOutput.containsKey(slot)) { - Expression expr = new Sum(leftSumSlotToOutput.get(slot).toSlot()); - newOutputExprs.add((NamedExpression) ne.withChildren(expr)); - } else if (rightSumSlotToOutput.containsKey(slot)) { - Expression expr = new Sum(rightSumSlotToOutput.get(slot).toSlot()); - newOutputExprs.add((NamedExpression) ne.withChildren(expr)); - } else { - throw new IllegalStateException("Slot " + slot + " not found in join output"); - } - } else { - newOutputExprs.add(ne); - } - } - return agg.withAggOutputChild(newOutputExprs, newJoin); - } } From 75c0d6c39850de3da13de3367bbb8aa8b5abe4f6 Mon Sep 17 00:00:00 2001 From: seawinde <149132972+seawinde@users.noreply.github.com> Date: Mon, 22 Jan 2024 13:04:04 +0800 Subject: [PATCH 119/200] [Improvement](nereids) Make sure to catch and record exception for every materialization context (#29953) 1. Make sure instance when change params of StructInfo,Predicates. 2. Catch and record exception for every materialization context, this make sure that if throw exception when one materialization context rewrite, it will not influence others. 3. Support to mv rewrite when hava count function when aggregate without group by --- .../java/org/apache/doris/mtmv/MTMVCache.java | 4 +- ...AbstractMaterializedViewAggregateRule.java | 114 ++---- .../mv/AbstractMaterializedViewJoinRule.java | 17 +- .../mv/AbstractMaterializedViewRule.java | 372 ++++++++++-------- .../mv/MaterializedViewAggregateRule.java | 2 +- .../rules/exploration/mv/Predicates.java | 28 +- .../rules/exploration/mv/StructInfo.java | 147 ++++--- .../rules/rewrite/NormalizeToSlot.java | 23 +- .../apache/doris/nereids/trees/TreeNode.java | 17 - .../trees/plans/logical/LogicalProject.java | 4 + .../doris/nereids/util/ExpressionUtils.java | 4 +- .../aggregate_with_roll_up.out | 6 + .../aggregate_with_roll_up.groovy | 131 +++--- 13 files changed, 483 insertions(+), 386 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCache.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCache.java index 3d776d9a7a42f6..a7ddeeb170ca82 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCache.java @@ -63,7 +63,9 @@ public static MTMVCache from(MTMV mtmv, ConnectContext connectContext) { StatementContext mvSqlStatementContext = new StatementContext(connectContext, new OriginStatement(mtmv.getQuerySql(), 0)); NereidsPlanner planner = new NereidsPlanner(mvSqlStatementContext); - + if (mvSqlStatementContext.getConnectContext().getStatementContext() == null) { + mvSqlStatementContext.getConnectContext().setStatementContext(mvSqlStatementContext); + } Plan mvRewrittenPlan = planner.plan(unboundMvPlan, PhysicalProperties.ANY, ExplainLevel.REWRITTEN_PLAN); Plan mvPlan = mvRewrittenPlan instanceof LogicalResultSink diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java index e47e15dd56b362..48b900ca745bbb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java @@ -23,8 +23,8 @@ import org.apache.doris.nereids.jobs.joinorder.hypergraph.node.AbstractNode; import org.apache.doris.nereids.jobs.joinorder.hypergraph.node.StructInfoNode; import org.apache.doris.nereids.rules.exploration.mv.StructInfo.PlanSplitContext; -import org.apache.doris.nereids.rules.exploration.mv.mapping.ExpressionMapping; import org.apache.doris.nereids.rules.exploration.mv.mapping.SlotMapping; +import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Any; import org.apache.doris.nereids.trees.expressions.Cast; import org.apache.doris.nereids.trees.expressions.ExprId; @@ -45,7 +45,6 @@ import org.apache.doris.nereids.util.ExpressionUtils; import com.google.common.collect.ArrayListMultimap; -import com.google.common.collect.HashMultimap; import com.google.common.collect.Multimap; import com.google.common.collect.Sets; @@ -64,7 +63,7 @@ */ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMaterializedViewRule { - protected static final Multimap + protected static final Multimap AGGREGATE_ROLL_UP_EQUIVALENT_FUNCTION_MAP = ArrayListMultimap.create(); static { @@ -95,7 +94,7 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate protected Plan rewriteQueryByView(MatchMode matchMode, StructInfo queryStructInfo, StructInfo viewStructInfo, - SlotMapping queryToViewSlotMapping, + SlotMapping viewToQuerySlotMapping, Plan tempRewritedPlan, MaterializationContext materializationContext) { // get view and query aggregate and top plan correspondingly @@ -115,12 +114,11 @@ protected Plan rewriteQueryByView(MatchMode matchMode, } // Firstly,if group by expression between query and view is equals, try to rewrite expression directly Plan queryTopPlan = queryTopPlanAndAggPair.key(); - SlotMapping viewToQurySlotMapping = queryToViewSlotMapping.inverse(); - if (isGroupByEquals(queryTopPlanAndAggPair, viewTopPlanAndAggPair, viewToQurySlotMapping)) { + if (isGroupByEquals(queryTopPlanAndAggPair, viewTopPlanAndAggPair, viewToQuerySlotMapping)) { List rewrittenQueryExpressions = rewriteExpression(queryTopPlan.getExpressions(), queryTopPlan, materializationContext.getMvExprToMvScanExprMapping(), - queryToViewSlotMapping, + viewToQuerySlotMapping, true); if (!rewrittenQueryExpressions.isEmpty()) { return new LogicalProject<>( @@ -133,15 +131,17 @@ protected Plan rewriteQueryByView(MatchMode matchMode, materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), Pair.of("Can not rewrite expression when no roll up", String.format("expressionToWrite = %s,\n mvExprToMvScanExprMapping = %s,\n" - + "queryToViewSlotMapping = %s", + + "viewToQuerySlotMapping = %s", queryTopPlan.getExpressions(), materializationContext.getMvExprToMvScanExprMapping(), - queryToViewSlotMapping))); + viewToQuerySlotMapping))); } // if view is scalar aggregate but query is not. Or if query is scalar aggregate but view is not // Should not rewrite - if (queryTopPlanAndAggPair.value().getGroupByExpressions().isEmpty() - || viewTopPlanAndAggPair.value().getGroupByExpressions().isEmpty()) { + List queryGroupByExpressions = queryTopPlanAndAggPair.value().getGroupByExpressions(); + List viewGroupByExpressions = viewTopPlanAndAggPair.value().getGroupByExpressions(); + if ((queryGroupByExpressions.isEmpty() && !viewGroupByExpressions.isEmpty()) + || (!queryGroupByExpressions.isEmpty() && viewGroupByExpressions.isEmpty())) { materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), Pair.of("only one the of query or view is scalar aggregate and " + "can not rewrite expression meanwhile", @@ -154,53 +154,42 @@ protected Plan rewriteQueryByView(MatchMode matchMode, // split the query top plan expressions to group expressions and functions, if can not, bail out. Pair, Set> queryGroupAndFunctionPair = topPlanSplitToGroupAndFunction(queryTopPlanAndAggPair); - // this map will be used to rewrite expression - Multimap needRollupExprMap = HashMultimap.create(); - Multimap groupRewrittenExprMap = HashMultimap.create(); - // permute the mv expr mapping to query based - Map mvExprToMvScanExprQueryBased = - materializationContext.getMvExprToMvScanExprMapping().keyPermute(viewToQurySlotMapping) - .flattenMap().get(0); Set queryTopPlanFunctionSet = queryGroupAndFunctionPair.value(); // try to rewrite, contains both roll up aggregate functions and aggregate group expression List finalAggregateExpressions = new ArrayList<>(); List finalGroupExpressions = new ArrayList<>(); - for (Expression topExpression : queryTopPlan.getExpressions()) { + List queryExpressions = queryTopPlan.getExpressions(); + // permute the mv expr mapping to query based + Map mvExprToMvScanExprQueryBased = + materializationContext.getMvExprToMvScanExprMapping().keyPermute(viewToQuerySlotMapping) + .flattenMap().get(0); + for (Expression topExpression : queryExpressions) { // if agg function, try to roll up and rewrite if (queryTopPlanFunctionSet.contains(topExpression)) { Expression queryFunctionShuttled = ExpressionUtils.shuttleExpressionWithLineage( topExpression, queryTopPlan); // try to roll up - AggregateFunction queryFunction = (AggregateFunction) queryFunctionShuttled.firstMatch( - expr -> expr instanceof AggregateFunction); - Function rollupAggregateFunction = rollup(queryFunction, queryFunctionShuttled, - mvExprToMvScanExprQueryBased); + List queryFunctions = + queryFunctionShuttled.collectFirst(expr -> expr instanceof AggregateFunction); + if (queryFunctions.isEmpty()) { + materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), + Pair.of("Can not found query function", + String.format("queryFunctionShuttled = %s", queryFunctionShuttled))); + return null; + } + Function rollupAggregateFunction = rollup((AggregateFunction) queryFunctions.get(0), + queryFunctionShuttled, mvExprToMvScanExprQueryBased); if (rollupAggregateFunction == null) { materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), Pair.of("Query function roll up fail", String.format("queryFunction = %s,\n queryFunctionShuttled = %s,\n" + "mvExprToMvScanExprQueryBased = %s", - queryFunction, queryFunctionShuttled, mvExprToMvScanExprQueryBased))); - return null; - } - // key is query need roll up expr, value is mv scan based roll up expr - needRollupExprMap.put(queryFunctionShuttled, rollupAggregateFunction); - // rewrite query function expression by mv expression - ExpressionMapping needRollupExprMapping = new ExpressionMapping(needRollupExprMap); - Expression rewrittenFunctionExpression = rewriteExpression(topExpression, - queryTopPlan, - needRollupExprMapping, - queryToViewSlotMapping, - false); - if (rewrittenFunctionExpression == null) { - materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), - Pair.of("Roll up expression can not rewrite by view", String.format( - "topExpression = %s,\n needRollupExprMapping = %s,\n queryToViewSlotMapping = %s", - topExpression, needRollupExprMapping, queryToViewSlotMapping))); + queryFunctions.get(0), queryFunctionShuttled, + mvExprToMvScanExprQueryBased))); return null; } - finalAggregateExpressions.add((NamedExpression) rewrittenFunctionExpression); + finalAggregateExpressions.add(new Alias(rollupAggregateFunction)); } else { // if group by expression, try to rewrite group by expression Expression queryGroupShuttledExpr = @@ -213,26 +202,9 @@ protected Plan rewriteQueryByView(MatchMode matchMode, mvExprToMvScanExprQueryBased, queryGroupShuttledExpr))); return null; } - groupRewrittenExprMap.put(queryGroupShuttledExpr, - mvExprToMvScanExprQueryBased.get(queryGroupShuttledExpr)); - // rewrite query group expression by mv expression - ExpressionMapping groupRewrittenExprMapping = new ExpressionMapping(groupRewrittenExprMap); - Expression rewrittenGroupExpression = rewriteExpression( - topExpression, - queryTopPlan, - groupRewrittenExprMapping, - queryToViewSlotMapping, - true); - if (rewrittenGroupExpression == null) { - materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), - Pair.of("Query dimensions can not be rewritten by view", - String.format("topExpression is %s,\n groupRewrittenExprMapping is %s,\n" - + "queryToViewSlotMapping = %s", - topExpression, groupRewrittenExprMapping, queryToViewSlotMapping))); - return null; - } - finalAggregateExpressions.add((NamedExpression) rewrittenGroupExpression); - finalGroupExpressions.add(rewrittenGroupExpression); + Expression expression = mvExprToMvScanExprQueryBased.get(queryGroupShuttledExpr); + finalAggregateExpressions.add((NamedExpression) expression); + finalGroupExpressions.add(expression); } } // add project to guarantee group by column ref is slot reference, @@ -271,7 +243,7 @@ protected Plan rewriteQueryByView(MatchMode matchMode, private boolean isGroupByEquals(Pair> queryTopPlanAndAggPair, Pair> viewTopPlanAndAggPair, - SlotMapping viewToQurySlotMapping) { + SlotMapping viewToQuerySlotMapping) { Plan queryTopPlan = queryTopPlanAndAggPair.key(); Plan viewTopPlan = viewTopPlanAndAggPair.key(); LogicalAggregate queryAggregate = queryTopPlanAndAggPair.value(); @@ -282,7 +254,7 @@ private boolean isGroupByEquals(Pair> queryTopPlanA Set viewGroupShuttledExpressionQueryBased = ExpressionUtils.shuttleExpressionWithLineage( viewAggregate.getGroupByExpressions(), viewTopPlan) .stream() - .map(expr -> ExpressionUtils.replace(expr, viewToQurySlotMapping.toSlotReferenceMap())) + .map(expr -> ExpressionUtils.replace(expr, viewToQuerySlotMapping.toSlotReferenceMap())) .collect(Collectors.toSet()); return queryGroupShuttledExpression.equals(viewGroupShuttledExpressionQueryBased); } @@ -309,19 +281,20 @@ private Function rollup(AggregateFunction queryAggregateFunction, } Expression rollupParam = null; Expression viewRollupFunction = null; - if (mvExprToMvScanExprQueryBased.containsKey(queryAggregateFunctionShuttled)) { - // function can rewrite by view + // handle simple aggregate function roll up which is not in the AGGREGATE_ROLL_UP_EQUIVALENT_FUNCTION_MAP + if (mvExprToMvScanExprQueryBased.containsKey(queryAggregateFunctionShuttled) + && AGGREGATE_ROLL_UP_EQUIVALENT_FUNCTION_MAP.keySet().stream() + .noneMatch(aggFunction -> aggFunction.equals(queryAggregateFunction))) { rollupParam = mvExprToMvScanExprQueryBased.get(queryAggregateFunctionShuttled); viewRollupFunction = queryAggregateFunctionShuttled; } else { - // function can not rewrite by view, try to use complex roll up param + // handle complex functions roll up // eg: query is count(distinct param), mv sql is bitmap_union(to_bitmap(param)) for (Expression mvExprShuttled : mvExprToMvScanExprQueryBased.keySet()) { if (!(mvExprShuttled instanceof Function)) { continue; } - if (isAggregateFunctionEquivalent(queryAggregateFunction, queryAggregateFunctionShuttled, - (Function) mvExprShuttled)) { + if (isAggregateFunctionEquivalent(queryAggregateFunction, (Function) mvExprShuttled)) { rollupParam = mvExprToMvScanExprQueryBased.get(mvExprShuttled); viewRollupFunction = mvExprShuttled; } @@ -429,13 +402,12 @@ protected boolean checkPattern(StructInfo structInfo) { * This will check the count(distinct a) in query is equivalent to bitmap_union(to_bitmap(a)) in mv, * and then check their arguments is equivalent. */ - private boolean isAggregateFunctionEquivalent(Function queryFunction, Expression queryFunctionShuttled, - Function viewFunction) { + private boolean isAggregateFunctionEquivalent(Function queryFunction, Function viewFunction) { if (queryFunction.equals(viewFunction)) { return true; } // check the argument of rollup function is equivalent to view function or not - for (Map.Entry> equivalentFunctionEntry : + for (Map.Entry> equivalentFunctionEntry : AGGREGATE_ROLL_UP_EQUIVALENT_FUNCTION_MAP.asMap().entrySet()) { if (equivalentFunctionEntry.getKey().equals(queryFunction)) { // check is have equivalent function or not diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewJoinRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewJoinRule.java index a482b13b5e76b8..57894ac17ca32d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewJoinRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewJoinRule.java @@ -23,6 +23,7 @@ import org.apache.doris.nereids.jobs.joinorder.hypergraph.node.AbstractNode; import org.apache.doris.nereids.jobs.joinorder.hypergraph.node.StructInfoNode; import org.apache.doris.nereids.rules.exploration.mv.mapping.SlotMapping; +import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.plans.Plan; @@ -47,7 +48,7 @@ public abstract class AbstractMaterializedViewJoinRule extends AbstractMateriali protected Plan rewriteQueryByView(MatchMode matchMode, StructInfo queryStructInfo, StructInfo viewStructInfo, - SlotMapping queryToViewSlotMapping, + SlotMapping targetToSourceMapping, Plan tempRewritedPlan, MaterializationContext materializationContext) { // Rewrite top projects, represent the query projects by view @@ -55,19 +56,18 @@ protected Plan rewriteQueryByView(MatchMode matchMode, queryStructInfo.getExpressions(), queryStructInfo.getOriginalPlan(), materializationContext.getMvExprToMvScanExprMapping(), - queryToViewSlotMapping, + targetToSourceMapping, true ); // Can not rewrite, bail out - if (expressionsRewritten.isEmpty() - || expressionsRewritten.stream().anyMatch(expr -> !(expr instanceof NamedExpression))) { + if (expressionsRewritten.isEmpty()) { materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), Pair.of("Rewrite expressions by view in join fail", String.format("expressionToRewritten is %s,\n mvExprToMvScanExprMapping is %s,\n" - + "queryToViewSlotMapping = %s", + + "targetToSourceMapping = %s", queryStructInfo.getExpressions(), materializationContext.getMvExprToMvScanExprMapping(), - queryToViewSlotMapping))); + targetToSourceMapping))); return null; } // record the group id in materializationContext, and when rewrite again in @@ -77,7 +77,10 @@ protected Plan rewriteQueryByView(MatchMode matchMode, queryStructInfo.getOriginalPlan().getGroupExpression().get().getOwnerGroup().getGroupId()); } return new LogicalProject<>( - expressionsRewritten.stream().map(NamedExpression.class::cast).collect(Collectors.toList()), + expressionsRewritten.stream() + .map(expression -> expression instanceof NamedExpression ? expression : new Alias(expression)) + .map(NamedExpression.class::cast) + .collect(Collectors.toList()), tempRewritedPlan); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java index 352bccc019b656..ea035769b60b88 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java @@ -48,6 +48,7 @@ import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.util.ExpressionUtils; import org.apache.doris.nereids.util.TypeUtils; @@ -72,163 +73,222 @@ public abstract class AbstractMaterializedViewRule implements ExplorationRuleFac JoinType.LEFT_OUTER_JOIN); /** - * The abstract template method for query rewrite, it contains the main logic and different query - * pattern should override the sub logic. + * The abstract template method for query rewrite, it contains the main logic, try to rewrite query by + * multi materialization every time. if exception it will catch the exception and record it to + * materialization context. */ - protected List rewrite(Plan queryPlan, CascadesContext cascadesContext) { + public List rewrite(Plan queryPlan, CascadesContext cascadesContext) { + List rewrittenPlans = new ArrayList<>(); + // already rewrite or query is invalid, bail out + List queryStructInfos = checkQuery(queryPlan, cascadesContext); + if (queryStructInfos.isEmpty()) { + return rewrittenPlans; + } + for (MaterializationContext context : cascadesContext.getMaterializationContexts()) { + if (checkIfRewritten(queryPlan, context)) { + continue; + } + // TODO Just support only one query struct info, support multi later. + StructInfo queryStructInfo = queryStructInfos.get(0); + try { + rewrittenPlans.addAll(doRewrite(queryStructInfo, cascadesContext, context)); + } catch (Exception exception) { + context.recordFailReason(queryStructInfo.getOriginalPlanId(), + Pair.of("Materialized view rule exec fail", exception.toString())); + } + } + return rewrittenPlans; + } + + /** + * Check query is valid or not, if valid return the query struct infos, if invalid return empty list. + */ + protected List checkQuery(Plan queryPlan, CascadesContext cascadesContext) { + List validQueryStructInfos = new ArrayList<>(); List materializationContexts = cascadesContext.getMaterializationContexts(); - List rewriteResults = new ArrayList<>(); if (materializationContexts.isEmpty()) { - return rewriteResults; + return validQueryStructInfos; } List queryStructInfos = MaterializedViewUtils.extractStructInfo(queryPlan, cascadesContext); // TODO Just Check query queryPlan firstly, support multi later. StructInfo queryStructInfo = queryStructInfos.get(0); if (!checkPattern(queryStructInfo)) { - materializationContexts.forEach(ctx -> ctx.recordFailReason(queryStructInfo.getOriginalPlanId(), - Pair.of("Query struct info is invalid", - String.format("queryPlan is %s", queryPlan.treeString())))); + cascadesContext.getMaterializationContexts().forEach(ctx -> + ctx.recordFailReason(queryStructInfo.getOriginalPlanId(), + Pair.of("Query struct info is invalid", + String.format("queryPlan is %s", queryPlan.treeString()))) + ); + return validQueryStructInfos; + } + validQueryStructInfos.add(queryStructInfo); + return validQueryStructInfos; + } + + /** + * The abstract template method for query rewrite, it contains the main logic, try to rewrite query by + * only one materialization every time. Different query pattern should override the sub logic. + */ + protected List doRewrite(StructInfo queryStructInfo, CascadesContext cascadesContext, + MaterializationContext materializationContext) { + List rewriteResults = new ArrayList<>(); + List viewStructInfos = MaterializedViewUtils.extractStructInfo( + materializationContext.getMvPlan(), cascadesContext); + if (viewStructInfos.size() > 1) { + // view struct info should only have one + materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), + Pair.of("The num of view struct info is more then one", + String.format("mv plan is %s", materializationContext.getMvPlan().treeString()))); return rewriteResults; } - for (MaterializationContext materializationContext : materializationContexts) { - // already rewrite, bail out - if (checkIfRewritten(queryPlan, materializationContext)) { - continue; - } - List viewStructInfos = MaterializedViewUtils.extractStructInfo( - materializationContext.getMvPlan(), cascadesContext); - if (viewStructInfos.size() > 1) { - // view struct info should only have one - materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), - Pair.of("The num of view struct info is more then one", - String.format("mv plan is %s", materializationContext.getMvPlan().treeString()))); - return rewriteResults; - } - StructInfo viewStructInfo = viewStructInfos.get(0); - if (!checkPattern(viewStructInfo)) { + StructInfo viewStructInfo = viewStructInfos.get(0); + if (!checkPattern(viewStructInfo)) { + materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), + Pair.of("View struct info is invalid", + String.format(", view plan is %s", viewStructInfo.getOriginalPlan().treeString()))); + return rewriteResults; + } + MatchMode matchMode = decideMatchMode(queryStructInfo.getRelations(), viewStructInfo.getRelations()); + if (MatchMode.COMPLETE != matchMode) { + materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), + Pair.of("Match mode is invalid", String.format("matchMode is %s", matchMode))); + return rewriteResults; + } + List queryToViewTableMappings = RelationMapping.generate(queryStructInfo.getRelations(), + viewStructInfo.getRelations()); + // if any relation in query and view can not map, bail out. + if (queryToViewTableMappings == null) { + materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), + Pair.of("Query to view table mapping is null", "")); + return rewriteResults; + } + for (RelationMapping queryToViewTableMapping : queryToViewTableMappings) { + SlotMapping queryToViewSlotMapping = SlotMapping.generate(queryToViewTableMapping); + if (queryToViewSlotMapping == null) { materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), - Pair.of("View struct info is invalid", - String.format(", view plan is %s", viewStructInfo.getOriginalPlan().treeString()))); + Pair.of("Query to view slot mapping is null", "")); continue; } - MatchMode matchMode = decideMatchMode(queryStructInfo.getRelations(), viewStructInfo.getRelations()); - if (MatchMode.COMPLETE != matchMode) { + SlotMapping viewToQuerySlotMapping = queryToViewSlotMapping.inverse(); + LogicalCompatibilityContext compatibilityContext = LogicalCompatibilityContext.from( + queryToViewTableMapping, queryToViewSlotMapping, queryStructInfo, viewStructInfo); + ComparisonResult comparisonResult = StructInfo.isGraphLogicalEquals(queryStructInfo, viewStructInfo, + compatibilityContext); + if (comparisonResult.isInvalid()) { materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), - Pair.of("Match mode is invalid", String.format("matchMode is %s", matchMode))); + Pair.of("The graph logic between query and view is not consistent", + comparisonResult.getErrorMessage())); continue; } - List queryToViewTableMappings = RelationMapping.generate(queryStructInfo.getRelations(), - viewStructInfo.getRelations()); - // if any relation in query and view can not map, bail out. - if (queryToViewTableMappings == null) { + SplitPredicate compensatePredicates = predicatesCompensate(queryStructInfo, viewStructInfo, + viewToQuerySlotMapping, comparisonResult, cascadesContext); + // Can not compensate, bail out + if (compensatePredicates.isInvalid()) { materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), - Pair.of("Query to view table mapping is null", "")); - return rewriteResults; + Pair.of("Predicate compensate fail", + String.format("query predicates = %s,\n query equivalenceClass = %s, \n" + + "view predicates = %s,\n query equivalenceClass = %s\n", + queryStructInfo.getPredicates(), + queryStructInfo.getEquivalenceClass(), + viewStructInfo.getPredicates(), + viewStructInfo.getEquivalenceClass()))); + continue; } - for (RelationMapping queryToViewTableMapping : queryToViewTableMappings) { - SlotMapping queryToViewSlotMapping = SlotMapping.generate(queryToViewTableMapping); - if (queryToViewSlotMapping == null) { + Plan rewrittenPlan; + Plan mvScan = materializationContext.getMvScanPlan(); + Plan originalPlan = queryStructInfo.getOriginalPlan(); + if (compensatePredicates.isAlwaysTrue()) { + rewrittenPlan = mvScan; + } else { + // Try to rewrite compensate predicates by using mv scan + List rewriteCompensatePredicates = rewriteExpression(compensatePredicates.toList(), + originalPlan, materializationContext.getMvExprToMvScanExprMapping(), + viewToQuerySlotMapping, true); + if (rewriteCompensatePredicates.isEmpty()) { materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), - Pair.of("Query to view slot mapping is null", "")); + Pair.of("Rewrite compensate predicate by view fail", String.format( + "compensatePredicates = %s,\n mvExprToMvScanExprMapping = %s,\n" + + "viewToQuerySlotMapping = %s", + compensatePredicates, + materializationContext.getMvExprToMvScanExprMapping(), + viewToQuerySlotMapping))); continue; } - LogicalCompatibilityContext compatibilityContext = LogicalCompatibilityContext.from( - queryToViewTableMapping, queryToViewSlotMapping, queryStructInfo, viewStructInfo); - ComparisonResult comparisonResult = StructInfo.isGraphLogicalEquals(queryStructInfo, viewStructInfo, - compatibilityContext); - if (comparisonResult.isInvalid()) { - materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), - Pair.of("The graph logic between query and view is not consistent", - comparisonResult.getErrorMessage())); - continue; - } - SplitPredicate compensatePredicates = predicatesCompensate(queryStructInfo, viewStructInfo, - queryToViewSlotMapping, comparisonResult, cascadesContext); - // Can not compensate, bail out - if (compensatePredicates.isInvalid()) { - materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), - Pair.of("Predicate compensate fail", - String.format("query predicates = %s,\n query equivalenceClass = %s, \n" - + "view predicates = %s,\n query equivalenceClass = %s\n", - queryStructInfo.getPredicates(), - queryStructInfo.getEquivalenceClass(), - viewStructInfo.getPredicates(), - viewStructInfo.getEquivalenceClass()))); - continue; - } - Plan rewrittenPlan; - Plan mvScan = materializationContext.getMvScanPlan(); - if (compensatePredicates.isAlwaysTrue()) { - rewrittenPlan = mvScan; - } else { - // Try to rewrite compensate predicates by using mv scan - List rewriteCompensatePredicates = rewriteExpression(compensatePredicates.toList(), - queryPlan, materializationContext.getMvExprToMvScanExprMapping(), queryToViewSlotMapping, - true); - if (rewriteCompensatePredicates.isEmpty()) { - materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), - Pair.of("Rewrite compensate predicate by view fail", String.format( - "compensatePredicates = %s,\n mvExprToMvScanExprMapping = %s,\n" - + "queryToViewSlotMapping = %s", - compensatePredicates, - materializationContext.getMvExprToMvScanExprMapping(), - queryToViewSlotMapping))); - continue; - } - rewrittenPlan = new LogicalFilter<>(Sets.newHashSet(rewriteCompensatePredicates), mvScan); - } - // Rewrite query by view - rewrittenPlan = rewriteQueryByView(matchMode, queryStructInfo, viewStructInfo, queryToViewSlotMapping, - rewrittenPlan, materializationContext); - if (rewrittenPlan == null) { - continue; - } - // run rbo job on mv rewritten plan - CascadesContext rewrittenPlanContext = CascadesContext.initContext( - cascadesContext.getStatementContext(), rewrittenPlan, - cascadesContext.getCurrentJobContext().getRequiredProperties()); - Rewriter.getWholeTreeRewriter(rewrittenPlanContext).execute(); - rewrittenPlan = rewrittenPlanContext.getRewritePlan(); - if (!checkOutput(queryPlan, rewrittenPlan, materializationContext)) { - continue; - } - // check the partitions used by rewritten plan is valid or not - Set invalidPartitionsQueryUsed = - calcInvalidPartitions(rewrittenPlan, materializationContext, cascadesContext); - if (!invalidPartitionsQueryUsed.isEmpty()) { - materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), - Pair.of("Check partition query used validation fail", - String.format("the partition used by query is invalid by materialized view," - + "invalid partition info query used is %s", - materializationContext.getMTMV().getPartitions().stream() - .filter(partition -> - invalidPartitionsQueryUsed.contains(partition.getId())) - .collect(Collectors.toSet())))); - continue; - } - recordIfRewritten(queryPlan, materializationContext); - rewriteResults.add(rewrittenPlan); + rewrittenPlan = new LogicalFilter<>(Sets.newHashSet(rewriteCompensatePredicates), mvScan); + } + // Rewrite query by view + rewrittenPlan = rewriteQueryByView(matchMode, queryStructInfo, viewStructInfo, viewToQuerySlotMapping, + rewrittenPlan, materializationContext); + if (rewrittenPlan == null) { + continue; } + rewrittenPlan = rewriteByRules(cascadesContext, rewrittenPlan, originalPlan); + if (!isOutputValid(originalPlan, rewrittenPlan)) { + ObjectId planObjId = originalPlan.getGroupExpression().map(GroupExpression::getId) + .orElseGet(() -> new ObjectId(-1)); + materializationContext.recordFailReason(planObjId, Pair.of( + "RewrittenPlan output logical properties is different with target group", + String.format("planOutput logical properties = %s,\n" + + "groupOutput logical properties = %s", rewrittenPlan.getLogicalProperties(), + originalPlan.getLogicalProperties()))); + continue; + } + // check the partitions used by rewritten plan is valid or not + Set invalidPartitionsQueryUsed = + calcInvalidPartitions(rewrittenPlan, materializationContext, cascadesContext); + if (!invalidPartitionsQueryUsed.isEmpty()) { + materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(), + Pair.of("Check partition query used validation fail", + String.format("the partition used by query is invalid by materialized view," + + "invalid partition info query used is %s", + materializationContext.getMTMV().getPartitions().stream() + .filter(partition -> + invalidPartitionsQueryUsed.contains(partition.getId())) + .collect(Collectors.toSet())))); + continue; + } + recordIfRewritten(originalPlan, materializationContext); + rewriteResults.add(rewrittenPlan); } return rewriteResults; } + /** + * Rewrite by rules and try to make output is the same after optimize by rules + */ + protected Plan rewriteByRules(CascadesContext cascadesContext, Plan rewrittenPlan, Plan originPlan) { + // run rbo job on mv rewritten plan + CascadesContext rewrittenPlanContext = CascadesContext.initContext( + cascadesContext.getStatementContext(), rewrittenPlan, + cascadesContext.getCurrentJobContext().getRequiredProperties()); + Rewriter.getWholeTreeRewriter(rewrittenPlanContext).execute(); + rewrittenPlan = rewrittenPlanContext.getRewritePlan(); + List originPlanOutput = originPlan.getOutput(); + List rewrittenPlanOutput = rewrittenPlan.getOutput(); + if (originPlanOutput.size() != rewrittenPlanOutput.size()) { + return null; + } + List expressions = new ArrayList<>(); + // should add project above rewritten plan if top plan is not project, if aggregate above will nu + if (!isOutputValid(originPlan, rewrittenPlan)) { + for (int i = 0; i < originPlanOutput.size(); i++) { + expressions.add(((NamedExpression) normalizeExpression(originPlanOutput.get(i), + rewrittenPlanOutput.get(i)))); + } + return new LogicalProject<>(expressions, rewrittenPlan, false); + } + return rewrittenPlan; + } + /** * Check the logical properties of rewritten plan by mv is the same with source plan + * if same return true, if different return false */ - protected boolean checkOutput(Plan sourcePlan, Plan rewrittenPlan, MaterializationContext materializationContext) { + protected boolean isOutputValid(Plan sourcePlan, Plan rewrittenPlan) { if (sourcePlan.getGroupExpression().isPresent() && !rewrittenPlan.getLogicalProperties() .equals(sourcePlan.getGroupExpression().get().getOwnerGroup().getLogicalProperties())) { - ObjectId planObjId = sourcePlan.getGroupExpression().map(GroupExpression::getId) - .orElseGet(() -> new ObjectId(-1)); - materializationContext.recordFailReason(planObjId, Pair.of( - "RewrittenPlan output logical properties is different with target group", - String.format("planOutput logical properties = %s,\n" - + "groupOutput logical properties = %s", rewrittenPlan.getLogicalProperties(), - sourcePlan.getGroupExpression().get().getOwnerGroup().getLogicalProperties()))); return false; } - return true; + return sourcePlan.getLogicalProperties().equals(rewrittenPlan.getLogicalProperties()); } /** @@ -271,7 +331,7 @@ protected Set calcInvalidPartitions(Plan rewrittenPlan, MaterializationCon * Rewrite query by view, for aggregate or join rewriting should be different inherit class implementation */ protected Plan rewriteQueryByView(MatchMode matchMode, StructInfo queryStructInfo, StructInfo viewStructInfo, - SlotMapping queryToViewSlotMapping, Plan tempRewritedPlan, MaterializationContext materializationContext) { + SlotMapping viewToQuerySlotMapping, Plan tempRewritedPlan, MaterializationContext materializationContext) { return tempRewritedPlan; } @@ -298,7 +358,7 @@ protected Plan rewriteQueryByView(MatchMode matchMode, StructInfo queryStructInf * target */ protected List rewriteExpression(List sourceExpressionsToWrite, Plan sourcePlan, - ExpressionMapping targetExpressionMapping, SlotMapping sourceToTargetMapping, + ExpressionMapping targetExpressionMapping, SlotMapping targetToSourceMapping, boolean targetExpressionNeedSourceBased) { // Firstly, rewrite the target expression using source with inverse mapping // then try to use the target expression to represent the query. if any of source expressions @@ -307,14 +367,13 @@ protected List rewriteExpression(List sourceEx List sourceShuttledExpressions = ExpressionUtils.shuttleExpressionWithLineage( sourceExpressionsToWrite, sourcePlan); ExpressionMapping expressionMappingKeySourceBased = targetExpressionNeedSourceBased - ? targetExpressionMapping.keyPermute(sourceToTargetMapping.inverse()) : targetExpressionMapping; + ? targetExpressionMapping.keyPermute(targetToSourceMapping) : targetExpressionMapping; // target to target replacement expression mapping, because mv is 1:1 so get first element List> flattenExpressionMap = expressionMappingKeySourceBased.flattenMap(); Map targetToTargetReplacementMapping = flattenExpressionMap.get(0); List rewrittenExpressions = new ArrayList<>(); - for (int index = 0; index < sourceShuttledExpressions.size(); index++) { - Expression expressionShuttledToRewrite = sourceShuttledExpressions.get(index); + for (Expression expressionShuttledToRewrite : sourceShuttledExpressions) { if (expressionShuttledToRewrite instanceof Literal) { rewrittenExpressions.add(expressionShuttledToRewrite); continue; @@ -327,39 +386,31 @@ protected List rewriteExpression(List sourceEx // if contains any slot to rewrite, which means can not be rewritten by target, bail out return ImmutableList.of(); } - Expression sourceExpression = sourceExpressionsToWrite.get(index); - if (sourceExpression instanceof NamedExpression - && replacedExpression.nullable() != sourceExpression.nullable()) { - // if enable join eliminate, query maybe inner join and mv maybe outer join. - // If the slot is at null generate side, the nullable maybe different between query and view - // So need to force to consistent. - replacedExpression = sourceExpression.nullable() - ? new Nullable(replacedExpression) : new NonNullable(replacedExpression); - } - if (sourceExpression instanceof NamedExpression) { - NamedExpression sourceNamedExpression = (NamedExpression) sourceExpression; - replacedExpression = new Alias(sourceNamedExpression.getExprId(), replacedExpression, - sourceNamedExpression.getName()); - } rewrittenExpressions.add(replacedExpression); } return rewrittenExpressions; } /** - * Rewrite single expression, the logic is the same with above + * Normalize expression with query, keep the consistency of exprId and nullable props with + * query */ - protected Expression rewriteExpression(Expression sourceExpressionsToWrite, Plan sourcePlan, - ExpressionMapping targetExpressionMapping, SlotMapping sourceToTargetMapping, - boolean targetExpressionNeedSourceBased) { - List expressionToRewrite = new ArrayList<>(); - expressionToRewrite.add(sourceExpressionsToWrite); - List rewrittenExpressions = rewriteExpression(expressionToRewrite, sourcePlan, - targetExpressionMapping, sourceToTargetMapping, targetExpressionNeedSourceBased); - if (rewrittenExpressions.isEmpty()) { - return null; + protected Expression normalizeExpression(Expression sourceExpression, Expression replacedExpression) { + if (sourceExpression instanceof NamedExpression + && replacedExpression.nullable() != sourceExpression.nullable()) { + // if enable join eliminate, query maybe inner join and mv maybe outer join. + // If the slot is at null generate side, the nullable maybe different between query and view + // So need to force to consistent. + replacedExpression = sourceExpression.nullable() + ? new Nullable(replacedExpression) : new NonNullable(replacedExpression); + } + if (sourceExpression instanceof NamedExpression + && !sourceExpression.equals(replacedExpression)) { + NamedExpression sourceNamedExpression = (NamedExpression) sourceExpression; + replacedExpression = new Alias(sourceNamedExpression.getExprId(), replacedExpression, + sourceNamedExpression.getName()); } - return rewrittenExpressions.get(0); + return replacedExpression; } /** @@ -371,7 +422,7 @@ protected Expression rewriteExpression(Expression sourceExpressionsToWrite, Plan protected SplitPredicate predicatesCompensate( StructInfo queryStructInfo, StructInfo viewStructInfo, - SlotMapping queryToViewSlotMapping, + SlotMapping viewToQuerySlotMapping, ComparisonResult comparisonResult, CascadesContext cascadesContext ) { @@ -379,15 +430,16 @@ protected SplitPredicate predicatesCompensate( List queryPulledUpExpressions = ImmutableList.copyOf(comparisonResult.getQueryExpressions()); // set pulled up expression to queryStructInfo predicates and update related predicates if (!queryPulledUpExpressions.isEmpty()) { - queryStructInfo.addPredicates(queryPulledUpExpressions); + queryStructInfo = queryStructInfo.withPredicates( + queryStructInfo.getPredicates().merge(queryPulledUpExpressions)); } List viewPulledUpExpressions = ImmutableList.copyOf(comparisonResult.getViewExpressions()); // set pulled up expression to viewStructInfo predicates and update related predicates if (!viewPulledUpExpressions.isEmpty()) { - viewStructInfo.addPredicates(viewPulledUpExpressions); + viewStructInfo = viewStructInfo.withPredicates( + viewStructInfo.getPredicates().merge(viewPulledUpExpressions)); } // viewEquivalenceClass to query based - SlotMapping viewToQuerySlotMapping = queryToViewSlotMapping.inverse(); // equal predicate compensate final Set equalCompensateConjunctions = Predicates.compensateEquivalence( queryStructInfo, @@ -422,11 +474,11 @@ protected SplitPredicate predicatesCompensate( // query has not null reject predicates, so return return SplitPredicate.INVALID_INSTANCE; } + SlotMapping queryToViewMapping = viewToQuerySlotMapping.inverse(); Set queryUsedNeedRejectNullSlotsViewBased = nullRejectPredicates.stream() .map(expression -> TypeUtils.isNotNull(expression).orElse(null)) .filter(Objects::nonNull) - .map(expr -> ExpressionUtils.replace((Expression) expr, - queryToViewSlotMapping.toSlotReferenceMap())) + .map(expr -> ExpressionUtils.replace((Expression) expr, queryToViewMapping.toSlotReferenceMap())) .collect(Collectors.toSet()); if (requireNoNullableViewSlot.stream().anyMatch( set -> Sets.intersection(set, queryUsedNeedRejectNullSlotsViewBased).isEmpty())) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateRule.java index 9059499d3810d8..8e0f8d6f71748b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateRule.java @@ -28,7 +28,7 @@ /** * This is responsible for aggregate rewriting according to different pattern - * */ + */ public class MaterializedViewAggregateRule extends AbstractMaterializedViewAggregateRule { public static final MaterializedViewAggregateRule INSTANCE = new MaterializedViewAggregateRule(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/Predicates.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/Predicates.java index 93bd1d314b1d4c..472e49d3b43a92 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/Predicates.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/Predicates.java @@ -29,6 +29,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; +import java.util.Collection; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -39,32 +40,30 @@ import java.util.stream.Collectors; /** - * This record the predicates which can be pulled up or some other type predicates + * This record the predicates which can be pulled up or some other type predicates. + * Also contains the necessary method for predicates process */ public class Predicates { // Predicates that can be pulled up - private final Set pulledUpPredicates = new HashSet<>(); + private final Set pulledUpPredicates; - private Predicates() { + public Predicates(Set pulledUpPredicates) { + this.pulledUpPredicates = pulledUpPredicates; } - public static Predicates of() { - return new Predicates(); - } - - public static Predicates of(List pulledUpPredicates) { - Predicates predicates = new Predicates(); - pulledUpPredicates.forEach(predicates::addPredicate); - return predicates; + public static Predicates of(Set pulledUpPredicates) { + return new Predicates(pulledUpPredicates); } public Set getPulledUpPredicates() { return pulledUpPredicates; } - public void addPredicate(Expression expression) { - this.pulledUpPredicates.add(expression); + public Predicates merge(Collection predicates) { + Set mergedPredicates = new HashSet<>(predicates); + mergedPredicates.addAll(this.pulledUpPredicates); + return new Predicates(mergedPredicates); } public Expression composedExpression() { @@ -98,8 +97,7 @@ public static Set compensateEquivalence(StructInfo queryStructInfo, if (queryEquivalenceClass.isEmpty() && viewEquivalenceClass.isEmpty()) { equalCompensateConjunctions.add(BooleanLiteral.TRUE); } - if (queryEquivalenceClass.isEmpty() - && !viewEquivalenceClass.isEmpty()) { + if (queryEquivalenceClass.isEmpty() && !viewEquivalenceClass.isEmpty()) { return null; } EquivalenceClassSetMapping queryToViewEquivalenceMapping = diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java index d79153bdc8ef08..3451d8e7c442ea 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java @@ -17,7 +17,9 @@ package org.apache.doris.nereids.rules.exploration.mv; +import org.apache.doris.common.Pair; import org.apache.doris.nereids.jobs.joinorder.hypergraph.HyperGraph; +import org.apache.doris.nereids.jobs.joinorder.hypergraph.edge.JoinEdge; import org.apache.doris.nereids.jobs.joinorder.hypergraph.node.StructInfoNode; import org.apache.doris.nereids.memo.Group; import org.apache.doris.nereids.memo.GroupExpression; @@ -44,6 +46,7 @@ import org.apache.doris.nereids.trees.plans.visitor.ExpressionLineageReplacer; import org.apache.doris.nereids.util.ExpressionUtils; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -60,6 +63,8 @@ /** * StructInfo for plan, this contains necessary info for query rewrite by materialized view + * the struct info is used by all materialization, so it's struct info should only get, should not + * modify, if wanting to modify, should copy and then modify */ public class StructInfo { public static final JoinPatternChecker JOIN_PATTERN_CHECKER = new JoinPatternChecker(); @@ -70,58 +75,76 @@ public class StructInfo { private static final PredicateCollector PREDICATE_COLLECTOR = new PredicateCollector(); // source data private final Plan originalPlan; - private ObjectId originalPlanId; + private final ObjectId originalPlanId; private final HyperGraph hyperGraph; - private boolean valid = true; + private final boolean valid; // derived data following // top plan which may include project or filter, except for join and scan - private Plan topPlan; + private final Plan topPlan; // bottom plan which top plan only contain join or scan. this is needed by hyper graph - private Plan bottomPlan; - private final List relations = new ArrayList<>(); + private final Plan bottomPlan; + private final List relations; // this is for LogicalCompatibilityContext later - private final Map relationIdStructInfoNodeMap = new HashMap<>(); + private final Map relationIdStructInfoNodeMap; // this recorde the predicates which can pull up, not shuttled private Predicates predicates; // split predicates is shuttled - private SplitPredicate splitPredicate; - private EquivalenceClass equivalenceClass; + private final SplitPredicate splitPredicate; + private final EquivalenceClass equivalenceClass; // Key is the expression shuttled and the value is the origin expression // this is for building LogicalCompatibilityContext later. - private final Map shuttledHashConjunctsToConjunctsMap = new HashMap<>(); + private final Map shuttledHashConjunctsToConjunctsMap; // Record the exprId and the corresponding expr map, this is used by expression shuttled - private final Map namedExprIdAndExprMapping = new HashMap<>(); + private final Map namedExprIdAndExprMapping; - private StructInfo(Plan originalPlan, @Nullable Plan topPlan, @Nullable Plan bottomPlan, HyperGraph hyperGraph) { + /** + * The construct method for StructInfo + */ + public StructInfo(Plan originalPlan, ObjectId originalPlanId, HyperGraph hyperGraph, boolean valid, Plan topPlan, + Plan bottomPlan, List relations, + Map relationIdStructInfoNodeMap, + @Nullable Predicates predicates, + Map shuttledHashConjunctsToConjunctsMap, + Map namedExprIdAndExprMapping) { this.originalPlan = originalPlan; - this.originalPlanId = originalPlan.getGroupExpression() - .map(GroupExpression::getId).orElseGet(() -> new ObjectId(-1)); + this.originalPlanId = originalPlanId; this.hyperGraph = hyperGraph; + this.valid = valid; this.topPlan = topPlan; this.bottomPlan = bottomPlan; - init(); - } - - private void init() { - // split the top plan to two parts by join node - if (topPlan == null || bottomPlan == null) { - PlanSplitContext planSplitContext = new PlanSplitContext(Sets.newHashSet(LogicalJoin.class)); - originalPlan.accept(PLAN_SPLITTER, planSplitContext); - this.bottomPlan = planSplitContext.getBottomPlan(); - this.topPlan = planSplitContext.getTopPlan(); + this.relations = relations; + this.relationIdStructInfoNodeMap = relationIdStructInfoNodeMap; + this.predicates = predicates; + if (predicates == null) { + // collect predicate from top plan which not in hyper graph + Set topPlanPredicates = new HashSet<>(); + topPlan.accept(PREDICATE_COLLECTOR, topPlanPredicates); + this.predicates = Predicates.of(topPlanPredicates); } - collectStructInfoFromGraph(); - initPredicates(); + Pair derivedPredicates = predicatesDerive(this.predicates, originalPlan); + this.splitPredicate = derivedPredicates.key(); + this.equivalenceClass = derivedPredicates.value(); + this.shuttledHashConjunctsToConjunctsMap = shuttledHashConjunctsToConjunctsMap; + this.namedExprIdAndExprMapping = namedExprIdAndExprMapping; } - public void addPredicates(List canPulledUpExpressions) { - canPulledUpExpressions.forEach(this.predicates::addPredicate); - predicatesDerive(); + /** + * Construct StructInfo with new predicates + */ + public StructInfo withPredicates(Predicates predicates) { + return new StructInfo(this.originalPlan, this.originalPlanId, this.hyperGraph, this.valid, this.topPlan, + this.bottomPlan, this.relations, this.relationIdStructInfoNodeMap, predicates, + this.shuttledHashConjunctsToConjunctsMap, this.namedExprIdAndExprMapping); } - private void collectStructInfoFromGraph() { + private static boolean collectStructInfoFromGraph(HyperGraph hyperGraph, + Plan topPlan, + Map shuttledHashConjunctsToConjunctsMap, + Map namedExprIdAndExprMapping, + ImmutableList.Builder relationBuilder, + Map relationIdStructInfoNodeMap) { // Collect expression from join condition in hyper graph - this.hyperGraph.getJoinEdges().forEach(edge -> { + for (JoinEdge edge : hyperGraph.getJoinEdges()) { List hashJoinConjuncts = edge.getHashJoinConjuncts(); // shuttle expression in edge for the build of LogicalCompatibilityContext later. // Record the exprId to expr map in the processing to strut info @@ -132,34 +155,31 @@ private void collectStructInfoFromGraph() { Lists.newArrayList(conjunctExpr), ImmutableSet.of(), ImmutableSet.of()); - this.topPlan.accept(ExpressionLineageReplacer.INSTANCE, replaceContext); + topPlan.accept(ExpressionLineageReplacer.INSTANCE, replaceContext); // Replace expressions by expression map List replacedExpressions = replaceContext.getReplacedExpressions(); shuttledHashConjunctsToConjunctsMap.put(replacedExpressions.get(0), conjunctExpr); // Record this, will be used in top level expression shuttle later, see the method // ExpressionLineageReplacer#visitGroupPlan - this.namedExprIdAndExprMapping.putAll(replaceContext.getExprIdExpressionMap()); + namedExprIdAndExprMapping.putAll(replaceContext.getExprIdExpressionMap()); }); List otherJoinConjuncts = edge.getOtherJoinConjuncts(); if (!otherJoinConjuncts.isEmpty()) { - this.valid = false; + return false; } - }); - if (!this.isValid()) { - return; } // Collect relations from hyper graph which in the bottom plan - this.hyperGraph.getNodes().forEach(node -> { + hyperGraph.getNodes().forEach(node -> { // plan relation collector and set to map Plan nodePlan = node.getPlan(); List nodeRelations = new ArrayList<>(); nodePlan.accept(RELATION_COLLECTOR, nodeRelations); - this.relations.addAll(nodeRelations); + relationBuilder.addAll(nodeRelations); // every node should only have one relation, this is for LogicalCompatibilityContext relationIdStructInfoNodeMap.put(nodeRelations.get(0).getRelationId(), (StructInfoNode) node); }); // Collect expression from where in hyper graph - this.hyperGraph.getFilterEdges().forEach(filterEdge -> { + hyperGraph.getFilterEdges().forEach(filterEdge -> { List filterExpressions = filterEdge.getExpressions(); filterExpressions.forEach(predicate -> { // this is used for LogicalCompatibilityContext @@ -168,28 +188,18 @@ private void collectStructInfoFromGraph() { ExpressionUtils.shuttleExpressionWithLineage(predicate, topPlan), predicate)); }); }); - } - - private void initPredicates() { - // Collect predicate from top plan which not in hyper graph - this.predicates = Predicates.of(); - Set topPlanPredicates = new HashSet<>(); - topPlan.accept(PREDICATE_COLLECTOR, topPlanPredicates); - topPlanPredicates.forEach(this.predicates::addPredicate); - predicatesDerive(); + return true; } // derive some useful predicate by predicates - private void predicatesDerive() { + private Pair predicatesDerive(Predicates predicates, Plan originalPlan) { // construct equivalenceClass according to equals predicates List shuttledExpression = ExpressionUtils.shuttleExpressionWithLineage( - new ArrayList<>(this.predicates.getPulledUpPredicates()), originalPlan).stream() + new ArrayList<>(predicates.getPulledUpPredicates()), originalPlan).stream() .map(Expression.class::cast) .collect(Collectors.toList()); SplitPredicate splitPredicate = Predicates.splitPredicates(ExpressionUtils.and(shuttledExpression)); - this.splitPredicate = splitPredicate; - - this.equivalenceClass = new EquivalenceClass(); + EquivalenceClass equivalenceClass = new EquivalenceClass(); for (Expression expression : ExpressionUtils.extractConjunction(splitPredicate.getEqualPredicate())) { if (expression instanceof Literal) { continue; @@ -201,6 +211,7 @@ private void predicatesDerive() { (SlotReference) equalTo.getArguments().get(1)); } } + return Pair.of(splitPredicate, equivalenceClass); } /** @@ -216,11 +227,39 @@ public static List of(Plan originalPlan) { List structInfos = HyperGraph.toStructInfo(planSplitContext.getBottomPlan()); return structInfos.stream() - .map(hyperGraph -> new StructInfo(originalPlan, planSplitContext.getTopPlan(), + .map(hyperGraph -> StructInfo.of(originalPlan, planSplitContext.getTopPlan(), planSplitContext.getBottomPlan(), hyperGraph)) .collect(Collectors.toList()); } + /** + * The construct method for init StructInfo + */ + public static StructInfo of(Plan originalPlan, @Nullable Plan topPlan, @Nullable Plan bottomPlan, + HyperGraph hyperGraph) { + ObjectId originalPlanId = originalPlan.getGroupExpression() + .map(GroupExpression::getId).orElseGet(() -> new ObjectId(-1)); + // if any of topPlan or bottomPlan is null, split the top plan to two parts by join node + if (topPlan == null || bottomPlan == null) { + PlanSplitContext planSplitContext = new PlanSplitContext(Sets.newHashSet(LogicalJoin.class)); + originalPlan.accept(PLAN_SPLITTER, planSplitContext); + bottomPlan = planSplitContext.getBottomPlan(); + topPlan = planSplitContext.getTopPlan(); + } + // collect struct info fromGraph + ImmutableList.Builder relationBuilder = ImmutableList.builder(); + Map relationIdStructInfoNodeMap = new HashMap<>(); + Map shuttledHashConjunctsToConjunctsMap = new HashMap<>(); + Map namedExprIdAndExprMapping = new HashMap<>(); + boolean valid = collectStructInfoFromGraph(hyperGraph, topPlan, shuttledHashConjunctsToConjunctsMap, + namedExprIdAndExprMapping, + relationBuilder, + relationIdStructInfoNodeMap); + return new StructInfo(originalPlan, originalPlanId, hyperGraph, valid, topPlan, bottomPlan, + relationBuilder.build(), relationIdStructInfoNodeMap, null, shuttledHashConjunctsToConjunctsMap, + namedExprIdAndExprMapping); + } + /** * Build Struct info from group. * Maybe return multi structInfo when original plan already be rewritten by mv diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/NormalizeToSlot.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/NormalizeToSlot.java index 41f384ac776d78..1cd56ad1298421 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/NormalizeToSlot.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/NormalizeToSlot.java @@ -38,10 +38,14 @@ import java.util.stream.Collectors; import javax.annotation.Nullable; -/** NormalizeToSlot */ +/** + * NormalizeToSlot + */ public interface NormalizeToSlot { - /** NormalizeSlotContext */ + /** + * NormalizeSlotContext + */ class NormalizeToSlotContext { private final Map normalizeToSlotMap; @@ -51,11 +55,11 @@ public NormalizeToSlotContext(Map normalizeT /** * build normalization context by follow step. - * 1. collect all exists alias by input parameters existsAliases build a reverted map: expr -> alias - * 2. for all input source expressions, use existsAliasMap to construct triple: - * origin expr, pushed expr and alias to replace origin expr, - * see more detail in {@link NormalizeToSlotTriplet} - * 3. construct a map: original expr -> triple constructed by step 2 + * 1. collect all exists alias by input parameters existsAliases build a reverted map: expr -> alias + * 2. for all input source expressions, use existsAliasMap to construct triple: + * origin expr, pushed expr and alias to replace origin expr, + * see more detail in {@link NormalizeToSlotTriplet} + * 3. construct a map: original expr -> triple constructed by step 2 */ public static NormalizeToSlotContext buildContext( Set existsAliases, Collection sourceExpressions) { @@ -65,7 +69,6 @@ public static NormalizeToSlotContext buildContext( for (Alias existsAlias : existsAliases) { existsAliasMap.put(existsAlias.child(), existsAlias); } - for (Expression expression : sourceExpressions) { if (normalizeToSlotMap.containsKey(expression)) { continue; @@ -186,7 +189,9 @@ public Expression visitWindow(WindowExpression windowExpression, } } - /** NormalizeToSlotTriplet */ + /** + * NormalizeToSlotTriplet + */ class NormalizeToSlotTriplet { // which expression need to normalized to slot? // e.g. `a + 1` diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/TreeNode.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/TreeNode.java index 00ac71eaf24c33..3519a983fd114a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/TreeNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/TreeNode.java @@ -199,23 +199,6 @@ default boolean anyMatch(Predicate> predicate) { return false; } - /** - * iterate top down and test predicate if any matched. Top-down traverse implicitly. - * @param predicate predicate - * @return the first node which match the predicate - */ - default TreeNode firstMatch(Predicate> predicate) { - if (predicate.test(this)) { - return this; - } - for (NODE_TYPE child : children()) { - if (child.anyMatch(predicate)) { - return child; - } - } - return this; - } - /** * Collect the nodes that satisfied the predicate. */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java index 5e7ffc1e815470..cdacfe95b66e9a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java @@ -61,6 +61,10 @@ public LogicalProject(List projects, CHILD_TYPE child) { this(projects, ImmutableList.of(), false, true, ImmutableList.of(child)); } + public LogicalProject(List projects, CHILD_TYPE child, boolean canEliminate) { + this(projects, ImmutableList.of(), false, canEliminate, ImmutableList.of(child)); + } + public LogicalProject(List projects, List excepts, boolean isDistinct, List child) { this(projects, excepts, isDistinct, true, Optional.empty(), Optional.empty(), child); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java index 7a2d9fc27d5be9..fd96ceecb939f2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java @@ -231,7 +231,9 @@ public static List shuttleExpressionWithLineage(List targetTypes, Set tableIdentifiers) { - + if (expressions.isEmpty()) { + return ImmutableList.of(); + } ExpressionLineageReplacer.ExpressionReplaceContext replaceContext = new ExpressionLineageReplacer.ExpressionReplaceContext( expressions.stream().map(Expression.class::cast).collect(Collectors.toList()), diff --git a/regression-test/data/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.out b/regression-test/data/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.out index 10c593bd5f06bf..17104fbd732213 100644 --- a/regression-test/data/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.out +++ b/regression-test/data/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.out @@ -255,3 +255,9 @@ -- !query29_0_after -- 8 +-- !query29_1_before -- +0 178.10 1.20 8 + +-- !query29_1_after -- +0 178.10 1.20 8 + diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy index cffb031cb17031..4d001af4128b75 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy @@ -171,7 +171,7 @@ suite("aggregate_with_roll_up") { } } - def check_rewrite_with_force_analyze = { mv_sql, query_sql, mv_name -> + def check_rewrite_but_not_chose = { mv_sql, query_sql, mv_name -> sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name}""" sql""" @@ -182,16 +182,14 @@ suite("aggregate_with_roll_up") { AS ${mv_sql} """ - sql "analyze table ${mv_name} with sync;" - sql "analyze table lineitem with sync;" - sql "analyze table orders with sync;" - sql "analyze table partsupp with sync;" - def job_name = getJobName(db, mv_name); waitingMTMVTaskFinished(job_name) explain { sql("${query_sql}") - contains("${mv_name}(${mv_name})") + check {result -> + def splitResult = result.split("MaterializedViewRewriteSuccessButNotChose") + splitResult.length == 2 ? splitResult[1].contains(mv_name) : false + } } } @@ -436,19 +434,21 @@ suite("aggregate_with_roll_up") { "o_orderdate, " + "l_partkey, " + "l_suppkey" - def query17_0 = "select t1.l_partkey, t1.l_suppkey, l_shipdate, " + - "sum(o_totalprice), " + - "max(o_totalprice), " + - "min(o_totalprice), " + - "count(*), " + - "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " + - "from lineitem t1 " + - "left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " + - "where o_orderdate = '2023-12-11' " + - "group by " + - "l_shipdate, " + - "l_partkey, " + - "l_suppkey" + def query17_0 = """ + select t1.l_partkey, t1.l_suppkey, l_shipdate, + sum(o_totalprice), + max(o_totalprice), + min(o_totalprice), + count(*), + count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) + from lineitem t1 + left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate + where o_orderdate = '2023-12-11' + group by + l_shipdate, + l_partkey, + l_suppkey; + """ order_qt_query17_0_before "${query17_0}" check_rewrite(mv17_0, query17_0, "mv17_0") order_qt_query17_0_after "${query17_0}" @@ -888,34 +888,38 @@ suite("aggregate_with_roll_up") { // single table // filter + use roll up dimension - def mv1_1 = "select o_orderdate, o_shippriority, o_comment, " + - "sum(o_totalprice) as sum_total, " + - "max(o_totalprice) as max_total, " + - "min(o_totalprice) as min_total, " + - "count(*) as count_all, " + - "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, " + - "bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 " + - "from orders " + - "group by " + - "o_orderdate, " + - "o_shippriority, " + - "o_comment " - def query1_1 = "select o_shippriority, o_comment, " + - "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as cnt_1, " + - "count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2, " + - "sum(o_totalprice), " + - "max(o_totalprice), " + - "min(o_totalprice), " + - "count(*) " + - "from orders " + - "where o_orderdate = '2023-12-09' " + - "group by " + - "o_shippriority, " + - "o_comment " + def mv1_1 = """ + select o_orderdate, o_shippriority, o_comment, + sum(o_totalprice) as sum_total, + max(o_totalprice) as max_total, + min(o_totalprice) as min_total, + count(*) as count_all, + bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, + bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 + from orders + group by + o_orderdate, + o_shippriority, + o_comment; + """ + def query1_1 = """ + select o_shippriority, o_comment, + count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as cnt_1, + count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2, + sum(o_totalprice), + max(o_totalprice), + min(o_totalprice), + count(*) + from orders + where o_orderdate = '2023-12-09' + group by + o_shippriority, + o_comment; + """ order_qt_query1_1_before "${query1_1}" - // rewrite success, for cbo chose, should force analyze + // rewrite success, but not chose // because data volume is small and mv plan is almost same to query plan - check_rewrite_with_force_analyze(mv1_1, query1_1, "mv1_1") + check_rewrite_but_not_chose(mv1_1, query1_1, "mv1_1") order_qt_query1_1_after "${query1_1}" sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_1""" @@ -947,9 +951,9 @@ suite("aggregate_with_roll_up") { "o_comment " order_qt_query2_0_before "${query2_0}" - // rewrite success, for cbo chose, should force analyze + // rewrite success, but not chose // because data volume is small and mv plan is almost same to query plan - check_rewrite_with_force_analyze(mv2_0, query2_0, "mv2_0") + check_rewrite_but_not_chose(mv2_0, query2_0, "mv2_0") order_qt_query2_0_after "${query2_0}" sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_0""" @@ -1078,8 +1082,8 @@ suite("aggregate_with_roll_up") { ifnull(o_totalprice, 0) as price_with_no_null from lineitem left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate - ) - select + ) + select count(1) count_all from cte_view_1 cte_view @@ -1090,4 +1094,31 @@ suite("aggregate_with_roll_up") { check_rewrite(mv29_0, query29_0, "mv29_0") order_qt_query29_0_after "${query29_0}" sql """ DROP MATERIALIZED VIEW IF EXISTS mv29_0""" + + // mv and query both are scalar aggregate + def mv29_1 = """ + select + sum(o_totalprice) as sum_total, + max(o_totalprice) as max_total, + min(o_totalprice) as min_total, + count(*) as count_all, + bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, + bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 + from lineitem + left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate; + """ + def query29_1 = """ + select + count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2, + sum(o_totalprice), + min(o_totalprice), + count(*) + from lineitem + left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate; + """ + + order_qt_query29_1_before "${query29_1}" + check_rewrite(mv29_1, query29_1, "mv29_1") + order_qt_query29_1_after "${query29_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv29_1""" } From 27f64c549e32aca6df333f43fa7d993a0a1d6d6d Mon Sep 17 00:00:00 2001 From: HHoflittlefish777 <77738092+HHoflittlefish777@users.noreply.github.com> Date: Mon, 22 Jan 2024 13:15:46 +0800 Subject: [PATCH 120/200] [fix](routine-load) optimize error msg when meet out of range (#30118) --- .../doris/load/routineload/RoutineLoadJob.java | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java index 0a947701ef506b..889d240ce291d6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java @@ -1142,11 +1142,24 @@ public void afterAborted(TransactionState txnState, boolean txnOperated, String if (txnStatusChangeReasonString != null) { txnStatusChangeReason = TransactionState.TxnStatusChangeReason.fromString(txnStatusChangeReasonString); + String msg; if (txnStatusChangeReason != null) { switch (txnStatusChangeReason) { case OFFSET_OUT_OF_RANGE: + msg = "be " + taskBeId + " abort task," + + " task id: " + routineLoadTaskInfo.getId() + + " job id: " + routineLoadTaskInfo.getJobId() + + " with reason: " + txnStatusChangeReasonString + + " the offset used by job does not exist in kafka," + + " please check the offset," + + " using the Alter ROUTINE LOAD command to modify it," + + " and resume the job"; + updateState(JobState.PAUSED, + new ErrorReason(InternalErrorCode.TASKS_ABORT_ERR, msg), + false /* not replay */); + return; case PAUSE: - String msg = "be " + taskBeId + " abort task " + msg = "be " + taskBeId + " abort task " + "with reason: " + txnStatusChangeReasonString; updateState(JobState.PAUSED, new ErrorReason(InternalErrorCode.TASKS_ABORT_ERR, msg), From f97dbd159d87bdc552696b1fde2f4ee3f474d77e Mon Sep 17 00:00:00 2001 From: HHoflittlefish777 <77738092+HHoflittlefish777@users.noreply.github.com> Date: Mon, 22 Jan 2024 13:16:03 +0800 Subject: [PATCH 121/200] [fix](routineload) check offset when schedule tasks (#30136) --- .../load/routineload/KafkaRoutineLoadJob.java | 23 ++++++--- .../doris/load/routineload/KafkaTaskInfo.java | 2 +- .../load/routineload/RoutineLoadTaskInfo.java | 2 +- .../routineload/RoutineLoadTaskScheduler.java | 16 +++---- .../test_routine_load_error.groovy | 47 +++++++++++++++++++ 5 files changed, 74 insertions(+), 16 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java index d7a090c23e5787..faad0a0248a10b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java @@ -704,7 +704,7 @@ public void replayModifyProperties(AlterRoutineLoadJobOperationLog log) { // check if given partitions has more data to consume. // 'partitionIdToOffset' to the offset to be consumed. - public boolean hasMoreDataToConsume(UUID taskId, Map partitionIdToOffset) { + public boolean hasMoreDataToConsume(UUID taskId, Map partitionIdToOffset) throws UserException { for (Map.Entry entry : partitionIdToOffset.entrySet()) { if (cachedPartitionWithLatestOffsets.containsKey(entry.getKey()) && entry.getValue() < cachedPartitionWithLatestOffsets.get(entry.getKey())) { @@ -734,11 +734,22 @@ public boolean hasMoreDataToConsume(UUID taskId, Map partitionIdT // check again for (Map.Entry entry : partitionIdToOffset.entrySet()) { - if (cachedPartitionWithLatestOffsets.containsKey(entry.getKey()) - && entry.getValue() < cachedPartitionWithLatestOffsets.get(entry.getKey())) { - LOG.debug("has more data to consume. offsets to be consumed: {}, latest offsets: {}, task {}, job {}", - partitionIdToOffset, cachedPartitionWithLatestOffsets, taskId, id); - return true; + Integer partitionId = entry.getKey(); + if (cachedPartitionWithLatestOffsets.containsKey(partitionId)) { + long partitionLatestOffset = cachedPartitionWithLatestOffsets.get(partitionId); + long recordPartitionOffset = entry.getValue(); + if (recordPartitionOffset < partitionLatestOffset) { + LOG.debug("has more data to consume. offsets to be consumed: {}," + + " latest offsets: {}, task {}, job {}", + partitionIdToOffset, cachedPartitionWithLatestOffsets, taskId, id); + return true; + } else if (recordPartitionOffset > partitionLatestOffset) { + String msg = "offset set in job: " + recordPartitionOffset + + " is greater than kafka latest offset: " + + partitionLatestOffset + " partition id: " + + partitionId; + throw new UserException(msg); + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java index 2075e5548e5e43..a8d387a2f6d9b6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java @@ -119,7 +119,7 @@ protected String getTaskDataSourceProperties() { } @Override - boolean hasMoreDataToConsume() { + boolean hasMoreDataToConsume() throws UserException { KafkaRoutineLoadJob routineLoadJob = (KafkaRoutineLoadJob) routineLoadManager.getJob(jobId); return routineLoadJob.hasMoreDataToConsume(id, partitionIdToOffset); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java index fd8e8182321564..9c28dbfc6a8448 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java @@ -209,7 +209,7 @@ public List getTaskShowInfo() { abstract String getTaskDataSourceProperties(); - abstract boolean hasMoreDataToConsume(); + abstract boolean hasMoreDataToConsume() throws UserException; @Override public boolean equals(Object obj) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskScheduler.java index 0f594a2d508f12..a6cc796027a428 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskScheduler.java @@ -128,15 +128,15 @@ private void scheduleOneTask(RoutineLoadTaskInfo routineLoadTaskInfo) throws Exc return; } - // check if topic has more data to consume - if (!routineLoadTaskInfo.hasMoreDataToConsume()) { - needScheduleTasksQueue.put(routineLoadTaskInfo); - return; - } - - // allocate BE slot for this task. - // this should be done before txn begin, or the txn may be begun successfully but failed to be allocated. try { + // check if topic has more data to consume + if (!routineLoadTaskInfo.hasMoreDataToConsume()) { + needScheduleTasksQueue.put(routineLoadTaskInfo); + return; + } + + // allocate BE slot for this task. + // this should be done before txn begin, or the txn may be begun successfully but failed to be allocated. if (!allocateTaskToBe(routineLoadTaskInfo)) { // allocate failed, push it back to the queue to wait next scheduling needScheduleTasksQueue.put(routineLoadTaskInfo); diff --git a/regression-test/suites/load_p0/routine_load/test_routine_load_error.groovy b/regression-test/suites/load_p0/routine_load/test_routine_load_error.groovy index b04610740d96a9..c4180e35f4df40 100644 --- a/regression-test/suites/load_p0/routine_load/test_routine_load_error.groovy +++ b/regression-test/suites/load_p0/routine_load/test_routine_load_error.groovy @@ -91,4 +91,51 @@ suite("test_routine_load_error","p0") { sql "stop routine load for testTableNoExist" } } + + // test out of range + if (enabled != null && enabled.equalsIgnoreCase("true")) { + def jobName = "testOutOfRange" + try { + sql """ + CREATE ROUTINE LOAD ${jobName} + COLUMNS TERMINATED BY "|" + PROPERTIES + ( + "max_batch_interval" = "5", + "max_batch_rows" = "300000", + "max_batch_size" = "209715200" + ) + FROM KAFKA + ( + "kafka_broker_list" = "${externalEnvIp}:${kafka_port}", + "kafka_partitions" = "0", + "kafka_topic" = "multi_table_load_invalid_table", + "kafka_offsets" = "100" + ); + """ + sql "sync" + + def count = 0 + while (true) { + sleep(1000) + def res = sql "show routine load for ${jobName}" + def state = res[0][8].toString() + log.info("routine load state: ${res[0][8].toString()}".toString()) + log.info("routine load statistic: ${res[0][14].toString()}".toString()) + log.info("reason of state changed: ${res[0][17].toString()}".toString()) + if (state != "PAUSED") { + count++ + if (count > 60) { + assertEquals(1, 2) + } + continue; + } + log.info("reason of state changed: ${res[0][17].toString()}".toString()) + assertTrue(res[0][17].toString().contains("is greater than kafka latest offset")) + break; + } + } finally { + sql "stop routine load for ${jobName}" + } + } } \ No newline at end of file From 25da0567ae5311258c2969e17f5f7dbf5df5e037 Mon Sep 17 00:00:00 2001 From: Jibing-Li <64681310+Jibing-Li@users.noreply.github.com> Date: Mon, 22 Jan 2024 13:20:34 +0800 Subject: [PATCH 122/200] [improvement](statistics) Optimize drop stats operation (#30144) Before, drop stats operation need to call columns * followers times of isMaster() function and the same times of rpc to drop remote column stats. This pr is to reduce the rpc calls and use more efficient way to check master node instead of using isMaster() --- .../apache/doris/analysis/DropStatsStmt.java | 19 +++++ .../doris/service/FrontendServiceImpl.java | 12 +++- .../doris/statistics/AnalysisManager.java | 70 ++++++++++++++----- .../statistics/InvalidateStatsTarget.java | 48 +++++++++++++ .../doris/statistics/StatisticsCache.java | 17 +---- 5 files changed, 133 insertions(+), 33 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/statistics/InvalidateStatsTarget.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropStatsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropStatsStmt.java index ac08f01f31e70e..5e3bd20c0f1f04 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropStatsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropStatsStmt.java @@ -53,7 +53,10 @@ public class DropStatsStmt extends DdlStmt { private Set columnNames; // Flag to drop external table row count in table_statistics. private boolean dropTableRowCount; + private boolean isAllColumns; + private long catalogId; + private long dbId; private long tblId; public DropStatsStmt(boolean dropExpired) { @@ -100,10 +103,13 @@ public void analyze(Analyzer analyzer) throws UserException { DatabaseIf db = catalog.getDbOrAnalysisException(dbName); TableIf table = db.getTableOrAnalysisException(tblName); tblId = table.getId(); + dbId = db.getId(); + catalogId = catalog.getId(); // check permission checkAnalyzePriv(db.getFullName(), table.getName()); // check columnNames if (columnNames != null) { + isAllColumns = false; for (String cName : columnNames) { if (table.getColumn(cName) == null) { ErrorReport.reportAnalysisException( @@ -115,6 +121,7 @@ public void analyze(Analyzer analyzer) throws UserException { } } } else { + isAllColumns = true; columnNames = table.getColumns().stream().map(Column::getName).collect(Collectors.toSet()); } } @@ -123,10 +130,22 @@ public long getTblId() { return tblId; } + public long getDbId() { + return dbId; + } + + public long getCatalogIdId() { + return catalogId; + } + public Set getColumnNames() { return columnNames; } + public boolean isAllColumns() { + return isAllColumns; + } + public boolean dropTableRowCount() { return dropTableRowCount; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index a9ae485d8f9902..f9e3372bf28076 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -95,9 +95,12 @@ import org.apache.doris.qe.StmtExecutor; import org.apache.doris.qe.VariableMgr; import org.apache.doris.service.arrowflight.FlightSqlConnectProcessor; +import org.apache.doris.statistics.AnalysisManager; import org.apache.doris.statistics.ColumnStatistic; +import org.apache.doris.statistics.InvalidateStatsTarget; import org.apache.doris.statistics.ResultRow; import org.apache.doris.statistics.StatisticsCacheKey; +import org.apache.doris.statistics.TableStatsMeta; import org.apache.doris.statistics.query.QueryStats; import org.apache.doris.system.Backend; import org.apache.doris.system.Frontend; @@ -3041,8 +3044,13 @@ public TStatus updateStatsCache(TUpdateFollowerStatsCacheRequest request) throws @Override public TStatus invalidateStatsCache(TInvalidateFollowerStatsCacheRequest request) throws TException { - StatisticsCacheKey k = GsonUtils.GSON.fromJson(request.key, StatisticsCacheKey.class); - Env.getCurrentEnv().getStatisticsCache().invalidate(k.tableId, k.idxId, k.colName); + InvalidateStatsTarget target = GsonUtils.GSON.fromJson(request.key, InvalidateStatsTarget.class); + AnalysisManager analysisManager = Env.getCurrentEnv().getAnalysisManager(); + TableStatsMeta tableStats = analysisManager.findTableStatsStatus(target.tableId); + if (tableStats == null) { + return new TStatus(TStatusCode.OK); + } + analysisManager.invalidateLocalStats(target.catalogId, target.dbId, target.tableId, target.columns, tableStats); return new TStatus(TStatusCode.OK); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java index 8f927694dc79a5..fe64fb14142fd5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java @@ -58,6 +58,9 @@ import org.apache.doris.statistics.AnalysisInfo.ScheduleType; import org.apache.doris.statistics.util.DBObjects; import org.apache.doris.statistics.util.StatisticsUtil; +import org.apache.doris.system.Frontend; +import org.apache.doris.system.SystemInfoService; +import org.apache.doris.thrift.TInvalidateFollowerStatsCacheRequest; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; @@ -646,23 +649,16 @@ public void dropStats(DropStatsStmt dropStatsStmt) throws DdlException { } Set cols = dropStatsStmt.getColumnNames(); + long catalogId = dropStatsStmt.getCatalogIdId(); + long dbId = dropStatsStmt.getDbId(); long tblId = dropStatsStmt.getTblId(); TableStatsMeta tableStats = findTableStatsStatus(dropStatsStmt.getTblId()); if (tableStats == null) { return; } - if (cols == null) { - tableStats.reset(); - } else { - dropStatsStmt.getColumnNames().forEach(tableStats::removeColumn); - StatisticsCache statisticsCache = Env.getCurrentEnv().getStatisticsCache(); - for (String col : cols) { - statisticsCache.syncInvalidate(tblId, -1L, col); - } - tableStats.updatedTime = 0; - } - tableStats.userInjected = false; - logCreateTableStats(tableStats); + invalidateLocalStats(catalogId, dbId, tblId, cols, tableStats); + // Drop stats ddl is master only operation. + invalidateRemoteStats(catalogId, dbId, tblId, cols, dropStatsStmt.isAllColumns()); StatisticsRepository.dropStatistics(tblId, cols); } @@ -671,15 +667,55 @@ public void dropStats(TableIf table) throws DdlException { if (tableStats == null) { return; } + long catalogId = table.getDatabase().getCatalog().getId(); + long dbId = table.getDatabase().getId(); + long tableId = table.getId(); Set cols = table.getBaseSchema().stream().map(Column::getName).collect(Collectors.toSet()); + invalidateLocalStats(catalogId, dbId, tableId, cols, tableStats); + // Drop stats ddl is master only operation. + invalidateRemoteStats(catalogId, dbId, tableId, cols, true); + StatisticsRepository.dropStatistics(table.getId(), cols); + } + + public void invalidateLocalStats(long catalogId, long dbId, long tableId, + Set columns, TableStatsMeta tableStats) { + if (tableStats == null) { + return; + } StatisticsCache statisticsCache = Env.getCurrentEnv().getStatisticsCache(); - for (String col : cols) { - tableStats.removeColumn(col); - statisticsCache.syncInvalidate(table.getId(), -1L, col); + if (columns == null) { + TableIf table = StatisticsUtil.findTable(catalogId, dbId, tableId); + columns = table.getBaseSchema().stream().map(Column::getName).collect(Collectors.toSet()); + } + for (String column : columns) { + tableStats.removeColumn(column); + statisticsCache.invalidate(tableId, -1, column); } tableStats.updatedTime = 0; - logCreateTableStats(tableStats); - StatisticsRepository.dropStatistics(table.getId(), cols); + tableStats.userInjected = false; + } + + public void invalidateRemoteStats(long catalogId, long dbId, long tableId, + Set columns, boolean isAllColumns) { + InvalidateStatsTarget target = new InvalidateStatsTarget(catalogId, dbId, tableId, columns, isAllColumns); + TInvalidateFollowerStatsCacheRequest request = new TInvalidateFollowerStatsCacheRequest(); + request.key = GsonUtils.GSON.toJson(target); + StatisticsCache statisticsCache = Env.getCurrentEnv().getStatisticsCache(); + SystemInfoService.HostInfo selfNode = Env.getCurrentEnv().getSelfNode(); + boolean success = true; + for (Frontend frontend : Env.getCurrentEnv().getFrontends(null)) { + // Skip master + if (selfNode.equals(frontend.getHost())) { + continue; + } + success = success && statisticsCache.invalidateStats(frontend, request); + } + if (!success) { + // If any rpc failed, use edit log to sync table stats to non-master FEs. + LOG.warn("Failed to invalidate all remote stats by rpc for table {}, use edit log.", tableId); + TableStatsMeta tableStats = findTableStatsStatus(tableId); + logCreateTableStats(tableStats); + } } public void handleKillAnalyzeStmt(KillAnalysisJobStmt killAnalysisJobStmt) throws DdlException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/InvalidateStatsTarget.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/InvalidateStatsTarget.java new file mode 100644 index 00000000000000..e49048f89466a9 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/InvalidateStatsTarget.java @@ -0,0 +1,48 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.statistics; + +import com.google.gson.annotations.SerializedName; + +import java.util.Set; + +public class InvalidateStatsTarget { + + @SerializedName("catalogId") + public final long catalogId; + + @SerializedName("dbId") + public final long dbId; + + @SerializedName("tableId") + public final long tableId; + + @SerializedName("columns") + public final Set columns; + + public InvalidateStatsTarget(long catalogId, long dbId, long tableId, Set columns, boolean isAllColumns) { + this.catalogId = catalogId; + this.dbId = dbId; + this.tableId = tableId; + if (isAllColumns) { + this.columns = null; + } else { + this.columns = columns; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCache.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCache.java index fbec9a60fa00e8..0cf2808222e2b8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCache.java @@ -137,19 +137,6 @@ public void invalidate(long tblId, long idxId, String colName) { columnStatisticsCache.synchronous().invalidate(new StatisticsCacheKey(tblId, idxId, colName)); } - public void syncInvalidate(long tblId, long idxId, String colName) { - StatisticsCacheKey cacheKey = new StatisticsCacheKey(tblId, idxId, colName); - columnStatisticsCache.synchronous().invalidate(cacheKey); - TInvalidateFollowerStatsCacheRequest request = new TInvalidateFollowerStatsCacheRequest(); - request.key = GsonUtils.GSON.toJson(cacheKey); - for (Frontend frontend : Env.getCurrentEnv().getFrontends(FrontendNodeType.FOLLOWER)) { - if (StatisticsUtil.isMaster(frontend)) { - continue; - } - invalidateStats(frontend, request); - } - } - public void updateColStatsCache(long tblId, long idxId, String colName, ColumnStatistic statistic) { columnStatisticsCache.synchronous().put(new StatisticsCacheKey(tblId, idxId, colName), Optional.of(statistic)); } @@ -261,7 +248,7 @@ public void sendStats(Frontend frontend, TUpdateFollowerStatsCacheRequest update } @VisibleForTesting - public void invalidateStats(Frontend frontend, TInvalidateFollowerStatsCacheRequest request) { + public boolean invalidateStats(Frontend frontend, TInvalidateFollowerStatsCacheRequest request) { TNetworkAddress address = new TNetworkAddress(frontend.getHost(), frontend.getRpcPort()); FrontendService.Client client = null; try { @@ -269,11 +256,13 @@ public void invalidateStats(Frontend frontend, TInvalidateFollowerStatsCacheRequ client.invalidateStatsCache(request); } catch (Throwable t) { LOG.warn("Failed to sync invalidate to follower: {}", address, t); + return false; } finally { if (client != null) { ClientPool.frontendPool.returnObject(address, client); } } + return true; } public void putCache(StatisticsCacheKey k, ColumnStatistic c) { From f137d4ea4c2bb7037cad23d033df72d5765b74a7 Mon Sep 17 00:00:00 2001 From: HappenLee Date: Mon, 22 Jan 2024 13:24:02 +0800 Subject: [PATCH 123/200] [temporary](log) Add temporary log for debug insert into lost result (#30140) --- be/src/vec/exec/vunion_node.cpp | 4 +++- be/src/vec/sink/writer/vtablet_writer.cpp | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/be/src/vec/exec/vunion_node.cpp b/be/src/vec/exec/vunion_node.cpp index 8ee258ca05678e..e77fd9fee9097b 100644 --- a/be/src/vec/exec/vunion_node.cpp +++ b/be/src/vec/exec/vunion_node.cpp @@ -235,7 +235,9 @@ Status VUnionNode::get_next_const(RuntimeState* state, Block* block) { } } block->set_columns(std::move(mblock.mutable_columns())); - + LOG(INFO) << "temporary log query id: " << print_id(state->query_id()) + << ", instance id: " << print_id(state->fragment_instance_id()) + << ", block rows: " << block->rows(); // some insert query like "insert into string_test select 1, repeat('a', 1024 * 1024);" // the const expr will be in output expr cause the union node return a empty block. so here we // need add one row to make sure the union node exec const expr return at least one row diff --git a/be/src/vec/sink/writer/vtablet_writer.cpp b/be/src/vec/sink/writer/vtablet_writer.cpp index 7af7b115c43aec..0ca2f7d8ffb45d 100644 --- a/be/src/vec/sink/writer/vtablet_writer.cpp +++ b/be/src/vec/sink/writer/vtablet_writer.cpp @@ -1603,7 +1603,9 @@ Status VTabletWriter::write(doris::vectorized::Block& input_block) { if (_state->query_options().dry_run_query) { return status; } - + LOG(INFO) << "temporary log query id: " << print_id(_state->query_id()) + << ", instance id: " << print_id(_state->fragment_instance_id()) + << ", block rows: " << input_block.rows(); // check out of limit RETURN_IF_ERROR(_send_new_partition_batch()); From f124a78fe7f3e9395ec1c2294dc58828155f3d1e Mon Sep 17 00:00:00 2001 From: yangshijie Date: Mon, 22 Jan 2024 13:25:28 +0800 Subject: [PATCH 124/200] [docs](ip function) add ipv4/v6 datatype and some ip functions' docs (#30194) --- .../ip-functions/ipv4-cidr-to-range.md | 72 ++++++++++++++++ .../ip-functions/ipv6-cidr-to-range.md | 72 ++++++++++++++++ .../ip-functions/is-ip-address-in-range.md | 65 ++++++++++++++ .../ip-functions/is-ipv4-string.md | 80 +++++++++++++++++ .../ip-functions/is-ipv6-string.md | 80 +++++++++++++++++ .../ip-functions/to-ipv4-or-default.md | 65 ++++++++++++++ .../ip-functions/to-ipv4-or-null.md | 65 ++++++++++++++ .../sql-functions/ip-functions/to-ipv4.md | 60 +++++++++++++ .../ip-functions/to-ipv6-or-default.md | 65 ++++++++++++++ .../ip-functions/to-ipv6-or-null.md | 65 ++++++++++++++ .../sql-functions/ip-functions/to-ipv6.md | 60 +++++++++++++ .../sql-reference/Data-Types/IPV4.md | 85 +++++++++++++++++++ .../sql-reference/Data-Types/IPV6.md | 83 ++++++++++++++++++ .../sql-reference/Data-Types/STRUCT.md | 2 +- docs/sidebars.json | 17 +++- .../ip-functions/ipv4-cidr-to-range.md | 72 ++++++++++++++++ .../ip-functions/ipv6-cidr-to-range.md | 72 ++++++++++++++++ .../ip-functions/is-ip-address-in-range.md | 65 ++++++++++++++ .../ip-functions/is-ipv4-string.md | 80 +++++++++++++++++ .../ip-functions/is-ipv6-string.md | 80 +++++++++++++++++ .../ip-functions/to-ipv4-or-default.md | 65 ++++++++++++++ .../ip-functions/to-ipv4-or-null.md | 65 ++++++++++++++ .../sql-functions/ip-functions/to-ipv4.md | 60 +++++++++++++ .../ip-functions/to-ipv6-or-default.md | 65 ++++++++++++++ .../ip-functions/to-ipv6-or-null.md | 65 ++++++++++++++ .../sql-functions/ip-functions/to-ipv6.md | 60 +++++++++++++ .../sql-reference/Data-Types/IPV4.md | 85 +++++++++++++++++++ .../sql-reference/Data-Types/IPV6.md | 83 ++++++++++++++++++ 28 files changed, 1850 insertions(+), 3 deletions(-) create mode 100644 docs/en/docs/sql-manual/sql-functions/ip-functions/ipv4-cidr-to-range.md create mode 100644 docs/en/docs/sql-manual/sql-functions/ip-functions/ipv6-cidr-to-range.md create mode 100644 docs/en/docs/sql-manual/sql-functions/ip-functions/is-ip-address-in-range.md create mode 100644 docs/en/docs/sql-manual/sql-functions/ip-functions/is-ipv4-string.md create mode 100644 docs/en/docs/sql-manual/sql-functions/ip-functions/is-ipv6-string.md create mode 100644 docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv4-or-default.md create mode 100644 docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv4-or-null.md create mode 100644 docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv4.md create mode 100644 docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv6-or-default.md create mode 100644 docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv6-or-null.md create mode 100644 docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv6.md create mode 100644 docs/en/docs/sql-manual/sql-reference/Data-Types/IPV4.md create mode 100644 docs/en/docs/sql-manual/sql-reference/Data-Types/IPV6.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/ipv4-cidr-to-range.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/ipv6-cidr-to-range.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/is-ip-address-in-range.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/is-ipv4-string.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/is-ipv6-string.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv4-or-default.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv4-or-null.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv4.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv6-or-default.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv6-or-null.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv6.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/IPV4.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/IPV6.md diff --git a/docs/en/docs/sql-manual/sql-functions/ip-functions/ipv4-cidr-to-range.md b/docs/en/docs/sql-manual/sql-functions/ip-functions/ipv4-cidr-to-range.md new file mode 100644 index 00000000000000..f5367a577f0599 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/ip-functions/ipv4-cidr-to-range.md @@ -0,0 +1,72 @@ +--- +{ +"title": "IPV4_CIDR_TO_RANGE", +"language": "en" +} +--- + + + +## IPV4_CIDR_TO_RANGE + + + +IPV4_CIDR_TO_RANGE + + + +### description + +#### Syntax + +`STRUCT IPV4_CIDR_TO_RANGE(IPV4 ip_v4, INT16 cidr)` + +Receive an IPv4 and an Int16 value containing CIDR. Returns a struct that contains two IPv4 fields representing the lower range (min) and higher range (max) of the subnet, respectively. + +### notice + +`If the input parameter is NULL, return NULL, indicating invalid input` + +### example + +``` +mysql> SELECT ipv4_cidr_to_range(ipv4_string_to_num('192.168.5.2'), 16); ++-----------------------------------------------------------+ +| ipv4_cidr_to_range(ipv4_string_to_num('192.168.5.2'), 16) | ++-----------------------------------------------------------+ +| {"min": "192.168.0.0", "max": "192.168.255.255"} | ++-----------------------------------------------------------+ + +mysql> SELECT ipv4_cidr_to_range(to_ipv4('192.168.5.2'), 16); ++--------------------------------------------------+ +| ipv4_cidr_to_range(to_ipv4('192.168.5.2'), 16) | ++--------------------------------------------------+ +| {"min": "192.168.0.0", "max": "192.168.255.255"} | ++--------------------------------------------------+ + +mysql> SELECT ipv4_cidr_to_range(NULL, NULL); ++--------------------------------+ +| ipv4_cidr_to_range(NULL, NULL) | ++--------------------------------+ +| NULL | ++--------------------------------+ +``` + +### keywords + +IPV4_CIDR_TO_RANGE, IP diff --git a/docs/en/docs/sql-manual/sql-functions/ip-functions/ipv6-cidr-to-range.md b/docs/en/docs/sql-manual/sql-functions/ip-functions/ipv6-cidr-to-range.md new file mode 100644 index 00000000000000..bf5c74c167d272 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/ip-functions/ipv6-cidr-to-range.md @@ -0,0 +1,72 @@ +--- +{ +"title": "IPV6_CIDR_TO_RANGE", +"language": "en" +} +--- + + + +## IPV6_CIDR_TO_RANGE + + + +IPV6_CIDR_TO_RANGE + + + +### description + +#### Syntax + +`STRUCT IPV6_CIDR_TO_RANGE(IPV6 ip_v6, INT16 cidr)` + +Receive an IPv6 and an Int16 value containing CIDR. Returns a struct that contains two IPv6 fields representing the lower range (min) and higher range (max) of the subnet, respectively. + +### notice + +`If the input parameter is NULL, return NULL, indicating invalid input` + +### example + +``` +mysql> SELECT ipv6_cidr_to_range(ipv6_string_to_num('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32); ++---------------------------------------------------------------------------------------+ +| ipv6_cidr_to_range(ipv6_string_to_num('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32) | ++---------------------------------------------------------------------------------------+ +| {"min": "2001:db8::", "max": "2001:db8:ffff:ffff:ffff:ffff:ffff:ffff"} | ++---------------------------------------------------------------------------------------+ + +mysql> SELECT ipv6_cidr_to_range(to_ipv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32); ++----------------------------------------------------------------------------+ +| ipv6_cidr_to_range(to_ipv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32) | ++----------------------------------------------------------------------------+ +| {"min": "2001:db8::", "max": "2001:db8:ffff:ffff:ffff:ffff:ffff:ffff"} | ++----------------------------------------------------------------------------+ + +mysql> SELECT ipv6_cidr_to_range(NULL, NULL); ++--------------------------------+ +| ipv6_cidr_to_range(NULL, NULL) | ++--------------------------------+ +| NULL | ++--------------------------------+ +``` + +### keywords + +IPV6_CIDR_TO_RANGE, IP diff --git a/docs/en/docs/sql-manual/sql-functions/ip-functions/is-ip-address-in-range.md b/docs/en/docs/sql-manual/sql-functions/ip-functions/is-ip-address-in-range.md new file mode 100644 index 00000000000000..7a47537f85526a --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/ip-functions/is-ip-address-in-range.md @@ -0,0 +1,65 @@ +--- +{ +"title": "IS_IP_ADDRESS_IN_RANGE", +"language": "en" +} +--- + + + +## IS_IP_ADDRESS_IN_RANGE + + + +IS_IP_ADDRESS_IN_RANGE + + + +### description + +#### Syntax + +`BOOLEAN IS_IP_ADDRESS_IN_RANGE(STRING ip_str, STRING cidr_prefix)` + +Determine whether the IP (IPv4 or IPv6) address is included in the network represented by CIDR notation. If yes, return true; otherwise, return false. + +### notice + +`ip_str and cidr_prefix both cannot be NULL` + +### example + +``` +mysql> SELECT is_ip_address_in_range('127.0.0.1', '127.0.0.0/8'); ++----------------------------------------------------+ +| is_ip_address_in_range('127.0.0.1', '127.0.0.0/8') | ++----------------------------------------------------+ +| 1 | ++----------------------------------------------------+ + +mysql> SELECT is_ip_address_in_range('::ffff:192.168.0.1', '::ffff:192.168.0.4/128'); ++------------------------------------------------------------------------+ +| is_ip_address_in_range('::ffff:192.168.0.1', '::ffff:192.168.0.4/128') | ++------------------------------------------------------------------------+ +| 0 | ++------------------------------------------------------------------------+ +``` + +### keywords + +IS_IP_ADDRESS_IN_RANGE, IP diff --git a/docs/en/docs/sql-manual/sql-functions/ip-functions/is-ipv4-string.md b/docs/en/docs/sql-manual/sql-functions/ip-functions/is-ipv4-string.md new file mode 100644 index 00000000000000..9aca93afc0acab --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/ip-functions/is-ipv4-string.md @@ -0,0 +1,80 @@ +--- +{ +"title": "IS_IPV4_STRING", +"language": "en" +} +--- + + + +## IS_IPV4_STRING + + + +IS_IPV4_STRING + + + +### description + +#### Syntax + +`BOOLEAN IS_IPV4_STRING(STRING ipv4_str)` + +Receive an IPv4 address in the form of a string as a parameter. If it is a correctly formatted and valid IPv4 address, return true; On the contrary, return false. + +### notice + +`If the input parameter is NULL, return NULL, indicating invalid input` + +### example + +``` +mysql> select is_ipv4_string(NULL); ++----------------------+ +| is_ipv4_string(NULL) | ++----------------------+ +| NULL | ++----------------------+ + +mysql> CREATE TABLE `test_is_ipv4_string` ( + `id` int, + `ip_v4` string + ) ENGINE=OLAP + DISTRIBUTED BY HASH(`id`) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + +mysql> insert into test_is_ipv4_string values(0, NULL), (1, '0.0.0.'), (2, ''), (3, '.'), (4, '255.255.255.255'); + +mysql> select id, is_ipv4_string(ip_v4) from test_is_ipv4_string order by id; ++------+-----------------------+ +| id | is_ipv4_string(ip_v4) | ++------+-----------------------+ +| 0 | NULL | +| 1 | 0 | +| 2 | 0 | +| 3 | 0 | +| 4 | 1 | ++------+-----------------------+ +``` + +### keywords + +IS_IPV4_STRING, IP diff --git a/docs/en/docs/sql-manual/sql-functions/ip-functions/is-ipv6-string.md b/docs/en/docs/sql-manual/sql-functions/ip-functions/is-ipv6-string.md new file mode 100644 index 00000000000000..6f3731e26181da --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/ip-functions/is-ipv6-string.md @@ -0,0 +1,80 @@ +--- +{ +"title": "IS_IPV6_STRING", +"language": "en" +} +--- + + + +## IS_IPV6_STRING + + + +IS_IPV6_STRING + + + +### description + +#### Syntax + +`BOOLEAN IS_IPV6_STRING(STRING ipv6_str)` + +Receive an IPv6 address in the form of a string as a parameter, and return true if it is a properly formatted and valid IPv6 address; On the contrary, return false. + +### notice + +`If the input parameter is NULL, return NULL, indicating invalid input` + +### example + +``` +mysql> select is_ipv6_string(NULL); ++----------------------+ +| is_ipv6_string(NULL) | ++----------------------+ +| NULL | ++----------------------+ + +mysql> CREATE TABLE `test_is_ipv6_string` ( + `id` int, + `ip_v6` string + ) ENGINE=OLAP + DISTRIBUTED BY HASH(`id`) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + +mysql> insert into test_is_ipv6_string values(0, NULL), (1, '::'), (2, ''), (3, '2001:1b70:a1:610::b102:2'), (4, 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffffg'); + +mysql> select id, is_ipv6_string(ip_v6) from test_is_ipv6_string order by id; ++------+-----------------------+ +| id | is_ipv6_string(ip_v6) | ++------+-----------------------+ +| 0 | NULL | +| 1 | 1 | +| 2 | 0 | +| 3 | 1 | +| 4 | 0 | ++------+-----------------------+ +``` + +### keywords + +IS_IPV6_STRING, IP diff --git a/docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv4-or-default.md b/docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv4-or-default.md new file mode 100644 index 00000000000000..d5770906b881ec --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv4-or-default.md @@ -0,0 +1,65 @@ +--- +{ +"title": "TO_IPV4_OR_DEFAULT", +"language": "en" +} +--- + + + +## TO_IPV4_OR_DEFAULT + + + +TO_IPV4_OR_DEFAULT + + + +### description + +#### Syntax + +`IPV4 TO_IPV4_OR_DEFAULT(STRING ipv4_str)` + +Same as to_ipv4, but if the IPv4 address has an invalid format, it returns 0.0.0.0 (0 as IPv4). + +### notice + +`If input is NULL, return 0.0.0.0 (0 as IPv4).` + +### example + +``` +mysql> select to_ipv4_or_default('.'); ++-------------------------+ +| to_ipv4_or_default('.') | ++-------------------------+ +| 0.0.0.0 | ++-------------------------+ + +mysql> select to_ipv4_or_default(NULL); ++--------------------------+ +| to_ipv4_or_default(NULL) | ++--------------------------+ +| 0.0.0.0 | ++--------------------------+ +``` + +### keywords + +TO_IPV4_OR_DEFAULT, IP diff --git a/docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv4-or-null.md b/docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv4-or-null.md new file mode 100644 index 00000000000000..35377b233a73fc --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv4-or-null.md @@ -0,0 +1,65 @@ +--- +{ +"title": "TO_IPV4_OR_NULL", +"language": "en" +} +--- + + + +## TO_IPV4_OR_NULL + + + +TO_IPV4_OR_NULL + + + +### description + +#### Syntax + +`IPV4 TO_IPV4_OR_NULL(STRING ipv4_str)` + +Same as to_ipv4, but if the IPv4 address has an invalid format, it returns NULL. + +### notice + +`If input is NULL, return NULL.` + +### example + +``` +mysql> select to_ipv4_or_null('.'); ++----------------------+ +| to_ipv4_or_null('.') | ++----------------------+ +| NULL | ++----------------------+ + +mysql> select to_ipv4_or_null(NULL); ++-----------------------+ +| to_ipv4_or_null(NULL) | ++-----------------------+ +| NULL | ++-----------------------+ +``` + +### keywords + +TO_IPV4_OR_NULL, IP diff --git a/docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv4.md b/docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv4.md new file mode 100644 index 00000000000000..24747ae0ed051e --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv4.md @@ -0,0 +1,60 @@ +--- +{ +"title": "TO_IPV4", +"language": "en" +} +--- + + + +## TO_IPV4 + + + +TO_IPV4 + + + +### description + +#### Syntax + +`IPV4 TO_IPV4(STRING ipv4_str)` + +This function like ipv4_string_to_num that takes a string form of IPv4 address and returns value of IPv4 type, +which is binary equal to value returned by ipv4_string_to_num. +If the IPv4 address has an invalid format, throw an exception. + +### notice + +`Input cannot be NULL. If it is NULL, an exception will be thrown.` + +### example + +``` +mysql> select to_ipv4('255.255.255.255'); ++----------------------------+ +| to_ipv4('255.255.255.255') | ++----------------------------+ +| 255.255.255.255 | ++----------------------------+ +``` + +### keywords + +TO_IPV4, IP diff --git a/docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv6-or-default.md b/docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv6-or-default.md new file mode 100644 index 00000000000000..c0cc118e9bb9e5 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv6-or-default.md @@ -0,0 +1,65 @@ +--- +{ +"title": "TO_IPV6_OR_DEFAULT", +"language": "en" +} +--- + + + +## TO_IPV6_OR_DEFAULT + + + +TO_IPV6_OR_DEFAULT + + + +### description + +#### Syntax + +`IPV6 TO_IPV6_OR_DEFAULT(STRING ipv6_str)` + +Same as to_ipv6, but if the IPv6 address has an invalid format, it returns :: (0 as IPv6). + +### notice + +`If input is NULL, return :: (0 as IPv6).` + +### example + +``` +mysql> select to_ipv6_or_default('.'); ++-------------------------+ +| to_ipv6_or_default('.') | ++-------------------------+ +| :: | ++-------------------------+ + +mysql> select to_ipv6_or_default(NULL); ++--------------------------+ +| to_ipv6_or_default(NULL) | ++--------------------------+ +| :: | ++--------------------------+ +``` + +### keywords + +TO_IPV6_OR_DEFAULT, IP diff --git a/docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv6-or-null.md b/docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv6-or-null.md new file mode 100644 index 00000000000000..863cb383973670 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv6-or-null.md @@ -0,0 +1,65 @@ +--- +{ +"title": "TO_IPV6_OR_NULL", +"language": "en" +} +--- + + + +## TO_IPV6_OR_NULL + + + +TO_IPV6_OR_NULL + + + +### description + +#### Syntax + +`IPV6 TO_IPV6_OR_NULL(STRING ipv6_str)` + +Same as to_ipv6, but if the IPv6 address has an invalid format, it returns NULL. + +### notice + +`If input is NULL, return NULL.` + +### example + +``` +mysql> select to_ipv6_or_null('.'); ++----------------------+ +| to_ipv6_or_null('.') | ++----------------------+ +| NULL | ++----------------------+ + +mysql> select to_ipv6_or_null(NULL); ++-----------------------+ +| to_ipv6_or_null(NULL) | ++-----------------------+ +| NULL | ++-----------------------+ +``` + +### keywords + +TO_IPV6_OR_NULL, IP diff --git a/docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv6.md b/docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv6.md new file mode 100644 index 00000000000000..aad7d42b94fd92 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/ip-functions/to-ipv6.md @@ -0,0 +1,60 @@ +--- +{ +"title": "TO_IPV6", +"language": "en" +} +--- + + + +## TO_IPV6 + + + +TO_IPV6 + + + +### description + +#### Syntax + +`IPV6 TO_IPV6(STRING ipv6_str)` + +Convert a string form of IPv6 address to IPv6 type. +If the IPv6 address has an invalid format, throw an exception. +Similar to ipv6_string_to_num function, which converts IPv6 address to binary format. + +### notice + +`Input cannot be NULL. If it is NULL, an exception will be thrown.` + +### example + +``` +mysql> select to_ipv6('::'); ++---------------+ +| to_ipv6('::') | ++---------------+ +| :: | ++---------------+ +``` + +### keywords + +TO_IPV6, IP diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Types/IPV4.md b/docs/en/docs/sql-manual/sql-reference/Data-Types/IPV4.md new file mode 100644 index 00000000000000..07ab836e4ebdf2 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-reference/Data-Types/IPV4.md @@ -0,0 +1,85 @@ +--- +{ + "title": "IPV4", + "language": "en" +} +--- + + + +## IPV4 + + + +IPV4 + + + +### description + +IPV4 +IPv4 type, stored in the form of UInt32 in 4 bytes, used to represent IPv4 addresses. +The range of values is ['0.0.0.0', '255.255.255.255']. + +`Inputs that exceed the value range or have invalid format will return NULL` + +### example + +Create table example: + +``` +CREATE TABLE ipv4_test ( + `id` int, + `ip_v4` ipv4 +) ENGINE=OLAP +DISTRIBUTED BY HASH(`id`) BUCKETS 4 +PROPERTIES ( +"replication_allocation" = "tag.location.default: 1" +); +``` + +Insert data example: + +``` +insert into ipv4_test values(1, '0.0.0.0'); +insert into ipv4_test values(2, '127.0.0.1'); +insert into ipv4_test values(3, '59.50.185.152'); +insert into ipv4_test values(4, '255.255.255.255'); +insert into ipv4_test values(5, '255.255.255.256'); // invalid data +``` + +Select data example: + +``` +mysql> select * from ipv4_test order by id; ++------+-----------------+ +| id | ip_v4 | ++------+-----------------+ +| 1 | 0.0.0.0 | +| 2 | 127.0.0.1 | +| 3 | 59.50.185.152 | +| 4 | 255.255.255.255 | +| 5 | NULL | ++------+-----------------+ +``` + +### keywords + +IPV4 diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Types/IPV6.md b/docs/en/docs/sql-manual/sql-reference/Data-Types/IPV6.md new file mode 100644 index 00000000000000..dcf46813240de2 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-reference/Data-Types/IPV6.md @@ -0,0 +1,83 @@ +--- +{ + "title": "IPV6", + "language": "en" +} +--- + + + +## IPV6 + + + +IPV6 + + + +### description + +IPV6 +IPv6 type, stored in Int128 format in 16 bytes, used to represent IPv6 addresses. +The range of values is ['::', 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff']. + +`Inputs that exceed the value range or have invalid format will return NULL` + +### example + +Create table example: + +``` +CREATE TABLE ipv6_test ( + `id` int, + `ip_v6` ipv6 +) ENGINE=OLAP +DISTRIBUTED BY HASH(`id`) BUCKETS 4 +PROPERTIES ( +"replication_allocation" = "tag.location.default: 1" +); +``` + +Insert data example: + +``` +insert into ipv6_test values(1, '::'); +insert into ipv6_test values(2, '2001:16a0:2:200a::2'); +insert into ipv6_test values(3, 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'); +insert into ipv6_test values(4, 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffffg'); // invalid data +``` + +Select data example: + +``` +mysql> select * from ipv6_test order by id; ++------+-----------------------------------------+ +| id | ip_v6 | ++------+-----------------------------------------+ +| 1 | :: | +| 2 | 2001:16a0:2:200a::2 | +| 3 | ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff | +| 4 | NULL | ++------+-----------------------------------------+ +``` + +### keywords + +IPV6 diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Types/STRUCT.md b/docs/en/docs/sql-manual/sql-reference/Data-Types/STRUCT.md index abab41488894f6..ba0faaf6375514 100644 --- a/docs/en/docs/sql-manual/sql-reference/Data-Types/STRUCT.md +++ b/docs/en/docs/sql-manual/sql-reference/Data-Types/STRUCT.md @@ -62,7 +62,7 @@ DATEV2, DATETIME, DATETIMEV2, CHAR, VARCHAR, STRING We have a todo list for future version: ``` -TODO:支持嵌套 STRUCT 或其他的复杂类型 +TODO: Supports nested Struct or other complex types ``` ### example diff --git a/docs/sidebars.json b/docs/sidebars.json index bda6b8da3105ce..cc8e5cfe39d3f6 100644 --- a/docs/sidebars.json +++ b/docs/sidebars.json @@ -816,7 +816,18 @@ "sql-manual/sql-functions/ip-functions/ipv6-string-to-num-or-default", "sql-manual/sql-functions/ip-functions/ipv6-string-to-num-or-null", "sql-manual/sql-functions/ip-functions/is-ipv4-compat", - "sql-manual/sql-functions/ip-functions/is-ipv4-mapped" + "sql-manual/sql-functions/ip-functions/is-ipv4-mapped", + "sql-manual/sql-functions/ip-functions/ipv4-cidr-to-range", + "sql-manual/sql-functions/ip-functions/ipv6-cidr-to-range", + "sql-manual/sql-functions/ip-functions/is-ip-address-in-range", + "sql-manual/sql-functions/ip-functions/is-ipv4-string", + "sql-manual/sql-functions/ip-functions/is-ipv6-string", + "sql-manual/sql-functions/ip-functions/to-ipv4", + "sql-manual/sql-functions/ip-functions/to-ipv4-or-default", + "sql-manual/sql-functions/ip-functions/to-ipv4-or-null", + "sql-manual/sql-functions/ip-functions/to-ipv6", + "sql-manual/sql-functions/ip-functions/to-ipv6-or-default", + "sql-manual/sql-functions/ip-functions/to-ipv6-or-null" ] }, { @@ -926,7 +937,9 @@ "sql-manual/sql-reference/Data-Types/STRUCT", "sql-manual/sql-reference/Data-Types/JSON", "sql-manual/sql-reference/Data-Types/AGG_STATE", - "sql-manual/sql-reference/Data-Types/VARIANT" + "sql-manual/sql-reference/Data-Types/VARIANT", + "sql-manual/sql-reference/Data-Types/IPV4", + "sql-manual/sql-reference/Data-Types/IPV6" ] }, { diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/ipv4-cidr-to-range.md b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/ipv4-cidr-to-range.md new file mode 100644 index 00000000000000..ca1803db6304e1 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/ipv4-cidr-to-range.md @@ -0,0 +1,72 @@ +--- +{ +"title": "IPV4_CIDR_TO_RANGE", +"language": "zh-CN" +} +--- + + + +## IPV4_CIDR_TO_RANGE + + + +IPV4_CIDR_TO_RANGE + + + +### description + +#### Syntax + +`STRUCT IPV4_CIDR_TO_RANGE(IPV4 ip_v4, INT16 cidr)` + +接收一个IPv4和一个包含CIDR的Int16值。返回一个结构体,其中包含两个IPv4字段分别表示子网的较低范围(min)和较高范围(max)。 + +### notice + +`如果入参为NULL,则返回NULL,表示无效输入` + +### example + +``` +mysql> SELECT ipv4_cidr_to_range(ipv4_string_to_num('192.168.5.2'), 16); ++------------------------------------------------------------+ +| ipv4_cidr_to_range(ipv4_string_to_num('192.168.5.2'), 16) | ++------------------------------------------------------------+ +| {"min": "192.168.0.0", "max": "192.168.255.255"} | ++------------------------------------------------------------+ + +mysql> SELECT ipv4_cidr_to_range(to_ipv4('192.168.5.2'), 16); ++--------------------------------------------------+ +| ipv4_cidr_to_range(to_ipv4('192.168.5.2'), 16) | ++--------------------------------------------------+ +| {"min": "192.168.0.0", "max": "192.168.255.255"} | ++--------------------------------------------------+ + +mysql> SELECT ipv4_cidr_to_range(NULL, NULL); ++--------------------------------+ +| ipv4_cidr_to_range(NULL, NULL) | ++--------------------------------+ +| NULL | ++--------------------------------+ +``` + +### keywords + +IPV4_CIDR_TO_RANGE, IP diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/ipv6-cidr-to-range.md b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/ipv6-cidr-to-range.md new file mode 100644 index 00000000000000..a88fa5651e1070 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/ipv6-cidr-to-range.md @@ -0,0 +1,72 @@ +--- +{ +"title": "IPV6_CIDR_TO_RANGE", +"language": "zh-CN" +} +--- + + + +## IPV6_CIDR_TO_RANGE + + + +IPV6_CIDR_TO_RANGE + + + +### description + +#### Syntax + +`STRUCT IPV6_CIDR_TO_RANGE(IPV6 ip_v6, INT16 cidr)` + +接收一个IPv6和一个包含CIDR的Int16值。返回一个结构体,其中包含两个IPv6字段分别表示子网的较低范围(min)和较高范围(max)。 + +### notice + +`如果入参为NULL,则返回NULL,表示无效输入` + +### example + +``` +mysql> SELECT ipv6_cidr_to_range(ipv6_string_to_num('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32); ++---------------------------------------------------------------------------------------+ +| ipv6_cidr_to_range(ipv6_string_to_num('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32) | ++---------------------------------------------------------------------------------------+ +| {"min": "2001:db8::", "max": "2001:db8:ffff:ffff:ffff:ffff:ffff:ffff"} | ++---------------------------------------------------------------------------------------+ + +mysql> SELECT ipv6_cidr_to_range(to_ipv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32); ++----------------------------------------------------------------------------+ +| ipv6_cidr_to_range(to_ipv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32) | ++----------------------------------------------------------------------------+ +| {"min": "2001:db8::", "max": "2001:db8:ffff:ffff:ffff:ffff:ffff:ffff"} | ++----------------------------------------------------------------------------+ + +mysql> SELECT ipv6_cidr_to_range(NULL, NULL); ++--------------------------------+ +| ipv6_cidr_to_range(NULL, NULL) | ++--------------------------------+ +| NULL | ++--------------------------------+ +``` + +### keywords + +IPV6_CIDR_TO_RANGE, IP diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/is-ip-address-in-range.md b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/is-ip-address-in-range.md new file mode 100644 index 00000000000000..16c8d31a9b38fb --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/is-ip-address-in-range.md @@ -0,0 +1,65 @@ +--- +{ +"title": "IS_IP_ADDRESS_IN_RANGE", +"language": "zh-CN" +} +--- + + + +## IS_IP_ADDRESS_IN_RANGE + + + +IS_IP_ADDRESS_IN_RANGE + + + +### description + +#### Syntax + +`BOOLEAN IS_IP_ADDRESS_IN_RANGE(STRING ip_str, STRING cidr_prefix)` + +判断IP(IPv4或IPv6)地址是否包含在以CIDR表示法表示的网络中。如果是,则返回true,否则返回false。 + +### notice + +`入参ip_str和cidr_prefix均不能为NULL` + +### example + +``` +mysql> SELECT is_ip_address_in_range('127.0.0.1', '127.0.0.0/8'); ++----------------------------------------------------+ +| is_ip_address_in_range('127.0.0.1', '127.0.0.0/8') | ++----------------------------------------------------+ +| 1 | ++----------------------------------------------------+ + +mysql> SELECT is_ip_address_in_range('::ffff:192.168.0.1', '::ffff:192.168.0.4/128'); ++------------------------------------------------------------------------+ +| is_ip_address_in_range('::ffff:192.168.0.1', '::ffff:192.168.0.4/128') | ++------------------------------------------------------------------------+ +| 0 | ++------------------------------------------------------------------------+ +``` + +### keywords + +IS_IP_ADDRESS_IN_RANGE, IP diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/is-ipv4-string.md b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/is-ipv4-string.md new file mode 100644 index 00000000000000..8cd691ab878e04 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/is-ipv4-string.md @@ -0,0 +1,80 @@ +--- +{ +"title": "IS_IPV4_STRING", +"language": "zh-CN" +} +--- + + + +## IS_IPV4_STRING + + + +IS_IPV4_STRING + + + +### description + +#### Syntax + +`BOOLEAN IS_IPV4_STRING(STRING ipv4_str)` + +接收一个表示形式为字符串的IPv4地址作为参数,如果为格式正确且合法的IPv4地址,返回true;反之,返回false。 + +### notice + +`如果入参为NULL,则返回NULL,表示无效输入` + +### example + +``` +mysql> select is_ipv4_string(NULL); ++----------------------+ +| is_ipv4_string(NULL) | ++----------------------+ +| NULL | ++----------------------+ + +mysql> CREATE TABLE `test_is_ipv4_string` ( + `id` int, + `ip_v4` string + ) ENGINE=OLAP + DISTRIBUTED BY HASH(`id`) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + +mysql> insert into test_is_ipv4_string values(0, NULL), (1, '0.0.0.'), (2, ''), (3, '.'), (4, '255.255.255.255'); + +mysql> select id, is_ipv4_string(ip_v4) from test_is_ipv4_string order by id; ++------+-----------------------+ +| id | is_ipv4_string(ip_v4) | ++------+-----------------------+ +| 0 | NULL | +| 1 | 0 | +| 2 | 0 | +| 3 | 0 | +| 4 | 1 | ++------+-----------------------+ +``` + +### keywords + +IS_IPV4_STRING, IP diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/is-ipv6-string.md b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/is-ipv6-string.md new file mode 100644 index 00000000000000..5aa9ca6087ee00 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/is-ipv6-string.md @@ -0,0 +1,80 @@ +--- +{ +"title": "IS_IPV6_STRING", +"language": "zh-CN" +} +--- + + + +## IS_IPV6_STRING + + + +IS_IPV6_STRING + + + +### description + +#### Syntax + +`BOOLEAN IS_IPV6_STRING(STRING ipv6_str)` + +接收一个表示形式为字符串的IPv6地址作为参数,如果为格式正确且合法的IPv6地址,返回true;反之,返回false。 + +### notice + +`如果入参为NULL,则返回NULL,表示无效输入` + +### example + +``` +mysql> select is_ipv6_string(NULL); ++----------------------+ +| is_ipv6_string(NULL) | ++----------------------+ +| NULL | ++----------------------+ + +mysql> CREATE TABLE `test_is_ipv6_string` ( + `id` int, + `ip_v6` string + ) ENGINE=OLAP + DISTRIBUTED BY HASH(`id`) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + +mysql> insert into test_is_ipv6_string values(0, NULL), (1, '::'), (2, ''), (3, '2001:1b70:a1:610::b102:2'), (4, 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffffg'); + +mysql> select id, is_ipv6_string(ip_v6) from test_is_ipv6_string order by id; ++------+-----------------------+ +| id | is_ipv6_string(ip_v6) | ++------+-----------------------+ +| 0 | NULL | +| 1 | 1 | +| 2 | 0 | +| 3 | 1 | +| 4 | 0 | ++------+-----------------------+ +``` + +### keywords + +IS_IPV6_STRING, IP diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv4-or-default.md b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv4-or-default.md new file mode 100644 index 00000000000000..f33cf4ea61a541 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv4-or-default.md @@ -0,0 +1,65 @@ +--- +{ +"title": "TO_IPV4_OR_DEFAULT", +"language": "zh-CN" +} +--- + + + +## TO_IPV4_OR_DEFAULT + + + +TO_IPV4_OR_DEFAULT + + + +### description + +#### Syntax + +`IPV4 TO_IPV4_OR_DEFAULT(STRING ipv4_str)` + +与to_ipv4函数类似,但如果IPv4地址的格式非法,则返回0.0.0.0。 + +### notice + +`入参ipv4_str如果为NULL,则返回0.0.0.0。` + +### example + +``` +mysql> select to_ipv4_or_default('.'); ++-------------------------+ +| to_ipv4_or_default('.') | ++-------------------------+ +| 0.0.0.0 | ++-------------------------+ + +mysql> select to_ipv4_or_default(NULL); ++--------------------------+ +| to_ipv4_or_default(NULL) | ++--------------------------+ +| 0.0.0.0 | ++--------------------------+ +``` + +### keywords + +TO_IPV4_OR_DEFAULT, IP diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv4-or-null.md b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv4-or-null.md new file mode 100644 index 00000000000000..9ffa174e0bba21 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv4-or-null.md @@ -0,0 +1,65 @@ +--- +{ +"title": "TO_IPV4_OR_NULL", +"language": "zh-CN" +} +--- + + + +## TO_IPV4_OR_NULL + + + +TO_IPV4_OR_NULL + + + +### description + +#### Syntax + +`IPV4 TO_IPV4_OR_NULL(STRING ipv4_str)` + +与to_ipv4函数类似,但如果IPv4地址的格式非法,则返回NULL。 + +### notice + +`入参ipv4_str如果为NULL,则返回NULL。` + +### example + +``` +mysql> select to_ipv4_or_null('.'); ++----------------------+ +| to_ipv4_or_null('.') | ++----------------------+ +| NULL | ++----------------------+ + +mysql> select to_ipv4_or_null(NULL); ++-----------------------+ +| to_ipv4_or_null(NULL) | ++-----------------------+ +| NULL | ++-----------------------+ +``` + +### keywords + +TO_IPV4_OR_NULL, IP diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv4.md b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv4.md new file mode 100644 index 00000000000000..f484da2905c656 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv4.md @@ -0,0 +1,60 @@ +--- +{ +"title": "TO_IPV4", +"language": "zh-CN" +} +--- + + + +## TO_IPV4 + + + +TO_IPV4 + + + +### description + +#### Syntax + +`IPV4 TO_IPV4(STRING ipv4_str)` + +该函数类似ipv4_string_to_num,输入IPv4地址的字符串形式,并返回IPv4类型的值。 +该值的二进制形式等于ipv4_string_to_num函数返回值的二进制形式。 +如果IPv4地址为非法格式,则抛出异常。 + +### notice + +`入参ipv4_str不能为NULL,若为NULL,则抛出异常。` + +### example + +``` +mysql> select to_ipv4('255.255.255.255'); ++----------------------------+ +| to_ipv4('255.255.255.255') | ++----------------------------+ +| 255.255.255.255 | ++----------------------------+ +``` + +### keywords + +TO_IPV4, IP diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv6-or-default.md b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv6-or-default.md new file mode 100644 index 00000000000000..679fbf1e85b18b --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv6-or-default.md @@ -0,0 +1,65 @@ +--- +{ +"title": "TO_IPV6_OR_DEFAULT", +"language": "zh-CN" +} +--- + + + +## TO_IPV6_OR_DEFAULT + + + +TO_IPV6_OR_DEFAULT + + + +### description + +#### Syntax + +`IPV6 TO_IPV6_OR_DEFAULT(STRING ipv6_str)` + +与to_ipv6函数类似,但如果IPv6地址的格式非法,则返回::。 + +### notice + +`入参ipv6_str如果为NULL,则返回::。` + +### example + +``` +mysql> select to_ipv6_or_default('.'); ++-------------------------+ +| to_ipv6_or_default('.') | ++-------------------------+ +| :: | ++-------------------------+ + +mysql> select to_ipv6_or_default(NULL); ++--------------------------+ +| to_ipv6_or_default(NULL) | ++--------------------------+ +| :: | ++--------------------------+ +``` + +### keywords + +TO_IPV6_OR_DEFAULT, IP diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv6-or-null.md b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv6-or-null.md new file mode 100644 index 00000000000000..f09fd1cf69126e --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv6-or-null.md @@ -0,0 +1,65 @@ +--- +{ +"title": "TO_IPV6_OR_NULL", +"language": "zh-CN" +} +--- + + + +## TO_IPV6_OR_NULL + + + +TO_IPV6_OR_NULL + + + +### description + +#### Syntax + +`IPV6 TO_IPV6_OR_NULL(STRING ipv6_str)` + +与to_ipv6函数类似,但如果IPv6地址的格式非法,则返回NULL。 + +### notice + +`入参ipv6_str如果为NULL,则返回NULL。` + +### example + +``` +mysql> select to_ipv6_or_null('.'); ++----------------------+ +| to_ipv6_or_null('.') | ++----------------------+ +| NULL | ++----------------------+ + +mysql> select to_ipv6_or_null(NULL); ++-----------------------+ +| to_ipv6_or_null(NULL) | ++-----------------------+ +| NULL | ++-----------------------+ +``` + +### keywords + +TO_IPV6_OR_NULL, IP diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv6.md b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv6.md new file mode 100644 index 00000000000000..ac7297204d232b --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/to-ipv6.md @@ -0,0 +1,60 @@ +--- +{ +"title": "TO_IPV6", +"language": "zh-CN" +} +--- + + + +## TO_IPV6 + + + +TO_IPV6 + + + +### description + +#### Syntax + +`IPV6 TO_IPV6(STRING ipv6_str)` + +该函数类似ipv6_string_to_num,输入IPv6地址的字符串形式,并返回IPv6类型的值。 +该值的二进制形式等于ipv6_string_to_num函数返回值的二进制形式。 +如果IPv6地址为非法格式,则抛出异常。 + +### notice + +`入参ipv6_str不能为NULL,若为NULL,则抛出异常。` + +### example + +``` +mysql> select to_ipv6('::'); ++---------------+ +| to_ipv6('::') | ++---------------+ +| :: | ++---------------+ +``` + +### keywords + +TO_IPV6, IP diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/IPV4.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/IPV4.md new file mode 100644 index 00000000000000..2766a20580e32c --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/IPV4.md @@ -0,0 +1,85 @@ +--- +{ + "title": "IPV4", + "language": "zh-CN" +} +--- + + + +## IPV4 + + + +IPV4 + + + +### description + +IPV4 +IPv4类型,以UInt32的形式存储在4个字节中,用于表示IPv4地址。 +取值范围是 ['0.0.0.0', '255.255.255.255']。 + +`超出取值范围或者格式非法的输入将返回NULL` + +### example + +建表示例如下: + +``` +CREATE TABLE ipv4_test ( + `id` int, + `ip_v4` ipv4 +) ENGINE=OLAP +DISTRIBUTED BY HASH(`id`) BUCKETS 4 +PROPERTIES ( +"replication_allocation" = "tag.location.default: 1" +); +``` + +插入数据示例: + +``` +insert into ipv4_test values(1, '0.0.0.0'); +insert into ipv4_test values(2, '127.0.0.1'); +insert into ipv4_test values(3, '59.50.185.152'); +insert into ipv4_test values(4, '255.255.255.255'); +insert into ipv4_test values(5, '255.255.255.256'); // invalid data +``` + +查询数据示例: + +``` +mysql> select * from ipv4_test order by id; ++------+-----------------+ +| id | ip_v4 | ++------+-----------------+ +| 1 | 0.0.0.0 | +| 2 | 127.0.0.1 | +| 3 | 59.50.185.152 | +| 4 | 255.255.255.255 | +| 5 | NULL | ++------+-----------------+ +``` + +### keywords + +IPV4 diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/IPV6.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/IPV6.md new file mode 100644 index 00000000000000..678b404a672d99 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/IPV6.md @@ -0,0 +1,83 @@ +--- +{ + "title": "IPV6", + "language": "zh-CN" +} +--- + + + +## IPV6 + + + +IPV6 + + + +### description + +IPV6 +IPv6类型,以Int128的形式存储在16个字节中,用于表示IPv6地址。 +取值范围是 ['::', 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff']。 + +`超出取值范围或者格式非法的输入将返回NULL` + +### example + +建表示例如下: + +``` +CREATE TABLE ipv6_test ( + `id` int, + `ip_v6` ipv6 +) ENGINE=OLAP +DISTRIBUTED BY HASH(`id`) BUCKETS 4 +PROPERTIES ( +"replication_allocation" = "tag.location.default: 1" +); +``` + +插入数据示例: + +``` +insert into ipv6_test values(1, '::'); +insert into ipv6_test values(2, '2001:16a0:2:200a::2'); +insert into ipv6_test values(3, 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'); +insert into ipv6_test values(4, 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffffg'); // invalid data +``` + +查询数据示例: + +``` +mysql> select * from ipv6_test order by id; ++------+-----------------------------------------+ +| id | ip_v6 | ++------+-----------------------------------------+ +| 1 | :: | +| 2 | 2001:16a0:2:200a::2 | +| 3 | ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff | +| 4 | NULL | ++------+-----------------------------------------+ +``` + +### keywords + +IPV6 From bb2c90a82b749a324bac7256a407237fe0b94525 Mon Sep 17 00:00:00 2001 From: zhiqiang Date: Mon, 22 Jan 2024 13:27:46 +0800 Subject: [PATCH 125/200] [feature](bitwise function) bit_count/bit_shift_left/bit_shift_right implementation (#30046) --- be/src/vec/functions/function_bit_count.cpp | 61 ++++ be/src/vec/functions/function_bit_shift.cpp | 90 +++++ .../vec/functions/simple_function_factory.h | 4 + .../bitwise-functions/bitcount.md | 52 +++ .../bitwise-functions/bitshfitleft.md | 71 ++++ .../bitwise-functions/bitshfitright.md | 79 ++++ .../bitwise-functions/bitcount.md | 52 +++ .../bitwise-functions/bitshiftleft.md | 71 ++++ .../bitwise-functions/bitshiftright.md | 78 ++++ .../doris/catalog/BuiltinScalarFunctions.java | 6 + .../functions/scalar/BitCount.java | 67 ++++ .../functions/scalar/BitShiftLeft.java | 61 ++++ .../functions/scalar/BitShiftRight.java | 63 ++++ .../visitor/ScalarFunctionVisitor.java | 15 + gensrc/script/doris_builtins_functions.py | 11 +- .../data/correctness_p0/test_bit_function.out | 132 +++++++ .../correctness_p0/test_bit_shift_lagency.out | 345 ++++++++++++++++++ .../correctness_p0/test_bit_shift_nereids.out | 345 ++++++++++++++++++ .../correctness_p0/test_bit_function.groovy | 63 ++++ .../test_bit_shift_lagency.groovy | 105 ++++++ .../test_bit_shift_nereids.groovy | 105 ++++++ 21 files changed, 1875 insertions(+), 1 deletion(-) create mode 100644 be/src/vec/functions/function_bit_count.cpp create mode 100644 be/src/vec/functions/function_bit_shift.cpp create mode 100644 docs/en/docs/sql-manual/sql-functions/bitwise-functions/bitcount.md create mode 100644 docs/en/docs/sql-manual/sql-functions/bitwise-functions/bitshfitleft.md create mode 100644 docs/en/docs/sql-manual/sql-functions/bitwise-functions/bitshfitright.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/bitwise-functions/bitcount.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/bitwise-functions/bitshiftleft.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/bitwise-functions/bitshiftright.md create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/BitCount.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/BitShiftLeft.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/BitShiftRight.java create mode 100644 regression-test/data/correctness_p0/test_bit_shift_lagency.out create mode 100644 regression-test/data/correctness_p0/test_bit_shift_nereids.out create mode 100644 regression-test/suites/correctness_p0/test_bit_shift_lagency.groovy create mode 100644 regression-test/suites/correctness_p0/test_bit_shift_nereids.groovy diff --git a/be/src/vec/functions/function_bit_count.cpp b/be/src/vec/functions/function_bit_count.cpp new file mode 100644 index 00000000000000..66ef9d4c4f723e --- /dev/null +++ b/be/src/vec/functions/function_bit_count.cpp @@ -0,0 +1,61 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include +#include +#include +#include + +#include "common/exception.h" +#include "common/status.h" +#include "gutil/integral_types.h" +#include "util/radix_sort.h" +#include "vec/core/types.h" +#include "vec/functions/function_unary_arithmetic.h" +#include "vec/functions/simple_function_factory.h" + +namespace doris::vectorized { + +struct NameBitCount { + static constexpr auto name = "bit_count"; +}; + +template +struct BitCountImpl { + // No unsigned type in Java. So we need signed number as return type + // Int8_MAX = 127 + using ResultType = std::conditional_t= 128, Int16, Int8>; + + static inline ResultType apply(T a) { + if constexpr (std::is_same_v || std::is_same_v || + std::is_same_v || std::is_same_v || + std::is_same_v) { + return std::popcount(static_cast>(a)); + } else { + throw Exception(ErrorCode::INVALID_ARGUMENT, + "bit_count only support using INTEGER as operator"); + } + } +}; + +using FunctionBitCount = FunctionUnaryArithmetic; + +void register_function_bit_count(SimpleFunctionFactory& factory) { + factory.register_function(); +} + +} // namespace doris::vectorized diff --git a/be/src/vec/functions/function_bit_shift.cpp b/be/src/vec/functions/function_bit_shift.cpp new file mode 100644 index 00000000000000..36a5c59e3cb62c --- /dev/null +++ b/be/src/vec/functions/function_bit_shift.cpp @@ -0,0 +1,90 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#include +#include +#include +#include +#include +#include + +#include "common/exception.h" +#include "common/logging.h" +#include "common/status.h" +#include "gutil/integral_types.h" +#include "util/radix_sort.h" +#include "vec/core/types.h" +#include "vec/functions/function_binary_arithmetic.h" +#include "vec/functions/simple_function_factory.h" + +namespace doris::vectorized { + +struct NameBitShiftLeft { + static constexpr auto name = "bit_shift_left"; +}; + +struct NameBitShiftRight { + static constexpr auto name = "bit_shift_right"; +}; + +template +struct BitShiftLeftImpl { + using ResultType = typename NumberTraits::ResultOfBit::Type; + + template + static inline Result apply(A a, B b) { + if constexpr (!std::is_same_v || !std::is_same_v) { + throw Exception(ErrorCode::NOT_FOUND, + "bit_shift_left only supports [BIGINT, TINYINT] as operator"); + } else { + // return zero if b < 0, keep consistent with mysql + // cast to unsigned so that we can do logical shift by default, keep consistent with mysql + return b < 0 ? 0 + : static_cast::type>(a) + << static_cast(b); + } + } +}; + +template +struct BitShiftRightImpl { + using ResultType = typename NumberTraits::ResultOfBit::Type; + + template + static inline Result apply(A a, B b) { + if constexpr (!std::is_same_v || !std::is_same_v) { + throw Exception(ErrorCode::NOT_FOUND, + "bit_shift_right only supports [BIGINT, TINYINT] as operator"); + } else { + // return zero if b < 0, keep consistent with mysql + // cast to unsigned so that we can do logical shift by default, keep consistent with mysql + return b < 0 ? 0 + : static_cast::type>(a) >> + static_cast(b); + } + } +}; + +using FunctionBitShiftLeft = FunctionBinaryArithmetic; +using FunctionBitShiftRight = FunctionBinaryArithmetic; + +void register_function_bit_shift(SimpleFunctionFactory& factory) { + factory.register_function(); + factory.register_function(); +} + +} // namespace doris::vectorized diff --git a/be/src/vec/functions/simple_function_factory.h b/be/src/vec/functions/simple_function_factory.h index 0992239e99b3e0..5c47198fd6bd78 100644 --- a/be/src/vec/functions/simple_function_factory.h +++ b/be/src/vec/functions/simple_function_factory.h @@ -48,6 +48,8 @@ void register_function_multiply(SimpleFunctionFactory& factory); void register_function_divide(SimpleFunctionFactory& factory); void register_function_int_div(SimpleFunctionFactory& factory); void register_function_bit(SimpleFunctionFactory& factory); +void register_function_bit_count(SimpleFunctionFactory& factory); +void register_function_bit_shift(SimpleFunctionFactory& factory); void register_function_math(SimpleFunctionFactory& factory); void register_function_modulo(SimpleFunctionFactory& factory); void register_function_bitmap(SimpleFunctionFactory& factory); @@ -229,6 +231,8 @@ class SimpleFunctionFactory { register_function_int_div(instance); register_function_modulo(instance); register_function_bit(instance); + register_function_bit_count(instance); + register_function_bit_shift(instance); register_function_is_null(instance); register_function_is_not_null(instance); register_function_nullables(instance); diff --git a/docs/en/docs/sql-manual/sql-functions/bitwise-functions/bitcount.md b/docs/en/docs/sql-manual/sql-functions/bitwise-functions/bitcount.md new file mode 100644 index 00000000000000..2518e35f2f125f --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/bitwise-functions/bitcount.md @@ -0,0 +1,52 @@ +--- +{ +"title": "BIT_COUNT", +"language": "en" +} +--- + + + +## bit_count +### description +#### Syntax + +`BIT_COUNT(Integer-type x)` + +Returns the exist count of one in 2's complement represent of integer x. + +Integer-type could be: TINYINT、SMALLINT、INT、BIGINT、LARGEINT + +### example + +``` +select "0b11111111", bit_count(-1) +-------------- + ++--------------+---------------+ +| '0b11111111' | bit_count(-1) | ++--------------+---------------+ +| 0b11111111 | 8 | ++--------------+---------------+ +``` + +### keywords + + BITCOUNT, BIT_COUNT diff --git a/docs/en/docs/sql-manual/sql-functions/bitwise-functions/bitshfitleft.md b/docs/en/docs/sql-manual/sql-functions/bitwise-functions/bitshfitleft.md new file mode 100644 index 00000000000000..b48313c242b971 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/bitwise-functions/bitshfitleft.md @@ -0,0 +1,71 @@ +--- +{ +"title": "BIT_SHIFT_LEFT", +"language": "en" +} +--- + + + +## bit_shift_left +### description +#### syntax + +`BIT_SHIFT_LEFT(BIGINT x, TINYINT c)` + +Do logical left shift to `BIGINT` type x by c bits, and return result as a `BIGINT`. +Return zero if `c` is less than 0. + +### example +Normal case +```sql +select 8 as x, number as c, bit_shift_left(8, number) as bit_shift_left from numbers("number"="5") +-------------- + ++------+------+----------------+ +| x | c | bit_shift_left | ++------+------+----------------+ +| 8 | 0 | 8 | +| 8 | 1 | 16 | +| 8 | 2 | 32 | +| 8 | 3 | 64 | +| 8 | 4 | 128 | ++------+------+----------------+ +5 rows in set (0.04 sec) +``` +Left shift result of `9223372036854775807` which is `BIGINT_MAX` by 1 bit will get -2. +```sql +WITH tbl AS ( + SELECT 9223372036854775807 AS BIGINT_MAX +) +SELECT BIGINT_MAX, bit_shift_left(BIGINT_MAX, 1) +FROM tbl +-------------- + ++---------------------+-------------------------------+ +| BIGINT_MAX | bit_shift_left(BIGINT_MAX, 1) | ++---------------------+-------------------------------+ +| 9223372036854775807 | -2 | ++---------------------+-------------------------------+ +1 row in set (0.05 sec) +``` +### keywords + + BITSHIFT, BITSHIFTLEFT diff --git a/docs/en/docs/sql-manual/sql-functions/bitwise-functions/bitshfitright.md b/docs/en/docs/sql-manual/sql-functions/bitwise-functions/bitshfitright.md new file mode 100644 index 00000000000000..8d44c3421bea5f --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/bitwise-functions/bitshfitright.md @@ -0,0 +1,79 @@ +--- +{ +"title": "BIT_SHIFT_RIGHT", +"language": "en" +} +--- + + + +## bit_shift_right +### description +#### syntax + +`BIT_SHIFT_RIGHT(BIGINT x, TINYINT c)` + +Return result of logical right shift of `BIGINT` type x by c bits. + +### example +Normal case +```sql +select 1024 as x, number as c, bit_shift_right(1024, number) as bit_shift_right from numbers("number"="5") +-------------- + ++------+------+-----------------+ +| x | c | bit_shift_right | ++------+------+-----------------+ +| 1024 | 0 | 1024 | +| 1024 | 1 | 512 | +| 1024 | 2 | 256 | +| 1024 | 3 | 128 | +| 1024 | 4 | 64 | ++------+------+-----------------+ +5 rows in set (0.03 sec) +``` +Logical right shift `BIGINT` -1 by 1 bits gets `BIGINT_MAX` +```sql +select bit_shift_right(-1, 1) +-------------- + ++------------------------+ +| bit_shift_right(-1, 1) | ++------------------------+ +| 9223372036854775807 | ++------------------------+ +``` +Return zero if `c` is less than 0 +```sql +select bit_shift_right(100, -1) +-------------- + ++--------------------------+ +| bit_shift_right(100, -1) | ++--------------------------+ +| 0 | ++--------------------------+ +1 row in set (0.04 sec) +``` + + +### keywords + + BITSHIFT, BITSHIFTRIGHT diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/bitwise-functions/bitcount.md b/docs/zh-CN/docs/sql-manual/sql-functions/bitwise-functions/bitcount.md new file mode 100644 index 00000000000000..9dfdd7e9d49738 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/bitwise-functions/bitcount.md @@ -0,0 +1,52 @@ +--- +{ +"title": "BIT_COUNT", +"language": "zh-CH" +} +--- + + + +## bit_count +### description +#### Syntax + +`BIT_COUNT(Integer-type x)` + +统计整型 x 的二的补码表示中 1 的个数。 + +整型可以是:TINYINT、SMALLINT、INT、BIGINT、LARGEINT + +### example + +``` +select "0b11111111", bit_count(-1) +-------------- + ++--------------+---------------+ +| '0b11111111' | bit_count(-1) | ++--------------+---------------+ +| 0b11111111 | 8 | ++--------------+---------------+ +``` + +### keywords + + BITCOUNT, BIT_COUNT diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/bitwise-functions/bitshiftleft.md b/docs/zh-CN/docs/sql-manual/sql-functions/bitwise-functions/bitshiftleft.md new file mode 100644 index 00000000000000..1729b7ac75a3b1 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/bitwise-functions/bitshiftleft.md @@ -0,0 +1,71 @@ +--- +{ +"title": "BIT_SHIFT_LEFT", +"language": "zh-CN" +} +--- + + + +## bit_shift_left +### description +#### syntax + +`BIT_SHIFT_LEFT(BIGINT x, TINYINT c)` + +将 BIGINT 类型的 x 向左移动 c 位,并将结果作为 BIGINT 返回。 +如果 c 小于 0,则返回零。 + +### example +```sql +select 8 as x, number as c, bit_shift_left(8, number) as bit_shift_left from numbers("number"="5") +-------------- + ++------+------+----------------+ +| x | c | bit_shift_left | ++------+------+----------------+ +| 8 | 0 | 8 | +| 8 | 1 | 16 | +| 8 | 2 | 32 | +| 8 | 3 | 64 | +| 8 | 4 | 128 | ++------+------+----------------+ +5 rows in set (0.04 sec) +``` +对于 BIGINT 类型的最大值 9223372036854775807(即 BIGINT_MAX),进行一位左移的结果将得到 -2。 +```sql +WITH tbl AS ( + SELECT 9223372036854775807 AS BIGINT_MAX +) +SELECT BIGINT_MAX, bit_shift_left(BIGINT_MAX, 1) +FROM tbl +-------------- + ++---------------------+-------------------------------+ +| BIGINT_MAX | bit_shift_left(BIGINT_MAX, 1) | ++---------------------+-------------------------------+ +| 9223372036854775807 | -2 | ++---------------------+-------------------------------+ +1 row in set (0.05 sec) +``` + +### keywords + + BITSHIFT, BITSHIFTLEFT diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/bitwise-functions/bitshiftright.md b/docs/zh-CN/docs/sql-manual/sql-functions/bitwise-functions/bitshiftright.md new file mode 100644 index 00000000000000..4308bff470ad2b --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/bitwise-functions/bitshiftright.md @@ -0,0 +1,78 @@ +--- +{ +"title": "BIT_SHIFT_RIGHT", +"language": "zh-CN" +} +--- + + +## bit_shift_right +### description +#### syntax + +`BIT_SHIFT_RIGHT(BIGINT x, TINYINT c)` + +返回对 BIGINT 类型 x 进行逻辑右移 c 位的结果。 + +### example +Normal case +```sql +select 1024 as x, number as c, bit_shift_right(1024, number) as bit_shift_right from numbers("number"="5") +-------------- + ++------+------+-----------------+ +| x | c | bit_shift_right | ++------+------+-----------------+ +| 1024 | 0 | 1024 | +| 1024 | 1 | 512 | +| 1024 | 2 | 256 | +| 1024 | 3 | 128 | +| 1024 | 4 | 64 | ++------+------+-----------------+ +5 rows in set (0.03 sec) +``` +BIGINT -1 逻辑右移一位得到的结果是 BIGINT_MAX + +```sql +select bit_shift_right(-1, 1) +-------------- + ++------------------------+ +| bit_shift_right(-1, 1) | ++------------------------+ +| 9223372036854775807 | ++------------------------+ +``` +如果 c 小于 0 得到的结果始终为 0 +```sql +select bit_shift_right(100, -1) +-------------- + ++--------------------------+ +| bit_shift_right(100, -1) | ++--------------------------+ +| 0 | ++--------------------------+ +1 row in set (0.04 sec) +``` + +### keywords + + BITSHIFT, BITSHIFTRIGHT diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java index fcc2a351bf0e72..492c561f0a1e28 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java @@ -73,7 +73,10 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Asin; import org.apache.doris.nereids.trees.expressions.functions.scalar.Atan; import org.apache.doris.nereids.trees.expressions.functions.scalar.Bin; +import org.apache.doris.nereids.trees.expressions.functions.scalar.BitCount; import org.apache.doris.nereids.trees.expressions.functions.scalar.BitLength; +import org.apache.doris.nereids.trees.expressions.functions.scalar.BitShiftLeft; +import org.apache.doris.nereids.trees.expressions.functions.scalar.BitShiftRight; import org.apache.doris.nereids.trees.expressions.functions.scalar.BitmapAnd; import org.apache.doris.nereids.trees.expressions.functions.scalar.BitmapAndCount; import org.apache.doris.nereids.trees.expressions.functions.scalar.BitmapAndNot; @@ -486,6 +489,7 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(Asin.class, "asin"), scalar(Atan.class, "atan"), scalar(Bin.class, "bin"), + scalar(BitCount.class, "bit_count"), scalar(BitLength.class, "bit_length"), scalar(BitmapAnd.class, "bitmap_and"), scalar(BitmapAndCount.class, "bitmap_and_count"), @@ -516,6 +520,8 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(BitmapToString.class, "bitmap_to_string"), scalar(BitmapXor.class, "bitmap_xor"), scalar(BitmapXorCount.class, "bitmap_xor_count"), + scalar(BitShiftLeft.class, "bit_shift_left"), + scalar(BitShiftRight.class, "bit_shift_right"), scalar(Cardinality.class, "array_size", "cardinality", "size"), scalar(Cbrt.class, "cbrt"), scalar(Ceil.class, "ceil", "ceiling"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/BitCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/BitCount.java new file mode 100644 index 00000000000000..5cbbf24df96721 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/BitCount.java @@ -0,0 +1,67 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; +import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.BigIntType; +import org.apache.doris.nereids.types.IntegerType; +import org.apache.doris.nereids.types.LargeIntType; +import org.apache.doris.nereids.types.SmallIntType; +import org.apache.doris.nereids.types.TinyIntType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** BitCount function */ + +public class BitCount extends ScalarFunction + implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullable { + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(TinyIntType.INSTANCE).args(TinyIntType.INSTANCE), + FunctionSignature.ret(TinyIntType.INSTANCE).args(SmallIntType.INSTANCE), + FunctionSignature.ret(TinyIntType.INSTANCE).args(IntegerType.INSTANCE), + FunctionSignature.ret(TinyIntType.INSTANCE).args(BigIntType.INSTANCE), + FunctionSignature.ret(SmallIntType.INSTANCE).args(LargeIntType.INSTANCE)); + + public BitCount(Expression arg) { + super("bit_count", arg); + } + + @Override + public BitCount withChildren(List children) { + Preconditions.checkArgument(children.size() == 1); + return new BitCount(children.get(0)); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitBitCount(this, context); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/BitShiftLeft.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/BitShiftLeft.java new file mode 100644 index 00000000000000..aa78d01e320c04 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/BitShiftLeft.java @@ -0,0 +1,61 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; +import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.BigIntType; +import org.apache.doris.nereids.types.TinyIntType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** BitShiftLeft function */ + +public class BitShiftLeft extends ScalarFunction + implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullable { + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(BigIntType.INSTANCE).args(BigIntType.INSTANCE, TinyIntType.INSTANCE)); + + public BitShiftLeft(Expression arg1, Expression arg2) { + super("bit_shift_left", arg1, arg2); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitBitShiftLeft(this, context); + } + + @Override + public BitShiftLeft withChildren(List children) { + Preconditions.checkArgument(children.size() == 2); + return new BitShiftLeft(children.get(0), children.get(1)); + } +} + diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/BitShiftRight.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/BitShiftRight.java new file mode 100644 index 00000000000000..ab2bce00a04321 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/BitShiftRight.java @@ -0,0 +1,63 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; +import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.BigIntType; +import org.apache.doris.nereids.types.TinyIntType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** BitShiftRight + * Do logical right shift to a BigInt value x by c bits. + */ + +public class BitShiftRight extends ScalarFunction + implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullable { + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(BigIntType.INSTANCE).args(BigIntType.INSTANCE, TinyIntType.INSTANCE)); + + public BitShiftRight(Expression arg1, Expression arg2) { + super("bit_shift_right", arg1, arg2); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitBitShiftRight(this, context); + } + + @Override + public BitShiftRight withChildren(List children) { + Preconditions.checkArgument(children.size() == 2); + return new BitShiftRight(children.get(0), children.get(1)); + } +} + diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java index 01823f78199856..8629d15b2bcfbb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java @@ -73,7 +73,10 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Asin; import org.apache.doris.nereids.trees.expressions.functions.scalar.Atan; import org.apache.doris.nereids.trees.expressions.functions.scalar.Bin; +import org.apache.doris.nereids.trees.expressions.functions.scalar.BitCount; import org.apache.doris.nereids.trees.expressions.functions.scalar.BitLength; +import org.apache.doris.nereids.trees.expressions.functions.scalar.BitShiftLeft; +import org.apache.doris.nereids.trees.expressions.functions.scalar.BitShiftRight; import org.apache.doris.nereids.trees.expressions.functions.scalar.BitmapAnd; import org.apache.doris.nereids.trees.expressions.functions.scalar.BitmapAndCount; import org.apache.doris.nereids.trees.expressions.functions.scalar.BitmapAndNot; @@ -625,6 +628,10 @@ default R visitBin(Bin bin, C context) { return visitScalarFunction(bin, context); } + default R visitBitCount(BitCount bitCount, C context) { + return visitScalarFunction(bitCount, context); + } + default R visitBitLength(BitLength bitLength, C context) { return visitScalarFunction(bitLength, context); } @@ -745,6 +752,14 @@ default R visitBitmapXorCount(BitmapXorCount bitmapXorCount, C context) { return visitScalarFunction(bitmapXorCount, context); } + default R visitBitShiftLeft(BitShiftLeft bitShiftLeft, C context) { + return visitScalarFunction(bitShiftLeft, context); + } + + default R visitBitShiftRight(BitShiftRight bitShiftRight, C context) { + return visitScalarFunction(bitShiftRight, context); + } + default R visitCardinality(Cardinality cardinality, C context) { return visitScalarFunction(cardinality, context); } diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py index 183b38bdfdae04..a9d2fe26dc2cb7 100644 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -51,6 +51,12 @@ [['bitand'], 'INT', ['INT', 'INT'], ''], [['bitand'], 'BIGINT', ['BIGINT', 'BIGINT'], ''], [['bitand'], 'LARGEINT', ['LARGEINT', 'LARGEINT'], ''], + + [['bit_count'], 'TINYINT', ['TINYINT'], ''], + [['bit_count'], 'TINYINT', ['SMALLINT'], ''], + [['bit_count'], 'TINYINT', ['INT'], ''], + [['bit_count'], 'TINYINT', ['BIGINT'], ''], + [['bit_count'], 'SMALLINT', ['LARGEINT'], ''], [['bitor'], 'TINYINT', ['TINYINT', 'TINYINT'], ''], [['bitor'], 'SMALLINT', ['SMALLINT', 'SMALLINT'], ''], @@ -68,7 +74,10 @@ [['bitnot'], 'SMALLINT', ['SMALLINT'], ''], [['bitnot'], 'INT', ['INT'], ''], [['bitnot'], 'BIGINT', ['BIGINT'], ''], - [['bitnot'], 'LARGEINT', ['LARGEINT'], ''] + [['bitnot'], 'LARGEINT', ['LARGEINT'], ''], + + [['bit_shift_left'], 'BIGINT', ['BIGINT', 'TINYINT'], ''], + [['bit_shift_right'], 'BIGINT', ['BIGINT', 'TINYINT'], ''] ], # map functions diff --git a/regression-test/data/correctness_p0/test_bit_function.out b/regression-test/data/correctness_p0/test_bit_function.out index 0fe60dde32c9f9..02334fcab28509 100644 --- a/regression-test/data/correctness_p0/test_bit_function.out +++ b/regression-test/data/correctness_p0/test_bit_function.out @@ -2,3 +2,135 @@ -- !select -- 64 123713 123649 -322 +-- !bit_count -- +0 +1 +1 +2 +1 +2 +2 +3 +1 +2 + +-- !bit_count -- +0 + +-- !bit_count -- +8 + +-- !bit_count -- +8 + +-- !bit_count -- +16 + +-- !bit_count -- +32 + +-- !bit_count -- +64 + +-- !bit_count -- +128 + +-- !bit_count_TINYINT_MAX -- +7 + +-- !bit_count_TINYINT_MIN -- +1 + +-- !bit_count_SMALLINT_MAX -- +15 + +-- !bit_count_SMALLINT_MIN -- +1 + +-- !bit_count_INT_MAX -- +31 + +-- !bit_count_INT_IN -- +1 + +-- !bit_count_INT64_MAX -- +63 + +-- !bit_count_INT64_MIN -- +1 + +-- !bit_count_INT128_MAX -- +127 127 + +-- !bit_count_INT128_MIN -- +1 1 + +-- !select -- +1 1 + +-- !bit_count -- +0 +1 +1 +2 +1 +2 +2 +3 +1 +2 + +-- !bit_count -- +0 + +-- !bit_count -- +8 + +-- !bit_count -- +8 + +-- !bit_count -- +16 + +-- !bit_count -- +32 + +-- !bit_count -- +64 + +-- !bit_count -- +128 + +-- !bit_count_TINYINT_MAX -- +7 + +-- !bit_count_TINYINT_MIN -- +1 + +-- !bit_count_SMALLINT_MAX -- +15 + +-- !bit_count_SMALLINT_MIN -- +1 + +-- !bit_count_INT_MAX -- +31 + +-- !bit_count_INT_IN -- +1 + +-- !bit_count_INT64_MAX -- +63 + +-- !bit_count_INT64_MIN -- +1 + +-- !bit_count_INT128_MAX -- +127 127 + +-- !bit_count_INT128_MIN -- +1 1 + +-- !select -- +1 1 + diff --git a/regression-test/data/correctness_p0/test_bit_shift_lagency.out b/regression-test/data/correctness_p0/test_bit_shift_lagency.out new file mode 100644 index 00000000000000..1c956fe3c743e4 --- /dev/null +++ b/regression-test/data/correctness_p0/test_bit_shift_lagency.out @@ -0,0 +1,345 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !comment -- +testing big_shift_left + +-- !numbers -- +0 +16 +32 +48 +64 +80 +96 +112 +128 +144 + +-- !0 -- +0 + +-- !1 -- +1024 + +-- !2 -- +-9223372036854775808 + +-- !3 -- +1 + +-- !select -- +254 + +-- !select -- +254 + +-- !int64_max -- +9223372036854775807 -2 + +-- !int64_min -- +-9223372036854775808 0 + +-- !select -- +127 -9223372036854775808 + +-- !select -- +-128 0 + +-- !select -- +0 254 65534 4294967294 -2 +1 -256 -65536 -4294967296 -2 + +-- !comment -- +testing big_shift_right + +-- !select -- +0 + +-- !select -- +0 + +-- !select -- +1 + +-- !select -- +0 + +-- !select -- +9223372036854775807 + +-- !select -- +2 + +-- !select -- +1 + +-- !select -- +-9223372036854775808 + +-- !select -- +0 + +-- !select -- +0 + +-- !select -- +1 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 + +-- !select -- +1 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 + diff --git a/regression-test/data/correctness_p0/test_bit_shift_nereids.out b/regression-test/data/correctness_p0/test_bit_shift_nereids.out new file mode 100644 index 00000000000000..1c956fe3c743e4 --- /dev/null +++ b/regression-test/data/correctness_p0/test_bit_shift_nereids.out @@ -0,0 +1,345 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !comment -- +testing big_shift_left + +-- !numbers -- +0 +16 +32 +48 +64 +80 +96 +112 +128 +144 + +-- !0 -- +0 + +-- !1 -- +1024 + +-- !2 -- +-9223372036854775808 + +-- !3 -- +1 + +-- !select -- +254 + +-- !select -- +254 + +-- !int64_max -- +9223372036854775807 -2 + +-- !int64_min -- +-9223372036854775808 0 + +-- !select -- +127 -9223372036854775808 + +-- !select -- +-128 0 + +-- !select -- +0 254 65534 4294967294 -2 +1 -256 -65536 -4294967296 -2 + +-- !comment -- +testing big_shift_right + +-- !select -- +0 + +-- !select -- +0 + +-- !select -- +1 + +-- !select -- +0 + +-- !select -- +9223372036854775807 + +-- !select -- +2 + +-- !select -- +1 + +-- !select -- +-9223372036854775808 + +-- !select -- +0 + +-- !select -- +0 + +-- !select -- +1 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 + +-- !select -- +1 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 + diff --git a/regression-test/suites/correctness_p0/test_bit_function.groovy b/regression-test/suites/correctness_p0/test_bit_function.groovy index 10b6e52de159f7..fc7b2c454b550a 100644 --- a/regression-test/suites/correctness_p0/test_bit_function.groovy +++ b/regression-test/suites/correctness_p0/test_bit_function.groovy @@ -18,4 +18,67 @@ suite("test_bit_functions") { sql "SET enable_nereids_planner=false" qt_select 'select bitand(123456, 321.0), bitor(123456, 321.0), bitxor(123456, 321.0), bitnot(321.0);' + + sql "SET enable_nereids_planner=false;" + qt_bit_count 'select bit_count(number) from numbers("number"="10");' + qt_bit_count 'select bit_count(0);' + qt_bit_count 'select bit_count(-1);' + qt_bit_count 'select bit_count(cast (-1 as tinyint));' // 8 + qt_bit_count 'select bit_count(cast (-1 as smallint));' // 16 + qt_bit_count 'select bit_count(cast (-1 as int));' // 32 + qt_bit_count 'select bit_count(cast (-1 as bigint));' // 64 + qt_bit_count 'select bit_count(cast (-1 as largeint));' // 128 + qt_bit_count_TINYINT_MAX 'select bit_count(cast (127 as tinyint));' // TINYINT_MAX + qt_bit_count_TINYINT_MIN 'select bit_count(cast (-128 as tinyint));' // TINYINT_MIN + qt_bit_count_SMALLINT_MAX 'select bit_count(cast (32767 as smallint));' // SMALLINT_MAX + qt_bit_count_SMALLINT_MIN 'select bit_count(cast (-32768 as smallint));' // SMALLINT_MIN + qt_bit_count_INT_MAX 'select bit_count(cast (2147483647 as int));' // INT_MAX + qt_bit_count_INT_IN 'select bit_count(cast (-2147483648 as int));' // INT_MIN + qt_bit_count_INT64_MAX 'select bit_count(cast (9223372036854775807 as bigint));' // INT64_MAX + qt_bit_count_INT64_MIN 'select bit_count(cast (-9223372036854775808 as bigint));' // INT64_MIN + // INT128_MAX + qt_bit_count_INT128_MAX """ + select bit_count(170141183460469231731687303715884105727), + bit_count(cast (170141183460469231731687303715884105727 as largeint)); + """ + // INT128_MIN + qt_bit_count_INT128_MIN """ + select bit_count(-170141183460469231731687303715884105728), + bit_count(cast (-170141183460469231731687303715884105728 as largeint)); + """ + + qt_select "select bit_count(bit_shift_right(-1, 63)), bit_count(bit_shift_right(-1, 63));" + + sql "SET enable_nereids_planner=true;" + sql "SET enable_fallback_to_original_planner=false;" + + qt_bit_count 'select bit_count(number) from numbers("number"="10");' + qt_bit_count 'select bit_count(0);' + qt_bit_count 'select bit_count(-1);' + qt_bit_count 'select bit_count(cast (-1 as tinyint));' // 8 + qt_bit_count 'select bit_count(cast (-1 as smallint));' // 16 + qt_bit_count 'select bit_count(cast (-1 as int));' // 32 + qt_bit_count 'select bit_count(cast (-1 as bigint));' // 64 + qt_bit_count 'select bit_count(cast (-1 as largeint));' // 128 + qt_bit_count_TINYINT_MAX 'select bit_count(cast (127 as tinyint));' // TINYINT_MAX + qt_bit_count_TINYINT_MIN 'select bit_count(cast (-128 as tinyint));' // TINYINT_MIN + qt_bit_count_SMALLINT_MAX 'select bit_count(cast (32767 as smallint));' // SMALLINT_MAX + qt_bit_count_SMALLINT_MIN 'select bit_count(cast (-32768 as smallint));' // SMALLINT_MIN + qt_bit_count_INT_MAX 'select bit_count(cast (2147483647 as int));' // INT_MAX + qt_bit_count_INT_IN 'select bit_count(cast (-2147483648 as int));' // INT_MIN + qt_bit_count_INT64_MAX 'select bit_count(cast (9223372036854775807 as bigint));' // INT64_MAX + qt_bit_count_INT64_MIN 'select bit_count(cast (-9223372036854775808 as bigint));' // INT64_MIN + // INT128_MAX + qt_bit_count_INT128_MAX """ + select bit_count(170141183460469231731687303715884105727), + bit_count(cast (170141183460469231731687303715884105727 as largeint)); + """ + // INT128_MIN + qt_bit_count_INT128_MIN """ + select bit_count(-170141183460469231731687303715884105728), + bit_count(cast (-170141183460469231731687303715884105728 as largeint)); + """ + + qt_select "select bit_count(bit_shift_right(-1, 63)), bit_count(bit_shift_right(-1, 63));" + } diff --git a/regression-test/suites/correctness_p0/test_bit_shift_lagency.groovy b/regression-test/suites/correctness_p0/test_bit_shift_lagency.groovy new file mode 100644 index 00000000000000..2db6ab0e901673 --- /dev/null +++ b/regression-test/suites/correctness_p0/test_bit_shift_lagency.groovy @@ -0,0 +1,105 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +suite("test_bit_shift_lagency") { + sql "SET enable_nereids_planner=false;" + + qt_comment """ + select "testing big_shift_left"; + """ + def INT64_MAX=9223372036854775807; + def INT64_MIN=-9223372036854775808; + def INT8_MAX=127 + def INT8_MIN=-128 + + // bit_shift_left + + qt_numbers 'select bit_shift_left(number, 4) from numbers("number"="10");' + qt_0 "select bit_shift_left(0, 0);" + qt_1 "select bit_shift_left(1, 10);" + qt_2 "select bit_shift_left(1, 63);" + qt_3 "select bit_shift_left(1, 64);" + qt_select "select bit_shift_left(127, 1);" + qt_select "select bit_shift_left(cast (127 as TINYINT), 1);" + qt_int64_max """ + WITH tbl AS ( + SELECT ${INT64_MAX} AS BIGINT_MAX) + SELECT BIGINT_MAX, bit_shift_left(BIGINT_MAX, 1) + FROM tbl + """ + qt_int64_min """ + WITH tbl AS ( + SELECT ${INT64_MIN} AS BIGINT_MIN) + SELECT BIGINT_MIN, bit_shift_left(BIGINT_MIN, 1) + FROM tbl + """ + qt_select """ + WITH tbl AS ( + SELECT ${INT8_MAX} AS TINYINT_MAX) + SELECT TINYINT_MAX, bit_shift_left(1, TINYINT_MAX) + FROM tbl + """ + qt_select """ + WITH tbl AS ( + SELECT ${INT8_MIN} AS TINYINT_MIN) + SELECT TINYINT_MIN, bit_shift_left(1, TINYINT_MIN) + FROM tbl + """ + + sql """ drop table if exists test_bit_shift_nereids; """ + sql """ + create table test_bit_shift_nereids ( + rid tinyint, + ti tinyint, + si smallint, + i int, + bi bigint + ) distributed by HASH(rid) + properties("replication_num"="1"); + """ + sql """ insert into test_bit_shift_nereids values + (0, 127, 32767, 2147483647, ${INT64_MAX}), + (1, -128, -32768, -2147483648, ${INT64_MAX})""" + + qt_select """ + select rid, + bit_shift_left(ti, 1), bit_shift_left(si, 1), + bit_shift_left(i, 1), bit_shift_left(bi, 1) + from test_bit_shift_nereids order by rid; """ + + // bit_shift_right + + qt_comment """ + select "testing big_shift_right"; + """ + + qt_select "SELECT bit_shift_right(0, 0);" + qt_select "SELECT bit_shift_right(0, 127);" + qt_select "SELECT bit_shift_right(${INT64_MAX}, 62);" + qt_select "SELECT bit_shift_right(${INT64_MAX}, 63);" + qt_select "SELECT bit_shift_right(${INT64_MAX}, 64);" + qt_select "SELECT bit_shift_right(${INT64_MIN}, 62);" + qt_select "SELECT bit_shift_right(${INT64_MIN}, 63);" + qt_select "SELECT bit_shift_right(${INT64_MIN}, 64);" + qt_select "SELECT bit_shift_right(1, ${INT8_MAX});" + qt_select "SELECT bit_shift_right(1, ${INT8_MIN});" + + // nagitave shift count + + qt_select """SELECT bit_shift_right(1, -number) from numbers("number"="129") order by number;""" + qt_select """SELECT bit_shift_right(1, -number) from numbers("number"="129") order by number;""" +} diff --git a/regression-test/suites/correctness_p0/test_bit_shift_nereids.groovy b/regression-test/suites/correctness_p0/test_bit_shift_nereids.groovy new file mode 100644 index 00000000000000..65f86877374f31 --- /dev/null +++ b/regression-test/suites/correctness_p0/test_bit_shift_nereids.groovy @@ -0,0 +1,105 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +suite("test_bit_shift_nereids") { + sql "SET enable_nereids_planner=true;" + sql "SET enable_fallback_to_original_planner=false;" + + qt_comment """ + select "testing big_shift_left"; + """ + def INT64_MAX=9223372036854775807; + def INT64_MIN=-9223372036854775808; + def INT8_MAX=127 + def INT8_MIN=-128 + + // bit_shift_left + + qt_numbers 'select bit_shift_left(number, 4) from numbers("number"="10");' + qt_0 "select bit_shift_left(0, 0);" + qt_1 "select bit_shift_left(1, 10);" + qt_2 "select bit_shift_left(1, 63);" + qt_3 "select bit_shift_left(1, 64);" + qt_select "select bit_shift_left(127, 1);" + qt_select "select bit_shift_left(cast (127 as TINYINT), 1);" + qt_int64_max """ + WITH tbl AS ( + SELECT ${INT64_MAX} AS BIGINT_MAX) + SELECT BIGINT_MAX, bit_shift_left(BIGINT_MAX, 1) + FROM tbl + """ + qt_int64_min """ + WITH tbl AS ( + SELECT ${INT64_MIN} AS BIGINT_MIN) + SELECT BIGINT_MIN, bit_shift_left(BIGINT_MIN, 1) + FROM tbl + """ + qt_select """ + WITH tbl AS ( + SELECT ${INT8_MAX} AS TINYINT_MAX) + SELECT TINYINT_MAX, bit_shift_left(1, TINYINT_MAX) + FROM tbl + """ + qt_select """ + WITH tbl AS ( + SELECT ${INT8_MIN} AS TINYINT_MIN) + SELECT TINYINT_MIN, bit_shift_left(1, TINYINT_MIN) + FROM tbl + """ + + sql """ drop table if exists test_bit_shift_nereids; """ + sql """ + create table test_bit_shift_nereids ( + rid tinyint, + ti tinyint, + si smallint, + i int, + bi bigint + ) distributed by HASH(rid) + properties("replication_num"="1"); + """ + sql """ insert into test_bit_shift_nereids values + (0, 127, 32767, 2147483647, ${INT64_MAX}), + (1, -128, -32768, -2147483648, ${INT64_MAX})""" + + qt_select """ + select rid, + bit_shift_left(ti, 1), bit_shift_left(si, 1), + bit_shift_left(i, 1), bit_shift_left(bi, 1) + from test_bit_shift_nereids order by rid; """ + // bit_shift_right + + qt_comment """ + select "testing big_shift_right"; + """ + + qt_select "SELECT bit_shift_right(0, 0);" + qt_select "SELECT bit_shift_right(0, 127);" + qt_select "SELECT bit_shift_right(${INT64_MAX}, 62);" + qt_select "SELECT bit_shift_right(${INT64_MAX}, 63);" + qt_select "SELECT bit_shift_right(${INT64_MAX}, 64);" + qt_select "SELECT bit_shift_right(${INT64_MIN}, 62);" + qt_select "SELECT bit_shift_right(${INT64_MIN}, 63);" + qt_select "SELECT bit_shift_right(${INT64_MIN}, 64);" + qt_select "SELECT bit_shift_right(1, ${INT8_MAX});" + qt_select "SELECT bit_shift_right(1, ${INT8_MIN});" + + // nagitave shift count + + qt_select """SELECT bit_shift_right(1, -number) from numbers("number"="129") order by number;""" + qt_select """SELECT bit_shift_right(1, -number) from numbers("number"="129") order by number;""" +} From 36591258499bc98fce0f4a26dfc2d33d7d6982ca Mon Sep 17 00:00:00 2001 From: starocean999 <40539150+starocean999@users.noreply.github.com> Date: Mon, 22 Jan 2024 13:35:55 +0800 Subject: [PATCH 126/200] [fix](planner) should return outputTupleDesc's id instead of tupleIds if outputTupleDesc is set in Plan Node (#30150) --- .../org/apache/doris/planner/PlanNode.java | 3 + .../test_nested_lateral_view.out | 15 +++++ .../test_nested_lateral_view.groovy | 55 +++++++++++++++++++ 3 files changed, 73 insertions(+) create mode 100644 regression-test/data/correctness_p0/test_nested_lateral_view.out create mode 100644 regression-test/suites/correctness_p0/test_nested_lateral_view.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/PlanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/PlanNode.java index 4cc535be09a7fc..24fc40be2e1f59 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/PlanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/PlanNode.java @@ -384,6 +384,9 @@ public ArrayList getOutputTblRefIds() { } public List getOutputTupleIds() { + if (outputTupleDesc != null) { + return Lists.newArrayList(outputTupleDesc.getId()); + } return tupleIds; } diff --git a/regression-test/data/correctness_p0/test_nested_lateral_view.out b/regression-test/data/correctness_p0/test_nested_lateral_view.out new file mode 100644 index 00000000000000..4bac7d8afe3418 --- /dev/null +++ b/regression-test/data/correctness_p0/test_nested_lateral_view.out @@ -0,0 +1,15 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_default -- +a,a,a,a 1 +a,a,a,a 1 +a,a,a,a 1 +a,a,a,a 1 +a,a,a,a 1 +a,a,a,a 1 +a,a,a,a 2 +a,a,a,a 2 +a,a,a,a 2 +a,a,a,a 2 +a,a,a,a 2 +a,a,a,a 2 + diff --git a/regression-test/suites/correctness_p0/test_nested_lateral_view.groovy b/regression-test/suites/correctness_p0/test_nested_lateral_view.groovy new file mode 100644 index 00000000000000..1d42331ee8808e --- /dev/null +++ b/regression-test/suites/correctness_p0/test_nested_lateral_view.groovy @@ -0,0 +1,55 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +suite("test_nested_lateral_view") { + sql """set enable_nereids_planner=true;""" + sql "SET enable_fallback_to_original_planner=false" + sql """drop table if exists ods_42378777c342d2da36d05db875393811;""" + sql """CREATE TABLE `ods_42378777c342d2da36d05db875393811` ( + `57CE5E02901753F7_06a28` varchar(*) NULL, + `57CE5E02520653F7_06a28` varchar(*) NULL, + `65F695F4519253F7_06a28` varchar(*) NULL, + `65E5671F659C6760_06a28` varchar(*) NULL, + `57CE5E027A7A683C_06a28` varchar(*) NULL, + `57CE5E02767E520653F7_06a28a4` varchar(*) NULL, + `65705B575C0F657070B9_06a28a4` varchar(*) NULL, + `qqqqqqqqqqqqqqqqqqqqqqqqqqqqqqqq_` DECIMAL(38, 10) NULL, + `5F88591A7B2653F7_06a28` varchar(*) NULL + ) ENGINE=OLAP + DUPLICATE KEY(`57CE5E02901753F7_06a28`) + COMMENT 'OLAP' + DISTRIBUTED BY RANDOM BUCKETS AUTO + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "is_being_synced" = "false", + "storage_format" = "V2", + "light_schema_change" = "true", + "disable_auto_compaction" = "false", + "enable_single_replica_compaction" = "false" + );""" + sql """INSERT INTO ods_42378777c342d2da36d05db875393811 (`57CE5E02901753F7_06a28`,`57CE5E02520653F7_06a28`,`65F695F4519253F7_06a28`,`65E5671F659C6760_06a28`,`57CE5E027A7A683C_06a28`,`57CE5E02767E520653F7_06a28a4`,`65705B575C0F657070B9_06a28a4`,qqqqqqqqqqqqqqqqqqqqqqqqqqqqqqqq_,`5F88591A7B2653F7_06a28`) VALUES + ('a,a,a,a','a;a;a;a','1899-12-31 15:11:22','2024-01-12 00:00:00','a a a a','a%a%a%a','33..33.*.@3',3.3300000000,''), + ('a,a,a,a','a;a;a;a','1899-12-31 00:11:22','2024-01-10 00:00:00','a a a a','a%a%a%a','99..99.*.@9',1.1100000000,'~!@#¥%……&*()——+=【】;‘,。、、'), + ('a,a,a,a','a;a;a;a','1899-12-31 12:11:22','2024-01-11 00:00:00','a a a a','a%a%a%a','22..22.*.@2',2.2200000000,'131.0'); + """ + qt_select_default """select * from (SELECT `t1`.`k0`, `f2`.`lt2` + FROM + ( + SELECT `t0`.`57CE5E02901753F7_06a28` as `k0` FROM `ods_42378777c342d2da36d05db875393811` `t0` + LATERAL VIEW explode(ARRAY(1,2)) `f1` AS `lt1`)`t1` + LATERAL VIEW explode(ARRAY(1,2)) `f2` AS `lt2` ) ttt order by 1,2; """ +} From 84ea93652d93ca8e315f31f85fb0645c3e003611 Mon Sep 17 00:00:00 2001 From: Calvin Kirs Date: Mon, 22 Jan 2024 14:46:40 +0800 Subject: [PATCH 127/200] [Fix](Job)Incorrect task query result of insert type (#30024) - IdToTask has no persistence, so the queried task will be lost once it is restarted. - The cancel task does not update metadata after being removed from the running task. - tvf displays an error when some fields in the query task result are empty - cycle scheduling job should not be STOP when task fail --- .../apache/doris/job/base/AbstractJob.java | 15 ++++-- .../job/extensions/insert/InsertJob.java | 46 ++++++++++++++----- .../job/extensions/insert/InsertTask.java | 30 ++++++------ .../apache/doris/job/manager/JobManager.java | 23 ++++++---- .../doris/job/scheduler/JobScheduler.java | 6 +-- .../suites/job_p0/test_base_insert_job.groovy | 24 ++++++---- 6 files changed, 95 insertions(+), 49 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java index 2416a6bca5f405..091ac158c1ce83 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java @@ -166,7 +166,7 @@ public void cancelTaskById(long taskId) throws JobException { throw new JobException("no running task"); } runningTasks.stream().filter(task -> task.getTaskId().equals(taskId)).findFirst() - .orElseThrow(() -> new JobException("no task id: " + taskId)).cancel(); + .orElseThrow(() -> new JobException("Not found task id: " + taskId)).cancel(); runningTasks.removeIf(task -> task.getTaskId().equals(taskId)); if (jobConfig.getExecuteType().equals(JobExecuteType.ONE_TIME)) { updateJobStatus(JobStatus.FINISHED); @@ -289,19 +289,19 @@ public void logUpdateOperation() { @Override public void onTaskFail(T task) throws JobException { - updateJobStatusIfEnd(); + updateJobStatusIfEnd(false); runningTasks.remove(task); } @Override public void onTaskSuccess(T task) throws JobException { - updateJobStatusIfEnd(); + updateJobStatusIfEnd(true); runningTasks.remove(task); } - private void updateJobStatusIfEnd() throws JobException { + private void updateJobStatusIfEnd(boolean taskSuccess) throws JobException { JobExecuteType executeType = getJobConfig().getExecuteType(); if (executeType.equals(JobExecuteType.MANUAL)) { return; @@ -309,7 +309,12 @@ private void updateJobStatusIfEnd() throws JobException { switch (executeType) { case ONE_TIME: case INSTANT: - Env.getCurrentEnv().getJobManager().getJob(jobId).updateJobStatus(JobStatus.FINISHED); + this.finishTimeMs = System.currentTimeMillis(); + if (taskSuccess) { + Env.getCurrentEnv().getJobManager().getJob(jobId).updateJobStatus(JobStatus.FINISHED); + } else { + Env.getCurrentEnv().getJobManager().getJob(jobId).updateJobStatus(JobStatus.STOPPED); + } break; case RECURRING: TimerDefinition timerDefinition = getJobConfig().getTimerDefinition(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java index ce918c426f8411..15e0c37987f6c7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java @@ -30,6 +30,7 @@ import org.apache.doris.common.ErrorReport; import org.apache.doris.common.FeConstants; import org.apache.doris.common.LabelAlreadyUsedException; +import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.io.Text; import org.apache.doris.common.util.LogBuilder; import org.apache.doris.common.util.LogKey; @@ -42,6 +43,7 @@ import org.apache.doris.job.common.TaskType; import org.apache.doris.job.exception.JobException; import org.apache.doris.load.FailMsg; +import org.apache.doris.load.loadv2.LoadJob; import org.apache.doris.load.loadv2.LoadStatistic; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.mysql.privilege.Privilege; @@ -272,14 +274,15 @@ public void recordTask(long id) { } if (CollectionUtils.isEmpty(historyTaskIdList)) { historyTaskIdList = new ConcurrentLinkedQueue<>(); - Env.getCurrentEnv().getEditLog().logUpdateJob(this); historyTaskIdList.add(id); + Env.getCurrentEnv().getEditLog().logUpdateJob(this); return; } historyTaskIdList.add(id); if (historyTaskIdList.size() >= Config.max_persistence_task_count) { historyTaskIdList.poll(); } + Env.getCurrentEnv().getEditLog().logUpdateJob(this); } @Override @@ -320,22 +323,44 @@ public List queryTasks() { } //TODO it's will be refactor, we will storage task info in job inner and query from it List taskIdList = new ArrayList<>(this.historyTaskIdList); + if (getJobConfig().getExecuteType().equals(JobExecuteType.INSTANT)) { + Collections.reverse(taskIdList); + return queryLoadTasksByTaskIds(taskIdList); + } + List loadJobs = Env.getCurrentEnv().getLoadManager().queryLoadJobsByJobIds(taskIdList); + if (CollectionUtils.isEmpty(loadJobs)) { + return new ArrayList<>(); + } + List tasks = new ArrayList<>(); + loadJobs.forEach(loadJob -> { + InsertTask task; + try { + task = new InsertTask(loadJob.getLabel(), loadJob.getDb().getFullName(), null, getCreateUser()); + task.setCreateTimeMs(loadJob.getCreateTimestamp()); + } catch (MetaNotFoundException e) { + log.warn("load job not found, job id is {}", loadJob.getId()); + return; + } + task.setJobId(getJobId()); + task.setTaskId(loadJob.getId()); + task.setJobInfo(loadJob); + tasks.add(task); + }); + return tasks; - Collections.reverse(taskIdList); - return queryLoadTasksByTaskIds(taskIdList); } public List queryLoadTasksByTaskIds(List taskIdList) { if (taskIdList.isEmpty()) { return new ArrayList<>(); } - List jobs = new ArrayList<>(); + List tasks = new ArrayList<>(); taskIdList.forEach(id -> { if (null != idToTasks.get(id)) { - jobs.add(idToTasks.get(id)); + tasks.add(idToTasks.get(id)); } }); - return jobs; + return tasks; } @Override @@ -354,14 +379,11 @@ public ShowResultSetMetaData getTaskMetaData() { } @Override - public void onTaskFail(InsertTask task) { - try { - updateJobStatus(JobStatus.STOPPED); + public void onTaskFail(InsertTask task) throws JobException { + if (getJobConfig().getExecuteType().equals(JobExecuteType.INSTANT)) { this.failMsg = new FailMsg(FailMsg.CancelType.LOAD_RUN_FAIL, task.getErrMsg()); - } catch (JobException e) { - throw new RuntimeException(e); } - getRunningTasks().remove(task); + super.onTaskFail(task); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java index e85e7a1b02708d..b5d8ea7fc177d9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java @@ -21,11 +21,11 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.ScalarType; -import org.apache.doris.common.FeConstants; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.task.AbstractTask; import org.apache.doris.load.FailMsg; +import org.apache.doris.load.loadv2.LoadJob; import org.apache.doris.load.loadv2.LoadStatistic; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.trees.plans.commands.InsertIntoTableCommand; @@ -89,7 +89,7 @@ public class InsertTask extends AbstractTask { @Getter @Setter - private InsertJob jobInfo; + private LoadJob jobInfo; private TaskType taskType = TaskType.PENDING; private MergeType mergeType = MergeType.APPEND; @@ -127,7 +127,7 @@ public InsertTask(String labelName, String currentDb, String sql, UserIdentity u } public InsertTask(String labelName, InsertIntoTableCommand insertInto, - ConnectContext ctx, StmtExecutor executor, LoadStatistic statistic) { + ConnectContext ctx, StmtExecutor executor, LoadStatistic statistic) { this.labelName = labelName; this.command = insertInto; this.userIdentity = ctx.getCurrentUserIdentity(); @@ -216,23 +216,27 @@ public TRow getTvfInfo() { // if task not start, load job is null,return pending task show info return getPendingTaskTVFInfo(); } - trow.addToColumnValue(new TCell().setStringVal(String.valueOf(jobInfo.getJobId()))); + trow.addToColumnValue(new TCell().setStringVal(String.valueOf(jobInfo.getId()))); trow.addToColumnValue(new TCell().setStringVal(String.valueOf(getJobId()))); trow.addToColumnValue(new TCell().setStringVal(labelName)); - trow.addToColumnValue(new TCell().setStringVal(jobInfo.getJobStatus().name())); + trow.addToColumnValue(new TCell().setStringVal(jobInfo.getState().name())); // err msg - String errMsg = FeConstants.null_string; + String errMsg = ""; if (failMsg != null) { errMsg = "type:" + failMsg.getCancelType() + "; msg:" + failMsg.getMsg(); } trow.addToColumnValue(new TCell().setStringVal(errMsg)); // create time - trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(jobInfo.getCreateTimeMs()))); + trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(jobInfo.getCreateTimestamp()))); // load end time - trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(jobInfo.getFinishTimeMs()))); + trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(jobInfo.getFinishTimestamp()))); // tracking url trow.addToColumnValue(new TCell().setStringVal(trackingUrl)); - trow.addToColumnValue(new TCell().setStringVal(loadStatistic.toJson())); + if (null != loadStatistic) { + trow.addToColumnValue(new TCell().setStringVal(loadStatistic.toJson())); + } else { + trow.addToColumnValue(new TCell().setStringVal("")); + } trow.addToColumnValue(new TCell().setStringVal(userIdentity.getQualifiedUser())); return trow; } @@ -244,11 +248,11 @@ private TRow getPendingTaskTVFInfo() { trow.addToColumnValue(new TCell().setStringVal(String.valueOf(getJobId()))); trow.addToColumnValue(new TCell().setStringVal(getJobId() + LABEL_SPLITTER + getTaskId())); trow.addToColumnValue(new TCell().setStringVal(getStatus().name())); - trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + trow.addToColumnValue(new TCell().setStringVal("")); trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(getCreateTimeMs()))); - trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); - trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); - trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + trow.addToColumnValue(new TCell().setStringVal("")); + trow.addToColumnValue(new TCell().setStringVal("")); + trow.addToColumnValue(new TCell().setStringVal("")); trow.addToColumnValue(new TCell().setStringVal(userIdentity.getQualifiedUser())); return trow; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java index b069fd5eca62de..7e8b01ce28775a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java @@ -137,16 +137,21 @@ public void unregisterJob(Long jobId) throws JobException { * @param ifExists is is true, if job not exist,we will ignore job not exist exception, else throw exception */ public void unregisterJob(String jobName, boolean ifExists) throws JobException { - T dropJob = null; - for (T job : jobMap.values()) { - if (job.getJobName().equals(jobName)) { - dropJob = job; + try { + T dropJob = null; + for (T job : jobMap.values()) { + if (job.getJobName().equals(jobName)) { + dropJob = job; + } } + if (dropJob == null && ifExists) { + return; + } + dropJob(dropJob, jobName); + } catch (Exception e) { + log.error("drop job error, jobName:" + jobName, e); + throw new JobException("unregister job error, jobName:" + jobName); } - if (dropJob == null && ifExists) { - return; - } - dropJob(dropJob, jobName); } private void dropJob(T dropJob, String jobName) throws JobException { @@ -284,6 +289,7 @@ public void cancelTaskById(String jobName, Long taskId) throws JobException { for (T job : jobMap.values()) { if (job.getJobName().equals(jobName)) { job.cancelTaskById(taskId); + job.logUpdateOperation(); return; } } @@ -378,6 +384,7 @@ private static boolean validState(JobState jobState, InsertJob loadJob) { } } + //todo it's not belong to JobManager public void cancelLoadJob(CancelLoadStmt cs) throws JobException, AnalysisException, DdlException { String dbName = cs.getDbName(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java index 597e39d96ed1a8..a104d3895e1485 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java @@ -184,8 +184,8 @@ private void executeTimerJobIdsWithinLastTenMinutesWindow() { } for (Map.Entry entry : jobMap.entrySet()) { T job = entry.getValue(); - if (job.getJobStatus().equals(JobStatus.FINISHED)) { - clearFinishedJob(job); + if (job.getJobStatus().equals(JobStatus.FINISHED) || job.getJobStatus().equals(JobStatus.STOPPED)) { + clearEndJob(job); continue; } if (!job.getJobStatus().equals(JobStatus.RUNNING) && !job.getJobConfig().checkIsTimerJob()) { @@ -195,7 +195,7 @@ private void executeTimerJobIdsWithinLastTenMinutesWindow() { } } - private void clearFinishedJob(T job) { + private void clearEndJob(T job) { if (job.getFinishTimeMs() + FINISHED_JOB_CLEANUP_THRESHOLD_TIME_MS < System.currentTimeMillis()) { return; } diff --git a/regression-test/suites/job_p0/test_base_insert_job.groovy b/regression-test/suites/job_p0/test_base_insert_job.groovy index f4db5907fa2d94..d9ebb83215239d 100644 --- a/regression-test/suites/job_p0/test_base_insert_job.groovy +++ b/regression-test/suites/job_p0/test_base_insert_job.groovy @@ -71,9 +71,18 @@ suite("test_base_insert_job") { CREATE JOB ${jobName} ON SCHEDULE every 1 second comment 'test' DO insert into ${tableName} (timestamp, type, user_id) values ('2023-03-18','1','12213'); """ Thread.sleep(2500) - def jobs = sql """select * from ${tableName}""" - println jobs - assert 3 >= jobs.size() >= (2 as Boolean) //at least 2 records, some times 3 records + sql """ + PAUSE JOB where jobname = '${jobName}' + """ + def tblDatas = sql """select * from ${tableName}""" + println tblDatas + assert 3 >= tblDatas.size() >= (2 as Boolean) //at least 2 records, some times 3 records + def pauseJobId = sql """select id from jobs("type"="insert") where Name='${jobName}'""" + def taskStatus = sql """select status from tasks("type"="insert") where jobid= '${pauseJobId.get(0).get(0)}'""" + println taskStatus + for (int i = 0; i < taskStatus.size(); i++) { + assert taskStatus.get(i).get(0) != "FAILED"||taskStatus.get(i).get(0) != "STOPPED"||taskStatus.get(i).get(0) != "STOPPED" + } sql """ CREATE JOB ${jobMixedName} ON SCHEDULE every 1 second DO insert into ${tableName} (timestamp, type, user_id) values ('2023-03-18','1','12213'); """ @@ -132,9 +141,8 @@ suite("test_base_insert_job") { sql """cancel task where jobName='${jobName}' and taskId= ${taskId}""" def cancelTask = sql """ select status from tasks("type"="insert") where jobid= ${onceJobId}""" println cancelTask - //check task status - assert cancelTask.size() == 1 - assert cancelTask.get(0).get(0) == "CANCELED" + //check task size is 0, cancel task where be deleted + assert cancelTask.size() == 0 // check table data def dataCount1 = sql """select count(1) from ${tableName}""" assert dataCount1.get(0).get(0) == 0 @@ -161,14 +169,14 @@ suite("test_base_insert_job") { assert job.size() == 1 def jobId = job.get(0).get(0); def tasks = sql """ select status from tasks("type"="insert") where jobid= ${jobId} """ - assert tasks.size() == 1 + assert tasks.size() == 0 sql """ RESUME JOB where jobname = '${jobName}' """ Thread.sleep(2500) def resumeTasks = sql """ select status from tasks("type"="insert") where jobid= ${jobId} """ println resumeTasks - assert resumeTasks.size() == 2 + assert resumeTasks.size() == 1 // assert same job name try { sql """ From 9b9e077b1d011e2cc74add9c622f18583602e79a Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Mon, 22 Jan 2024 14:47:18 +0800 Subject: [PATCH 128/200] [fix](Nereids) nullable not adjust in output exprs in result sink node (#30206) --- .../apache/doris/nereids/analyzer/UnboundResultSink.java | 5 +++++ .../apache/doris/nereids/analyzer/UnboundTableSink.java | 6 ++++++ .../doris/nereids/rules/rewrite/AdjustNullable.java | 8 ++++++++ .../plans/logical/LogicalDeferMaterializeResultSink.java | 7 +++++++ .../nereids/trees/plans/logical/LogicalFileSink.java | 2 +- .../nereids/trees/plans/logical/LogicalOlapTableSink.java | 5 +++++ .../nereids/trees/plans/logical/LogicalResultSink.java | 5 +++++ .../doris/nereids/trees/plans/logical/LogicalSink.java | 2 ++ 8 files changed, 39 insertions(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundResultSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundResultSink.java index 6f378e9911270a..10a6b0121e3b27 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundResultSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundResultSink.java @@ -21,6 +21,7 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation; import org.apache.doris.nereids.trees.plans.Plan; @@ -77,7 +78,11 @@ public Plan withGroupExprLogicalPropChildren(Optional groupExpr Optional logicalProperties, List children) { Preconditions.checkArgument(children.size() == 1, "UnboundResultSink only accepts one child"); return new UnboundResultSink<>(groupExpression, logicalProperties, children.get(0)); + } + @Override + public UnboundResultSink withOutputExprs(List outputExprs) { + throw new UnboundException("could not call withOutputExprs on UnboundResultSink"); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSink.java index 1bd4e994ac5637..89d67dc376661b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSink.java @@ -22,6 +22,7 @@ import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.UnboundLogicalProperties; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation; import org.apache.doris.nereids.trees.plans.Plan; @@ -137,6 +138,11 @@ public Plan withChildren(List children) { dmlCommandType, groupExpression, Optional.empty(), children.get(0)); } + @Override + public UnboundTableSink withOutputExprs(List outputExprs) { + throw new UnboundException("could not call withOutputExprs on UnboundTableSink"); + } + @Override public R accept(PlanVisitor visitor, C context) { return visitor.visitUnboundTableSink(this, context); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AdjustNullable.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AdjustNullable.java index 2aa095875ee689..8f70b86e4bdb20 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AdjustNullable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AdjustNullable.java @@ -40,6 +40,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalRepeat; import org.apache.doris.nereids.trees.plans.logical.LogicalSetOperation; +import org.apache.doris.nereids.trees.plans.logical.LogicalSink; import org.apache.doris.nereids.trees.plans.logical.LogicalSort; import org.apache.doris.nereids.trees.plans.logical.LogicalTopN; import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; @@ -78,6 +79,13 @@ public Plan visit(Plan plan, Map replaceMap) { return logicalPlan; } + @Override + public Plan visitLogicalSink(LogicalSink logicalSink, Map replaceMap) { + logicalSink = (LogicalSink) super.visit(logicalSink, replaceMap); + List newOutputExprs = updateExpressions(logicalSink.getOutputExprs(), replaceMap); + return logicalSink.withOutputExprs(newOutputExprs); + } + @Override public Plan visitLogicalAggregate(LogicalAggregate aggregate, Map replaceMap) { aggregate = (LogicalAggregate) super.visit(aggregate, replaceMap); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalDeferMaterializeResultSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalDeferMaterializeResultSink.java index ab6efb8977a686..dab393cb007b6d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalDeferMaterializeResultSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalDeferMaterializeResultSink.java @@ -21,6 +21,7 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PropagateFuncDeps; import org.apache.doris.nereids.trees.plans.algebra.Sink; @@ -82,6 +83,12 @@ public LogicalDeferMaterializeResultSink withChildren(List children) olapTable, selectedIndexId, Optional.empty(), Optional.empty(), children.get(0)); } + @Override + public LogicalDeferMaterializeResultSink withOutputExprs(List outputExprs) { + return new LogicalDeferMaterializeResultSink<>(logicalResultSink, olapTable, selectedIndexId, + Optional.empty(), Optional.empty(), child()); + } + @Override public R accept(PlanVisitor visitor, C context) { return visitor.visitLogicalDeferMaterializeResultSink(this, context); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileSink.java index 3ca559c51974d0..43e490c00c7a17 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileSink.java @@ -59,7 +59,7 @@ public LogicalFileSink(String filePath, String format, Map prope this.properties = ImmutableMap.copyOf(Objects.requireNonNull(properties)); } - public Plan withOutputExprs(List outputExprs) { + public LogicalFileSink withOutputExprs(List outputExprs) { return new LogicalFileSink<>(filePath, format, properties, outputExprs, child()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapTableSink.java index a1147a96170a80..1a298ea8c407b9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapTableSink.java @@ -113,6 +113,11 @@ public DMLCommandType getDmlCommandType() { return dmlCommandType; } + public LogicalOlapTableSink withOutputExprs(List outputExprs) { + return new LogicalOlapTableSink<>(database, targetTable, cols, partitionIds, outputExprs, isPartialUpdate, + dmlCommandType, Optional.empty(), Optional.empty(), child()); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalResultSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalResultSink.java index 564938dd8b101a..6312aee6235a2e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalResultSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalResultSink.java @@ -72,6 +72,11 @@ public LogicalResultSink withGroupExprLogicalPropChildren(Optional(outputExprs, groupExpression, logicalProperties, children.get(0)); } + @Override + public LogicalResultSink withOutputExprs(List outputExprs) { + return new LogicalResultSink<>(outputExprs, Optional.empty(), Optional.empty(), child()); + } + @Override public String toString() { return Utils.toSqlString("LogicalResultSink[" + id.asInt() + "]", diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSink.java index 028c0e93e0ae39..ea2fa49e4664bb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSink.java @@ -52,6 +52,8 @@ public List getOutputExprs() { return outputExprs; } + public abstract LogicalSink withOutputExprs(List outputExprs); + @Override public List getExpressions() { return outputExprs; From 1eabcc3d8b09c582057a386c26263a45cf4aaf8b Mon Sep 17 00:00:00 2001 From: Jerry Hu Date: Mon, 22 Jan 2024 15:04:27 +0800 Subject: [PATCH 129/200] [fix](exec_node) crashing caused by cancelled query in ExecNode (#30192) --- be/src/exec/exec_node.cpp | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/be/src/exec/exec_node.cpp b/be/src/exec/exec_node.cpp index 6bc6e07c5631c6..d65e8f96bd2cd8 100644 --- a/be/src/exec/exec_node.cpp +++ b/be/src/exec/exec_node.cpp @@ -558,22 +558,22 @@ Status ExecNode::get_next_after_projects( RuntimeState* state, vectorized::Block* block, bool* eos, const std::function& func, bool clear_data) { - Defer defer([block, this]() { - if (block && !block->empty()) { - COUNTER_UPDATE(_output_bytes_counter, block->allocated_bytes()); - COUNTER_UPDATE(_block_count_counter, 1); - } - }); if (_output_row_descriptor) { if (clear_data) { clear_origin_block(); } - auto status = func(state, &_origin_block, eos); - if (UNLIKELY(!status.ok())) return status; - return do_projections(&_origin_block, block); + RETURN_IF_ERROR(func(state, &_origin_block, eos)); + RETURN_IF_ERROR(do_projections(&_origin_block, block)); + } else { + RETURN_IF_ERROR(func(state, block, eos)); } _peak_memory_usage_counter->set(_mem_tracker->peak_consumption()); - return func(state, block, eos); + + if (block && !block->empty()) { + COUNTER_UPDATE(_output_bytes_counter, block->allocated_bytes()); + COUNTER_UPDATE(_block_count_counter, 1); + } + return Status::OK(); } Status ExecNode::sink(RuntimeState* state, vectorized::Block* input_block, bool eos) { From 6bf835f75a6576212160c8ade1533c124b9c4b1a Mon Sep 17 00:00:00 2001 From: chunping Date: Mon, 22 Jan 2024 15:09:23 +0800 Subject: [PATCH 130/200] [docker](container)create data path after rm residual data avoiding docker start fail (#30141) --- docker/thirdparties/run-thirdparties-docker.sh | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docker/thirdparties/run-thirdparties-docker.sh b/docker/thirdparties/run-thirdparties-docker.sh index 3ff8a29997662e..1a3faba5617dbe 100755 --- a/docker/thirdparties/run-thirdparties-docker.sh +++ b/docker/thirdparties/run-thirdparties-docker.sh @@ -198,8 +198,8 @@ if [[ "${RUN_MYSQL}" -eq 1 ]]; then sed -i "s/doris--/${CONTAINER_UID}/g" "${ROOT}"/docker-compose/mysql/mysql-5.7.yaml sudo docker compose -f "${ROOT}"/docker-compose/mysql/mysql-5.7.yaml --env-file "${ROOT}"/docker-compose/mysql/mysql-5.7.env down if [[ "${STOP}" -ne 1 ]]; then - sudo mkdir -p "${ROOT}"/docker-compose/mysql/data/ sudo rm "${ROOT}"/docker-compose/mysql/data/* -rf + sudo mkdir -p "${ROOT}"/docker-compose/mysql/data/ sudo docker compose -f "${ROOT}"/docker-compose/mysql/mysql-5.7.yaml --env-file "${ROOT}"/docker-compose/mysql/mysql-5.7.env up -d fi fi @@ -210,8 +210,8 @@ if [[ "${RUN_PG}" -eq 1 ]]; then sed -i "s/doris--/${CONTAINER_UID}/g" "${ROOT}"/docker-compose/postgresql/postgresql-14.yaml sudo docker compose -f "${ROOT}"/docker-compose/postgresql/postgresql-14.yaml --env-file "${ROOT}"/docker-compose/postgresql/postgresql-14.env down if [[ "${STOP}" -ne 1 ]]; then - sudo mkdir -p "${ROOT}"/docker-compose/postgresql/data/data sudo rm "${ROOT}"/docker-compose/postgresql/data/* -rf + sudo mkdir -p "${ROOT}"/docker-compose/postgresql/data/data sudo docker compose -f "${ROOT}"/docker-compose/postgresql/postgresql-14.yaml --env-file "${ROOT}"/docker-compose/postgresql/postgresql-14.env up -d fi fi @@ -222,8 +222,8 @@ if [[ "${RUN_ORACLE}" -eq 1 ]]; then sed -i "s/doris--/${CONTAINER_UID}/g" "${ROOT}"/docker-compose/oracle/oracle-11.yaml sudo docker compose -f "${ROOT}"/docker-compose/oracle/oracle-11.yaml --env-file "${ROOT}"/docker-compose/oracle/oracle-11.env down if [[ "${STOP}" -ne 1 ]]; then - sudo mkdir -p "${ROOT}"/docker-compose/oracle/data/ sudo rm "${ROOT}"/docker-compose/oracle/data/* -rf + sudo mkdir -p "${ROOT}"/docker-compose/oracle/data/ sudo docker compose -f "${ROOT}"/docker-compose/oracle/oracle-11.yaml --env-file "${ROOT}"/docker-compose/oracle/oracle-11.env up -d fi fi @@ -234,8 +234,8 @@ if [[ "${RUN_SQLSERVER}" -eq 1 ]]; then sed -i "s/doris--/${CONTAINER_UID}/g" "${ROOT}"/docker-compose/sqlserver/sqlserver.yaml sudo docker compose -f "${ROOT}"/docker-compose/sqlserver/sqlserver.yaml --env-file "${ROOT}"/docker-compose/sqlserver/sqlserver.env down if [[ "${STOP}" -ne 1 ]]; then - sudo mkdir -p "${ROOT}"/docker-compose/sqlserver/data/ sudo rm "${ROOT}"/docker-compose/sqlserver/data/* -rf + sudo mkdir -p "${ROOT}"/docker-compose/sqlserver/data/ sudo docker compose -f "${ROOT}"/docker-compose/sqlserver/sqlserver.yaml --env-file "${ROOT}"/docker-compose/sqlserver/sqlserver.env up -d fi fi @@ -246,8 +246,8 @@ if [[ "${RUN_CLICKHOUSE}" -eq 1 ]]; then sed -i "s/doris--/${CONTAINER_UID}/g" "${ROOT}"/docker-compose/clickhouse/clickhouse.yaml sudo docker compose -f "${ROOT}"/docker-compose/clickhouse/clickhouse.yaml --env-file "${ROOT}"/docker-compose/clickhouse/clickhouse.env down if [[ "${STOP}" -ne 1 ]]; then - sudo mkdir -p "${ROOT}"/docker-compose/clickhouse/data/ sudo rm "${ROOT}"/docker-compose/clickhouse/data/* -rf + sudo mkdir -p "${ROOT}"/docker-compose/clickhouse/data/ sudo docker compose -f "${ROOT}"/docker-compose/clickhouse/clickhouse.yaml --env-file "${ROOT}"/docker-compose/clickhouse/clickhouse.env up -d fi fi From c2d731d506e84b4124e44f38241268533f309f5e Mon Sep 17 00:00:00 2001 From: xy Date: Mon, 22 Jan 2024 15:10:46 +0800 Subject: [PATCH 131/200] [fix](Nereids) Fixed a bug where the execution plan was incorrect after ddl (#30107) should only compare column name when generate data dist info of PhysicalOlapScan Co-authored-by: xingying01 --- .../LogicalOlapScanToPhysicalOlapScan.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalOlapScanToPhysicalOlapScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalOlapScanToPhysicalOlapScan.java index 6bcf22a9adc1e0..43436355ae17aa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalOlapScanToPhysicalOlapScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalOlapScanToPhysicalOlapScan.java @@ -94,7 +94,10 @@ private DistributionSpec convertDistribution(LogicalOlapScan olapScan) { if (hashColumns.size() != hashDistributionInfo.getDistributionColumns().size()) { for (Slot slot : baseOutput) { for (Column column : hashDistributionInfo.getDistributionColumns()) { - if (((SlotReference) slot).getColumn().get().equals(column)) { + // If the length of the column in the bucket key changes after DDL, the length cannot be + // determined. As a result, some bucket fields are lost in the query execution plan. + // So here we use the column name to avoid this problem + if (((SlotReference) slot).getColumn().get().getName().equalsIgnoreCase(column.getName())) { hashColumns.add(slot.getExprId()); } } @@ -108,7 +111,10 @@ private DistributionSpec convertDistribution(LogicalOlapScan olapScan) { List hashColumns = Lists.newArrayList(); for (Slot slot : output) { for (Column column : hashDistributionInfo.getDistributionColumns()) { - if (((SlotReference) slot).getColumn().get().equals(column)) { + // If the length of the column in the bucket key changes after DDL, the length cannot be + // determined. As a result, some bucket fields are lost in the query execution plan. + // So here we use the column name to avoid this problem + if (((SlotReference) slot).getColumn().get().getName().equalsIgnoreCase(column.getName())) { hashColumns.add(slot.getExprId()); } } From 01c227e7ce93b672464359eb1984827e789b8326 Mon Sep 17 00:00:00 2001 From: Petrichor <31833513+vinlee19@users.noreply.github.com> Date: Mon, 22 Jan 2024 15:39:28 +0800 Subject: [PATCH 132/200] [Improvement](doc) add table-buckets doc for flink connector (#29672) --- docs/en/docs/ecosystem/flink-doris-connector.md | 4 ++-- docs/zh-CN/docs/ecosystem/flink-doris-connector.md | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/docs/ecosystem/flink-doris-connector.md b/docs/en/docs/ecosystem/flink-doris-connector.md index 090f515bfc98fe..f1ef66b1cb529d 100644 --- a/docs/en/docs/ecosystem/flink-doris-connector.md +++ b/docs/en/docs/ecosystem/flink-doris-connector.md @@ -531,12 +531,12 @@ insert into doris_sink select id,name,bank,age from cdc_mysql_source; | --postgres-conf | Postgres CDCSource configuration, e.g. --postgres-conf hostname=127.0.0.1, you can find [here](https://ververica.github.io/flink-cdc-connectors/master/content/connectors/postgres-cdc.html) View all configurations Postgres-CDC where hostname/username/password/database-name/schema-name/slot.name is required. | | --sqlserver-conf | SQLServer CDCSource configuration, for example --sqlserver-conf hostname=127.0.0.1, you can find it [here](https://ververica.github.io/flink-cdc-connectors/master/content/connectors/sqlserver-cdc.html) View all configurations SQLServer-CDC, where hostname/username/password/database-name/schema-name is required. | | --sink-conf | All configurations of Doris Sink can be found [here](https://doris.apache.org/zh-CN/docs/dev/ecosystem/flink-doris-connector/#%E9%80%9A%E7%94%A8%E9%85%8D%E7%BD%AE%E9%A1%B9) View the complete configuration items. | -| --table-conf | The configuration items of the Doris table, that is, the content contained in properties. For example --table-conf replication_num=1 | +| --table-conf | The configuration items of the Doris table(The exception is table-buckets, non-properties attributes), that is, the content contained in properties. For example `--table-conf replication_num=1`, and the `--table-conf table-buckets="tbl1:10,tbl2:20,a.*:30,b.*:40,.*:50"` option specifies the number of buckets for different tables based on the order of regular expressions. If there is no match, the table is created with the default setting of BUCKETS AUTO. | | --ignore-default-value | Turn off the default value of synchronizing mysql table structure. It is suitable for synchronizing mysql data to doris when the field has a default value but the actual inserted data is null. Reference [here](https://github.com/apache/doris-flink-connector/pull/152) | | --use-new-schema-change | Whether to use the new schema change to support synchronization of MySQL multi-column changes and default values. Reference [here](https://github.com/apache/doris-flink-connector/pull/167) | | --single-sink | Whether to use a single Sink to synchronize all tables. When turned on, newly created tables in the upstream can also be automatically recognized and tables automatically created. | | --multi-to-one-origin | When writing multiple upstream tables into the same table, the configuration of the source table, for example: --multi-to-one-origin="a\_.\*|b_.\*", Reference [here](https://github.com/apache/doris-flink-connector/pull/208) | -| --multi-to-one-target | Used with multi-to-one-origin, the configuration of the target table, such as:--multi-to-one-target="a\|b" | +| --multi-to-one-target | Used with multi-to-one-origin, the configuration of the target table, such as: --multi-to-one-target="a\|b" | >Note: When synchronizing, you need to add the corresponding Flink CDC dependencies in the $FLINK_HOME/lib directory, such as flink-sql-connector-mysql-cdc-${version}.jar, flink-sql-connector-oracle-cdc-${version}.jar diff --git a/docs/zh-CN/docs/ecosystem/flink-doris-connector.md b/docs/zh-CN/docs/ecosystem/flink-doris-connector.md index 6d9b2a8d976c6d..321c7da429829b 100644 --- a/docs/zh-CN/docs/ecosystem/flink-doris-connector.md +++ b/docs/zh-CN/docs/ecosystem/flink-doris-connector.md @@ -532,7 +532,7 @@ insert into doris_sink select id,name,bank,age from cdc_mysql_source; | --postgres-conf | Postgres CDCSource 配置,例如--postgres-conf hostname=127.0.0.1 ,您可以在[这里](https://ververica.github.io/flink-cdc-connectors/master/content/connectors/postgres-cdc.html)查看所有配置Postgres-CDC,其中hostname/username/password/database-name/schema-name/slot.name 是必需的。 | | --sqlserver-conf | SQLServer CDCSource 配置,例如--sqlserver-conf hostname=127.0.0.1 ,您可以在[这里](https://ververica.github.io/flink-cdc-connectors/master/content/connectors/sqlserver-cdc.html)查看所有配置SQLServer-CDC,其中hostname/username/password/database-name/schema-name 是必需的。 | | --sink-conf | Doris Sink 的所有配置,可以在[这里](https://doris.apache.org/zh-CN/docs/dev/ecosystem/flink-doris-connector/#%E9%80%9A%E7%94%A8%E9%85%8D%E7%BD%AE%E9%A1%B9)查看完整的配置项。 | -| --table-conf | Doris表的配置项,即properties中包含的内容。 例如 --table-conf replication_num=1 | +| --table-conf | Doris表的配置项,即properties中包含的内容(其中table-buckets例外,非properties属性)。 例如 `--table-conf replication_num=1`, 而 `--table-conf table-buckets="tbl1:10,tbl2:20,a.*:30,b.*:40,.*:50"`表示按照正则表达式顺序指定不同表的buckets数量,如果没有匹配到则采用BUCKETS AUTO建表。 | | --ignore-default-value | 关闭同步mysql表结构的默认值。适用于同步mysql数据到doris时,字段有默认值,但实际插入数据为null情况。参考[#152](https://github.com/apache/doris-flink-connector/pull/152) | | --use-new-schema-change | 是否使用新的schema change,支持同步mysql多列变更、默认值。参考[#167](https://github.com/apache/doris-flink-connector/pull/167) | | --single-sink | 是否使用单个Sink同步所有表,开启后也可自动识别上游新创建的表,自动创建表。 | From acc064565e771133d5279f91de927d27d7f01921 Mon Sep 17 00:00:00 2001 From: yangshijie Date: Mon, 22 Jan 2024 16:19:47 +0800 Subject: [PATCH 133/200] [feature](function) support ip functions named to_ipv4[or_default, or_null](string) and to_ipv6[or_default, or_null](string) (#29838) --- be/src/vec/functions/function_ip.cpp | 26 +- be/src/vec/functions/function_ip.h | 223 ++++++++++++++---- .../doris/catalog/BuiltinScalarFunctions.java | 12 + .../expressions/functions/scalar/ToIpv4.java | 67 ++++++ .../functions/scalar/ToIpv4OrDefault.java | 67 ++++++ .../functions/scalar/ToIpv4OrNull.java | 67 ++++++ .../expressions/functions/scalar/ToIpv6.java | 67 ++++++ .../functions/scalar/ToIpv6OrDefault.java | 67 ++++++ .../functions/scalar/ToIpv6OrNull.java | 67 ++++++ .../visitor/ScalarFunctionVisitor.java | 30 +++ gensrc/script/doris_builtins_functions.py | 12 + .../ip_functions/test_to_ip_functions.out | 21 ++ .../ip_functions/test_to_ip_functions.groovy | 49 ++++ 13 files changed, 720 insertions(+), 55 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv4.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv4OrDefault.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv4OrNull.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv6.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv6OrDefault.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv6OrNull.java create mode 100644 regression-test/data/query_p0/sql_functions/ip_functions/test_to_ip_functions.out create mode 100644 regression-test/suites/query_p0/sql_functions/ip_functions/test_to_ip_functions.groovy diff --git a/be/src/vec/functions/function_ip.cpp b/be/src/vec/functions/function_ip.cpp index 349e095b4a2c2c..7eed9c1336417e 100644 --- a/be/src/vec/functions/function_ip.cpp +++ b/be/src/vec/functions/function_ip.cpp @@ -22,25 +22,27 @@ namespace doris::vectorized { void register_function_ip(SimpleFunctionFactory& factory) { factory.register_function(); factory.register_alias(FunctionIPv4NumToString::name, "inet_ntoa"); - factory.register_function>(); - factory.register_function>(); - factory.register_function>(); - factory.register_alias(FunctionIPv4StringToNum::name, - "inet_aton"); - + factory.register_function>(); + factory.register_function>(); + factory.register_function>(); + factory.register_alias(FunctionIPv4StringToNum::name, "inet_aton"); factory.register_function(); factory.register_alias(FunctionIPv6NumToString::name, "inet6_ntoa"); - factory.register_function>(); - factory.register_function>(); - factory.register_function>(); - factory.register_alias(FunctionIPv6StringToNum::name, - "inet6_aton"); - + factory.register_function>(); + factory.register_function>(); + factory.register_function>(); + factory.register_alias(FunctionIPv6StringToNum::name, "inet6_aton"); factory.register_function(); factory.register_function(); factory.register_function>(); factory.register_function>(); factory.register_function(); factory.register_function(); + factory.register_function>(); + factory.register_function>(); + factory.register_function>(); + factory.register_function>(); + factory.register_function>(); + factory.register_function>(); } } // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/functions/function_ip.h b/be/src/vec/functions/function_ip.h index 8a260b8fd82397..359cafe7e34305 100644 --- a/be/src/vec/functions/function_ip.h +++ b/be/src/vec/functions/function_ip.h @@ -117,13 +117,13 @@ class FunctionIPv4NumToString : public IFunction { } }; -enum class IPStringToNumExceptionMode : uint8_t { Throw, Default, Null }; +enum class IPExceptionMode : uint8_t { Throw, Default, Null }; static inline bool tryParseIPv4(const char* pos, Int64& result_value) { return parseIPv4whole(pos, reinterpret_cast(&result_value)); } -template +template ColumnPtr convertToIPv4(ColumnPtr column, const PaddedPODArray* null_map = nullptr) { const ColumnString* column_string = check_and_get_column(column.get()); @@ -138,7 +138,7 @@ ColumnPtr convertToIPv4(ColumnPtr column, const PaddedPODArray* null_map ColumnUInt8::MutablePtr col_null_map_to; ColumnUInt8::Container* vec_null_map_to = nullptr; - if constexpr (exception_mode == IPStringToNumExceptionMode::Null) { + if constexpr (exception_mode == IPExceptionMode::Null) { col_null_map_to = ColumnUInt8::create(column_size, false); vec_null_map_to = &col_null_map_to->get_data(); } @@ -156,7 +156,7 @@ ColumnPtr convertToIPv4(ColumnPtr column, const PaddedPODArray* null_map if (null_map && (*null_map)[i]) { vec_res[i] = 0; prev_offset = offsets_src[i]; - if constexpr (exception_mode == IPStringToNumExceptionMode::Null) { + if constexpr (exception_mode == IPExceptionMode::Null) { (*vec_null_map_to)[i] = true; } continue; @@ -169,11 +169,11 @@ ColumnPtr convertToIPv4(ColumnPtr column, const PaddedPODArray* null_map bool parse_result = tryParseIPv4(src.c_str(), vec_res[i]); if (!parse_result) { - if constexpr (exception_mode == IPStringToNumExceptionMode::Throw) { + if constexpr (exception_mode == IPExceptionMode::Throw) { throw Exception(ErrorCode::INVALID_ARGUMENT, "Invalid IPv4 value"); - } else if constexpr (exception_mode == IPStringToNumExceptionMode::Default) { + } else if constexpr (exception_mode == IPExceptionMode::Default) { vec_res[i] = 0; - } else if constexpr (exception_mode == IPStringToNumExceptionMode::Null) { + } else if constexpr (exception_mode == IPExceptionMode::Null) { (*vec_null_map_to)[i] = true; vec_res[i] = 0; } @@ -182,20 +182,20 @@ ColumnPtr convertToIPv4(ColumnPtr column, const PaddedPODArray* null_map prev_offset = offsets_src[i]; } - if constexpr (exception_mode == IPStringToNumExceptionMode::Null) + if constexpr (exception_mode == IPExceptionMode::Null) return ColumnNullable::create(std::move(col_res), std::move(col_null_map_to)); return col_res; } -template +template class FunctionIPv4StringToNum : public IFunction { public: - static constexpr auto name = exception_mode == IPStringToNumExceptionMode::Throw - ? "ipv4_string_to_num" - : (exception_mode == IPStringToNumExceptionMode::Default - ? "ipv4_string_to_num_or_default" - : "ipv4_string_to_num_or_null"); + static constexpr auto name = + exception_mode == IPExceptionMode::Throw + ? "ipv4_string_to_num" + : (exception_mode == IPExceptionMode::Default ? "ipv4_string_to_num_or_default" + : "ipv4_string_to_num_or_null"); static FunctionPtr create() { return std::make_shared>(); @@ -213,7 +213,7 @@ class FunctionIPv4StringToNum : public IFunction { } auto result_type = std::make_shared(); - if constexpr (exception_mode == IPStringToNumExceptionMode::Null) { + if constexpr (exception_mode == IPExceptionMode::Null) { return make_nullable(result_type); } @@ -343,8 +343,7 @@ class FunctionIPv6NumToString : public IFunction { }; namespace detail { -template +template ColumnPtr convertToIPv6(const StringColumnType& string_column, const PaddedPODArray* null_map = nullptr) { if constexpr (!std::is_same_v && @@ -359,7 +358,7 @@ ColumnPtr convertToIPv6(const StringColumnType& string_column, ColumnUInt8::MutablePtr col_null_map_to; ColumnUInt8::Container* vec_null_map_to = nullptr; - if constexpr (exception_mode == IPStringToNumExceptionMode::Null) { + if constexpr (exception_mode == IPExceptionMode::Null) { col_null_map_to = ColumnUInt8::create(column_size, false); vec_null_map_to = &col_null_map_to->get_data(); } @@ -420,9 +419,9 @@ ColumnPtr convertToIPv6(const StringColumnType& string_column, } if (null_map && (*null_map)[i]) { - if (exception_mode == IPStringToNumExceptionMode::Throw) { + if (exception_mode == IPExceptionMode::Throw) { throw Exception(ErrorCode::INVALID_ARGUMENT, "Invalid IPv6 value"); - } else if (exception_mode == IPStringToNumExceptionMode::Default) { + } else if (exception_mode == IPExceptionMode::Default) { std::fill_n(&vec_res[out_offset], offset_inc, 0); } else { std::fill_n(&vec_res[out_offset], offset_inc, 0); @@ -463,7 +462,7 @@ ColumnPtr convertToIPv6(const StringColumnType& string_column, col_res->insert_data(reinterpret_cast(res_value), IPV6_BINARY_LENGTH); } } else { - if (exception_mode == IPStringToNumExceptionMode::Throw) { + if (exception_mode == IPExceptionMode::Throw) { throw Exception(ErrorCode::INVALID_ARGUMENT, "Invalid IPv6 value"); } std::fill_n(&vec_res[out_offset], offset_inc, 0); @@ -471,21 +470,21 @@ ColumnPtr convertToIPv6(const StringColumnType& string_column, auto* column_string = assert_cast(col_res.get()); column_string->get_offsets().push_back((i + 1) * IPV6_BINARY_LENGTH); } - if constexpr (exception_mode == IPStringToNumExceptionMode::Null) { + if constexpr (exception_mode == IPExceptionMode::Null) { (*vec_null_map_to)[i] = true; } } src_offset = src_next_offset; } - if constexpr (exception_mode == IPStringToNumExceptionMode::Null) { + if constexpr (exception_mode == IPExceptionMode::Null) { return ColumnNullable::create(std::move(col_res), std::move(col_null_map_to)); } return col_res; } } // namespace detail -template +template ColumnPtr convertToIPv6(ColumnPtr column, const PaddedPODArray* null_map = nullptr) { if (const auto* column_input_string = check_and_get_column(column.get())) { auto result = @@ -497,14 +496,14 @@ ColumnPtr convertToIPv6(ColumnPtr column, const PaddedPODArray* null_map } } -template +template class FunctionIPv6StringToNum : public IFunction { public: - static constexpr auto name = exception_mode == IPStringToNumExceptionMode::Throw - ? "ipv6_string_to_num" - : (exception_mode == IPStringToNumExceptionMode::Default - ? "ipv6_string_to_num_or_default" - : "ipv6_string_to_num_or_null"); + static constexpr auto name = + exception_mode == IPExceptionMode::Throw + ? "ipv6_string_to_num" + : (exception_mode == IPExceptionMode::Default ? "ipv6_string_to_num_or_default" + : "ipv6_string_to_num_or_null"); static FunctionPtr create() { return std::make_shared>(); @@ -525,7 +524,7 @@ class FunctionIPv6StringToNum : public IFunction { auto result_type = std::make_shared(); - if constexpr (exception_mode == IPStringToNumExceptionMode::Null) { + if constexpr (exception_mode == IPExceptionMode::Null) { return make_nullable(result_type); } @@ -541,7 +540,7 @@ class FunctionIPv6StringToNum : public IFunction { if (column->is_nullable()) { const auto* column_nullable = assert_cast(column.get()); column = column_nullable->get_nested_column_ptr(); - if constexpr (exception_mode == IPStringToNumExceptionMode::Null) { + if constexpr (exception_mode == IPExceptionMode::Null) { null_map_column = column_nullable->get_null_map_column_ptr(); null_map = &column_nullable->get_null_map_data(); } @@ -640,29 +639,33 @@ class FunctionIsIPAddressInRange : public IFunction { Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, size_t result, size_t input_rows_count) const override { - ColumnPtr& addr_column = block.get_by_position(arguments[0]).column; - ColumnPtr& cidr_column = block.get_by_position(arguments[1]).column; + const auto& addr_column_with_type_and_name = block.get_by_position(arguments[0]); + const auto& cidr_column_with_type_and_name = block.get_by_position(arguments[1]); + WhichDataType addr_type(addr_column_with_type_and_name.type); + WhichDataType cidr_type(cidr_column_with_type_and_name.type); + const ColumnPtr& addr_column = addr_column_with_type_and_name.column; + const ColumnPtr& cidr_column = cidr_column_with_type_and_name.column; const ColumnString* str_addr_column = nullptr; const ColumnString* str_cidr_column = nullptr; - const NullMap* nullmap_addr = nullptr; - const NullMap* nullmap_cidr = nullptr; + const NullMap* null_map_addr = nullptr; + const NullMap* null_map_cidr = nullptr; - if (addr_column->is_nullable()) { + if (addr_type.is_nullable()) { const auto* addr_column_nullable = assert_cast(addr_column.get()); str_addr_column = check_and_get_column(addr_column_nullable->get_nested_column()); - nullmap_addr = &addr_column_nullable->get_null_map_data(); + null_map_addr = &addr_column_nullable->get_null_map_data(); } else { str_addr_column = check_and_get_column(addr_column.get()); } - if (cidr_column->is_nullable()) { + if (cidr_type.is_nullable()) { const auto* cidr_column_nullable = assert_cast(cidr_column.get()); str_cidr_column = check_and_get_column(cidr_column_nullable->get_nested_column()); - nullmap_cidr = &cidr_column_nullable->get_null_map_data(); + null_map_cidr = &cidr_column_nullable->get_null_map_data(); } else { str_cidr_column = check_and_get_column(cidr_column.get()); } @@ -683,12 +686,12 @@ class FunctionIsIPAddressInRange : public IFunction { auto& col_res_data = col_res->get_data(); for (size_t i = 0; i < input_rows_count; ++i) { - if (nullmap_addr && (*nullmap_addr)[i]) { + if (null_map_addr && (*null_map_addr)[i]) { throw Exception(ErrorCode::INVALID_ARGUMENT, "The arguments of function {} must be String, not NULL", get_name()); } - if (nullmap_cidr && (*nullmap_cidr)[i]) { + if (null_map_cidr && (*null_map_cidr)[i]) { throw Exception(ErrorCode::INVALID_ARGUMENT, "The arguments of function {} must be String, not NULL", get_name()); @@ -885,4 +888,138 @@ class FunctionIsIPv4Mapped : public IFunction { } }; -} // namespace doris::vectorized +template +inline constexpr auto to_ip_func_name() { + if constexpr (std::is_same_v) { + return exception_mode == IPExceptionMode::Throw + ? "to_ipv4" + : (exception_mode == IPExceptionMode::Default ? "to_ipv4_or_default" + : "to_ipv4_or_null"); + } else { + return exception_mode == IPExceptionMode::Throw + ? "to_ipv6" + : (exception_mode == IPExceptionMode::Default ? "to_ipv6_or_default" + : "to_ipv6_or_null"); + } +} + +template +class FunctionToIP : public IFunction { + static_assert(std::is_same_v || std::is_same_v); + +public: + static constexpr auto name = to_ip_func_name(); + + static FunctionPtr create() { return std::make_shared>(); } + + String get_name() const override { return name; } + + size_t get_number_of_arguments() const override { return 1; } + + DataTypePtr get_return_type_impl(const DataTypes& arguments) const override { + if (!is_string(remove_nullable(arguments[0]))) { + throw Exception(ErrorCode::INVALID_ARGUMENT, + "Illegal type {} of argument of function {}, expected String", + arguments[0]->get_name(), get_name()); + } + + DataTypePtr result_type; + + if constexpr (std::is_same_v) { + result_type = std::make_shared(); + } else { + result_type = std::make_shared(); + } + + if constexpr (exception_mode == IPExceptionMode::Null) { + return make_nullable(result_type); + } else { + return result_type; + } + } + + bool use_default_implementation_for_nulls() const override { return false; } + + Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, + size_t result, size_t input_rows_count) const override { + const auto& addr_column_with_type_and_name = block.get_by_position(arguments[0]); + WhichDataType addr_type(addr_column_with_type_and_name.type); + const ColumnPtr& addr_column = addr_column_with_type_and_name.column; + const ColumnString* str_addr_column = nullptr; + const NullMap* addr_null_map = nullptr; + + if (addr_type.is_nullable()) { + const auto* addr_column_nullable = + assert_cast(addr_column.get()); + str_addr_column = + check_and_get_column(addr_column_nullable->get_nested_column()); + addr_null_map = &addr_column_nullable->get_null_map_data(); + } else { + str_addr_column = check_and_get_column(addr_column.get()); + } + + auto col_res = ColumnVector::create(input_rows_count, 0); + auto res_null_map = ColumnUInt8::create(input_rows_count, 0); + auto& col_res_data = col_res->get_data(); + auto& res_null_map_data = res_null_map->get_data(); + + for (size_t i = 0; i < input_rows_count; ++i) { + if (addr_null_map && (*addr_null_map)[i]) { + if constexpr (exception_mode == IPExceptionMode::Throw) { + throw Exception(ErrorCode::INVALID_ARGUMENT, + "The arguments of function {} must be String, not NULL", + get_name()); + } else if constexpr (exception_mode == IPExceptionMode::Default) { + col_res_data[i] = 0; // '0.0.0.0' or '::' + continue; + } else { + res_null_map_data[i] = 1; + continue; + } + } + + if constexpr (std::is_same_v) { + StringRef ipv4_str = str_addr_column->get_data_at(i); + IPv4 ipv4_val = 0; + if (IPv4Value::from_string(ipv4_val, ipv4_str.data, ipv4_str.size)) { + col_res_data[i] = ipv4_val; + } else { + if constexpr (exception_mode == IPExceptionMode::Throw) { + throw Exception(ErrorCode::INVALID_ARGUMENT, "Invalid IPv4 value '{}'", + ipv4_str.to_string_view()); + } else if constexpr (exception_mode == IPExceptionMode::Default) { + col_res_data[i] = 0; // '0.0.0.0' + } else { + res_null_map_data[i] = 1; + } + } + } else { + StringRef ipv6_str = str_addr_column->get_data_at(i); + IPv6 ipv6_val = 0; + if (IPv6Value::from_string(ipv6_val, ipv6_str.data, ipv6_str.size)) { + col_res_data[i] = ipv6_val; + } else { + if constexpr (exception_mode == IPExceptionMode::Throw) { + throw Exception(ErrorCode::INVALID_ARGUMENT, "Invalid IPv6 value '{}'", + ipv6_str.to_string_view()); + } else if constexpr (exception_mode == IPExceptionMode::Default) { + col_res_data[i] = 0; // '::' + } else if constexpr (exception_mode == IPExceptionMode::Null) { + res_null_map_data[i] = 1; + } + } + } + } + + if constexpr (exception_mode == IPExceptionMode::Null) { + block.replace_by_position( + result, ColumnNullable::create(std::move(col_res), std::move(res_null_map))); + } else { + block.replace_by_position(result, std::move(col_res)); + } + + return Status::OK(); + } +}; + +} // namespace doris::vectorized \ No newline at end of file diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java index 492c561f0a1e28..1a40365b523305 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java @@ -393,6 +393,12 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.ToDate; import org.apache.doris.nereids.trees.expressions.functions.scalar.ToDateV2; import org.apache.doris.nereids.trees.expressions.functions.scalar.ToDays; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ToIpv4; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ToIpv4OrDefault; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ToIpv4OrNull; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ToIpv6; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ToIpv6OrDefault; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ToIpv6OrNull; import org.apache.doris.nereids.trees.expressions.functions.scalar.ToMonday; import org.apache.doris.nereids.trees.expressions.functions.scalar.ToQuantileState; import org.apache.doris.nereids.trees.expressions.functions.scalar.Tokenize; @@ -623,6 +629,12 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(IsIpv6String.class, "is_ipv6_string"), scalar(IsIpAddressInRange.class, "is_ip_address_in_range"), scalar(Ipv6CIDRToRange.class, "ipv6_cidr_to_range"), + scalar(ToIpv4.class, "to_ipv4"), + scalar(ToIpv4OrDefault.class, "to_ipv4_or_default"), + scalar(ToIpv4OrNull.class, "to_ipv4_or_null"), + scalar(ToIpv6.class, "to_ipv6"), + scalar(ToIpv6OrDefault.class, "to_ipv6_or_default"), + scalar(ToIpv6OrNull.class, "to_ipv6_or_null"), scalar(JsonArray.class, "json_array"), scalar(JsonObject.class, "json_object"), scalar(JsonQuote.class, "json_quote"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv4.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv4.java new file mode 100644 index 00000000000000..590c16a6000cce --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv4.java @@ -0,0 +1,67 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.IPv4Type; +import org.apache.doris.nereids.types.StringType; +import org.apache.doris.nereids.types.VarcharType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * scalar function to_ipv4 + */ +public class ToIpv4 extends ScalarFunction + implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(IPv4Type.INSTANCE).args(VarcharType.SYSTEM_DEFAULT), + FunctionSignature.ret(IPv4Type.INSTANCE).args(StringType.INSTANCE)); + + public ToIpv4(Expression arg0) { + super("to_ipv4", arg0); + } + + @Override + public ToIpv4 withChildren(List children) { + Preconditions.checkArgument(children.size() == 1, + "to_ipv4 accept 1 args, but got %s (%s)", + children.size(), + children); + return new ToIpv4(children.get(0)); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitToIpv4(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv4OrDefault.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv4OrDefault.java new file mode 100644 index 00000000000000..e77e8120d714f2 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv4OrDefault.java @@ -0,0 +1,67 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.IPv4Type; +import org.apache.doris.nereids.types.StringType; +import org.apache.doris.nereids.types.VarcharType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * scalar function to_ipv4_or_default + */ +public class ToIpv4OrDefault extends ScalarFunction + implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(IPv4Type.INSTANCE).args(VarcharType.SYSTEM_DEFAULT), + FunctionSignature.ret(IPv4Type.INSTANCE).args(StringType.INSTANCE)); + + public ToIpv4OrDefault(Expression arg0) { + super("to_ipv4_or_default", arg0); + } + + @Override + public ToIpv4OrDefault withChildren(List children) { + Preconditions.checkArgument(children.size() == 1, + "to_ipv4_or_default accept 1 args, but got %s (%s)", + children.size(), + children); + return new ToIpv4OrDefault(children.get(0)); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitToIpv4OrDefault(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv4OrNull.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv4OrNull.java new file mode 100644 index 00000000000000..940ae4c397182e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv4OrNull.java @@ -0,0 +1,67 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.IPv4Type; +import org.apache.doris.nereids.types.StringType; +import org.apache.doris.nereids.types.VarcharType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * scalar function to_ipv4_or_null + */ +public class ToIpv4OrNull extends ScalarFunction + implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(IPv4Type.INSTANCE).args(VarcharType.SYSTEM_DEFAULT), + FunctionSignature.ret(IPv4Type.INSTANCE).args(StringType.INSTANCE)); + + public ToIpv4OrNull(Expression arg0) { + super("to_ipv4_or_null", arg0); + } + + @Override + public ToIpv4OrNull withChildren(List children) { + Preconditions.checkArgument(children.size() == 1, + "to_ipv4_or_null accept 1 args, but got %s (%s)", + children.size(), + children); + return new ToIpv4OrNull(children.get(0)); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitToIpv4OrNull(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv6.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv6.java new file mode 100644 index 00000000000000..4261862147b891 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv6.java @@ -0,0 +1,67 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.IPv6Type; +import org.apache.doris.nereids.types.StringType; +import org.apache.doris.nereids.types.VarcharType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * scalar function to_ipv6 + */ +public class ToIpv6 extends ScalarFunction + implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(IPv6Type.INSTANCE).args(VarcharType.SYSTEM_DEFAULT), + FunctionSignature.ret(IPv6Type.INSTANCE).args(StringType.INSTANCE)); + + public ToIpv6(Expression arg0) { + super("to_ipv6", arg0); + } + + @Override + public ToIpv6 withChildren(List children) { + Preconditions.checkArgument(children.size() == 1, + "to_ipv6 accept 1 args, but got %s (%s)", + children.size(), + children); + return new ToIpv6(children.get(0)); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitToIpv6(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv6OrDefault.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv6OrDefault.java new file mode 100644 index 00000000000000..da9ec1083b034e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv6OrDefault.java @@ -0,0 +1,67 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.IPv6Type; +import org.apache.doris.nereids.types.StringType; +import org.apache.doris.nereids.types.VarcharType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * scalar function to_ipv6_or_default + */ +public class ToIpv6OrDefault extends ScalarFunction + implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(IPv6Type.INSTANCE).args(VarcharType.SYSTEM_DEFAULT), + FunctionSignature.ret(IPv6Type.INSTANCE).args(StringType.INSTANCE)); + + public ToIpv6OrDefault(Expression arg0) { + super("to_ipv6_or_default", arg0); + } + + @Override + public ToIpv6OrDefault withChildren(List children) { + Preconditions.checkArgument(children.size() == 1, + "to_ipv6_or_default accept 1 args, but got %s (%s)", + children.size(), + children); + return new ToIpv6OrDefault(children.get(0)); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitToIpv6OrDefault(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv6OrNull.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv6OrNull.java new file mode 100644 index 00000000000000..6a3a3e0973c0ee --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToIpv6OrNull.java @@ -0,0 +1,67 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.IPv6Type; +import org.apache.doris.nereids.types.StringType; +import org.apache.doris.nereids.types.VarcharType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * scalar function to_ipv6_or_null + */ +public class ToIpv6OrNull extends ScalarFunction + implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(IPv6Type.INSTANCE).args(VarcharType.SYSTEM_DEFAULT), + FunctionSignature.ret(IPv6Type.INSTANCE).args(StringType.INSTANCE)); + + public ToIpv6OrNull(Expression arg0) { + super("to_ipv6_or_null", arg0); + } + + @Override + public ToIpv6OrNull withChildren(List children) { + Preconditions.checkArgument(children.size() == 1, + "to_ipv6_or_null accept 1 args, but got %s (%s)", + children.size(), + children); + return new ToIpv6OrNull(children.get(0)); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitToIpv6OrNull(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java index 8629d15b2bcfbb..d10c384c55e14a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java @@ -383,6 +383,12 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.ToDate; import org.apache.doris.nereids.trees.expressions.functions.scalar.ToDateV2; import org.apache.doris.nereids.trees.expressions.functions.scalar.ToDays; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ToIpv4; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ToIpv4OrDefault; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ToIpv4OrNull; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ToIpv6; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ToIpv6OrDefault; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ToIpv6OrNull; import org.apache.doris.nereids.trees.expressions.functions.scalar.ToMonday; import org.apache.doris.nereids.trees.expressions.functions.scalar.ToQuantileState; import org.apache.doris.nereids.trees.expressions.functions.scalar.Tokenize; @@ -1196,6 +1202,30 @@ default R visitIpv6CIDRToRange(Ipv6CIDRToRange ipv6CIDRToRange, C context) { return visitScalarFunction(ipv6CIDRToRange, context); } + default R visitToIpv4(ToIpv4 toIpv4, C context) { + return visitScalarFunction(toIpv4, context); + } + + default R visitToIpv4OrDefault(ToIpv4OrDefault toIpv4OrDefault, C context) { + return visitScalarFunction(toIpv4OrDefault, context); + } + + default R visitToIpv4OrNull(ToIpv4OrNull toIpv4OrNull, C context) { + return visitScalarFunction(toIpv4OrNull, context); + } + + default R visitToIpv6(ToIpv6 toIpv6, C context) { + return visitScalarFunction(toIpv6, context); + } + + default R visitToIpv6OrDefault(ToIpv6OrDefault toIpv6OrDefault, C context) { + return visitScalarFunction(toIpv6OrDefault, context); + } + + default R visitToIpv6OrNull(ToIpv6OrNull toIpv6OrNull, C context) { + return visitScalarFunction(toIpv6OrNull, context); + } + default R visitJsonArray(JsonArray jsonArray, C context) { return visitScalarFunction(jsonArray, context); } diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py index a9d2fe26dc2cb7..9faf1e092ec4a7 100644 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -2035,6 +2035,18 @@ [['is_ip_address_in_range'], 'BOOLEAN', ['STRING', 'STRING'], 'ALWAYS_NOT_NULLABLE'], [['ipv6_cidr_to_range'], 'STRUCT', ['VARCHAR', 'SMALLINT'], ''], [['ipv6_cidr_to_range'], 'STRUCT', ['STRING', 'SMALLINT'], ''], + [['to_ipv4'], 'IPV4', ['VARCHAR'], 'ALWAYS_NOT_NULLABLE'], + [['to_ipv4'], 'IPV4', ['STRING'], 'ALWAYS_NOT_NULLABLE'], + [['to_ipv4_or_default'], 'IPV4', ['VARCHAR'], 'ALWAYS_NOT_NULLABLE'], + [['to_ipv4_or_default'], 'IPV4', ['STRING'], 'ALWAYS_NOT_NULLABLE'], + [['to_ipv4_or_null'], 'IPV4', ['VARCHAR'], 'ALWAYS_NULLABLE'], + [['to_ipv4_or_null'], 'IPV4', ['STRING'], 'ALWAYS_NULLABLE'], + [['to_ipv6'], 'IPV6', ['VARCHAR'], 'ALWAYS_NOT_NULLABLE'], + [['to_ipv6'], 'IPV6', ['STRING'], 'ALWAYS_NOT_NULLABLE'], + [['to_ipv6_or_default'], 'IPV6', ['VARCHAR'], 'ALWAYS_NOT_NULLABLE'], + [['to_ipv6_or_default'], 'IPV6', ['STRING'], 'ALWAYS_NOT_NULLABLE'], + [['to_ipv6_or_null'], 'IPV6', ['VARCHAR'], 'ALWAYS_NULLABLE'], + [['to_ipv6_or_null'], 'IPV6', ['STRING'], 'ALWAYS_NULLABLE'], ], "NonNullalbe": [ diff --git a/regression-test/data/query_p0/sql_functions/ip_functions/test_to_ip_functions.out b/regression-test/data/query_p0/sql_functions/ip_functions/test_to_ip_functions.out new file mode 100644 index 00000000000000..8015d0eef300a7 --- /dev/null +++ b/regression-test/data/query_p0/sql_functions/ip_functions/test_to_ip_functions.out @@ -0,0 +1,21 @@ +-- !sql -- +0.0.0.0 :: +192.168.0.1 ::1 +127.0.0.1 2001:1b70:a1:610::b102:2 +255.255.255.255 ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff + +-- !sql -- +0.0.0.0 :: +0.0.0.0 :: +192.168.0.1 ::1 +127.0.0.1 2001:1b70:a1:610::b102:2 +255.255.255.255 ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff +0.0.0.0 :: + +-- !sql -- +\N \N +0.0.0.0 :: +192.168.0.1 ::1 +127.0.0.1 2001:1b70:a1:610::b102:2 +255.255.255.255 ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff +\N \N diff --git a/regression-test/suites/query_p0/sql_functions/ip_functions/test_to_ip_functions.groovy b/regression-test/suites/query_p0/sql_functions/ip_functions/test_to_ip_functions.groovy new file mode 100644 index 00000000000000..aefe338b9599d2 --- /dev/null +++ b/regression-test/suites/query_p0/sql_functions/ip_functions/test_to_ip_functions.groovy @@ -0,0 +1,49 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. +suite("test_to_ip_functions") { + sql """ DROP TABLE IF EXISTS test_to_ip_functions """ + + sql """ SET enable_nereids_planner=true """ + sql """ SET enable_fallback_to_original_planner=false """ + + sql """ + CREATE TABLE `test_to_ip_functions` ( + `id` int, + `ip_v4` string, + `ip_v6` string + ) ENGINE=OLAP + DISTRIBUTED BY HASH(`id`) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + sql """ + insert into test_to_ip_functions values + (0, NULL, NULL), + (1, '0.0.0.0', '::'), + (2, '192.168.0.1', '::1'), + (3, '127.0.0.1', '2001:1b70:a1:610::b102:2'), + (4, '255.255.255.255', 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'), + (5, '255.255.255.256', 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffg') + """ + + qt_sql "select to_ipv4(ip_v4), to_ipv6(ip_v6) from test_to_ip_functions where id in (1, 2, 3, 4) order by id" + qt_sql "select to_ipv4_or_default(ip_v4), to_ipv6_or_default(ip_v6) from test_to_ip_functions order by id" + qt_sql "select to_ipv4_or_null(ip_v4), to_ipv6_or_null(ip_v6) from test_to_ip_functions order by id" + + sql "DROP TABLE test_to_ip_functions" +} \ No newline at end of file From b93b7d2b9b217f9680624310bb1c7759fd955f31 Mon Sep 17 00:00:00 2001 From: zclllyybb Date: Mon, 22 Jan 2024 17:46:26 +0800 Subject: [PATCH 134/200] [fix](auto-partition) Fix a concurrent bug (#30086) when incremental opening, it may be a mistake to send new packet from sender to reciever cuz' re-constructed send closure. now fixed it. --- be/src/runtime/tablets_channel.cpp | 2 +- be/src/vec/sink/writer/vtablet_writer.cpp | 51 +++++--- be/src/vec/sink/writer/vtablet_writer.h | 4 +- .../auto_partition/ddl/stress_destination.sql | 20 +++ .../auto_partition/ddl/stress_source.sql | 19 +++ .../stress_test_insert_into.yaml | 21 ++++ .../sql/multi_thread_load.groovy | 3 +- .../sql/stress_test_insert_into.groovy | 118 ++++++++++++++++++ 8 files changed, 215 insertions(+), 23 deletions(-) create mode 100644 regression-test/suites/partition_p1/auto_partition/ddl/stress_destination.sql create mode 100644 regression-test/suites/partition_p1/auto_partition/ddl/stress_source.sql create mode 100644 regression-test/suites/partition_p1/auto_partition/doris_dbgen_conf/stress_test_insert_into.yaml create mode 100644 regression-test/suites/partition_p1/auto_partition/sql/stress_test_insert_into.groovy diff --git a/be/src/runtime/tablets_channel.cpp b/be/src/runtime/tablets_channel.cpp index 7ead68d916f9bf..d248ad09d7af1a 100644 --- a/be/src/runtime/tablets_channel.cpp +++ b/be/src/runtime/tablets_channel.cpp @@ -152,7 +152,7 @@ Status BaseTabletsChannel::open(const PTabletWriterOpenRequest& request) { Status BaseTabletsChannel::incremental_open(const PTabletWriterOpenRequest& params) { SCOPED_TIMER(_incremental_open_timer); if (_state == kInitialized) { // haven't opened - return open(params); + RETURN_IF_ERROR(open(params)); } std::lock_guard l(_lock); std::vector* index_slots = nullptr; diff --git a/be/src/vec/sink/writer/vtablet_writer.cpp b/be/src/vec/sink/writer/vtablet_writer.cpp index 0ca2f7d8ffb45d..8eb1c4d743a530 100644 --- a/be/src/vec/sink/writer/vtablet_writer.cpp +++ b/be/src/vec/sink/writer/vtablet_writer.cpp @@ -310,6 +310,10 @@ void VNodeChannel::clear_all_blocks() { // no need to set _cancel_msg because the error will be // returned directly via "TabletSink::prepare()" method. Status VNodeChannel::init(RuntimeState* state) { + if (_inited) { + return Status::OK(); + } + SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker.get()); _task_exec_ctx = state->get_task_execution_context(); _tuple_desc = _parent->_output_tuple_desc; @@ -348,6 +352,25 @@ Status VNodeChannel::init(RuntimeState* state) { _cur_add_block_request->set_backend_id(_node_id); _cur_add_block_request->set_eos(false); + // add block closure + _send_block_callback = WriteBlockCallback::create_shared(); + _send_block_callback->addFailedHandler([this](bool is_last_rpc) { + auto ctx_lock = _task_exec_ctx.lock(); + if (ctx_lock == nullptr) { + return; + } + _add_block_failed_callback(is_last_rpc); + }); + + _send_block_callback->addSuccessHandler( + [this](const PTabletWriterAddBlockResult& result, bool is_last_rpc) { + auto ctx_lock = _task_exec_ctx.lock(); + if (ctx_lock == nullptr) { + return; + } + _add_block_success_callback(result, is_last_rpc); + }); + _name = fmt::format("VNodeChannel[{}-{}]", _index_channel->_index_id, _node_id); // The node channel will send _batch_size rows of data each rpc. When the // number of tablets is large, the number of data rows received by each @@ -356,18 +379,23 @@ Status VNodeChannel::init(RuntimeState* state) { // a relatively large value to improve the import performance. _batch_size = std::max(_batch_size, 8192); + _inited = true; return Status::OK(); } void VNodeChannel::_open_internal(bool is_incremental) { + if (_tablets_wait_open.empty()) { + return; + } SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker.get()); auto request = std::make_shared(); request->set_allocated_id(&_parent->_load_id); request->set_index_id(_index_channel->_index_id); request->set_txn_id(_parent->_txn_id); request->set_allocated_schema(_parent->_schema->to_protobuf()); + std::set deduper; - for (auto& tablet : _all_tablets) { + for (auto& tablet : _tablets_wait_open) { if (deduper.contains(tablet.tablet_id)) { continue; } @@ -375,7 +403,10 @@ void VNodeChannel::_open_internal(bool is_incremental) { ptablet->set_partition_id(tablet.partition_id); ptablet->set_tablet_id(tablet.tablet_id); deduper.insert(tablet.tablet_id); + _all_tablets.push_back(std::move(tablet)); } + _tablets_wait_open.clear(); + request->set_num_senders(_parent->_num_senders); request->set_need_gen_rollup(false); // Useless but it is a required field in pb request->set_load_mem_limit(_parent->_load_mem_limit); @@ -444,24 +475,6 @@ Status VNodeChannel::open_wait() { } } - // add block closure - _send_block_callback = WriteBlockCallback::create_shared(); - _send_block_callback->addFailedHandler([this](bool is_last_rpc) { - auto ctx_lock = _task_exec_ctx.lock(); - if (ctx_lock == nullptr) { - return; - } - _add_block_failed_callback(is_last_rpc); - }); - - _send_block_callback->addSuccessHandler( - [this](const PTabletWriterAddBlockResult& result, bool is_last_rpc) { - auto ctx_lock = _task_exec_ctx.lock(); - if (ctx_lock == nullptr) { - return; - } - _add_block_success_callback(result, is_last_rpc); - }); return status; } diff --git a/be/src/vec/sink/writer/vtablet_writer.h b/be/src/vec/sink/writer/vtablet_writer.h index cd2eafb1f24b19..68534cd3a16ec5 100644 --- a/be/src/vec/sink/writer/vtablet_writer.h +++ b/be/src/vec/sink/writer/vtablet_writer.h @@ -219,7 +219,7 @@ class VNodeChannel { ~VNodeChannel(); // called before open, used to add tablet located in this backend. called by IndexChannel::init - void add_tablet(const TTabletWithPartition& tablet) { _all_tablets.emplace_back(tablet); } + void add_tablet(const TTabletWithPartition& tablet) { _tablets_wait_open.emplace_back(tablet); } std::string debug_tablets() const { std::stringstream ss; for (const auto& tab : _all_tablets) { @@ -368,6 +368,7 @@ class VNodeChannel { std::vector>> _open_callbacks; std::vector _all_tablets; + std::vector _tablets_wait_open; // map from tablet_id to node_id where slave replicas locate in std::unordered_map> _slave_tablet_nodes; std::vector _tablet_commit_infos; @@ -388,6 +389,7 @@ class VNodeChannel { // The IndexChannel is definitely accessible until the NodeChannel is closed. std::mutex _closed_lock; bool _is_closed = false; + bool _inited = false; RuntimeState* _state = nullptr; // A context lock for callbacks, the callback has to lock the ctx, to avoid diff --git a/regression-test/suites/partition_p1/auto_partition/ddl/stress_destination.sql b/regression-test/suites/partition_p1/auto_partition/ddl/stress_destination.sql new file mode 100644 index 00000000000000..f1dbb790987a8d --- /dev/null +++ b/regression-test/suites/partition_p1/auto_partition/ddl/stress_destination.sql @@ -0,0 +1,20 @@ +CREATE TABLE `stress_destination` ( + `create_date` DATE NOT NULL, + `parent_org_id` VARCHAR(96) NULL, + `org_id` VARCHAR(100) NULL, + `org_name` VARCHAR(192) NULL, + `create_month` VARCHAR(11) NOT NULL, + `org_type` VARCHAR(192) NULL, + `sms_total` INT NULL, + `success_sms_total` INT NULL, + `sms_price_total` DOUBLE NULL, + `sms_total_sum` INT NULL, + `has_sub` INT NULL, + `order_num` INT NULL +) ENGINE=OLAP +UNIQUE KEY(`create_date`, `parent_org_id`, `org_id`) +AUTO PARTITION BY RANGE date_trunc(`create_date`,'day')() +DISTRIBUTED BY HASH(`create_date`, `org_id`) BUCKETS AUTO +PROPERTIES ( +"replication_allocation" = "tag.location.default: 1" +); diff --git a/regression-test/suites/partition_p1/auto_partition/ddl/stress_source.sql b/regression-test/suites/partition_p1/auto_partition/ddl/stress_source.sql new file mode 100644 index 00000000000000..d095e4119c7003 --- /dev/null +++ b/regression-test/suites/partition_p1/auto_partition/ddl/stress_source.sql @@ -0,0 +1,19 @@ +CREATE TABLE `stress_source` ( + `create_date` DATE NOT NULL, + `parent_org_id` VARCHAR(96) NULL, + `org_id` VARCHAR(100) NULL, + `org_name` VARCHAR(192) NULL, + `create_month` VARCHAR(11) NOT NULL, + `org_type` VARCHAR(192) NULL, + `sms_total` INT NULL, + `success_sms_total` INT NULL, + `sms_price_total` DOUBLE NULL, + `sms_total_sum` INT NULL, + `has_sub` INT NULL, + `order_num` INT NULL +) ENGINE=OLAP +UNIQUE KEY(`create_date`, `parent_org_id`, `org_id`) +DISTRIBUTED BY HASH(`create_date`, `org_id`) BUCKETS AUTO +PROPERTIES ( +"replication_allocation" = "tag.location.default: 1" +); diff --git a/regression-test/suites/partition_p1/auto_partition/doris_dbgen_conf/stress_test_insert_into.yaml b/regression-test/suites/partition_p1/auto_partition/doris_dbgen_conf/stress_test_insert_into.yaml new file mode 100644 index 00000000000000..a45fb01c316695 --- /dev/null +++ b/regression-test/suites/partition_p1/auto_partition/doris_dbgen_conf/stress_test_insert_into.yaml @@ -0,0 +1,21 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +tables: + stress_source: + create_date: + range: {min: "2023-07-01", max: "2024-01-10"} \ No newline at end of file diff --git a/regression-test/suites/partition_p1/auto_partition/sql/multi_thread_load.groovy b/regression-test/suites/partition_p1/auto_partition/sql/multi_thread_load.groovy index 25a9a49305d6a7..aebc022feeb0e6 100644 --- a/regression-test/suites/partition_p1/auto_partition/sql/multi_thread_load.groovy +++ b/regression-test/suites/partition_p1/auto_partition/sql/multi_thread_load.groovy @@ -20,9 +20,8 @@ import java.nio.file.Files import java.nio.file.Paths import java.net.URL import java.io.File -import java.util.concurrent.locks.ReentrantLock -suite("multi_thread_load") { +suite("multi_thread_load", "nonConcurrent") { // stress case should use resource fully // get doris-db from s3 def dirPath = context.file.parent def fatherPath = context.file.parentFile.parentFile.getPath() diff --git a/regression-test/suites/partition_p1/auto_partition/sql/stress_test_insert_into.groovy b/regression-test/suites/partition_p1/auto_partition/sql/stress_test_insert_into.groovy new file mode 100644 index 00000000000000..d2ce87248e06bd --- /dev/null +++ b/regression-test/suites/partition_p1/auto_partition/sql/stress_test_insert_into.groovy @@ -0,0 +1,118 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +import groovy.io.FileType +import java.nio.file.Files +import java.nio.file.Paths +import java.net.URL +import java.io.File + +suite("stress_test_insert_into") { + // get doris-db from s3 + def dirPath = context.file.parent + def fatherPath = context.file.parentFile.parentFile.getPath() + def fileName = "doris-dbgen" + def fileUrl = "${getS3Url()}/regression/doris-dbgen-23-10-18/doris-dbgen-23-10-20/doris-dbgen" + def filePath = Paths.get(dirPath, fileName) + if (!Files.exists(filePath)) { + new URL(fileUrl).withInputStream { inputStream -> + Files.copy(inputStream, filePath) + } + def file = new File(dirPath + "/" + fileName) + file.setExecutable(true) + } + + def rows = 100000 + + // load data via doris-dbgen + def doris_dbgen_create_data = { db_name, tb_name -> + def tableName = tb_name + + def jdbcUrl = context.config.jdbcUrl + def urlWithoutSchema = jdbcUrl.substring(jdbcUrl.indexOf("://") + 3) + def sql_ip = urlWithoutSchema.substring(0, urlWithoutSchema.indexOf(":")) + def sql_port + if (urlWithoutSchema.indexOf("/") >= 0) { + // e.g: jdbc:mysql://locahost:8080/?a=b + sql_port = urlWithoutSchema.substring(urlWithoutSchema.indexOf(":") + 1, urlWithoutSchema.indexOf("/")) + } else { + // e.g: jdbc:mysql://locahost:8080 + sql_port = urlWithoutSchema.substring(urlWithoutSchema.indexOf(":") + 1) + } + String feHttpAddress = context.config.feHttpAddress + def http_port = feHttpAddress.substring(feHttpAddress.indexOf(":") + 1) + + String realDb = db_name + String user = context.config.jdbcUser + String password = context.config.jdbcPassword + + def cm + if (password) { + cm = """ + ${dirPath}/doris-dbgen gen + --host ${sql_ip} + --sql-port ${sql_port} + --user ${user} + --password ${password} + --database ${realDb} + --table ${tableName} + --rows ${rows} + --http-port ${http_port} + --config ${fatherPath}/doris_dbgen_conf/stress_test_insert_into.yaml + """ + } else { + cm = """ + ${dirPath}/doris-dbgen gen + --host ${sql_ip} + --sql-port ${sql_port} + --user ${user} + --database ${realDb} + --table ${tableName} + --rows ${rows} + --http-port ${http_port} + --config ${fatherPath}/doris_dbgen_conf/stress_test_insert_into.yaml + """ + } + + logger.info("datagen: " + cm) + def proc = cm.execute() + def sout = new StringBuilder(), serr = new StringBuilder() + proc.consumeProcessOutput(sout, serr) + proc.waitForOrKill(1800000) + } + + def database_name = "regression_test_auto_partition_concurrent" + def table_src = "stress_source" + def table_dest = "stress_destination" + + sql """create database if not exists ${database_name};""" + sql """use ${database_name};""" + sql """drop table if exists ${table_src};""" + sql """drop table if exists ${table_dest};""" + sql new File("""${fatherPath}/ddl/stress_source.sql""").text + sql new File("""${fatherPath}/ddl/stress_destination.sql""").text + doris_dbgen_create_data(database_name, table_src) + + // TEST-BODY + def count_src = sql " select count() from ${table_src}; " + sql " insert into ${table_dest} select * from ${table_src} " + def count_dest = sql " select count() from ${table_dest}; " + // check data count + assertTrue(count_src[0][0] > 0) + assertEquals(count_src[0][0], count_dest[0][0]) + logger.info("got rows: ${count_src[0][0]}") +} From 78f8df336dfc869d960b212683b5418ee3df6eac Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Mon, 22 Jan 2024 17:53:25 +0800 Subject: [PATCH 135/200] [fix](Nereids) result nullable of sum distinct in scalar agg is wrong (#30221) --- .../AdjustAggregateNullableForEmptySet.java | 3 +- .../implementation/AggregateStrategies.java | 44 +++++++------------ .../trees/expressions/functions/agg/Sum.java | 6 +++ .../nereids_syntax_p0/agg_with_empty_set.out | 3 ++ .../agg_with_empty_set.groovy | 1 + 5 files changed, 26 insertions(+), 31 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AdjustAggregateNullableForEmptySet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AdjustAggregateNullableForEmptySet.java index 75400a8e6b0f33..86a70d35ccc087 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AdjustAggregateNullableForEmptySet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AdjustAggregateNullableForEmptySet.java @@ -90,8 +90,7 @@ public Expression visitWindow(WindowExpression windowExpression, Boolean alwaysN @Override public Expression visitNullableAggregateFunction(NullableAggregateFunction nullableAggregateFunction, Boolean alwaysNullable) { - return nullableAggregateFunction.isDistinct() ? nullableAggregateFunction - : nullableAggregateFunction.withAlwaysNullable(alwaysNullable); + return nullableAggregateFunction.withAlwaysNullable(alwaysNullable); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java index a0eb011ba92815..c9907ae7c3d89c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java @@ -50,7 +50,6 @@ import org.apache.doris.nereids.trees.expressions.functions.agg.Max; import org.apache.doris.nereids.trees.expressions.functions.agg.Min; import org.apache.doris.nereids.trees.expressions.functions.agg.MultiDistinctCount; -import org.apache.doris.nereids.trees.expressions.functions.agg.MultiDistinctSum; import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; import org.apache.doris.nereids.trees.expressions.functions.scalar.If; import org.apache.doris.nereids.trees.expressions.literal.Literal; @@ -108,9 +107,9 @@ public List buildRules() { logicalAggregate( logicalFilter( logicalOlapScan().when(this::isDupOrMowKeyTable).when(this::isInvertedIndexEnabledOnTable) - ).when(filter -> filter.getConjuncts().size() > 0)) + ).when(filter -> !filter.getConjuncts().isEmpty())) .when(agg -> enablePushDownCountOnIndex()) - .when(agg -> agg.getGroupByExpressions().size() == 0) + .when(agg -> agg.getGroupByExpressions().isEmpty()) .when(agg -> { Set funcs = agg.getAggregateFunctions(); return !funcs.isEmpty() && funcs.stream() @@ -128,9 +127,9 @@ public List buildRules() { logicalProject( logicalFilter( logicalOlapScan().when(this::isDupOrMowKeyTable).when(this::isInvertedIndexEnabledOnTable) - ).when(filter -> filter.getConjuncts().size() > 0))) + ).when(filter -> !filter.getConjuncts().isEmpty()))) .when(agg -> enablePushDownCountOnIndex()) - .when(agg -> agg.getGroupByExpressions().size() == 0) + .when(agg -> agg.getGroupByExpressions().isEmpty()) .when(agg -> { Set funcs = agg.getAggregateFunctions(); return !funcs.isEmpty() && funcs.stream().allMatch(f -> f instanceof Count && !f.isDistinct()); @@ -154,7 +153,7 @@ public List buildRules() { Expression childExpr = filter.getConjuncts().iterator().next().children().get(0); if (childExpr instanceof SlotReference) { Optional column = ((SlotReference) childExpr).getColumn(); - return column.isPresent() ? column.get().isDeleteSignColumn() : false; + return column.map(Column::isDeleteSignColumn).orElse(false); } return false; }) @@ -187,8 +186,7 @@ public List buildRules() { .children().get(0); if (childExpr instanceof SlotReference) { Optional column = ((SlotReference) childExpr).getColumn(); - return column.isPresent() ? column.get().isDeleteSignColumn() - : false; + return column.map(Column::isDeleteSignColumn).orElse(false); } return false; })) @@ -253,12 +251,12 @@ public List buildRules() { ), RuleType.ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT.build( basePattern - .when(agg -> agg.getDistinctArguments().size() == 0) + .when(agg -> agg.getDistinctArguments().isEmpty()) .thenApplyMulti(ctx -> onePhaseAggregateWithoutDistinct(ctx.root, ctx.connectContext)) ), RuleType.TWO_PHASE_AGGREGATE_WITHOUT_DISTINCT.build( basePattern - .when(agg -> agg.getDistinctArguments().size() == 0) + .when(agg -> agg.getDistinctArguments().isEmpty()) .thenApplyMulti(ctx -> twoPhaseAggregateWithoutDistinct(ctx.root, ctx.connectContext)) ), // RuleType.TWO_PHASE_AGGREGATE_WITH_COUNT_DISTINCT_MULTI.build( @@ -435,12 +433,7 @@ private boolean checkWhetherPushDownMinMax(Set aggregateFunct boolean onlyContainsSlotOrNumericCastSlot = aggregateFunctions.stream() .map(ExpressionTrait::getArguments) .flatMap(List::stream) - .allMatch(argument -> { - if (argument instanceof SlotReference) { - return true; - } - return false; - }); + .allMatch(argument -> argument instanceof SlotReference); if (!onlyContainsSlotOrNumericCastSlot) { return false; } @@ -457,19 +450,13 @@ private boolean checkWhetherPushDownMinMax(Set aggregateFunct } onlyContainsSlotOrNumericCastSlot = argumentsOfAggregateFunction .stream() - .allMatch(argument -> { - if (argument instanceof SlotReference) { - return true; - } - return false; - }); + .allMatch(argument -> argument instanceof SlotReference); if (!onlyContainsSlotOrNumericCastSlot) { return false; } Set aggUsedSlots = ExpressionUtils.collect(argumentsOfAggregateFunction, SlotReference.class::isInstance); - List usedSlotInTable = (List) Project.findProject(aggUsedSlots, - outPutSlots); + List usedSlotInTable = (List) Project.findProject(aggUsedSlots, outPutSlots); for (SlotReference slot : usedSlotInTable) { Column column = slot.getColumn().get(); PrimitiveType colType = column.getType().getPrimitiveType(); @@ -630,7 +617,7 @@ private LogicalAggregate storageLayerAggregate( if (logicalScan instanceof LogicalOlapScan) { PhysicalOlapScan physicalScan = (PhysicalOlapScan) new LogicalOlapScanToPhysicalOlapScan() .build() - .transform((LogicalOlapScan) logicalScan, cascadesContext) + .transform(logicalScan, cascadesContext) .get(0); if (project != null) { @@ -647,7 +634,7 @@ private LogicalAggregate storageLayerAggregate( } else if (logicalScan instanceof LogicalFileScan) { PhysicalFileScan physicalScan = (PhysicalFileScan) new LogicalFileScanToPhysicalFileScan() .build() - .transform((LogicalFileScan) logicalScan, cascadesContext) + .transform(logicalScan, cascadesContext) .get(0); if (project != null) { return aggregate.withChildren(ImmutableList.of( @@ -1193,8 +1180,7 @@ private List> twoPhaseAggregateWithDistinc return new AggregateExpression(nonDistinct, AggregateParam.LOCAL_RESULT); } else { Alias alias = nonDistinctAggFunctionToAliasPhase1.get(outputChild); - return new AggregateExpression( - aggregateFunction, bufferToResultParam, alias.toSlot()); + return new AggregateExpression(aggregateFunction, bufferToResultParam, alias.toSlot()); } } else { return outputChild; @@ -1582,7 +1568,7 @@ private AggregateFunction tryConvertToMultiDistinct(AggregateFunction function) return new MultiDistinctCount(function.getArgument(0), function.getArguments().subList(1, function.arity()).toArray(new Expression[0])); } else if (function instanceof Sum && function.isDistinct()) { - return new MultiDistinctSum(function.getArgument(0)); + return ((Sum) function).convertToMultiDistinct(); } else if (function instanceof GroupConcat && function.isDistinct()) { return ((GroupConcat) function).convertToMultiDistinct(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum.java index f0dbd839583857..0b00536d6a8826 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum.java @@ -78,6 +78,12 @@ public Sum(boolean distinct, boolean alwaysNullable, Expression arg) { super("sum", distinct, alwaysNullable, arg); } + public MultiDistinctSum convertToMultiDistinct() { + Preconditions.checkArgument(distinct, + "can't convert to multi_distinct_sum because there is no distinct args"); + return new MultiDistinctSum(false, alwaysNullable, child()); + } + @Override public void checkLegalityBeforeTypeCoercion() { DataType argType = child().getDataType(); diff --git a/regression-test/data/nereids_syntax_p0/agg_with_empty_set.out b/regression-test/data/nereids_syntax_p0/agg_with_empty_set.out index 1db851c70c12b8..ffe8f93eb58e1e 100644 --- a/regression-test/data/nereids_syntax_p0/agg_with_empty_set.out +++ b/regression-test/data/nereids_syntax_p0/agg_with_empty_set.out @@ -18,3 +18,6 @@ -- !select6 -- 0 \N \N \N \N +-- !ditinct_sum -- +\N + diff --git a/regression-test/suites/nereids_syntax_p0/agg_with_empty_set.groovy b/regression-test/suites/nereids_syntax_p0/agg_with_empty_set.groovy index 5fc117445a4d46..bdcb4526a2cd1d 100644 --- a/regression-test/suites/nereids_syntax_p0/agg_with_empty_set.groovy +++ b/regression-test/suites/nereids_syntax_p0/agg_with_empty_set.groovy @@ -29,4 +29,5 @@ suite("agg_with_empty_set") { (select min(c_custkey) from customer)""" qt_select6 """select count(c_custkey), max(c_custkey), min(c_custkey), avg(c_custkey), sum(c_custkey) from customer where c_custkey < (select min(c_custkey) from customer) having min(c_custkey) is null""" + qt_ditinct_sum """select sum(distinct ifnull(c_custkey, 0)) from customer where 1 = 0""" } \ No newline at end of file From c929d26996e76e6bb41ca1117b2efbd9d0a18c45 Mon Sep 17 00:00:00 2001 From: zclllyybb Date: Mon, 22 Jan 2024 18:04:22 +0800 Subject: [PATCH 136/200] [Fix](Expr&code-style) check prepare&open before every VExpr execute (#26673) --- be/src/exec/exec_node.cpp | 52 +++++++-------- be/src/exec/exec_node.h | 19 ++---- .../exec/multi_cast_data_stream_source.cpp | 8 +-- be/src/pipeline/exec/operator.h | 18 ++---- be/src/pipeline/exec/set_sink_operator.cpp | 16 ++--- be/src/pipeline/pipeline_fragment_context.cpp | 26 ++++---- be/src/pipeline/pipeline_fragment_context.h | 4 +- be/src/service/point_query_executor.cpp | 10 +-- be/src/vec/columns/column_nullable.cpp | 6 +- be/src/vec/columns/column_nullable.h | 4 +- be/src/vec/exec/join/vjoin_node_base.h | 19 +++--- be/src/vec/exec/vset_operation_node.cpp | 7 +-- be/src/vec/exec/vset_operation_node.h | 7 +-- be/src/vec/exec/vunion_node.cpp | 21 ++----- be/src/vec/exprs/vbitmap_predicate.cpp | 4 ++ be/src/vec/exprs/vbloom_predicate.cpp | 4 ++ be/src/vec/exprs/vcase_expr.cpp | 9 ++- be/src/vec/exprs/vcast_expr.cpp | 4 ++ be/src/vec/exprs/vcolumn_ref.h | 10 +++ be/src/vec/exprs/vdirect_in_predicate.h | 19 +++++- be/src/vec/exprs/vectorized_fn_call.cpp | 19 +++--- be/src/vec/exprs/vexpr.cpp | 62 +++++++++--------- be/src/vec/exprs/vexpr.h | 63 ++++++++++++------- be/src/vec/exprs/vexpr_context.cpp | 28 +++------ be/src/vec/exprs/vexpr_context.h | 15 ++--- be/src/vec/exprs/vin_predicate.cpp | 4 ++ be/src/vec/exprs/vlambda_function_call_expr.h | 12 +++- be/src/vec/exprs/vlambda_function_expr.h | 15 +++++ be/src/vec/exprs/vliteral.cpp | 19 ++++-- be/src/vec/exprs/vliteral.h | 5 ++ be/src/vec/exprs/vmatch_predicate.cpp | 5 +- be/src/vec/exprs/vruntimefilter_wrapper.cpp | 15 ++++- be/src/vec/exprs/vslot_ref.cpp | 11 ++++ be/src/vec/exprs/vslot_ref.h | 5 +- be/src/vec/exprs/vtuple_is_null_predicate.cpp | 9 +++ be/src/vec/exprs/vtuple_is_null_predicate.h | 4 +- be/src/vec/runtime/vdata_stream_recvr.h | 8 +-- 37 files changed, 329 insertions(+), 237 deletions(-) diff --git a/be/src/exec/exec_node.cpp b/be/src/exec/exec_node.cpp index d65e8f96bd2cd8..ac6e7eae9a0748 100644 --- a/be/src/exec/exec_node.cpp +++ b/be/src/exec/exec_node.cpp @@ -25,8 +25,8 @@ #include #include +#include #include -#include #include #include "common/compiler_util.h" // IWYU pragma: keep @@ -84,16 +84,10 @@ ExecNode::ExecNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl _tuple_ids(tnode.row_tuples), _row_descriptor(descs, tnode.row_tuples, tnode.nullable_tuples), _resource_profile(tnode.resource_profile), - _limit(tnode.limit), - _num_rows_returned(0), - _rows_returned_counter(nullptr), - _rows_returned_rate(nullptr), - _memory_used_counter(nullptr), - _peak_memory_usage_counter(nullptr), - _is_closed(false), - _ref(0) { + _limit(tnode.limit) { if (tnode.__isset.output_tuple_id) { - _output_row_descriptor.reset(new RowDescriptor(descs, {tnode.output_tuple_id}, {true})); + _output_row_descriptor = std::make_unique( + descs, std::vector {tnode.output_tuple_id}, std::vector {true}); } _query_statistics = std::make_shared(); } @@ -108,7 +102,7 @@ Status ExecNode::init(const TPlanNode& tnode, RuntimeState* state) { RETURN_IF_ERROR(vectorized::VExpr::create_expr_tree(tnode.vconjunct, context)); _conjuncts.emplace_back(context); } else if (tnode.__isset.conjuncts) { - for (auto& conjunct : tnode.conjuncts) { + for (const auto& conjunct : tnode.conjuncts) { vectorized::VExprContextSPtr context; RETURN_IF_ERROR(vectorized::VExpr::create_expr_tree(conjunct, context)); _conjuncts.emplace_back(context); @@ -136,8 +130,9 @@ Status ExecNode::prepare(RuntimeState* state) { _projection_timer = ADD_TIMER(_runtime_profile, "ProjectionTime"); _rows_returned_rate = runtime_profile()->add_derived_counter( ROW_THROUGHPUT_COUNTER, TUnit::UNIT_PER_SECOND, - std::bind(&RuntimeProfile::units_per_second, _rows_returned_counter, - runtime_profile()->total_time_counter()), + [this, capture0 = runtime_profile()->total_time_counter()] { + return RuntimeProfile::units_per_second(_rows_returned_counter, capture0); + }, ""); _memory_used_counter = ADD_LABEL_COUNTER(runtime_profile(), "MemoryUsage"); _peak_memory_usage_counter = _runtime_profile->AddHighWaterMarkCounter( @@ -150,13 +145,13 @@ Status ExecNode::prepare(RuntimeState* state) { RETURN_IF_ERROR(vectorized::VExpr::prepare(_projections, state, intermediate_row_desc())); - for (int i = 0; i < _children.size(); ++i) { - RETURN_IF_ERROR(_children[i]->prepare(state)); + for (auto& i : _children) { + RETURN_IF_ERROR(i->prepare(state)); } return Status::OK(); } -Status ExecNode::alloc_resource(doris::RuntimeState* state) { +Status ExecNode::alloc_resource(RuntimeState* state) { for (auto& conjunct : _conjuncts) { RETURN_IF_ERROR(conjunct->open(state)); } @@ -170,8 +165,8 @@ Status ExecNode::open(RuntimeState* state) { Status ExecNode::reset(RuntimeState* state) { _num_rows_returned = 0; - for (int i = 0; i < _children.size(); ++i) { - RETURN_IF_ERROR(_children[i]->reset(state)); + for (auto& i : _children) { + RETURN_IF_ERROR(i->reset(state)); } return Status::OK(); } @@ -199,8 +194,8 @@ Status ExecNode::close(RuntimeState* state) { _is_closed = true; Status result; - for (int i = 0; i < _children.size(); ++i) { - auto st = _children[i]->close(state); + for (auto& i : _children) { + auto st = i->close(state); if (result.ok() && !st.ok()) { result = st; } @@ -227,7 +222,7 @@ void ExecNode::add_runtime_exec_option(const std::string& str) { Status ExecNode::create_tree(RuntimeState* state, ObjectPool* pool, const TPlan& plan, const DescriptorTbl& descs, ExecNode** root) { - if (plan.nodes.size() == 0) { + if (plan.nodes.empty()) { *root = nullptr; return Status::OK(); } @@ -305,6 +300,7 @@ Status ExecNode::create_tree_helper(RuntimeState* state, ObjectPool* pool, return Status::OK(); } +// NOLINTBEGIN(readability-function-size) Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, ExecNode** node) { VLOG_CRITICAL << "tnode:\n" << apache::thrift::ThriftDebugString(tnode); @@ -428,8 +424,7 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN return Status::OK(); default: - std::map::const_iterator i = - _TPlanNodeType_VALUES_TO_NAMES.find(tnode.node_type); + auto i = _TPlanNodeType_VALUES_TO_NAMES.find(tnode.node_type); const char* str = "unknown node type"; if (i != _TPlanNodeType_VALUES_TO_NAMES.end()) { @@ -443,6 +438,7 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN return Status::OK(); } +// NOLINTEND(readability-function-size) std::string ExecNode::debug_string() const { std::stringstream out; @@ -459,9 +455,9 @@ void ExecNode::debug_string(int indentation_level, std::stringstream* out) const } *out << "]"; - for (int i = 0; i < _children.size(); ++i) { + for (auto* i : _children) { *out << "\n"; - _children[i]->debug_string(indentation_level + 1, out); + i->debug_string(indentation_level + 1, out); } } @@ -470,8 +466,8 @@ void ExecNode::collect_nodes(TPlanNodeType::type node_type, std::vectorpush_back(this); } - for (int i = 0; i < _children.size(); ++i) { - _children[i]->collect_nodes(node_type, nodes); + for (auto& i : _children) { + i->collect_nodes(node_type, nodes); } } @@ -488,7 +484,7 @@ void ExecNode::collect_scan_nodes(vector* nodes) { void ExecNode::init_runtime_profile(const std::string& name) { std::stringstream ss; ss << name << " (id=" << _id << ")"; - _runtime_profile.reset(new RuntimeProfile(ss.str())); + _runtime_profile = std::make_unique(ss.str()); _runtime_profile->set_metadata(_id); } diff --git a/be/src/exec/exec_node.h b/be/src/exec/exec_node.h index f4b49cba6f56fe..903122ecded1b6 100644 --- a/be/src/exec/exec_node.h +++ b/be/src/exec/exec_node.h @@ -21,10 +21,10 @@ #pragma once #include -#include -#include #include +#include +#include #include #include #include @@ -267,7 +267,7 @@ class ExecNode { const TBackendResourceProfile _resource_profile; int64_t _limit; // -1: no limit - int64_t _num_rows_returned; + int64_t _num_rows_returned = 0; std::unique_ptr _runtime_profile; @@ -303,15 +303,6 @@ class ExecNode { bool is_closed() const { return _is_closed; } - // TODO(zc) - /// Pointer to the containing SubplanNode or nullptr if not inside a subplan. - /// Set by SubplanNode::Init(). Not owned. - // SubplanNode* containing_subplan_; - - /// Returns true if this node is inside the right-hand side plan tree of a SubplanNode. - /// Valid to call in or after Prepare(). - bool is_in_subplan() const { return false; } - // Create a single exec node derived from thrift node; place exec node in 'pool'. static Status create_node(RuntimeState* state, ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, ExecNode** node); @@ -334,9 +325,9 @@ class ExecNode { ExecNode** root); friend class pipeline::OperatorBase; - bool _is_closed; + bool _is_closed = false; bool _is_resource_released = false; - std::atomic_int _ref; // used by pipeline operator to release resource. + std::atomic_int _ref = 0; // used by pipeline operator to release resource. }; } // namespace doris diff --git a/be/src/pipeline/exec/multi_cast_data_stream_source.cpp b/be/src/pipeline/exec/multi_cast_data_stream_source.cpp index a4f3ff55a5c023..6ac06ee5f10fb2 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_source.cpp +++ b/be/src/pipeline/exec/multi_cast_data_stream_source.cpp @@ -17,13 +17,11 @@ #include "multi_cast_data_stream_source.h" -#include - #include "common/status.h" #include "pipeline/exec/multi_cast_data_streamer.h" #include "pipeline/exec/operator.h" -#include "runtime/query_statistics.h" #include "vec/core/block.h" +#include "vec/core/materialize_block.h" namespace doris::pipeline { @@ -108,7 +106,7 @@ Status MultiCastDataStreamerSourceOperator::get_block(RuntimeState* state, vecto if (!_output_expr_contexts.empty() && output_block->rows() > 0) { RETURN_IF_ERROR(vectorized::VExprContext::get_output_block_after_execute_exprs( _output_expr_contexts, *output_block, block, true)); - materialize_block_inplace(*block); + vectorized::materialize_block_inplace(*block); } if (eos) { source_state = SourceState::FINISHED; @@ -176,7 +174,7 @@ Status MultiCastDataStreamerSourceOperatorX::get_block(RuntimeState* state, if (!local_state._output_expr_contexts.empty() && output_block->rows() > 0) { RETURN_IF_ERROR(vectorized::VExprContext::get_output_block_after_execute_exprs( local_state._output_expr_contexts, *output_block, block, true)); - materialize_block_inplace(*block); + vectorized::materialize_block_inplace(*block); } COUNTER_UPDATE(local_state._rows_returned_counter, block->rows()); if (eos) { diff --git a/be/src/pipeline/exec/operator.h b/be/src/pipeline/exec/operator.h index cd5fba5fee3f9d..bf41c670e0c036 100644 --- a/be/src/pipeline/exec/operator.h +++ b/be/src/pipeline/exec/operator.h @@ -19,20 +19,16 @@ #include #include -#include +#include #include #include -#include #include -#include #include #include #include "common/status.h" #include "exec/exec_node.h" -#include "pipeline/pipeline_x/dependency.h" -#include "runtime/memory/mem_tracker.h" #include "runtime/runtime_state.h" #include "util/runtime_profile.h" #include "vec/core/block.h" @@ -105,7 +101,7 @@ using OperatorBuilders = std::vector; class OperatorBuilderBase { public: - OperatorBuilderBase(int32_t id, const std::string& name) : _id(id), _name(name) {} + OperatorBuilderBase(int32_t id, std::string name) : _id(id), _name(std::move(name)) {} virtual ~OperatorBuilderBase() = default; @@ -333,10 +329,7 @@ class StreamingOperator : public OperatorBase { return Status::OK(); } - Status open(RuntimeState* state) override { - RETURN_IF_ERROR(_node->alloc_resource(state)); - return Status::OK(); - } + Status open(RuntimeState* state) override { return _node->alloc_resource(state); } Status sink(RuntimeState* state, vectorized::Block* in_block, SourceState source_state) override { @@ -413,8 +406,7 @@ class StatefulOperator : public StreamingOperator { public: StatefulOperator(OperatorBuilderBase* builder, ExecNode* node) : StreamingOperator(builder, node), - _child_block(vectorized::Block::create_shared()), - _child_source_state(SourceState::DEPEND_ON_SOURCE) {} + _child_block(vectorized::Block::create_shared()) {} virtual ~StatefulOperator() = default; @@ -454,7 +446,7 @@ class StatefulOperator : public StreamingOperator { protected: std::shared_ptr _child_block; - SourceState _child_source_state; + SourceState _child_source_state {SourceState::DEPEND_ON_SOURCE}; }; } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/set_sink_operator.cpp b/be/src/pipeline/exec/set_sink_operator.cpp index 6c18cab03f6f7d..cb106d76edb497 100644 --- a/be/src/pipeline/exec/set_sink_operator.cpp +++ b/be/src/pipeline/exec/set_sink_operator.cpp @@ -21,6 +21,7 @@ #include "pipeline/exec/operator.h" #include "vec/common/hash_table/hash_table_set_build.h" +#include "vec/core/materialize_block.h" #include "vec/exec/vset_operation_node.h" namespace doris { @@ -139,10 +140,10 @@ Status SetSinkOperatorX::_extract_build_column( block.get_by_position(result_col_id).column = block.get_by_position(result_col_id).column->convert_to_full_column_if_const(); - auto column = block.get_by_position(result_col_id).column.get(); + const auto* column = block.get_by_position(result_col_id).column.get(); - if (auto* nullable = check_and_get_column(*column)) { - auto& col_nested = nullable->get_nested_column(); + if (const auto* nullable = check_and_get_column(*column)) { + const auto& col_nested = nullable->get_nested_column(); if (local_state._shared_state->build_not_ignore_null[i]) { raw_ptrs[i] = nullable; } else { @@ -165,7 +166,7 @@ Status SetSinkLocalState::init(RuntimeState* state, LocalSinkState SCOPED_TIMER(_open_timer); _build_timer = ADD_TIMER(_profile, "BuildTime"); - Parent& parent = _parent->cast(); + auto& parent = _parent->cast(); _dependency->set_cur_child_id(parent._cur_child_id); _child_exprs.resize(parent._child_exprs.size()); for (size_t i = 0; i < _child_exprs.size(); i++) { @@ -175,16 +176,15 @@ Status SetSinkLocalState::init(RuntimeState* state, LocalSinkState _shared_state->child_quantity = parent._child_quantity; auto& child_exprs_lists = _shared_state->child_exprs_lists; - DCHECK(child_exprs_lists.size() == 0 || child_exprs_lists.size() == parent._child_quantity); - if (child_exprs_lists.size() == 0) { + DCHECK(child_exprs_lists.empty() || child_exprs_lists.size() == parent._child_quantity); + if (child_exprs_lists.empty()) { child_exprs_lists.resize(parent._child_quantity); } child_exprs_lists[parent._cur_child_id] = _child_exprs; _shared_state->hash_table_variants = std::make_unique(); - for (int i = 0; i < child_exprs_lists[0].size(); ++i) { - const auto& ctx = child_exprs_lists[0][i]; + for (const auto& ctx : child_exprs_lists[0]) { _shared_state->build_not_ignore_null.push_back(ctx->root()->is_nullable()); } _shared_state->hash_table_init(); diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp index b9c2382ce86162..538a2ce1bdb546 100644 --- a/be/src/pipeline/pipeline_fragment_context.cpp +++ b/be/src/pipeline/pipeline_fragment_context.cpp @@ -22,13 +22,15 @@ #include #include #include -#include + +#include // IWYU pragma: no_include #include #include #include // IWYU pragma: keep #include +#include #include #include #include @@ -212,8 +214,7 @@ PipelinePtr PipelineFragmentContext::add_pipeline(PipelinePtr parent, int idx) { return pipeline; } -Status PipelineFragmentContext::prepare(const doris::TPipelineFragmentParams& request, - const size_t idx) { +Status PipelineFragmentContext::prepare(const doris::TPipelineFragmentParams& request, size_t idx) { if (_prepared) { return Status::InternalError("Already prepared"); } @@ -299,16 +300,16 @@ Status PipelineFragmentContext::prepare(const doris::TPipelineFragmentParams& re << local_params.per_node_scan_ranges.size(); // set scan range in ScanNode - for (int i = 0; i < scan_nodes.size(); ++i) { + for (auto& i : scan_nodes) { // TODO(cmy): this "if...else" should be removed once all ScanNode are derived from VScanNode. - ExecNode* node = scan_nodes[i]; + ExecNode* node = i; if (typeid(*node) == typeid(vectorized::NewOlapScanNode) || typeid(*node) == typeid(vectorized::NewFileScanNode) || typeid(*node) == typeid(vectorized::NewOdbcScanNode) || typeid(*node) == typeid(vectorized::NewEsScanNode) || typeid(*node) == typeid(vectorized::VMetaScanNode) || typeid(*node) == typeid(vectorized::NewJdbcScanNode)) { - auto* scan_node = static_cast(scan_nodes[i]); + auto* scan_node = static_cast(i); auto scan_ranges = find_with_default(local_params.per_node_scan_ranges, scan_node->id(), no_scan_ranges); const bool shared_scan = @@ -316,7 +317,7 @@ Status PipelineFragmentContext::prepare(const doris::TPipelineFragmentParams& re scan_node->set_scan_ranges(_runtime_state.get(), scan_ranges); scan_node->set_shared_scan(_runtime_state.get(), shared_scan); } else { - ScanNode* scan_node = static_cast(node); + auto* scan_node = static_cast(node); auto scan_ranges = find_with_default(local_params.per_node_scan_ranges, scan_node->id(), no_scan_ranges); static_cast(scan_node->set_scan_ranges(_runtime_state.get(), scan_ranges)); @@ -850,10 +851,10 @@ Status PipelineFragmentContext::_create_sink(int sender_id, const TDataSink& thr {false}) : sink_->row_desc(); // 1. create the data stream sender sink - _multi_cast_stream_sink_senders[i].reset(new vectorized::VDataStreamSender( + _multi_cast_stream_sink_senders[i] = std::make_unique( _runtime_state.get(), _runtime_state->obj_pool(), sender_id, row_desc, thrift_sink.multi_cast_stream_sink.sinks[i], - thrift_sink.multi_cast_stream_sink.destinations[i])); + thrift_sink.multi_cast_stream_sink.destinations[i]); // 2. create and set the source operator of multi_cast_data_stream_source for new pipeline OperatorBuilderPtr source_op = @@ -941,9 +942,10 @@ Status PipelineFragmentContext::send_report(bool done) { _fragment_instance_id, _backend_num, _runtime_state.get(), - std::bind(&PipelineFragmentContext::update_status, this, std::placeholders::_1), - std::bind(&PipelineFragmentContext::cancel, this, std::placeholders::_1, - std::placeholders::_2), + [this](auto&& PH1) { return update_status(std::forward(PH1)); }, + [this](auto&& PH1, auto&& PH2) { + cancel(std::forward(PH1), std::forward(PH2)); + }, _query_ctx->get_query_statistics()}, std::dynamic_pointer_cast(shared_from_this())); } diff --git a/be/src/pipeline/pipeline_fragment_context.h b/be/src/pipeline/pipeline_fragment_context.h index 353e7a0658685e..a7a45d8f07fee9 100644 --- a/be/src/pipeline/pipeline_fragment_context.h +++ b/be/src/pipeline/pipeline_fragment_context.h @@ -67,7 +67,7 @@ class PipelineFragmentContext : public TaskExecutionContext { const std::function& call_back, const report_status_callback& report_status_cb); - virtual ~PipelineFragmentContext(); + ~PipelineFragmentContext() override; PipelinePtr add_pipeline(); @@ -89,7 +89,7 @@ class PipelineFragmentContext : public TaskExecutionContext { int32_t next_operator_builder_id() { return _next_operator_builder_id++; } - Status prepare(const doris::TPipelineFragmentParams& request, const size_t idx); + Status prepare(const doris::TPipelineFragmentParams& request, size_t idx); virtual Status prepare(const doris::TPipelineFragmentParams& request) { return Status::InternalError("Pipeline fragment context do not implement prepare"); diff --git a/be/src/service/point_query_executor.cpp b/be/src/service/point_query_executor.cpp index a86d5ed90b869b..83d059ba206dfc 100644 --- a/be/src/service/point_query_executor.cpp +++ b/be/src/service/point_query_executor.cpp @@ -26,6 +26,7 @@ #include #include +#include "common/status.h" #include "gutil/integral_types.h" #include "olap/lru_cache.h" #include "olap/olap_tuple.h" @@ -56,21 +57,22 @@ Status Reusable::init(const TDescriptorTable& t_desc_tbl, const std::vectorobj_pool(), t_desc_tbl, &_desc_tbl)); _runtime_state->set_desc_tbl(_desc_tbl); _block_pool.resize(block_size); - for (int i = 0; i < _block_pool.size(); ++i) { - _block_pool[i] = vectorized::Block::create_unique(tuple_desc()->slots(), 2); + for (auto& i : _block_pool) { + i = vectorized::Block::create_unique(tuple_desc()->slots(), 2); // Name is useless but cost space - _block_pool[i]->clear_names(); + i->clear_names(); } RETURN_IF_ERROR(vectorized::VExpr::create_expr_trees(output_exprs, _output_exprs_ctxs)); RowDescriptor row_desc(tuple_desc(), false); // Prepare the exprs to run. RETURN_IF_ERROR(vectorized::VExpr::prepare(_output_exprs_ctxs, _runtime_state.get(), row_desc)); + RETURN_IF_ERROR(vectorized::VExpr::open(_output_exprs_ctxs, _runtime_state.get())); _create_timestamp = butil::gettimeofday_ms(); _data_type_serdes = vectorized::create_data_type_serdes(tuple_desc()->slots()); _col_default_values.resize(tuple_desc()->slots().size()); for (int i = 0; i < tuple_desc()->slots().size(); ++i) { - auto slot = tuple_desc()->slots()[i]; + auto* slot = tuple_desc()->slots()[i]; _col_uid_to_idx[slot->col_unique_id()] = i; _col_default_values[i] = slot->col_default_value(); } diff --git a/be/src/vec/columns/column_nullable.cpp b/be/src/vec/columns/column_nullable.cpp index 426de2d4f70eee..8b0008d6e2e4e8 100644 --- a/be/src/vec/columns/column_nullable.cpp +++ b/be/src/vec/columns/column_nullable.cpp @@ -23,9 +23,7 @@ #include "vec/columns/column_const.h" #include "vec/common/arena.h" #include "vec/common/assert_cast.h" -#include "vec/common/nan_utils.h" #include "vec/common/sip_hash.h" -#include "vec/common/typeid_cast.h" #include "vec/core/sort_block.h" #include "vec/data_types/data_type.h" #include "vec/utils/util.hpp" @@ -571,7 +569,9 @@ bool ColumnNullable::has_null(size_t size) const { } ColumnPtr make_nullable(const ColumnPtr& column, bool is_nullable) { - if (is_column_nullable(*column)) return column; + if (is_column_nullable(*column)) { + return column; + } if (is_column_const(*column)) { return ColumnConst::create( diff --git a/be/src/vec/columns/column_nullable.h b/be/src/vec/columns/column_nullable.h index 83cbe82e328fd9..91128fb69a8cec 100644 --- a/be/src/vec/columns/column_nullable.h +++ b/be/src/vec/columns/column_nullable.h @@ -32,7 +32,6 @@ #include "olap/olap_common.h" #include "runtime/define_primitive_type.h" #include "vec/columns/column.h" -#include "vec/columns/column_impl.h" #include "vec/columns/column_vector.h" #include "vec/columns/columns_number.h" #include "vec/common/assert_cast.h" @@ -77,8 +76,7 @@ class ColumnNullable final : public COWHelper { null_map_->assume_mutable()); } - template ::value>::type> + template ::value>> static MutablePtr create(Args&&... args) { return Base::create(std::forward(args)...); } diff --git a/be/src/vec/exec/join/vjoin_node_base.h b/be/src/vec/exec/join/vjoin_node_base.h index a44bc5513a966f..c918e26e6fe594 100644 --- a/be/src/vec/exec/join/vjoin_node_base.h +++ b/be/src/vec/exec/join/vjoin_node_base.h @@ -23,7 +23,6 @@ #include #include #include -#include #include "common/status.h" #include "exec/exec_node.h" @@ -57,22 +56,20 @@ class VJoinNodeBase : public ExecNode { public: VJoinNodeBase(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); - virtual Status prepare(RuntimeState* state) override; + Status prepare(RuntimeState* state) override; - virtual Status close(RuntimeState* state) override; + Status close(RuntimeState* state) override; - virtual Status open(RuntimeState* state) override; + Status open(RuntimeState* state) override; - virtual const RowDescriptor& row_desc() const override { return *_output_row_desc; } + const RowDescriptor& row_desc() const override { return *_output_row_desc; } - virtual const RowDescriptor& intermediate_row_desc() const override { - return *_intermediate_row_desc; - } + const RowDescriptor& intermediate_row_desc() const override { return *_intermediate_row_desc; } - virtual Status alloc_resource(RuntimeState* state) override; - virtual void release_resource(RuntimeState* state) override; + Status alloc_resource(RuntimeState* state) override; + void release_resource(RuntimeState* state) override; - virtual Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override; + Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override; [[nodiscard]] bool can_terminate_early() override { return _short_circuit_for_probe; } diff --git a/be/src/vec/exec/vset_operation_node.cpp b/be/src/vec/exec/vset_operation_node.cpp index 4f4634fbd645e8..3c47638ef42e25 100644 --- a/be/src/vec/exec/vset_operation_node.cpp +++ b/be/src/vec/exec/vset_operation_node.cpp @@ -22,8 +22,6 @@ #include #include -#include -#include #include #include #include @@ -31,12 +29,9 @@ #include "runtime/define_primitive_type.h" #include "runtime/runtime_state.h" -#include "util/defer_op.h" #include "vec/columns/column_nullable.h" -#include "vec/common/columns_hashing.h" #include "vec/common/hash_table/hash_table_set_build.h" #include "vec/common/hash_table/hash_table_set_probe.h" -#include "vec/common/uint128.h" #include "vec/core/column_with_type_and_name.h" #include "vec/core/materialize_block.h" #include "vec/core/types.h" @@ -100,6 +95,8 @@ Status VSetOperationNode::init(const TPlanNode& tnode, RuntimeStat template Status VSetOperationNode::alloc_resource(RuntimeState* state) { SCOPED_TIMER(_exec_timer); + // will open projections + RETURN_IF_ERROR(ExecNode::alloc_resource(state)); // open result expr lists. for (const VExprContextSPtrs& exprs : _child_expr_lists) { RETURN_IF_ERROR(VExpr::open(exprs, state)); diff --git a/be/src/vec/exec/vset_operation_node.h b/be/src/vec/exec/vset_operation_node.h index ae600c6490de0c..b1ab9c4765049a 100644 --- a/be/src/vec/exec/vset_operation_node.h +++ b/be/src/vec/exec/vset_operation_node.h @@ -17,10 +17,7 @@ #pragma once -#include -#include - -#include +#include #include #include #include @@ -33,8 +30,6 @@ #include "vec/aggregate_functions/aggregate_function.h" #include "vec/columns/column.h" #include "vec/common/arena.h" -#include "vec/common/hash_table/hash_map.h" -#include "vec/common/string_ref.h" #include "vec/core/block.h" #include "vec/exec/join/process_hash_table_probe.h" #include "vec/exec/join/vhash_join_node.h" diff --git a/be/src/vec/exec/vunion_node.cpp b/be/src/vec/exec/vunion_node.cpp index e77fd9fee9097b..eac8aa1b167e4e 100644 --- a/be/src/vec/exec/vunion_node.cpp +++ b/be/src/vec/exec/vunion_node.cpp @@ -20,7 +20,6 @@ #include #include -#include #include #include #include @@ -66,8 +65,8 @@ Status VUnionNode::init(const TPlanNode& tnode, RuntimeState* state) { _const_expr_lists.push_back(ctxs); } // Create result_expr_ctx_lists_ from thrift exprs. - auto& result_texpr_lists = tnode.union_node.result_expr_lists; - for (auto& texprs : result_texpr_lists) { + const auto& result_texpr_lists = tnode.union_node.result_expr_lists; + for (const auto& texprs : result_texpr_lists) { VExprContextSPtrs ctxs; RETURN_IF_ERROR(VExpr::create_expr_trees(texprs, ctxs)); _child_expr_lists.push_back(ctxs); @@ -127,7 +126,6 @@ Status VUnionNode::alloc_resource(RuntimeState* state) { Status VUnionNode::get_next_pass_through(RuntimeState* state, Block* block) { DCHECK(!reached_limit()); - DCHECK(!is_in_subplan()); DCHECK_LT(_child_idx, _children.size()); DCHECK(is_child_passthrough(_child_idx)); if (_child_eos) { @@ -196,12 +194,6 @@ Status VUnionNode::get_next_materialized(RuntimeState* state, Block* block) { // incremented '_num_rows_returned' yet. DCHECK(!reached_limit()); if (_child_eos) { - // Unless we are inside a subplan expecting to call open()/get_next() on the child - // again, the child can be closed at this point. - // TODO: Recheck whether is_in_subplan() is right - // if (!is_in_subplan()) { - // child(_child_idx)->close(state); - // } ++_child_idx; } } @@ -275,7 +267,6 @@ Status VUnionNode::get_next(RuntimeState* state, Block* block, bool* eos) { // The previous child needs to be closed if passthrough was enabled for it. In the non // passthrough case, the child was already closed in the previous call to get_next(). DCHECK(is_child_passthrough(_to_close_child_idx)); - DCHECK(!is_in_subplan()); static_cast(child(_to_close_child_idx)->close(state)); _to_close_child_idx = -1; } @@ -317,8 +308,8 @@ void VUnionNode::debug_string(int indentation_level, std::stringstream* out) con *out << string(indentation_level * 2, ' '); *out << "_union(_first_materialized_child_idx=" << _first_materialized_child_idx << " _child_expr_lists=["; - for (int i = 0; i < _child_expr_lists.size(); ++i) { - *out << VExpr::debug_string(_child_expr_lists[i]) << ", "; + for (const auto& _child_expr_list : _child_expr_lists) { + *out << VExpr::debug_string(_child_expr_list) << ", "; } *out << "] \n"; ExecNode::debug_string(indentation_level, out); @@ -329,9 +320,9 @@ Status VUnionNode::materialize_block(Block* src_block, int child_idx, Block* res SCOPED_TIMER(_exec_timer); const auto& child_exprs = _child_expr_lists[child_idx]; ColumnsWithTypeAndName colunms; - for (size_t i = 0; i < child_exprs.size(); ++i) { + for (const auto& child_expr : child_exprs) { int result_column_id = -1; - RETURN_IF_ERROR(child_exprs[i]->execute(src_block, &result_column_id)); + RETURN_IF_ERROR(child_expr->execute(src_block, &result_column_id)); colunms.emplace_back(src_block->get_by_position(result_column_id)); } _child_row_idx += src_block->rows(); diff --git a/be/src/vec/exprs/vbitmap_predicate.cpp b/be/src/vec/exprs/vbitmap_predicate.cpp index 0e158298d85a4c..8116311247b5b9 100644 --- a/be/src/vec/exprs/vbitmap_predicate.cpp +++ b/be/src/vec/exprs/vbitmap_predicate.cpp @@ -66,19 +66,23 @@ doris::Status vectorized::VBitmapPredicate::prepare(doris::RuntimeState* state, auto column = child->data_type()->create_column(); argument_template.emplace_back(std::move(column), child->data_type(), child->expr_name()); } + _prepare_finished = true; return Status::OK(); } doris::Status vectorized::VBitmapPredicate::open(doris::RuntimeState* state, vectorized::VExprContext* context, FunctionContext::FunctionStateScope scope) { + DCHECK(_prepare_finished); RETURN_IF_ERROR(VExpr::open(state, context, scope)); + _open_finished = true; return Status::OK(); } doris::Status vectorized::VBitmapPredicate::execute(vectorized::VExprContext* context, doris::vectorized::Block* block, int* result_column_id) { + DCHECK(_open_finished || _getting_const_col); doris::vectorized::ColumnNumbers arguments(_children.size()); for (int i = 0; i < _children.size(); ++i) { int column_id = -1; diff --git a/be/src/vec/exprs/vbloom_predicate.cpp b/be/src/vec/exprs/vbloom_predicate.cpp index f72657c528a009..08f891b0e56548 100644 --- a/be/src/vec/exprs/vbloom_predicate.cpp +++ b/be/src/vec/exprs/vbloom_predicate.cpp @@ -60,12 +60,15 @@ Status VBloomPredicate::prepare(RuntimeState* state, const RowDescriptor& desc, } _be_exec_version = state->be_exec_version(); + _prepare_finished = true; return Status::OK(); } Status VBloomPredicate::open(RuntimeState* state, VExprContext* context, FunctionContext::FunctionStateScope scope) { + DCHECK(_prepare_finished); RETURN_IF_ERROR(VExpr::open(state, context, scope)); + _open_finished = true; return Status::OK(); } @@ -74,6 +77,7 @@ void VBloomPredicate::close(VExprContext* context, FunctionContext::FunctionStat } Status VBloomPredicate::execute(VExprContext* context, Block* block, int* result_column_id) { + DCHECK(_open_finished || _getting_const_col); doris::vectorized::ColumnNumbers arguments(_children.size()); for (int i = 0; i < _children.size(); ++i) { int column_id = -1; diff --git a/be/src/vec/exprs/vcase_expr.cpp b/be/src/vec/exprs/vcase_expr.cpp index e09d62bfb23f62..dee60b5a6f1c65 100644 --- a/be/src/vec/exprs/vcase_expr.cpp +++ b/be/src/vec/exprs/vcase_expr.cpp @@ -74,22 +74,26 @@ Status VCaseExpr::prepare(RuntimeState* state, const RowDescriptor& desc, VExprC } VExpr::register_function_context(state, context); + _prepare_finished = true; return Status::OK(); } Status VCaseExpr::open(RuntimeState* state, VExprContext* context, FunctionContext::FunctionStateScope scope) { - for (int i = 0; i < _children.size(); ++i) { - RETURN_IF_ERROR(_children[i]->open(state, context, scope)); + DCHECK(_prepare_finished); + for (auto& i : _children) { + RETURN_IF_ERROR(i->open(state, context, scope)); } RETURN_IF_ERROR(VExpr::init_function_context(context, scope, _function)); if (scope == FunctionContext::FRAGMENT_LOCAL) { RETURN_IF_ERROR(VExpr::get_const_col(context, nullptr)); } + _open_finished = true; return Status::OK(); } void VCaseExpr::close(VExprContext* context, FunctionContext::FunctionStateScope scope) { + DCHECK(_prepare_finished); VExpr::close_function_context(context, scope, _function); VExpr::close(context, scope); } @@ -98,6 +102,7 @@ Status VCaseExpr::execute(VExprContext* context, Block* block, int* result_colum if (is_const_and_have_executed()) { // const have execute in open function return get_result_from_const(block, _expr_name, result_column_id); } + DCHECK(_open_finished || _getting_const_col); ColumnNumbers arguments(_children.size()); for (int i = 0; i < _children.size(); i++) { int column_id = -1; diff --git a/be/src/vec/exprs/vcast_expr.cpp b/be/src/vec/exprs/vcast_expr.cpp index 3207ba5b5419a6..f322c1d2fae6f6 100644 --- a/be/src/vec/exprs/vcast_expr.cpp +++ b/be/src/vec/exprs/vcast_expr.cpp @@ -76,6 +76,7 @@ doris::Status VCastExpr::prepare(doris::RuntimeState* state, const doris::RowDes VExpr::register_function_context(state, context); _expr_name = fmt::format("(CAST {}({}) TO {})", child_name, child->data_type()->get_name(), _target_data_type_name); + _prepare_finished = true; return Status::OK(); } @@ -85,6 +86,7 @@ const DataTypePtr& VCastExpr::get_target_type() const { doris::Status VCastExpr::open(doris::RuntimeState* state, VExprContext* context, FunctionContext::FunctionStateScope scope) { + DCHECK(_prepare_finished); for (int i = 0; i < _children.size(); ++i) { RETURN_IF_ERROR(_children[i]->open(state, context, scope)); } @@ -92,6 +94,7 @@ doris::Status VCastExpr::open(doris::RuntimeState* state, VExprContext* context, if (scope == FunctionContext::FRAGMENT_LOCAL) { RETURN_IF_ERROR(VExpr::get_const_col(context, nullptr)); } + _open_finished = true; return Status::OK(); } @@ -102,6 +105,7 @@ void VCastExpr::close(VExprContext* context, FunctionContext::FunctionStateScope doris::Status VCastExpr::execute(VExprContext* context, doris::vectorized::Block* block, int* result_column_id) { + DCHECK(_open_finished || _getting_const_col); // for each child call execute int column_id = 0; RETURN_IF_ERROR(_children[0]->execute(context, block, &column_id)); diff --git a/be/src/vec/exprs/vcolumn_ref.h b/be/src/vec/exprs/vcolumn_ref.h index 25e35dec4a5a51..a763797880e361 100644 --- a/be/src/vec/exprs/vcolumn_ref.h +++ b/be/src/vec/exprs/vcolumn_ref.h @@ -43,10 +43,20 @@ class VColumnRef final : public VExpr { "VColumnRef have invalid slot id: {}, _column_name: {}, desc: {}", _column_id, _column_name, desc.debug_string()); } + _prepare_finished = true; + return Status::OK(); + } + + Status open(RuntimeState* state, VExprContext* context, + FunctionContext::FunctionStateScope scope) override { + DCHECK(_prepare_finished); + RETURN_IF_ERROR(VExpr::open(state, context, scope)); + _open_finished = true; return Status::OK(); } Status execute(VExprContext* context, Block* block, int* result_column_id) override { + DCHECK(_open_finished || _getting_const_col); *result_column_id = _column_id; return Status::OK(); } diff --git a/be/src/vec/exprs/vdirect_in_predicate.h b/be/src/vec/exprs/vdirect_in_predicate.h index 5211e013466975..a68a6c3121abaa 100644 --- a/be/src/vec/exprs/vdirect_in_predicate.h +++ b/be/src/vec/exprs/vdirect_in_predicate.h @@ -18,6 +18,7 @@ #pragma once #include "common/status.h" +#include "exprs/hybrid_set.h" #include "vec/exprs/vexpr.h" namespace doris::vectorized { @@ -29,7 +30,23 @@ class VDirectInPredicate final : public VExpr { : VExpr(node), _filter(nullptr), _expr_name("direct_in_predicate") {} ~VDirectInPredicate() override = default; + Status prepare(RuntimeState* state, const RowDescriptor& row_desc, + VExprContext* context) override { + RETURN_IF_ERROR_OR_PREPARED(VExpr::prepare(state, row_desc, context)); + _prepare_finished = true; + return Status::OK(); + } + + Status open(RuntimeState* state, VExprContext* context, + FunctionContext::FunctionStateScope scope) override { + DCHECK(_prepare_finished); + RETURN_IF_ERROR(VExpr::open(state, context, scope)); + _open_finished = true; + return Status::OK(); + } + Status execute(VExprContext* context, Block* block, int* result_column_id) override { + DCHECK(_open_finished || _getting_const_col); ColumnNumbers arguments(_children.size()); for (int i = 0; i < _children.size(); ++i) { int column_id = -1; @@ -47,7 +64,7 @@ class VDirectInPredicate final : public VExpr { if (argument_column->is_nullable()) { auto column_nested = static_cast(argument_column.get()) ->get_nested_column_ptr(); - auto& null_map = + const auto& null_map = static_cast(argument_column.get())->get_null_map_data(); _filter->find_batch_nullable(*column_nested, sz, null_map, res_data_column->get_data()); } else { diff --git a/be/src/vec/exprs/vectorized_fn_call.cpp b/be/src/vec/exprs/vectorized_fn_call.cpp index 48522b355009f6..bf38185f7dfdf6 100644 --- a/be/src/vec/exprs/vectorized_fn_call.cpp +++ b/be/src/vec/exprs/vectorized_fn_call.cpp @@ -21,8 +21,6 @@ #include // IWYU pragma: keep #include -#include -#include #include #include #include @@ -115,19 +113,21 @@ Status VectorizedFnCall::prepare(RuntimeState* state, const RowDescriptor& desc, VExpr::register_function_context(state, context); _function_name = _fn.name.function_name; _can_fast_execute = _function->can_fast_execute(); - + _prepare_finished = true; return Status::OK(); } Status VectorizedFnCall::open(RuntimeState* state, VExprContext* context, FunctionContext::FunctionStateScope scope) { - for (int i = 0; i < _children.size(); ++i) { - RETURN_IF_ERROR(_children[i]->open(state, context, scope)); + DCHECK(_prepare_finished); + for (auto& i : _children) { + RETURN_IF_ERROR(i->open(state, context, scope)); } RETURN_IF_ERROR(VExpr::init_function_context(context, scope, _function)); if (scope == FunctionContext::FRAGMENT_LOCAL) { RETURN_IF_ERROR(VExpr::get_const_col(context, nullptr)); } + _open_finished = true; return Status::OK(); } @@ -142,6 +142,7 @@ Status VectorizedFnCall::execute(VExprContext* context, vectorized::Block* block return get_result_from_const(block, _expr_name, result_column_id); } + DCHECK(_open_finished || _getting_const_col) << debug_string(); // TODO: not execute const expr again, but use the const column in function context vectorized::ColumnNumbers arguments(_children.size()); for (int i = 0; i < _children.size(); ++i) { @@ -188,9 +189,9 @@ bool VectorizedFnCall::fast_execute(FunctionContext* context, Block& block, block.get_by_name(result_column_name).column->convert_to_full_column_if_const(); auto& result_info = block.get_by_position(result); if (result_info.type->is_nullable()) { - block.replace_by_position(result, - ColumnNullable::create(std::move(result_column), - ColumnUInt8::create(input_rows_count, 0))); + block.replace_by_position( + result, + ColumnNullable::create(result_column, ColumnUInt8::create(input_rows_count, 0))); } else { block.replace_by_position(result, std::move(result_column)); } @@ -208,7 +209,7 @@ std::string VectorizedFnCall::debug_string() const { out << _expr_name; out << "]{"; bool first = true; - for (auto& input_expr : children()) { + for (const auto& input_expr : children()) { if (first) { first = false; } else { diff --git a/be/src/vec/exprs/vexpr.cpp b/be/src/vec/exprs/vexpr.cpp index 7270126e563110..4f6b984e8fcb24 100644 --- a/be/src/vec/exprs/vexpr.cpp +++ b/be/src/vec/exprs/vexpr.cpp @@ -17,6 +17,7 @@ #include "vec/exprs/vexpr.h" +#include #include #include @@ -27,7 +28,6 @@ #include "common/config.h" #include "common/exception.h" -#include "common/object_pool.h" #include "common/status.h" #include "vec/columns/column_vector.h" #include "vec/columns/columns_number.h" @@ -40,6 +40,7 @@ #include "vec/exprs/vcompound_pred.h" #include "vec/exprs/vectorized_fn_call.h" #include "vec/exprs/vexpr_context.h" +#include "vec/exprs/vexpr_fwd.h" #include "vec/exprs/vin_predicate.h" #include "vec/exprs/vinfo_func.h" #include "vec/exprs/vlambda_function_call_expr.h" @@ -55,6 +56,9 @@ namespace doris { class RowDescriptor; class RuntimeState; + +// NOLINTBEGIN(readability-function-cognitive-complexity) +// NOLINTBEGIN(readability-function-size) TExprNode create_texpr_node_from(const void* data, const PrimitiveType& type, int precision, int scale) { TExprNode node; @@ -146,6 +150,8 @@ TExprNode create_texpr_node_from(const void* data, const PrimitiveType& type, in } return node; } +// NOLINTEND(readability-function-size) +// NOLINTEND(readability-function-cognitive-complexity) } // namespace doris namespace doris::vectorized { @@ -162,9 +168,7 @@ bool VExpr::is_acting_on_a_slot(const VExpr& expr) { VExpr::VExpr(const TExprNode& node) : _node_type(node.node_type), _opcode(node.__isset.opcode ? node.opcode : TExprOpcode::INVALID_OPCODE), - _type(TypeDescriptor::from_thrift(node.type)), - _fn_context_index(-1), - _prepared(false) { + _type(TypeDescriptor::from_thrift(node.type)) { if (node.__isset.fn) { _fn = node.fn; } @@ -183,10 +187,7 @@ VExpr::VExpr(const TExprNode& node) VExpr::VExpr(const VExpr& vexpr) = default; VExpr::VExpr(TypeDescriptor type, bool is_slotref, bool is_nullable) - : _opcode(TExprOpcode::INVALID_OPCODE), - _type(std::move(type)), - _fn_context_index(-1), - _prepared(false) { + : _opcode(TExprOpcode::INVALID_OPCODE), _type(std::move(type)) { if (is_slotref) { _node_type = TExprNodeType::SLOT_REF; } @@ -221,11 +222,12 @@ Status VExpr::open(RuntimeState* state, VExprContext* context, } void VExpr::close(VExprContext* context, FunctionContext::FunctionStateScope scope) { - for (int i = 0; i < _children.size(); ++i) { - _children[i]->close(context, scope); + for (auto& i : _children) { + i->close(context, scope); } } +// NOLINTBEGIN(readability-function-size) Status VExpr::create_expr(const TExprNode& expr_node, VExprSPtr& expr) { try { switch (expr_node.node_type) { @@ -326,6 +328,7 @@ Status VExpr::create_expr(const TExprNode& expr_node, VExprSPtr& expr) { } return Status::OK(); } +// NOLINTEND(readability-function-size) Status VExpr::create_tree_from_thrift(const std::vector& nodes, int* node_idx, VExprSPtr& root_expr, VExprContextSPtr& ctx) { @@ -348,7 +351,7 @@ Status VExpr::create_tree_from_thrift(const std::vector& nodes, int* // non-recursive traversal std::stack> s; - s.push({root, root_children}); + s.emplace(root, root_children); while (!s.empty()) { auto& parent = s.top(); if (parent.second > 1) { @@ -366,14 +369,14 @@ Status VExpr::create_tree_from_thrift(const std::vector& nodes, int* parent.first->add_child(expr); int num_children = nodes[*node_idx].num_children; if (num_children > 0) { - s.push({expr, num_children}); + s.emplace(expr, num_children); } } return Status::OK(); } Status VExpr::create_expr_tree(const TExpr& texpr, VExprContextSPtr& ctx) { - if (texpr.nodes.size() == 0) { + if (texpr.nodes.empty()) { ctx = nullptr; return Status::OK(); } @@ -395,9 +398,9 @@ Status VExpr::create_expr_tree(const TExpr& texpr, VExprContextSPtr& ctx) { Status VExpr::create_expr_trees(const std::vector& texprs, VExprContextSPtrs& ctxs) { ctxs.clear(); - for (int i = 0; i < texprs.size(); ++i) { + for (const auto& texpr : texprs) { VExprContextSPtr ctx; - RETURN_IF_ERROR(create_expr_tree(texprs[i], ctx)); + RETURN_IF_ERROR(create_expr_tree(texpr, ctx)); ctxs.push_back(ctx); } return Status::OK(); @@ -412,8 +415,8 @@ Status VExpr::prepare(const VExprContextSPtrs& ctxs, RuntimeState* state, } Status VExpr::open(const VExprContextSPtrs& ctxs, RuntimeState* state) { - for (int i = 0; i < ctxs.size(); ++i) { - RETURN_IF_ERROR(ctxs[i]->open(state)); + for (const auto& ctx : ctxs) { + RETURN_IF_ERROR(ctx->open(state)); } return Status::OK(); } @@ -423,8 +426,8 @@ Status VExpr::clone_if_not_exists(const VExprContextSPtrs& ctxs, RuntimeState* s if (!new_ctxs.empty()) { // 'ctxs' was already cloned into '*new_ctxs', nothing to do. DCHECK_EQ(new_ctxs.size(), ctxs.size()); - for (int i = 0; i < new_ctxs.size(); ++i) { - DCHECK(new_ctxs[i]->_is_clone); + for (auto& new_ctx : new_ctxs) { + DCHECK(new_ctx->_is_clone); } return Status::OK(); } @@ -461,20 +464,15 @@ std::string VExpr::debug_string(const VExprSPtrs& exprs) { std::string VExpr::debug_string(const VExprContextSPtrs& ctxs) { VExprSPtrs exprs; - for (int i = 0; i < ctxs.size(); ++i) { - exprs.push_back(ctxs[i]->root()); + for (const auto& ctx : ctxs) { + exprs.push_back(ctx->root()); } return debug_string(exprs); } bool VExpr::is_constant() const { - for (int i = 0; i < _children.size(); ++i) { - if (!_children[i]->is_constant()) { - return false; - } - } - - return true; + return std::all_of(_children.begin(), _children.end(), + [](const VExprSPtr& expr) { return expr->is_constant(); }); } Status VExpr::get_const_col(VExprContext* context, @@ -494,7 +492,11 @@ Status VExpr::get_const_col(VExprContext* context, // If block is empty, some functions will produce no result. So we insert a column with // single value here. block.insert({ColumnUInt8::create(1), std::make_shared(), ""}); + + _getting_const_col = true; RETURN_IF_ERROR(execute(context, &block, &result)); + _getting_const_col = false; + DCHECK(result != -1); const auto& column = block.get_by_position(result).column; _constant_col = std::make_shared(column); @@ -507,8 +509,8 @@ Status VExpr::get_const_col(VExprContext* context, void VExpr::register_function_context(RuntimeState* state, VExprContext* context) { std::vector arg_types; - for (int i = 0; i < _children.size(); ++i) { - arg_types.push_back(_children[i]->type()); + for (auto& i : _children) { + arg_types.push_back(i->type()); } _fn_context_index = context->register_function_context(state, _type, arg_types); diff --git a/be/src/vec/exprs/vexpr.h b/be/src/vec/exprs/vexpr.h index b6a2b4ac6bdd91..a852afeb2d27e7 100644 --- a/be/src/vec/exprs/vexpr.h +++ b/be/src/vec/exprs/vexpr.h @@ -21,15 +21,14 @@ #include #include #include -#include +#include #include #include #include #include #include -#include "common/factory_creator.h" #include "common/status.h" #include "runtime/define_primitive_type.h" #include "runtime/large_int_value.h" @@ -57,10 +56,9 @@ namespace vectorized { #define RETURN_IF_ERROR_OR_PREPARED(stmt) \ if (_prepared) { \ return Status::OK(); \ - } else { \ - _prepared = true; \ - RETURN_IF_ERROR(stmt); \ - } + } \ + _prepared = true; \ + RETURN_IF_ERROR(stmt); // VExpr should be used as shared pointer because it will be passed between classes // like runtime filter to scan node, or from scannode to scanner. We could not make sure @@ -106,6 +104,14 @@ class VExpr { virtual Status open(RuntimeState* state, VExprContext* context, FunctionContext::FunctionStateScope scope); + // before execute, check if expr has been parepared+opened. + [[maybe_unused]] Status ready_status() const { + if (_prepare_finished && _open_finished) { + return Status::OK(); + } + return Status::InternalError(expr_name() + " is not ready when execute"); + } + virtual Status execute(VExprContext* context, Block* block, int* result_column_id) = 0; /// Subclasses overriding this function should call VExpr::Close(). @@ -156,6 +162,8 @@ class VExpr { static std::string debug_string(const VExprSPtrs& exprs); static std::string debug_string(const VExprContextSPtrs& ctxs); + void set_getting_const_col(bool val = true) { _getting_const_col = val; } + bool is_and_expr() const { return _fn.name.function_name == "and"; } virtual bool is_compound_predicate() const { return false; } @@ -254,63 +262,69 @@ class VExpr { /// Index to pass to ExprContext::fn_context() to retrieve this expr's FunctionContext. /// Set in RegisterFunctionContext(). -1 if this expr does not need a FunctionContext and /// doesn't call RegisterFunctionContext(). - int _fn_context_index; + int _fn_context_index = -1; // If this expr is constant, this will store and cache the value generated by // get_const_col() std::shared_ptr _constant_col; - bool _prepared; + bool _prepared = false; // for base class VExpr + bool _getting_const_col = + false; // if true, current execute() is in prepare() (that is, can't check _prepared) + // for concrete classes + bool _prepare_finished = false; + bool _open_finished = false; }; } // namespace vectorized +// NOLINTBEGIN(readability-function-size) template Status create_texpr_literal_node(const void* data, TExprNode* node, int precision = 0, int scale = 0) { if constexpr (T == TYPE_BOOLEAN) { - auto origin_value = reinterpret_cast(data); + const auto* origin_value = reinterpret_cast(data); TBoolLiteral boolLiteral; (*node).__set_node_type(TExprNodeType::BOOL_LITERAL); boolLiteral.__set_value(*origin_value); (*node).__set_bool_literal(boolLiteral); (*node).__set_type(create_type_desc(PrimitiveType::TYPE_BOOLEAN)); } else if constexpr (T == TYPE_TINYINT) { - auto origin_value = reinterpret_cast(data); + const auto* origin_value = reinterpret_cast(data); (*node).__set_node_type(TExprNodeType::INT_LITERAL); TIntLiteral intLiteral; intLiteral.__set_value(*origin_value); (*node).__set_int_literal(intLiteral); (*node).__set_type(create_type_desc(PrimitiveType::TYPE_TINYINT)); } else if constexpr (T == TYPE_SMALLINT) { - auto origin_value = reinterpret_cast(data); + const auto* origin_value = reinterpret_cast(data); (*node).__set_node_type(TExprNodeType::INT_LITERAL); TIntLiteral intLiteral; intLiteral.__set_value(*origin_value); (*node).__set_int_literal(intLiteral); (*node).__set_type(create_type_desc(PrimitiveType::TYPE_SMALLINT)); } else if constexpr (T == TYPE_INT) { - auto origin_value = reinterpret_cast(data); + const auto* origin_value = reinterpret_cast(data); (*node).__set_node_type(TExprNodeType::INT_LITERAL); TIntLiteral intLiteral; intLiteral.__set_value(*origin_value); (*node).__set_int_literal(intLiteral); (*node).__set_type(create_type_desc(PrimitiveType::TYPE_INT)); } else if constexpr (T == TYPE_BIGINT) { - auto origin_value = reinterpret_cast(data); + const auto* origin_value = reinterpret_cast(data); (*node).__set_node_type(TExprNodeType::INT_LITERAL); TIntLiteral intLiteral; intLiteral.__set_value(*origin_value); (*node).__set_int_literal(intLiteral); (*node).__set_type(create_type_desc(PrimitiveType::TYPE_BIGINT)); } else if constexpr (T == TYPE_LARGEINT) { - auto origin_value = reinterpret_cast(data); + const auto* origin_value = reinterpret_cast(data); (*node).__set_node_type(TExprNodeType::LARGE_INT_LITERAL); TLargeIntLiteral large_int_literal; large_int_literal.__set_value(LargeIntValue::to_string(*origin_value)); (*node).__set_large_int_literal(large_int_literal); (*node).__set_type(create_type_desc(PrimitiveType::TYPE_LARGEINT)); } else if constexpr ((T == TYPE_DATE) || (T == TYPE_DATETIME) || (T == TYPE_TIME)) { - auto origin_value = reinterpret_cast(data); + const auto* origin_value = reinterpret_cast(data); TDateLiteral date_literal; char convert_buffer[30]; origin_value->to_string(convert_buffer); @@ -325,7 +339,7 @@ Status create_texpr_literal_node(const void* data, TExprNode* node, int precisio (*node).__set_type(create_type_desc(PrimitiveType::TYPE_TIME)); } } else if constexpr (T == TYPE_DATEV2) { - auto origin_value = reinterpret_cast*>(data); + const auto* origin_value = reinterpret_cast*>(data); TDateLiteral date_literal; char convert_buffer[30]; origin_value->to_string(convert_buffer); @@ -334,7 +348,7 @@ Status create_texpr_literal_node(const void* data, TExprNode* node, int precisio (*node).__set_node_type(TExprNodeType::DATE_LITERAL); (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DATEV2)); } else if constexpr (T == TYPE_DATETIMEV2) { - auto origin_value = reinterpret_cast*>(data); + const auto* origin_value = reinterpret_cast*>(data); TDateLiteral date_literal; char convert_buffer[30]; origin_value->to_string(convert_buffer); @@ -343,28 +357,28 @@ Status create_texpr_literal_node(const void* data, TExprNode* node, int precisio (*node).__set_node_type(TExprNodeType::DATE_LITERAL); (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DATETIMEV2)); } else if constexpr (T == TYPE_DECIMALV2) { - auto origin_value = reinterpret_cast(data); + const auto* origin_value = reinterpret_cast(data); (*node).__set_node_type(TExprNodeType::DECIMAL_LITERAL); TDecimalLiteral decimal_literal; decimal_literal.__set_value(origin_value->to_string()); (*node).__set_decimal_literal(decimal_literal); (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DECIMALV2, precision, scale)); } else if constexpr (T == TYPE_DECIMAL32) { - auto origin_value = reinterpret_cast*>(data); + const auto* origin_value = reinterpret_cast*>(data); (*node).__set_node_type(TExprNodeType::DECIMAL_LITERAL); TDecimalLiteral decimal_literal; decimal_literal.__set_value(origin_value->to_string(scale)); (*node).__set_decimal_literal(decimal_literal); (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DECIMAL32, precision, scale)); } else if constexpr (T == TYPE_DECIMAL64) { - auto origin_value = reinterpret_cast*>(data); + const auto* origin_value = reinterpret_cast*>(data); (*node).__set_node_type(TExprNodeType::DECIMAL_LITERAL); TDecimalLiteral decimal_literal; decimal_literal.__set_value(origin_value->to_string(scale)); (*node).__set_decimal_literal(decimal_literal); (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DECIMAL64, precision, scale)); } else if constexpr (T == TYPE_DECIMAL128I) { - auto origin_value = reinterpret_cast*>(data); + const auto* origin_value = reinterpret_cast*>(data); (*node).__set_node_type(TExprNodeType::DECIMAL_LITERAL); TDecimalLiteral decimal_literal; decimal_literal.__set_value(origin_value->to_string(scale)); @@ -378,21 +392,21 @@ Status create_texpr_literal_node(const void* data, TExprNode* node, int precisio (*node).__set_decimal_literal(decimal_literal); (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DECIMAL256, precision, scale)); } else if constexpr (T == TYPE_FLOAT) { - auto origin_value = reinterpret_cast(data); + const auto* origin_value = reinterpret_cast(data); (*node).__set_node_type(TExprNodeType::FLOAT_LITERAL); TFloatLiteral float_literal; float_literal.__set_value(*origin_value); (*node).__set_float_literal(float_literal); (*node).__set_type(create_type_desc(PrimitiveType::TYPE_FLOAT)); } else if constexpr (T == TYPE_DOUBLE) { - auto origin_value = reinterpret_cast(data); + const auto* origin_value = reinterpret_cast(data); (*node).__set_node_type(TExprNodeType::FLOAT_LITERAL); TFloatLiteral float_literal; float_literal.__set_value(*origin_value); (*node).__set_float_literal(float_literal); (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DOUBLE)); } else if constexpr ((T == TYPE_STRING) || (T == TYPE_CHAR) || (T == TYPE_VARCHAR)) { - auto origin_value = reinterpret_cast(data); + const auto* origin_value = reinterpret_cast(data); (*node).__set_node_type(TExprNodeType::STRING_LITERAL); TStringLiteral string_literal; string_literal.__set_value(origin_value->to_string()); @@ -403,6 +417,7 @@ Status create_texpr_literal_node(const void* data, TExprNode* node, int precisio } return Status::OK(); } +// NOLINTEND(readability-function-size) TExprNode create_texpr_node_from(const void* data, const PrimitiveType& type, int precision = 0, int scale = 0); diff --git a/be/src/vec/exprs/vexpr_context.cpp b/be/src/vec/exprs/vexpr_context.cpp index 35eaae5c607dd9..cebb7dd2e53f94 100644 --- a/be/src/vec/exprs/vexpr_context.cpp +++ b/be/src/vec/exprs/vexpr_context.cpp @@ -17,17 +17,14 @@ #include "vec/exprs/vexpr_context.h" -#include #include #include #include "common/compiler_util.h" // IWYU pragma: keep #include "common/exception.h" -#include "common/object_pool.h" #include "runtime/runtime_state.h" #include "runtime/thread_context.h" #include "udf/udf.h" -#include "util/stack_util.h" #include "vec/columns/column_const.h" #include "vec/core/column_with_type_and_name.h" #include "vec/core/columns_with_type_and_name.h" @@ -38,13 +35,6 @@ class RowDescriptor; } // namespace doris namespace doris::vectorized { -VExprContext::VExprContext(const VExprSPtr& expr) - : _root(expr), - _is_clone(false), - _prepared(false), - _opened(false), - _last_result_column_id(-1) {} - VExprContext::~VExprContext() { // In runtime filter, only create expr context to get expr root, will not call // prepare or open, so that it is not need to call close. And call close may core @@ -154,19 +144,19 @@ Status VExprContext::execute_conjuncts(const VExprContextSPtrs& ctxs, return execute_conjuncts(ctxs, filters, false, block, result_filter, can_filter_all); } -// TODO Performance Optimization +// TODO: Performance Optimization Status VExprContext::execute_conjuncts(const VExprContextSPtrs& ctxs, const std::vector* filters, - const bool accept_null, Block* block, + bool accept_null, Block* block, IColumn::Filter* result_filter, bool* can_filter_all) { DCHECK(result_filter->size() == block->rows()); *can_filter_all = false; auto* __restrict result_filter_data = result_filter->data(); - for (auto& ctx : ctxs) { + for (const auto& ctx : ctxs) { int result_column_id = -1; RETURN_IF_ERROR(ctx->execute(block, &result_column_id)); ColumnPtr& filter_column = block->get_by_position(result_column_id).column; - if (auto* nullable_column = check_and_get_column(*filter_column)) { + if (const auto* nullable_column = check_and_get_column(*filter_column)) { size_t column_size = nullable_column->size(); if (column_size == 0) { *can_filter_all = true; @@ -175,9 +165,9 @@ Status VExprContext::execute_conjuncts(const VExprContextSPtrs& ctxs, const ColumnPtr& nested_column = nullable_column->get_nested_column_ptr(); const IColumn::Filter& filter = assert_cast(*nested_column).get_data(); - auto* __restrict filter_data = filter.data(); + const auto* __restrict filter_data = filter.data(); const size_t size = filter.size(); - auto* __restrict null_map_data = nullable_column->get_null_map_data().data(); + const auto* __restrict null_map_data = nullable_column->get_null_map_data().data(); if (accept_null) { for (size_t i = 0; i < size; ++i) { @@ -194,7 +184,7 @@ Status VExprContext::execute_conjuncts(const VExprContextSPtrs& ctxs, return Status::OK(); } } - } else if (auto* const_column = check_and_get_column(*filter_column)) { + } else if (const auto* const_column = check_and_get_column(*filter_column)) { // filter all if (!const_column->get_bool(0)) { *can_filter_all = true; @@ -204,7 +194,7 @@ Status VExprContext::execute_conjuncts(const VExprContextSPtrs& ctxs, } else { const IColumn::Filter& filter = assert_cast(*filter_column).get_data(); - auto* __restrict filter_data = filter.data(); + const auto* __restrict filter_data = filter.data(); const size_t size = filter.size(); for (size_t i = 0; i < size; ++i) { @@ -297,7 +287,7 @@ Status VExprContext::get_output_block_after_execute_exprs( auto rows = input_block.rows(); vectorized::Block tmp_block(input_block.get_columns_with_type_and_name()); vectorized::ColumnsWithTypeAndName result_columns; - for (auto& vexpr_ctx : output_vexpr_ctxs) { + for (const auto& vexpr_ctx : output_vexpr_ctxs) { int result_column_id = -1; RETURN_IF_ERROR(vexpr_ctx->execute(&tmp_block, &result_column_id)); DCHECK(result_column_id != -1); diff --git a/be/src/vec/exprs/vexpr_context.h b/be/src/vec/exprs/vexpr_context.h index db5c4c87d8d6fd..70bd37b1878d95 100644 --- a/be/src/vec/exprs/vexpr_context.h +++ b/be/src/vec/exprs/vexpr_context.h @@ -20,6 +20,7 @@ #include #include +#include #include #include "common/factory_creator.h" @@ -40,7 +41,7 @@ class VExprContext { ENABLE_FACTORY_CREATOR(VExprContext); public: - VExprContext(const VExprSPtr& expr); + VExprContext(VExprSPtr expr) : _root(std::move(expr)) {} ~VExprContext(); [[nodiscard]] Status prepare(RuntimeState* state, const RowDescriptor& row_desc); [[nodiscard]] Status open(RuntimeState* state); @@ -76,7 +77,7 @@ class VExprContext { [[nodiscard]] static Status execute_conjuncts(const VExprContextSPtrs& ctxs, const std::vector* filters, - const bool accept_null, Block* block, + bool accept_null, Block* block, IColumn::Filter* result_filter, bool* can_filter_all); @@ -121,7 +122,7 @@ class VExprContext { _prepared = other._prepared; _opened = other._opened; - for (auto& fn : other._fn_contexts) { + for (const auto& fn : other._fn_contexts) { _fn_contexts.emplace_back(fn->clone()); } @@ -152,17 +153,17 @@ class VExprContext { VExprSPtr _root; /// True if this context came from a Clone() call. Used to manage FunctionStateScope. - bool _is_clone; + bool _is_clone = false; /// Variables keeping track of current state. - bool _prepared; - bool _opened; + bool _prepared = false; + bool _opened = false; /// FunctionContexts for each registered expression. The FunctionContexts are created /// and owned by this VExprContext. std::vector> _fn_contexts; - int _last_result_column_id; + int _last_result_column_id = -1; /// The depth of expression-tree. int _depth_num = 0; diff --git a/be/src/vec/exprs/vin_predicate.cpp b/be/src/vec/exprs/vin_predicate.cpp index 9a25d3a223089d..896b2a903d34d0 100644 --- a/be/src/vec/exprs/vin_predicate.cpp +++ b/be/src/vec/exprs/vin_predicate.cpp @@ -73,11 +73,13 @@ Status VInPredicate::prepare(RuntimeState* state, const RowDescriptor& desc, } VExpr::register_function_context(state, context); + _prepare_finished = true; return Status::OK(); } Status VInPredicate::open(RuntimeState* state, VExprContext* context, FunctionContext::FunctionStateScope scope) { + DCHECK(_prepare_finished); for (int i = 0; i < _children.size(); ++i) { RETURN_IF_ERROR(_children[i]->open(state, context, scope)); } @@ -85,6 +87,7 @@ Status VInPredicate::open(RuntimeState* state, VExprContext* context, if (scope == FunctionContext::FRAGMENT_LOCAL) { RETURN_IF_ERROR(VExpr::get_const_col(context, nullptr)); } + _open_finished = true; return Status::OK(); } @@ -97,6 +100,7 @@ Status VInPredicate::execute(VExprContext* context, Block* block, int* result_co if (is_const_and_have_executed()) { // const have execute in open function return get_result_from_const(block, _expr_name, result_column_id); } + DCHECK(_open_finished || _getting_const_col); // TODO: not execute const expr again, but use the const column in function context doris::vectorized::ColumnNumbers arguments(_children.size()); for (int i = 0; i < _children.size(); ++i) { diff --git a/be/src/vec/exprs/vlambda_function_call_expr.h b/be/src/vec/exprs/vlambda_function_call_expr.h index 4467849802715f..44d22b1f9ebbea 100644 --- a/be/src/vec/exprs/vlambda_function_call_expr.h +++ b/be/src/vec/exprs/vlambda_function_call_expr.h @@ -34,6 +34,8 @@ class VLambdaFunctionCallExpr : public VExpr { VLambdaFunctionCallExpr(const TExprNode& node) : VExpr(node) {} ~VLambdaFunctionCallExpr() override = default; + const std::string& expr_name() const override { return _expr_name; } + Status prepare(RuntimeState* state, const RowDescriptor& desc, VExprContext* context) override { RETURN_IF_ERROR_OR_PREPARED(VExpr::prepare(state, desc, context)); @@ -48,12 +50,20 @@ class VLambdaFunctionCallExpr : public VExpr { return Status::InternalError("Lambda Function {} is not implemented.", _fn.name.function_name); } + _prepare_finished = true; return Status::OK(); } - const std::string& expr_name() const override { return _expr_name; } + Status open(RuntimeState* state, VExprContext* context, + FunctionContext::FunctionStateScope scope) override { + DCHECK(_prepare_finished); + RETURN_IF_ERROR(VExpr::open(state, context, scope)); + _open_finished = true; + return Status::OK(); + } Status execute(VExprContext* context, Block* block, int* result_column_id) override { + DCHECK(_open_finished || _getting_const_col); return _lambda_function->execute(context, block, result_column_id, _data_type, _children); } diff --git a/be/src/vec/exprs/vlambda_function_expr.h b/be/src/vec/exprs/vlambda_function_expr.h index 6d84abb937f5de..94571712e40d34 100644 --- a/be/src/vec/exprs/vlambda_function_expr.h +++ b/be/src/vec/exprs/vlambda_function_expr.h @@ -28,7 +28,22 @@ class VLambdaFunctionExpr final : public VExpr { VLambdaFunctionExpr(const TExprNode& node) : VExpr(node) {} ~VLambdaFunctionExpr() override = default; + Status prepare(RuntimeState* state, const RowDescriptor& desc, VExprContext* context) override { + RETURN_IF_ERROR_OR_PREPARED(VExpr::prepare(state, desc, context)); + _prepare_finished = true; + return Status::OK(); + } + + Status open(RuntimeState* state, VExprContext* context, + FunctionContext::FunctionStateScope scope) override { + DCHECK(_prepare_finished); + RETURN_IF_ERROR(VExpr::open(state, context, scope)); + _open_finished = true; + return Status::OK(); + } + Status execute(VExprContext* context, Block* block, int* result_column_id) override { + DCHECK(_open_finished || _getting_const_col); return get_child(0)->execute(context, block, result_column_id); } diff --git a/be/src/vec/exprs/vliteral.cpp b/be/src/vec/exprs/vliteral.cpp index 03d1659eee63a6..c7fbb081675089 100644 --- a/be/src/vec/exprs/vliteral.cpp +++ b/be/src/vec/exprs/vliteral.cpp @@ -47,9 +47,7 @@ #include "vec/data_types/data_type_decimal.h" #include "vec/runtime/vdatetime_value.h" -namespace doris { - -namespace vectorized { +namespace doris::vectorized { class VExprContext; void VLiteral::init(const TExprNode& node) { @@ -58,8 +56,20 @@ void VLiteral::init(const TExprNode& node) { _column_ptr = _data_type->create_column_const(1, field); } +Status VLiteral::prepare(RuntimeState* state, const RowDescriptor& desc, VExprContext* context) { + RETURN_IF_ERROR_OR_PREPARED(VExpr::prepare(state, desc, context)); + return Status::OK(); +} + +Status VLiteral::open(RuntimeState* state, VExprContext* context, + FunctionContext::FunctionStateScope scope) { + RETURN_IF_ERROR(VExpr::open(state, context, scope)); + return Status::OK(); +} + Status VLiteral::execute(VExprContext* context, vectorized::Block* block, int* result_column_id) { // Literal expr should return least one row. + // sometimes we just use a VLiteral without open or prepare. so can't check it at this moment size_t row_size = std::max(block->rows(), _column_ptr->size()); *result_column_id = VExpr::insert_param(block, {_column_ptr, _data_type, _expr_name}, row_size); return Status::OK(); @@ -86,5 +96,4 @@ std::string VLiteral::debug_string() const { return out.str(); } -} // namespace vectorized -} // namespace doris +} // namespace doris::vectorized diff --git a/be/src/vec/exprs/vliteral.h b/be/src/vec/exprs/vliteral.h index 78879c00d04966..d443478ada5530 100644 --- a/be/src/vec/exprs/vliteral.h +++ b/be/src/vec/exprs/vliteral.h @@ -42,7 +42,12 @@ class VLiteral : public VExpr { init(node); } } + + Status prepare(RuntimeState* state, const RowDescriptor& desc, VExprContext* context) override; + Status open(RuntimeState* state, VExprContext* context, + FunctionContext::FunctionStateScope scope) override; Status execute(VExprContext* context, Block* block, int* result_column_id) override; + const std::string& expr_name() const override { return _expr_name; } std::string debug_string() const override; diff --git a/be/src/vec/exprs/vmatch_predicate.cpp b/be/src/vec/exprs/vmatch_predicate.cpp index 23a34aae5acea0..17326b5b23b767 100644 --- a/be/src/vec/exprs/vmatch_predicate.cpp +++ b/be/src/vec/exprs/vmatch_predicate.cpp @@ -91,12 +91,13 @@ Status VMatchPredicate::prepare(RuntimeState* state, const RowDescriptor& desc, VExpr::register_function_context(state, context); _expr_name = fmt::format("{}({})", _fn.name.function_name, child_expr_name); _function_name = _fn.name.function_name; - + _prepare_finished = true; return Status::OK(); } Status VMatchPredicate::open(RuntimeState* state, VExprContext* context, FunctionContext::FunctionStateScope scope) { + DCHECK(_prepare_finished); for (int i = 0; i < _children.size(); ++i) { RETURN_IF_ERROR(_children[i]->open(state, context, scope)); } @@ -107,6 +108,7 @@ Status VMatchPredicate::open(RuntimeState* state, VExprContext* context, if (scope == FunctionContext::FRAGMENT_LOCAL) { RETURN_IF_ERROR(VExpr::get_const_col(context, nullptr)); } + _open_finished = true; return Status::OK(); } @@ -116,6 +118,7 @@ void VMatchPredicate::close(VExprContext* context, FunctionContext::FunctionStat } Status VMatchPredicate::execute(VExprContext* context, Block* block, int* result_column_id) { + DCHECK(_open_finished || _getting_const_col); // TODO: not execute const expr again, but use the const column in function context doris::vectorized::ColumnNumbers arguments(_children.size()); for (int i = 0; i < _children.size(); ++i) { diff --git a/be/src/vec/exprs/vruntimefilter_wrapper.cpp b/be/src/vec/exprs/vruntimefilter_wrapper.cpp index 62ef2bbdb646b7..c623355d6730b8 100644 --- a/be/src/vec/exprs/vruntimefilter_wrapper.cpp +++ b/be/src/vec/exprs/vruntimefilter_wrapper.cpp @@ -52,12 +52,16 @@ Status VRuntimeFilterWrapper::prepare(RuntimeState* state, const RowDescriptor& VExprContext* context) { RETURN_IF_ERROR_OR_PREPARED(_impl->prepare(state, desc, context)); _expr_name = fmt::format("VRuntimeFilterWrapper({})", _impl->expr_name()); + _prepare_finished = true; return Status::OK(); } Status VRuntimeFilterWrapper::open(RuntimeState* state, VExprContext* context, FunctionContext::FunctionStateScope scope) { - return _impl->open(state, context, scope); + DCHECK(_prepare_finished); + RETURN_IF_ERROR(_impl->open(state, context, scope)); + _open_finished = true; + return Status::OK(); } void VRuntimeFilterWrapper::close(VExprContext* context, @@ -66,6 +70,7 @@ void VRuntimeFilterWrapper::close(VExprContext* context, } Status VRuntimeFilterWrapper::execute(VExprContext* context, Block* block, int* result_column_id) { + DCHECK(_open_finished || _getting_const_col); if (_always_true) { auto res_data_column = ColumnVector::create(block->rows(), 1); size_t num_columns_without_result = block->columns(); @@ -80,7 +85,15 @@ Status VRuntimeFilterWrapper::execute(VExprContext* context, Block* block, int* return Status::OK(); } else { _scan_rows += block->rows(); + + if (_getting_const_col) { + _impl->set_getting_const_col(true); + } RETURN_IF_ERROR(_impl->execute(context, block, result_column_id)); + if (_getting_const_col) { + _impl->set_getting_const_col(false); + } + uint8_t* data = nullptr; const ColumnWithTypeAndName& result_column = block->get_by_position(*result_column_id); if (is_column_const(*result_column.column)) { diff --git a/be/src/vec/exprs/vslot_ref.cpp b/be/src/vec/exprs/vslot_ref.cpp index 5a34999accb16e..b683a1fb15696f 100644 --- a/be/src/vec/exprs/vslot_ref.cpp +++ b/be/src/vec/exprs/vslot_ref.cpp @@ -51,6 +51,7 @@ Status VSlotRef::prepare(doris::RuntimeState* state, const doris::RowDescriptor& RETURN_IF_ERROR_OR_PREPARED(VExpr::prepare(state, desc, context)); DCHECK_EQ(_children.size(), 0); if (_slot_id == -1) { + _prepare_finished = true; return Status::OK(); } const SlotDescriptor* slot_desc = state->desc_tbl().get_slot_descriptor(_slot_id); @@ -63,6 +64,7 @@ Status VSlotRef::prepare(doris::RuntimeState* state, const doris::RowDescriptor& if (!context->force_materialize_slot() && !slot_desc->need_materialize()) { // slot should be ignored manually _column_id = -1; + _prepare_finished = true; return Status::OK(); } _column_id = desc.get_column_id(_slot_id, context->force_materialize_slot()); @@ -72,6 +74,15 @@ Status VSlotRef::prepare(doris::RuntimeState* state, const doris::RowDescriptor& *_column_name, _slot_id, desc.debug_string(), slot_desc->debug_string(), state->desc_tbl().debug_string()); } + _prepare_finished = true; + return Status::OK(); +} + +Status VSlotRef::open(RuntimeState* state, VExprContext* context, + FunctionContext::FunctionStateScope scope) { + DCHECK(_prepare_finished); + RETURN_IF_ERROR(VExpr::open(state, context, scope)); + _open_finished = true; return Status::OK(); } diff --git a/be/src/vec/exprs/vslot_ref.h b/be/src/vec/exprs/vslot_ref.h index 2084ae187159ea..c30ac64041ff05 100644 --- a/be/src/vec/exprs/vslot_ref.h +++ b/be/src/vec/exprs/vslot_ref.h @@ -38,8 +38,11 @@ class VSlotRef final : public VExpr { public: VSlotRef(const TExprNode& node); VSlotRef(const SlotDescriptor* desc); - Status execute(VExprContext* context, Block* block, int* result_column_id) override; Status prepare(RuntimeState* state, const RowDescriptor& desc, VExprContext* context) override; + Status open(RuntimeState* state, VExprContext* context, + FunctionContext::FunctionStateScope scope) override; + Status execute(VExprContext* context, Block* block, int* result_column_id) override; + const std::string& expr_name() const override; std::string debug_string() const override; bool is_constant() const override { return false; } diff --git a/be/src/vec/exprs/vtuple_is_null_predicate.cpp b/be/src/vec/exprs/vtuple_is_null_predicate.cpp index b17428bfc000c6..641e34590a41df 100644 --- a/be/src/vec/exprs/vtuple_is_null_predicate.cpp +++ b/be/src/vec/exprs/vtuple_is_null_predicate.cpp @@ -48,11 +48,20 @@ Status VTupleIsNullPredicate::prepare(RuntimeState* state, const RowDescriptor& DCHECK_EQ(0, _children.size()); _column_to_check = _is_left_null_side ? desc.num_materialized_slots() : desc.num_materialized_slots() + 1; + _prepare_finished = true; + return Status::OK(); +} +Status VTupleIsNullPredicate::open(RuntimeState* state, VExprContext* context, + FunctionContext::FunctionStateScope scope) { + DCHECK(_prepare_finished); + RETURN_IF_ERROR(VExpr::open(state, context, scope)); + _open_finished = true; return Status::OK(); } Status VTupleIsNullPredicate::execute(VExprContext* context, Block* block, int* result_column_id) { + DCHECK(_open_finished || _getting_const_col); *result_column_id = _column_to_check; return Status::OK(); } diff --git a/be/src/vec/exprs/vtuple_is_null_predicate.h b/be/src/vec/exprs/vtuple_is_null_predicate.h index 9d3b794fb8b82a..c42e7300d1d100 100644 --- a/be/src/vec/exprs/vtuple_is_null_predicate.h +++ b/be/src/vec/exprs/vtuple_is_null_predicate.h @@ -42,8 +42,10 @@ class VTupleIsNullPredicate final : public VExpr { public: explicit VTupleIsNullPredicate(const TExprNode& node); ~VTupleIsNullPredicate() override = default; - Status execute(VExprContext* context, Block* block, int* result_column_id) override; Status prepare(RuntimeState* state, const RowDescriptor& desc, VExprContext* context) override; + Status open(RuntimeState* state, VExprContext* context, + FunctionContext::FunctionStateScope scope) override; + Status execute(VExprContext* context, Block* block, int* result_column_id) override; [[nodiscard]] bool is_constant() const override { return false; } diff --git a/be/src/vec/runtime/vdata_stream_recvr.h b/be/src/vec/runtime/vdata_stream_recvr.h index 141a5c54b64a59..2da6f9f920bf05 100644 --- a/be/src/vec/runtime/vdata_stream_recvr.h +++ b/be/src/vec/runtime/vdata_stream_recvr.h @@ -20,11 +20,11 @@ #include #include #include -#include -#include #include #include +#include +#include #include #include #include @@ -42,14 +42,10 @@ #include "common/object_pool.h" #include "common/status.h" #include "runtime/descriptors.h" -#include "runtime/query_statistics.h" #include "runtime/task_execution_context.h" #include "util/runtime_profile.h" #include "util/stopwatch.hpp" -#include "vec/columns/column.h" #include "vec/core/block.h" -#include "vec/core/column_with_type_and_name.h" -#include "vec/core/materialize_block.h" #include "vec/exprs/vexpr_fwd.h" namespace doris { From b61a057bb06c3db9e949c01282dba7150a96c768 Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Mon, 22 Jan 2024 18:36:15 +0800 Subject: [PATCH 137/200] [fix](Nereids) user identify syntax is wrong (#30226) --- .../src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 709ee61f54adfc..1f3e11675381c5 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -176,7 +176,7 @@ identifierOrText ; userIdentify - : user=identifierOrText (AT (host=identifierOrText | LEFT_PAREN host=identifierOrText RIGHT_PAREN))? + : user=identifierOrText (ATSIGN (host=identifierOrText | LEFT_PAREN host=identifierOrText RIGHT_PAREN))? ; From 3922c9ac01837cfafa7a5644afbe208eb3ed7417 Mon Sep 17 00:00:00 2001 From: Lei Zhang <27994433+SWJTU-ZhangLei@users.noreply.github.com> Date: Mon, 22 Jan 2024 18:48:02 +0800 Subject: [PATCH 138/200] [feature](merge-cloud) Add CloudGlobalTransactionMgr (#30117) --- .../org/apache/doris/alter/AlterJobV2.java | 4 +- .../apache/doris/alter/IndexChangeJob.java | 5 +- .../org/apache/doris/alter/RollupJobV2.java | 5 +- .../doris/alter/SchemaChangeHandler.java | 73 +- .../apache/doris/alter/SchemaChangeJobV2.java | 5 +- .../java/org/apache/doris/catalog/Env.java | 10 +- .../org/apache/doris/catalog/EnvFactory.java | 9 +- .../doris/catalog/TabletInvertedIndex.java | 4 +- .../apache/doris/clone/TabletSchedCtx.java | 2 +- .../apache/doris/clone/TabletScheduler.java | 39 +- .../apache/doris/cloud/catalog/CloudEnv.java | 14 + .../doris/cloud/catalog/CloudEnvFactory.java | 6 + .../CloudGlobalTransactionMgr.java | 836 ++++++++++++++++++ .../doris/cloud/transaction/TxnUtil.java | 357 ++++++++ .../doris/common/proc/TransDbProcDir.java | 4 +- .../common/proc/TransPartitionProcNode.java | 4 +- .../doris/common/proc/TransProcDir.java | 4 +- .../doris/common/proc/TransStateProcDir.java | 4 +- .../doris/common/proc/TransTablesProcDir.java | 4 +- .../doris/httpv2/rest/GetStreamLoadState.java | 8 +- .../java/org/apache/doris/load/DeleteJob.java | 4 +- .../apache/doris/load/loadv2/LoadManager.java | 2 +- .../doris/load/loadv2/SparkLoadJob.java | 2 +- .../doris/load/routineload/KafkaProgress.java | 9 + .../RLTaskTxnCommitAttachment.java | 18 + .../load/sync/canal/CanalSyncChannel.java | 4 +- .../org/apache/doris/qe/ShowExecutor.java | 4 +- .../DbUsedDataQuotaInfoCollector.java | 2 +- .../transaction/GlobalTransactionMgr.java | 430 ++++----- .../GlobalTransactionMgrIface.java | 184 ++++ .../transaction/PublishVersionDaemon.java | 2 +- .../doris/transaction/TransactionState.java | 16 + .../apache/doris/alter/RollupJobV2Test.java | 6 +- .../doris/alter/SchemaChangeJobV2Test.java | 4 +- .../CloudGlobalTransactionMgrTest.java | 407 +++++++++ .../DatabaseTransactionMgrTest.java | 4 +- .../transaction/GlobalTransactionMgrTest.java | 4 +- 37 files changed, 2191 insertions(+), 308 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgrIface.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgrTest.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java index c63623e8b8d764..828a7650c588f9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java @@ -217,7 +217,7 @@ public synchronized void run() { default: break; } - } catch (AlterCancelException e) { + } catch (Exception e) { cancelImpl(e.getMessage()); } } @@ -260,7 +260,7 @@ protected boolean checkTableStable(Database db) throws AlterCancelException { } } - protected abstract void runPendingJob() throws AlterCancelException; + protected abstract void runPendingJob() throws Exception; protected abstract void runWaitingTxnJob() throws AlterCancelException; diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/IndexChangeJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/IndexChangeJob.java index e6f4c4e0a0e4e5..57a5f9d9ec6df3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/IndexChangeJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/IndexChangeJob.java @@ -117,7 +117,7 @@ public IndexChangeJob() { this.jobState = JobState.WAITING_TXN; } - public IndexChangeJob(long jobId, long dbId, long tableId, String tableName) { + public IndexChangeJob(long jobId, long dbId, long tableId, String tableName) throws Exception { this.jobId = jobId; this.dbId = dbId; this.tableId = tableId; @@ -125,8 +125,7 @@ public IndexChangeJob(long jobId, long dbId, long tableId, String tableName) { this.createTimeMs = System.currentTimeMillis(); this.jobState = JobState.WAITING_TXN; - this.watershedTxnId = Env.getCurrentGlobalTransactionMgr() - .getTransactionIDGenerator().getNextTransactionId(); + this.watershedTxnId = Env.getCurrentGlobalTransactionMgr().getNextTransactionId(); } public long getJobId() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java index aa61eaf9062a7e..bc9c5b0ae438ab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java @@ -203,7 +203,7 @@ private void initAnalyzer() throws AnalysisException { * 3. Get a new transaction id, then set job's state to WAITING_TXN */ @Override - protected void runPendingJob() throws AlterCancelException { + protected void runPendingJob() throws Exception { Preconditions.checkState(jobState == JobState.PENDING, jobState); LOG.info("begin to send create rollup replica tasks. job: {}", jobId); @@ -331,8 +331,7 @@ protected void runPendingJob() throws AlterCancelException { tbl.writeUnlock(); } - this.watershedTxnId = Env.getCurrentGlobalTransactionMgr() - .getTransactionIDGenerator().getNextTransactionId(); + this.watershedTxnId = Env.getCurrentGlobalTransactionMgr().getNextTransactionId(); this.jobState = JobState.WAITING_TXN; // write edit log diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index dbb9ee1466d0c8..a3f8977332b06d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -2891,45 +2891,50 @@ public void buildOrDeleteTableInvertedIndices(Database db, OlapTable olapTable, throw new DdlException("Nothing is changed. please check your alter stmt."); } - for (Map.Entry> entry : changedIndexIdToSchema.entrySet()) { - long originIndexId = entry.getKey(); - for (Partition partition : olapTable.getPartitions()) { - // create job - long jobId = Env.getCurrentEnv().getNextId(); - IndexChangeJob indexChangeJob = new IndexChangeJob( - jobId, db.getId(), olapTable.getId(), olapTable.getName()); - indexChangeJob.setOriginIndexId(originIndexId); - indexChangeJob.setAlterInvertedIndexInfo(isDropOp, alterIndexes); - long partitionId = partition.getId(); - String partitionName = partition.getName(); - boolean found = false; - for (Set partitions : invertedIndexOnPartitions.values()) { - if (partitions.contains(partitionName)) { - found = true; - break; + try { + for (Map.Entry> entry : changedIndexIdToSchema.entrySet()) { + long originIndexId = entry.getKey(); + for (Partition partition : olapTable.getPartitions()) { + // create job + long jobId = Env.getCurrentEnv().getNextId(); + IndexChangeJob indexChangeJob = new IndexChangeJob( + jobId, db.getId(), olapTable.getId(), olapTable.getName()); + indexChangeJob.setOriginIndexId(originIndexId); + indexChangeJob.setAlterInvertedIndexInfo(isDropOp, alterIndexes); + long partitionId = partition.getId(); + String partitionName = partition.getName(); + boolean found = false; + for (Set partitions : invertedIndexOnPartitions.values()) { + if (partitions.contains(partitionName)) { + found = true; + break; + } + } + if (!found) { + continue; } - } - if (!found) { - continue; - } - if (hasIndexChangeJobOnPartition(originIndexId, db.getId(), olapTable.getId(), - partitionName, alterIndexes, isDropOp)) { - throw new DdlException("partition " + partitionName + " has been built specified index." - + " please check your build stmt."); - } + if (hasIndexChangeJobOnPartition(originIndexId, db.getId(), olapTable.getId(), + partitionName, alterIndexes, isDropOp)) { + throw new DdlException("partition " + partitionName + " has been built specified index." + + " please check your build stmt."); + } - indexChangeJob.setPartitionId(partitionId); - indexChangeJob.setPartitionName(partitionName); + indexChangeJob.setPartitionId(partitionId); + indexChangeJob.setPartitionName(partitionName); - addIndexChangeJob(indexChangeJob); + addIndexChangeJob(indexChangeJob); - // write edit log - Env.getCurrentEnv().getEditLog().logIndexChangeJob(indexChangeJob); - LOG.info("finish create table's inverted index job. table: {}, partition: {}, job: {}", - olapTable.getName(), partitionName, jobId); - } // end for partition - } // end for index + // write edit log + Env.getCurrentEnv().getEditLog().logIndexChangeJob(indexChangeJob); + LOG.info("finish create table's inverted index job. table: {}, partition: {}, job: {}", + olapTable.getName(), partitionName, jobId); + } // end for partition + } // end for index + } catch (Exception e) { + LOG.warn("Exception:", e); + throw new UserException(e.getMessage()); + } } public boolean hasIndexChangeJobOnPartition( diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java index 5c74164ae33c36..0dc4ec50350847 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java @@ -204,7 +204,7 @@ private void pruneMeta() { * 3. Get a new transaction id, then set job's state to WAITING_TXN */ @Override - protected void runPendingJob() throws AlterCancelException { + protected void runPendingJob() throws Exception { Preconditions.checkState(jobState == JobState.PENDING, jobState); LOG.info("begin to send create replica tasks. job: {}", jobId); Database db = Env.getCurrentInternalCatalog() @@ -342,8 +342,7 @@ protected void runPendingJob() throws AlterCancelException { tbl.writeUnlock(); } - this.watershedTxnId = Env.getCurrentGlobalTransactionMgr() - .getTransactionIDGenerator().getNextTransactionId(); + this.watershedTxnId = Env.getCurrentGlobalTransactionMgr().getNextTransactionId(); this.jobState = JobState.WAITING_TXN; // write edit log diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index 2c0ff9f692eb7f..fb697c04f8d4f4 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -269,7 +269,7 @@ import org.apache.doris.thrift.TStatusCode; import org.apache.doris.thrift.TStorageMedium; import org.apache.doris.transaction.DbUsedDataQuotaInfoCollector; -import org.apache.doris.transaction.GlobalTransactionMgr; +import org.apache.doris.transaction.GlobalTransactionMgrIface; import org.apache.doris.transaction.PublishVersionDaemon; import com.google.common.base.Joiner; @@ -441,7 +441,7 @@ public class Env { private BrokerMgr brokerMgr; private ResourceMgr resourceMgr; - private GlobalTransactionMgr globalTransactionMgr; + private GlobalTransactionMgrIface globalTransactionMgr; private DeployManager deployManager; @@ -688,7 +688,7 @@ public Env(boolean isCheckpointCatalog) { this.brokerMgr = new BrokerMgr(); this.resourceMgr = new ResourceMgr(); - this.globalTransactionMgr = new GlobalTransactionMgr(this); + this.globalTransactionMgr = EnvFactory.getInstance().createGlobalTransactionMgr(this); this.tabletStatMgr = new TabletStatMgr(); @@ -795,11 +795,11 @@ public ResourceMgr getResourceMgr() { return resourceMgr; } - public static GlobalTransactionMgr getCurrentGlobalTransactionMgr() { + public static GlobalTransactionMgrIface getCurrentGlobalTransactionMgr() { return getCurrentEnv().globalTransactionMgr; } - public GlobalTransactionMgr getGlobalTransactionMgr() { + public GlobalTransactionMgrIface getGlobalTransactionMgr() { return globalTransactionMgr; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/EnvFactory.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/EnvFactory.java index 9ecee918322141..000c974a8fd62c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/EnvFactory.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/EnvFactory.java @@ -22,14 +22,15 @@ import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.system.SystemInfoService; +import org.apache.doris.transaction.GlobalTransactionMgr; +import org.apache.doris.transaction.GlobalTransactionMgrIface; import java.lang.reflect.Type; import java.util.Map; public class EnvFactory { - public EnvFactory() { - } + public EnvFactory() {} private static class SingletonHolder { private static final EnvFactory INSTANCE = @@ -88,4 +89,8 @@ public DynamicPartitionProperty createDynamicPartitionProperty(Map transactionIds = backendTabletInfo.getTransactionIds(); - GlobalTransactionMgr transactionMgr = Env.getCurrentGlobalTransactionMgr(); + GlobalTransactionMgrIface transactionMgr = Env.getCurrentGlobalTransactionMgr(); for (Long transactionId : transactionIds) { TransactionState transactionState = transactionMgr.getTransactionState(tabletMeta.getDbId(), transactionId); diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java index e37ba3315d596f..16767c16a4ba09 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java +++ b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java @@ -1156,7 +1156,7 @@ public void finishCloneTask(CloneTask cloneTask, TFinishTaskRequest request) replica.setNeedFurtherRepair(true); try { long furtherRepairWatermarkTxnTd = Env.getCurrentGlobalTransactionMgr() - .getTransactionIDGenerator().getNextTransactionId(); + .getNextTransactionId(); replica.setFurtherRepairWatermarkTxnTd(furtherRepairWatermarkTxnTd); LOG.info("new replica {} of tablet {} set further repair watermark id {}", replica, tabletId, furtherRepairWatermarkTxnTd); diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java index 59c62f719064ef..dadf4a0b30705e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java @@ -1126,41 +1126,36 @@ private void deleteReplicaInternal(TabletSchedCtx tabletCtx, LOG.info("set replica {} on backend {} of tablet {} state to DECOMMISSION due to reason {}", replica.getId(), replica.getBackendId(), tabletCtx.getTabletId(), reason); } + try { + long preWatermarkTxnId = replica.getPreWatermarkTxnId(); + if (preWatermarkTxnId == -1) { + preWatermarkTxnId = Env.getCurrentGlobalTransactionMgr() + .getTransactionIDGenerator().getNextTransactionId(); + replica.setPreWatermarkTxnId(preWatermarkTxnId); + LOG.info("set decommission replica {} on backend {} of tablet {} pre watermark txn id {}", + replica.getId(), replica.getBackendId(), tabletCtx.getTabletId(), preWatermarkTxnId); + } - long preWatermarkTxnId = replica.getPreWatermarkTxnId(); - if (preWatermarkTxnId == -1) { - preWatermarkTxnId = Env.getCurrentGlobalTransactionMgr() - .getTransactionIDGenerator().getNextTransactionId(); - replica.setPreWatermarkTxnId(preWatermarkTxnId); - LOG.info("set decommission replica {} on backend {} of tablet {} pre watermark txn id {}", - replica.getId(), replica.getBackendId(), tabletCtx.getTabletId(), preWatermarkTxnId); - } - - long postWatermarkTxnId = replica.getPostWatermarkTxnId(); - if (postWatermarkTxnId == -1) { - try { + long postWatermarkTxnId = replica.getPostWatermarkTxnId(); + if (postWatermarkTxnId == -1) { if (!Env.getCurrentGlobalTransactionMgr().isPreviousTransactionsFinished(preWatermarkTxnId, tabletCtx.getDbId(), tabletCtx.getTblId(), tabletCtx.getPartitionId())) { throw new SchedException(Status.SCHEDULE_FAILED, SubCode.WAITING_DECOMMISSION, "wait txn before pre watermark txn " + preWatermarkTxnId + " to be finished"); } - } catch (AnalysisException e) { - throw new SchedException(Status.UNRECOVERABLE, e.getMessage()); + postWatermarkTxnId = Env.getCurrentGlobalTransactionMgr().getNextTransactionId(); + + replica.setPostWatermarkTxnId(postWatermarkTxnId); + LOG.info("set decommission replica {} on backend {} of tablet {} post watermark txn id {}", + replica.getId(), replica.getBackendId(), tabletCtx.getTabletId(), postWatermarkTxnId); } - postWatermarkTxnId = Env.getCurrentGlobalTransactionMgr() - .getTransactionIDGenerator().getNextTransactionId(); - replica.setPostWatermarkTxnId(postWatermarkTxnId); - LOG.info("set decommission replica {} on backend {} of tablet {} post watermark txn id {}", - replica.getId(), replica.getBackendId(), tabletCtx.getTabletId(), postWatermarkTxnId); - } - try { if (!Env.getCurrentGlobalTransactionMgr().isPreviousTransactionsFinished(postWatermarkTxnId, tabletCtx.getDbId(), tabletCtx.getTblId(), tabletCtx.getPartitionId())) { throw new SchedException(Status.SCHEDULE_FAILED, SubCode.WAITING_DECOMMISSION, "wait txn before post watermark txn " + postWatermarkTxnId + " to be finished"); } - } catch (AnalysisException e) { + } catch (Exception e) { throw new SchedException(Status.UNRECOVERABLE, e.getMessage()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java index 85162fec825246..d6e6b51f2b2d50 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java @@ -18,10 +18,14 @@ package org.apache.doris.cloud.catalog; import org.apache.doris.catalog.Env; +import org.apache.doris.common.io.CountingDataOutputStream; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.io.DataInputStream; +import java.io.IOException; + public class CloudEnv extends Env { private static final Logger LOG = LogManager.getLogger(CloudEnv.class); @@ -30,5 +34,15 @@ public CloudEnv(boolean isCheckpointCatalog) { super(isCheckpointCatalog); } + @Override + public long loadTransactionState(DataInputStream dis, long checksum) throws IOException { + // for CloudGlobalTransactionMgr do nothing. + return checksum; + } + + @Override + public long saveTransactionState(CountingDataOutputStream dos, long checksum) throws IOException { + return checksum; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnvFactory.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnvFactory.java index 46b916b9dbb856..42173592c7a570 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnvFactory.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnvFactory.java @@ -27,9 +27,11 @@ import org.apache.doris.cloud.common.util.CloudPropertyAnalyzer; import org.apache.doris.cloud.datasource.CloudInternalCatalog; import org.apache.doris.cloud.system.CloudSystemInfoService; +import org.apache.doris.cloud.transaction.CloudGlobalTransactionMgr; import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.system.SystemInfoService; +import org.apache.doris.transaction.GlobalTransactionMgrIface; import java.lang.reflect.Type; import java.util.Map; @@ -99,4 +101,8 @@ public DynamicPartitionProperty createDynamicPartitionProperty(Map tableIdList, String label, TxnCoordinator coordinator, + LoadJobSourceType sourceType, long timeoutSecond) + throws AnalysisException, LabelAlreadyUsedException, BeginTransactionException, DuplicatedRequestException, + QuotaExceedException, MetaNotFoundException { + return beginTransaction(dbId, tableIdList, label, null, coordinator, sourceType, -1, timeoutSecond); + } + + @Override + public long beginTransaction(long dbId, List tableIdList, String label, TUniqueId requestId, + TxnCoordinator coordinator, LoadJobSourceType sourceType, long listenerId, long timeoutSecond) + throws AnalysisException, LabelAlreadyUsedException, BeginTransactionException, DuplicatedRequestException, + QuotaExceedException, MetaNotFoundException { + + LOG.info("try to begin transaction, dbId: {}, label: {}", dbId, label); + if (Config.disable_load_job) { + throw new AnalysisException("disable_load_job is set to true, all load jobs are prevented"); + } + + switch (sourceType) { + case BACKEND_STREAMING: + checkValidTimeoutSecond(timeoutSecond, Config.max_stream_load_timeout_second, + Config.min_load_timeout_second); + break; + default: + checkValidTimeoutSecond(timeoutSecond, Config.max_load_timeout_second, Config.min_load_timeout_second); + } + + BeginTxnResponse beginTxnResponse = null; + int retryTime = 0; + + try { + Preconditions.checkNotNull(coordinator); + Preconditions.checkNotNull(label); + FeNameFormat.checkLabel(label); + + TxnInfoPB.Builder txnInfoBuilder = TxnInfoPB.newBuilder(); + txnInfoBuilder.setDbId(dbId); + txnInfoBuilder.addAllTableIds(tableIdList); + txnInfoBuilder.setLabel(label); + txnInfoBuilder.setListenerId(listenerId); + + if (requestId != null) { + UniqueIdPB.Builder uniqueIdBuilder = UniqueIdPB.newBuilder(); + uniqueIdBuilder.setHi(requestId.getHi()); + uniqueIdBuilder.setLo(requestId.getLo()); + txnInfoBuilder.setRequestId(uniqueIdBuilder); + } + + txnInfoBuilder.setCoordinator(TxnUtil.txnCoordinatorToPb(coordinator)); + txnInfoBuilder.setLoadJobSourceType(LoadJobSourceTypePB.forNumber(sourceType.value())); + txnInfoBuilder.setTimeoutMs(timeoutSecond * 1000); + txnInfoBuilder.setPrecommitTimeoutMs(Config.stream_load_default_precommit_timeout_second * 1000); + + final BeginTxnRequest beginTxnRequest = BeginTxnRequest.newBuilder() + .setTxnInfo(txnInfoBuilder.build()) + .setCloudUniqueId(Config.cloud_unique_id) + .build(); + + while (retryTime < Config.cloud_meta_service_rpc_failed_retry_times) { + LOG.debug("retryTime:{}, beginTxnRequest:{}", retryTime, beginTxnRequest); + beginTxnResponse = MetaServiceProxy.getInstance().beginTxn(beginTxnRequest); + LOG.debug("retryTime:{}, beginTxnResponse:{}", retryTime, beginTxnResponse); + + if (beginTxnResponse.getStatus().getCode() != MetaServiceCode.KV_TXN_CONFLICT) { + break; + } + LOG.info("beginTxn KV_TXN_CONFLICT, retryTime:{}", retryTime); + backoff(); + retryTime++; + continue; + } + + Preconditions.checkNotNull(beginTxnResponse); + Preconditions.checkNotNull(beginTxnResponse.getStatus()); + } catch (Exception e) { + LOG.warn("beginTxn failed, exception:", e); + throw new BeginTransactionException("beginTxn failed, errMsg:" + e.getMessage()); + } + + if (beginTxnResponse.getStatus().getCode() != MetaServiceCode.OK) { + switch (beginTxnResponse.getStatus().getCode()) { + case TXN_DUPLICATED_REQ: + throw new DuplicatedRequestException(DebugUtil.printId(requestId), + beginTxnResponse.getDupTxnId(), beginTxnResponse.getStatus().getMsg()); + case TXN_LABEL_ALREADY_USED: + throw new LabelAlreadyUsedException(label); + default: + if (MetricRepo.isInit) { + MetricRepo.COUNTER_TXN_REJECT.increase(1L); + } + throw new BeginTransactionException(beginTxnResponse.getStatus().getMsg()); + } + } + + long txnId = beginTxnResponse.getTxnId(); + if (MetricRepo.isInit) { + MetricRepo.COUNTER_TXN_BEGIN.increase(1L); + } + return txnId; + } + + @Override + public void preCommitTransaction2PC(Database db, List
tableList, long transactionId, + List tabletCommitInfos, long timeoutMillis, TxnCommitAttachment txnCommitAttachment) + throws UserException { + Preconditions.checkState(false, "should not implement this in derived class"); + } + + @Override + public void commitTransaction(long dbId, List
tableList, + long transactionId, List tabletCommitInfos) + throws UserException { + commitTransaction(dbId, tableList, transactionId, tabletCommitInfos, null); + } + + @Override + public void commitTransaction(long dbId, List
tableList, long transactionId, + List tabletCommitInfos, TxnCommitAttachment txnCommitAttachment) + throws UserException { + commitTransaction(dbId, tableList, transactionId, tabletCommitInfos, txnCommitAttachment, false); + } + + private void commitTransaction(long dbId, List
tableList, long transactionId, + List tabletCommitInfos, TxnCommitAttachment txnCommitAttachment, boolean is2PC) + throws UserException { + + LOG.info("try to commit transaction, transactionId: {}", transactionId); + if (Config.disable_load_job) { + throw new TransactionCommitFailedException( + "disable_load_job is set to true, all load jobs are not allowed"); + } + + + CommitTxnRequest.Builder builder = CommitTxnRequest.newBuilder(); + builder.setDbId(dbId) + .setTxnId(transactionId) + .setIs2Pc(is2PC) + .setCloudUniqueId(Config.cloud_unique_id); + + if (txnCommitAttachment != null) { + if (txnCommitAttachment instanceof LoadJobFinalOperation) { + LoadJobFinalOperation loadJobFinalOperation = (LoadJobFinalOperation) txnCommitAttachment; + builder.setCommitAttachment(TxnUtil + .loadJobFinalOperationToPb(loadJobFinalOperation)); + } else if (txnCommitAttachment instanceof RLTaskTxnCommitAttachment) { + RLTaskTxnCommitAttachment rlTaskTxnCommitAttachment = (RLTaskTxnCommitAttachment) txnCommitAttachment; + builder.setCommitAttachment(TxnUtil + .rlTaskTxnCommitAttachmentToPb(rlTaskTxnCommitAttachment)); + } else { + throw new UserException("invalid txnCommitAttachment"); + } + } + + final CommitTxnRequest commitTxnRequest = builder.build(); + CommitTxnResponse commitTxnResponse = null; + int retryTime = 0; + + try { + while (retryTime < Config.cloud_meta_service_rpc_failed_retry_times) { + LOG.debug("retryTime:{}, commitTxnRequest:{}", retryTime, commitTxnRequest); + commitTxnResponse = MetaServiceProxy.getInstance().commitTxn(commitTxnRequest); + LOG.debug("retryTime:{}, commitTxnResponse:{}", retryTime, commitTxnResponse); + if (commitTxnResponse.getStatus().getCode() != MetaServiceCode.KV_TXN_CONFLICT) { + break; + } + // sleep random [20, 200] ms, avoid txn conflict + LOG.info("commitTxn KV_TXN_CONFLICT, transactionId:{}, retryTime:{}", transactionId, retryTime); + backoff(); + retryTime++; + continue; + } + + Preconditions.checkNotNull(commitTxnResponse); + Preconditions.checkNotNull(commitTxnResponse.getStatus()); + } catch (Exception e) { + LOG.warn("commitTxn failed, transactionId:{}, exception:", transactionId, e); + throw new UserException("commitTxn() failed, errMsg:" + e.getMessage()); + } + + if (commitTxnResponse.getStatus().getCode() != MetaServiceCode.OK + && commitTxnResponse.getStatus().getCode() != MetaServiceCode.TXN_ALREADY_VISIBLE) { + LOG.warn("commitTxn failed, transactionId:{}, retryTime:{}, commitTxnResponse:{}", + transactionId, retryTime, commitTxnResponse); + StringBuilder internalMsgBuilder = + new StringBuilder("commitTxn failed, transactionId:"); + internalMsgBuilder.append(transactionId); + internalMsgBuilder.append(" code:"); + internalMsgBuilder.append(commitTxnResponse.getStatus().getCode()); + throw new UserException("internal error, " + internalMsgBuilder.toString()); + } + + TransactionState txnState = TxnUtil.transactionStateFromPb(commitTxnResponse.getTxnInfo()); + TxnStateChangeCallback cb = callbackFactory.getCallback(txnState.getCallbackId()); + if (cb != null) { + LOG.info("commitTxn, run txn callback, transactionId:{} callbackId:{}, txnState:{}", + txnState.getTransactionId(), txnState.getCallbackId(), txnState); + cb.afterCommitted(txnState, true); + cb.afterVisible(txnState, true); + } + if (MetricRepo.isInit) { + MetricRepo.COUNTER_TXN_SUCCESS.increase(1L); + MetricRepo.HISTO_TXN_EXEC_LATENCY.update(txnState.getCommitTime() - txnState.getPrepareTime()); + } + } + + @Override + public boolean commitAndPublishTransaction(DatabaseIf db, List
tableList, long transactionId, + List tabletCommitInfos, long timeoutMillis) + throws UserException { + return commitAndPublishTransaction(db, tableList, transactionId, tabletCommitInfos, timeoutMillis, null); + } + + @Override + public boolean commitAndPublishTransaction(DatabaseIf db, List
tableList, long transactionId, + List tabletCommitInfos, long timeoutMillis, + TxnCommitAttachment txnCommitAttachment) throws UserException { + commitTransaction(db.getId(), tableList, transactionId, tabletCommitInfos, txnCommitAttachment); + return true; + } + + @Override + public void commitTransaction2PC(Database db, List
tableList, long transactionId, long timeoutMillis) + throws UserException { + commitTransaction(db.getId(), tableList, transactionId, null, null, true); + } + + @Override + public void abortTransaction(Long dbId, Long transactionId, String reason) throws UserException { + abortTransaction(dbId, transactionId, reason, null, null); + } + + @Override + public void abortTransaction(Long dbId, Long transactionId, String reason, + TxnCommitAttachment txnCommitAttachment, List
tableList) throws UserException { + LOG.info("try to abort transaction, dbId:{}, transactionId:{}", dbId, transactionId); + + AbortTxnRequest.Builder builder = AbortTxnRequest.newBuilder(); + builder.setDbId(dbId); + builder.setTxnId(transactionId); + builder.setReason(reason); + builder.setCloudUniqueId(Config.cloud_unique_id); + + final AbortTxnRequest abortTxnRequest = builder.build(); + AbortTxnResponse abortTxnResponse = null; + int retryTime = 0; + try { + while (retryTime < Config.cloud_meta_service_rpc_failed_retry_times) { + LOG.debug("retryTime:{}, abortTxnRequest:{}", retryTime, abortTxnRequest); + abortTxnResponse = MetaServiceProxy + .getInstance().abortTxn(abortTxnRequest); + LOG.debug("retryTime:{}, abortTxnResponse:{}", retryTime, abortTxnResponse); + if (abortTxnResponse.getStatus().getCode() != MetaServiceCode.KV_TXN_CONFLICT) { + break; + } + // sleep random [20, 200] ms, avoid txn conflict + LOG.info("abortTxn KV_TXN_CONFLICT, transactionId:{}, retryTime:{}", transactionId, retryTime); + backoff(); + retryTime++; + continue; + } + Preconditions.checkNotNull(abortTxnResponse); + Preconditions.checkNotNull(abortTxnResponse.getStatus()); + } catch (RpcException e) { + LOG.warn("abortTxn failed, transactionId:{}, Exception", transactionId, e); + throw new UserException("abortTxn failed, errMsg:" + e.getMessage()); + } + + TransactionState txnState = TxnUtil.transactionStateFromPb(abortTxnResponse.getTxnInfo()); + TxnStateChangeCallback cb = callbackFactory.getCallback(txnState.getCallbackId()); + if (cb != null) { + LOG.info("run txn callback, txnId:{} callbackId:{}", txnState.getTransactionId(), + txnState.getCallbackId()); + cb.afterAborted(txnState, true, txnState.getReason()); + } + if (MetricRepo.isInit) { + MetricRepo.COUNTER_TXN_FAILED.increase(1L); + } + } + + @Override + public void abortTransaction(Long dbId, String label, String reason) throws UserException { + LOG.info("try to abort transaction, dbId:{}, label:{}", dbId, label); + + AbortTxnRequest.Builder builder = AbortTxnRequest.newBuilder(); + builder.setDbId(dbId); + builder.setLabel(label); + builder.setReason(reason); + builder.setCloudUniqueId(Config.cloud_unique_id); + + final AbortTxnRequest abortTxnRequest = builder.build(); + AbortTxnResponse abortTxnResponse = null; + int retryTime = 0; + + try { + while (retryTime < Config.cloud_meta_service_rpc_failed_retry_times) { + LOG.debug("retyTime:{}, abortTxnRequest:{}", retryTime, abortTxnRequest); + abortTxnResponse = MetaServiceProxy + .getInstance().abortTxn(abortTxnRequest); + LOG.debug("retryTime:{}, abortTxnResponse:{}", retryTime, abortTxnResponse); + if (abortTxnResponse.getStatus().getCode() != MetaServiceCode.KV_TXN_CONFLICT) { + break; + } + + // sleep random [20, 200] ms, avoid txn conflict + LOG.info("abortTxn KV_TXN_CONFLICT, dbId:{}, label:{}, retryTime:{}", dbId, label, retryTime); + backoff(); + retryTime++; + continue; + } + Preconditions.checkNotNull(abortTxnResponse); + Preconditions.checkNotNull(abortTxnResponse.getStatus()); + } catch (Exception e) { + LOG.warn("abortTxn failed, label:{}, exception:", label, e); + throw new UserException("abortTxn failed, errMsg:" + e.getMessage()); + } + + TransactionState txnState = TxnUtil.transactionStateFromPb(abortTxnResponse.getTxnInfo()); + TxnStateChangeCallback cb = callbackFactory.getCallback(txnState.getCallbackId()); + if (cb == null) { + LOG.info("no callback to run for this txn, txnId:{} callbackId:{}", txnState.getTransactionId(), + txnState.getCallbackId()); + return; + } + + LOG.info("run txn callback, txnId:{} callbackId:{}", txnState.getTransactionId(), txnState.getCallbackId()); + cb.afterAborted(txnState, true, txnState.getReason()); + if (MetricRepo.isInit) { + MetricRepo.COUNTER_TXN_FAILED.increase(1L); + } + } + + @Override + public void abortTransaction2PC(Long dbId, long transactionId, List
tableList) throws UserException { + LOG.info("try to abortTransaction2PC, dbId:{}, transactionId:{}", dbId, transactionId); + abortTransaction(dbId, transactionId, "User Abort", null, null); + LOG.info("abortTransaction2PC successfully, dbId:{}, transactionId:{}", dbId, transactionId); + } + + @Override + public List getReadyToPublishTransactions() { + //do nothing for CloudGlobalTransactionMgr + return new ArrayList(); + } + + @Override + public boolean existCommittedTxns(Long dbId, Long tableId, Long partitionId) { + //do nothing for CloudGlobalTransactionMgr + return false; + } + + @Override + public void finishTransaction(long dbId, long transactionId) throws UserException { + throw new UserException("Disallow to call finishTransaction()"); + } + + @Override + public boolean isPreviousTransactionsFinished(long endTransactionId, long dbId, List tableIdList) + throws AnalysisException { + LOG.info("isPreviousTransactionsFinished(), endTransactionId:{}, dbId:{}, tableIdList:{}", + endTransactionId, dbId, tableIdList); + + if (endTransactionId <= 0) { + throw new AnalysisException("Invaid endTransactionId:" + endTransactionId); + } + CheckTxnConflictRequest.Builder builder = CheckTxnConflictRequest.newBuilder(); + builder.setDbId(dbId); + builder.setEndTxnId(endTransactionId); + builder.addAllTableIds(tableIdList); + builder.setCloudUniqueId(Config.cloud_unique_id); + + final CheckTxnConflictRequest checkTxnConflictRequest = builder.build(); + CheckTxnConflictResponse checkTxnConflictResponse = null; + try { + LOG.info("CheckTxnConflictRequest:{}", checkTxnConflictRequest); + checkTxnConflictResponse = MetaServiceProxy + .getInstance().checkTxnConflict(checkTxnConflictRequest); + LOG.info("CheckTxnConflictResponse: {}", checkTxnConflictResponse); + } catch (RpcException e) { + throw new AnalysisException(e.getMessage()); + } + + if (checkTxnConflictResponse.getStatus().getCode() != MetaServiceCode.OK) { + throw new AnalysisException(checkTxnConflictResponse.getStatus().getMsg()); + } + return checkTxnConflictResponse.getFinished(); + } + + public boolean isPreviousTransactionsFinished(long endTransactionId, long dbId, long tableId, + long partitionId) throws AnalysisException { + throw new AnalysisException(NOT_SUPPORTED_MSG); + } + + public boolean isPreviousNonTimeoutTxnFinished(long endTransactionId, long dbId, List tableIdList) + throws AnalysisException { + LOG.info("isPreviousNonTimeoutTxnFinished(), endTransactionId:{}, dbId:{}, tableIdList:{}", + endTransactionId, dbId, tableIdList); + + if (endTransactionId <= 0) { + throw new AnalysisException("Invaid endTransactionId:" + endTransactionId); + } + CheckTxnConflictRequest.Builder builder = CheckTxnConflictRequest.newBuilder(); + builder.setDbId(dbId); + builder.setEndTxnId(endTransactionId); + builder.addAllTableIds(tableIdList); + builder.setCloudUniqueId(Config.cloud_unique_id); + builder.setIgnoreTimeoutTxn(true); + + final CheckTxnConflictRequest checkTxnConflictRequest = builder.build(); + CheckTxnConflictResponse checkTxnConflictResponse = null; + try { + LOG.info("CheckTxnConflictRequest:{}", checkTxnConflictRequest); + checkTxnConflictResponse = MetaServiceProxy + .getInstance().checkTxnConflict(checkTxnConflictRequest); + LOG.info("CheckTxnConflictResponse: {}", checkTxnConflictResponse); + } catch (RpcException e) { + throw new AnalysisException(e.getMessage()); + } + + if (checkTxnConflictResponse.getStatus().getCode() != MetaServiceCode.OK) { + throw new AnalysisException(checkTxnConflictResponse.getStatus().getMsg()); + } + return checkTxnConflictResponse.getFinished(); + } + + @Override + public void removeExpiredAndTimeoutTxns() { + // do nothing in cloud mode + } + + public void cleanLabel(Long dbId, String label, boolean isReplay) throws Exception { + LOG.info("try to cleanLabel dbId: {}, label:{}", dbId, label); + CleanTxnLabelRequest.Builder builder = CleanTxnLabelRequest.newBuilder(); + builder.setDbId(dbId).setCloudUniqueId(Config.cloud_unique_id); + + if (!Strings.isNullOrEmpty(label)) { + builder.addLabels(label); + } + + final CleanTxnLabelRequest cleanTxnLabelRequest = builder.build(); + CleanTxnLabelResponse cleanTxnLabelResponse = null; + int retryTime = 0; + + try { + // 5 times retry is enough for clean label + while (retryTime < 5) { + LOG.debug("retryTime:{}, cleanTxnLabel:{}", retryTime, cleanTxnLabelRequest); + cleanTxnLabelResponse = MetaServiceProxy.getInstance().cleanTxnLabel(cleanTxnLabelRequest); + LOG.debug("retryTime:{}, cleanTxnLabel:{}", retryTime, cleanTxnLabelResponse); + if (cleanTxnLabelResponse.getStatus().getCode() != MetaServiceCode.KV_TXN_CONFLICT) { + break; + } + // sleep random [20, 200] ms, avoid txn conflict + LOG.info("cleanTxnLabel KV_TXN_CONFLICT, dbId:{}, label:{}, retryTime:{}", dbId, label, retryTime); + backoff(); + retryTime++; + continue; + } + + Preconditions.checkNotNull(cleanTxnLabelResponse); + Preconditions.checkNotNull(cleanTxnLabelResponse.getStatus()); + } catch (Exception e) { + LOG.warn("cleanTxnLabel failed, dbId:{}, exception:", dbId, e); + throw new UserException("cleanTxnLabel failed, errMsg:" + e.getMessage()); + } + + if (cleanTxnLabelResponse.getStatus().getCode() != MetaServiceCode.OK) { + LOG.warn("cleanTxnLabel failed, dbId:{} label:{} retryTime:{} cleanTxnLabelResponse:{}", + dbId, label, retryTime, cleanTxnLabelResponse); + throw new UserException("cleanTxnLabel failed, errMsg:" + cleanTxnLabelResponse.getStatus().getMsg()); + } + return; + } + + @Override + public void updateMultiTableRunningTransactionTableIds(Long dbId, Long transactionId, List tableIds) + throws UserException { + throw new UserException(NOT_SUPPORTED_MSG); + } + + @Override + public void putTransactionTableNames(Long dbId, Long transactionId, List tableIds) + throws Exception { + throw new Exception(NOT_SUPPORTED_MSG); + } + + @Override + public TWaitingTxnStatusResult getWaitingTxnStatus(TWaitingTxnStatusRequest request) + throws AnalysisException, TimeoutException { + long dbId = request.getDbId(); + int commitTimeoutSec = Config.commit_timeout_second; + for (int i = 0; i < commitTimeoutSec; ++i) { + Env.getCurrentInternalCatalog().getDbOrAnalysisException(dbId); + TWaitingTxnStatusResult statusResult = new TWaitingTxnStatusResult(); + statusResult.status = new TStatus(); + TransactionStatus txnStatus = null; + if (request.isSetTxnId()) { + long txnId = request.getTxnId(); + TransactionState txnState = Env.getCurrentGlobalTransactionMgr() + .getTransactionState(dbId, txnId); + if (txnState == null) { + throw new AnalysisException("txn does not exist: " + txnId); + } + txnStatus = txnState.getTransactionStatus(); + if (!txnState.getReason().trim().isEmpty()) { + statusResult.status.setErrorMsgsIsSet(true); + statusResult.status.addToErrorMsgs(txnState.getReason()); + } + } else { + txnStatus = getLabelState(dbId, request.getLabel()); + } + if (txnStatus == TransactionStatus.UNKNOWN || txnStatus.isFinalStatus()) { + statusResult.setTxnStatusId(txnStatus.value()); + return statusResult; + } + try { + Thread.sleep(1000L); + } catch (InterruptedException e) { + LOG.info("commit sleep exception.", e); + } + } + throw new TimeoutException("Operation is timeout"); + } + + @Override + public void updateDatabaseUsedQuotaData(long dbId, long usedQuotaDataBytes) throws AnalysisException { + // do nothing in cloud mode + } + + @Override + public void abortTxnWhenCoordinateBeDown(String coordinateHost, int limit) { + // do nothing in cloud mode + } + + @Override + public TransactionStatus getLabelState(long dbId, String label) throws AnalysisException { + throw new AnalysisException(NOT_SUPPORTED_MSG); + } + + @Override + public Long getTransactionId(Long dbId, String label) throws AnalysisException { + throw new AnalysisException(NOT_SUPPORTED_MSG); + } + + @Override + public TransactionState getTransactionState(long dbId, long transactionId) { + LOG.info("try to get transaction state, dbId:{}, transactionId:{}", dbId, transactionId); + GetTxnRequest.Builder builder = GetTxnRequest.newBuilder(); + builder.setDbId(dbId); + builder.setTxnId(transactionId); + builder.setCloudUniqueId(Config.cloud_unique_id); + + final GetTxnRequest getTxnRequest = builder.build(); + GetTxnResponse getTxnResponse = null; + try { + LOG.info("getTxnRequest:{}", getTxnRequest); + getTxnResponse = MetaServiceProxy + .getInstance().getTxn(getTxnRequest); + LOG.info("getTxnRequest: {}", getTxnResponse); + } catch (RpcException e) { + LOG.info("getTransactionState exception: {}", e.getMessage()); + return null; + } + + if (getTxnResponse.getStatus().getCode() != MetaServiceCode.OK || !getTxnResponse.hasTxnInfo()) { + LOG.info("getTransactionState exception: {}, {}", getTxnResponse.getStatus().getCode(), + getTxnResponse.getStatus().getMsg()); + return null; + } + return TxnUtil.transactionStateFromPb(getTxnResponse.getTxnInfo()); + } + + @Override + public Long getTransactionIdByLabel(Long dbId, String label, List statusList) + throws UserException { + throw new UserException(NOT_SUPPORTED_MSG); + } + + @Override + public List getPreCommittedTxnList(Long dbId) throws AnalysisException { + // todo + return new ArrayList(); + } + + @Override + public int getTransactionNum() { + return 0; + } + + @Override + public Long getNextTransactionId() throws UserException { + GetCurrentMaxTxnRequest.Builder builder = GetCurrentMaxTxnRequest.newBuilder(); + builder.setCloudUniqueId(Config.cloud_unique_id); + + final GetCurrentMaxTxnRequest getCurrentMaxTxnRequest = builder.build(); + GetCurrentMaxTxnResponse getCurrentMaxTxnResponse = null; + try { + LOG.info("GetCurrentMaxTxnRequest:{}", getCurrentMaxTxnRequest); + getCurrentMaxTxnResponse = MetaServiceProxy + .getInstance().getCurrentMaxTxnId(getCurrentMaxTxnRequest); + LOG.info("GetCurrentMaxTxnResponse: {}", getCurrentMaxTxnResponse); + } catch (RpcException e) { + LOG.warn("getNextTransactionId() RpcException: {}", e.getMessage()); + throw new UserException("getNextTransactionId() RpcException: " + e.getMessage()); + } + + if (getCurrentMaxTxnResponse.getStatus().getCode() != MetaServiceCode.OK) { + LOG.info("getNextTransactionId() failed, code: {}, msg: {}", + getCurrentMaxTxnResponse.getStatus().getCode(), getCurrentMaxTxnResponse.getStatus().getMsg()); + throw new UserException("getNextTransactionId() failed, msg:" + + getCurrentMaxTxnResponse.getStatus().getMsg()); + } + return getCurrentMaxTxnResponse.getCurrentMaxTxnId(); + } + + @Override + public int getRunningTxnNums(Long dbId) throws AnalysisException { + return 0; + } + + @Override + public long getTxnNumByStatus(TransactionStatus status) { + // TODO Auto-generated method stub + return 0; + } + + @Override + public long getAllRunningTxnNum() { + return 0; + } + + @Override + public long getAllPublishTxnNum() { + return 0; + } + + /** + * backoff policy implement by sleep random ms in [20ms, 200ms] + */ + private void backoff() { + int randomMillis = 20 + (int) (Math.random() * (200 - 20)); + try { + Thread.sleep(randomMillis); + } catch (InterruptedException e) { + LOG.info("InterruptedException: ", e); + } + } + + @Override + public TransactionIdGenerator getTransactionIDGenerator() throws Exception { + throw new Exception(NOT_SUPPORTED_MSG); + } + + @Override + public List> getDbInfo() throws AnalysisException { + throw new AnalysisException(NOT_SUPPORTED_MSG); + } + + @Override + public List> getDbTransStateInfo(Long dbId) throws AnalysisException { + throw new AnalysisException(NOT_SUPPORTED_MSG); + } + + @Override + public List> getDbTransInfo(Long dbId, boolean running, int limit) throws AnalysisException { + throw new AnalysisException(NOT_SUPPORTED_MSG); + } + + @Override + public List> getDbTransInfoByStatus(Long dbId, TransactionStatus status) throws AnalysisException { + throw new AnalysisException(NOT_SUPPORTED_MSG); + } + + @Override + public List> getSingleTranInfo(long dbId, long txnId) throws AnalysisException { + throw new AnalysisException(NOT_SUPPORTED_MSG); + } + + @Override + public List> getTableTransInfo(long dbId, long txnId) throws AnalysisException { + throw new AnalysisException(NOT_SUPPORTED_MSG); + } + + @Override + public List> getPartitionTransInfo(long dbId, long tid, long tableId) + throws AnalysisException { + throw new AnalysisException(NOT_SUPPORTED_MSG); + } + + @Override + public void write(DataOutput out) throws IOException { + throw new IOException(NOT_SUPPORTED_MSG); + } + + @Override + public void readFields(DataInput in) throws IOException { + throw new IOException(NOT_SUPPORTED_MSG); + } + + @Override + public void replayUpsertTransactionState(TransactionState transactionState) throws Exception { + throw new Exception(NOT_SUPPORTED_MSG); + } + + @Deprecated + public void replayDeleteTransactionState(TransactionState transactionState) throws Exception { + throw new Exception(NOT_SUPPORTED_MSG); + } + + @Override + public void replayBatchRemoveTransactions(BatchRemoveTransactionsOperation operation) throws Exception { + throw new Exception(NOT_SUPPORTED_MSG); + } + + @Override + public void replayBatchRemoveTransactionV2(BatchRemoveTransactionsOperationV2 operation) + throws Exception { + throw new Exception(NOT_SUPPORTED_MSG); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java new file mode 100644 index 00000000000000..eeffb8e3be11ee --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java @@ -0,0 +1,357 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.cloud.transaction; + +import org.apache.doris.cloud.proto.Cloud.RLTaskTxnCommitAttachmentPB; +import org.apache.doris.cloud.proto.Cloud.RoutineLoadProgressPB; +import org.apache.doris.cloud.proto.Cloud.TxnCommitAttachmentPB; +import org.apache.doris.cloud.proto.Cloud.TxnCommitAttachmentPB.LoadJobFinalOperationPB; +import org.apache.doris.cloud.proto.Cloud.TxnCommitAttachmentPB.LoadJobFinalOperationPB.EtlStatusPB; +import org.apache.doris.cloud.proto.Cloud.TxnCommitAttachmentPB.LoadJobFinalOperationPB.FailMsgPB; +import org.apache.doris.cloud.proto.Cloud.TxnCommitAttachmentPB.LoadJobFinalOperationPB.JobStatePB; +import org.apache.doris.cloud.proto.Cloud.TxnCoordinatorPB; +import org.apache.doris.cloud.proto.Cloud.TxnInfoPB; +import org.apache.doris.cloud.proto.Cloud.TxnSourceTypePB; +import org.apache.doris.cloud.proto.Cloud.UniqueIdPB; +import org.apache.doris.load.EtlStatus; +import org.apache.doris.load.FailMsg; +import org.apache.doris.load.loadv2.JobState; +import org.apache.doris.load.loadv2.LoadJobFinalOperation; +import org.apache.doris.load.routineload.KafkaProgress; +import org.apache.doris.load.routineload.RLTaskTxnCommitAttachment; +import org.apache.doris.thrift.TEtlState; +import org.apache.doris.thrift.TUniqueId; +import org.apache.doris.transaction.TransactionState; +import org.apache.doris.transaction.TransactionState.LoadJobSourceType; +import org.apache.doris.transaction.TransactionState.TxnCoordinator; +import org.apache.doris.transaction.TransactionState.TxnSourceType; +import org.apache.doris.transaction.TransactionStatus; +import org.apache.doris.transaction.TxnCommitAttachment; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; + +public class TxnUtil { + private static final Logger LOG = LogManager.getLogger(TxnUtil.class); + + public static EtlStatusPB.EtlStatePB etlStateToPb(TEtlState tEtlState) { + switch (tEtlState) { + case RUNNING: + return EtlStatusPB.EtlStatePB.RUNNING; + case FINISHED: + return EtlStatusPB.EtlStatePB.FINISHED; + case CANCELLED: + return EtlStatusPB.EtlStatePB.CANCELLED; + default: + return EtlStatusPB.EtlStatePB.UNKNOWN; + } + } + + public static TEtlState etlStateFromPb(EtlStatusPB.EtlStatePB etlStatePB) { + switch (etlStatePB) { + case RUNNING: + return TEtlState.RUNNING; + case FINISHED: + return TEtlState.FINISHED; + case CANCELLED: + return TEtlState.CANCELLED; + default: + return TEtlState.UNKNOWN; + } + } + + public static JobStatePB jobStateToPb(JobState jobState) { + switch (jobState) { + case PENDING: + return JobStatePB.PENDING; + case ETL: + return JobStatePB.ETL; + case LOADING: + return JobStatePB.LOADING; + case COMMITTED: + return JobStatePB.COMMITTED; + case FINISHED: + return JobStatePB.FINISHED; + case CANCELLED: + return JobStatePB.CANCELLED; + default: + return JobStatePB.UNKNOWN; + } + } + + public static JobState jobStateFromPb(JobStatePB jobStatePb) { + switch (jobStatePb) { + case PENDING: + return JobState.PENDING; + case ETL: + return JobState.ETL; + case LOADING: + return JobState.LOADING; + case COMMITTED: + return JobState.COMMITTED; + case FINISHED: + return JobState.FINISHED; + case CANCELLED: + return JobState.CANCELLED; + default: + return JobState.UNKNOWN; + } + } + + public static FailMsgPB failMsgToPb(FailMsg failMsg) { + FailMsgPB.Builder builder = FailMsgPB.newBuilder(); + builder.setMsg(failMsg.getMsg()); + + switch (failMsg.getCancelType()) { + case USER_CANCEL: + builder.setCancelType(FailMsgPB.CancelTypePB.USER_CANCEL); + break; + case ETL_SUBMIT_FAIL: + builder.setCancelType(FailMsgPB.CancelTypePB.ETL_SUBMIT_FAIL); + break; + case ETL_RUN_FAIL: + builder.setCancelType(FailMsgPB.CancelTypePB.ETL_RUN_FAIL); + break; + case ETL_QUALITY_UNSATISFIED: + builder.setCancelType(FailMsgPB.CancelTypePB.ETL_QUALITY_UNSATISFIED); + break; + case LOAD_RUN_FAIL: + builder.setCancelType(FailMsgPB.CancelTypePB.LOAD_RUN_FAIL); + break; + case TIMEOUT: + builder.setCancelType(FailMsgPB.CancelTypePB.TIMEOUT); + break; + case TXN_UNKNOWN: + builder.setCancelType(FailMsgPB.CancelTypePB.TXN_UNKNOWN); + break; + default: + builder.setCancelType(FailMsgPB.CancelTypePB.UNKNOWN); + break; + } + return builder.build(); + } + + public static FailMsg failMsgFromPb(FailMsgPB failMsgPb) { + FailMsg failMsg = new FailMsg(); + failMsg.setMsg(failMsgPb.getMsg()); + switch (failMsgPb.getCancelType()) { + case USER_CANCEL: + failMsg.setCancelType(FailMsg.CancelType.USER_CANCEL); + break; + case ETL_SUBMIT_FAIL: + failMsg.setCancelType(FailMsg.CancelType.ETL_SUBMIT_FAIL); + break; + case ETL_RUN_FAIL: + failMsg.setCancelType(FailMsg.CancelType.ETL_RUN_FAIL); + break; + case ETL_QUALITY_UNSATISFIED: + failMsg.setCancelType(FailMsg.CancelType.ETL_QUALITY_UNSATISFIED); + break; + case LOAD_RUN_FAIL: + failMsg.setCancelType(FailMsg.CancelType.LOAD_RUN_FAIL); + break; + case TIMEOUT: + failMsg.setCancelType(FailMsg.CancelType.TIMEOUT); + break; + case TXN_UNKNOWN: + failMsg.setCancelType(FailMsg.CancelType.TXN_UNKNOWN); + break; + default: + failMsg.setCancelType(FailMsg.CancelType.UNKNOWN); + break; + } + return failMsg; + } + + public static EtlStatusPB etlStatusToPb(EtlStatus etlStatus) { + EtlStatusPB.Builder builder = EtlStatusPB.newBuilder(); + builder.setState(TxnUtil.etlStateToPb(etlStatus.getState())) + .setTrackingUrl(etlStatus.getTrackingUrl()) + .putAllStats(etlStatus.getStats()) + .putAllCounters(etlStatus.getCounters()); + return builder.build(); + } + + public static EtlStatus etlStatusFromPb(EtlStatusPB etlStatusPB) { + EtlStatus etlStatus = new EtlStatus(); + + etlStatus.setState(TxnUtil.etlStateFromPb(etlStatusPB.getState())); + etlStatus.setTrackingUrl(etlStatusPB.getTrackingUrl()); + etlStatus.setStats(etlStatusPB.getStats()); + etlStatus.setCounters(etlStatusPB.getCounters()); + return etlStatus; + } + + public static TxnCommitAttachmentPB loadJobFinalOperationToPb(LoadJobFinalOperation loadJobFinalOperation) { + LOG.info("loadJobFinalOperation:{}", loadJobFinalOperation); + TxnCommitAttachmentPB.Builder attachementBuilder = TxnCommitAttachmentPB.newBuilder(); + attachementBuilder.setType(TxnCommitAttachmentPB.Type.LODD_JOB_FINAL_OPERATION); + + TxnCommitAttachmentPB.LoadJobFinalOperationPB.Builder builder = + TxnCommitAttachmentPB.LoadJobFinalOperationPB.newBuilder(); + + builder.setId(loadJobFinalOperation.getId()) + .setLoadingStatus(TxnUtil.etlStatusToPb(loadJobFinalOperation.getLoadingStatus())) + .setProgress(loadJobFinalOperation.getProgress()) + .setLoadStartTimestamp(loadJobFinalOperation.getLoadStartTimestamp()) + .setFinishTimestamp(loadJobFinalOperation.getFinishTimestamp()) + .setJobState(TxnUtil.jobStateToPb(loadJobFinalOperation.getJobState())); + + if (loadJobFinalOperation.getFailMsg() != null) { + builder.setFailMsg(TxnUtil.failMsgToPb(loadJobFinalOperation.getFailMsg())); + } + + attachementBuilder.setLoadJobFinalOperation(builder.build()); + return attachementBuilder.build(); + } + + public static TxnCommitAttachmentPB rlTaskTxnCommitAttachmentToPb(RLTaskTxnCommitAttachment + rtTaskTxnCommitAttachment) { + LOG.info("rtTaskTxnCommitAttachment:{}", rtTaskTxnCommitAttachment); + TxnCommitAttachmentPB.Builder attachementBuilder = TxnCommitAttachmentPB.newBuilder(); + attachementBuilder.setType(TxnCommitAttachmentPB.Type.RT_TASK_TXN_COMMIT_ATTACHMENT); + + RLTaskTxnCommitAttachmentPB.Builder builder = + RLTaskTxnCommitAttachmentPB.newBuilder(); + + UniqueIdPB.Builder taskIdBuilder = UniqueIdPB.newBuilder(); + taskIdBuilder.setHi(rtTaskTxnCommitAttachment.getTaskId().getHi()); + taskIdBuilder.setLo(rtTaskTxnCommitAttachment.getTaskId().getLo()); + + RoutineLoadProgressPB.Builder progressBuilder = RoutineLoadProgressPB.newBuilder(); + progressBuilder.putAllPartitionToOffset(((KafkaProgress) rtTaskTxnCommitAttachment.getProgress()) + .getOffsetByPartition()); + + builder.setJobId(rtTaskTxnCommitAttachment.getJobId()) + .setTaskId(taskIdBuilder) + .setFilteredRows(rtTaskTxnCommitAttachment.getFilteredRows()) + .setLoadedRows(rtTaskTxnCommitAttachment.getLoadedRows()) + .setProgress(progressBuilder) + .setUnselectedRows(rtTaskTxnCommitAttachment.getUnselectedRows()) + .setReceivedBytes(rtTaskTxnCommitAttachment.getReceivedBytes()) + .setTaskExecutionTimeMs(rtTaskTxnCommitAttachment.getTaskExecutionTimeMs()); + + if (rtTaskTxnCommitAttachment.getErrorLogUrl() != null) { + builder.setErrorLogUrl(rtTaskTxnCommitAttachment.getErrorLogUrl()); + } + + attachementBuilder.setRlTaskTxnCommitAttachment(builder.build()); + return attachementBuilder.build(); + } + + public static RLTaskTxnCommitAttachment rtTaskTxnCommitAttachmentFromPb( + TxnCommitAttachmentPB txnCommitAttachmentPB) { + RLTaskTxnCommitAttachmentPB rlTaskTxnCommitAttachmentPB = txnCommitAttachmentPB.getRlTaskTxnCommitAttachment(); + LOG.debug("RLTaskTxnCommitAttachmentPB={}", rlTaskTxnCommitAttachmentPB); + return new RLTaskTxnCommitAttachment(txnCommitAttachmentPB.getRlTaskTxnCommitAttachment()); + } + + public static LoadJobFinalOperation loadJobFinalOperationFromPb(TxnCommitAttachmentPB txnCommitAttachmentPB) { + LoadJobFinalOperationPB loadJobFinalOperationPB = txnCommitAttachmentPB.getLoadJobFinalOperation(); + LOG.debug("loadJobFinalOperationPB={}", loadJobFinalOperationPB); + FailMsg failMsg = loadJobFinalOperationPB.hasFailMsg() + ? TxnUtil.failMsgFromPb(loadJobFinalOperationPB.getFailMsg()) : null; + return new LoadJobFinalOperation(loadJobFinalOperationPB.getId(), + TxnUtil.etlStatusFromPb(loadJobFinalOperationPB.getLoadingStatus()), + loadJobFinalOperationPB.getProgress(), loadJobFinalOperationPB.getLoadStartTimestamp(), + loadJobFinalOperationPB.getFinishTimestamp(), + TxnUtil.jobStateFromPb(loadJobFinalOperationPB.getJobState()), failMsg); + } + + public static TxnCoordinatorPB txnCoordinatorToPb(TxnCoordinator txnCoordinator) { + TxnCoordinatorPB.Builder builder = TxnCoordinatorPB.newBuilder(); + builder.setSourceType(TxnSourceTypePB.forNumber(txnCoordinator.sourceType.value())); + builder.setIp(txnCoordinator.ip); + return builder.build(); + } + + public static TxnCoordinator txnCoordinatorFromPb(TxnCoordinatorPB txnCoordinatorPB) { + TxnCoordinator txnCoordinator = new TxnCoordinator(); + txnCoordinator.sourceType = TxnSourceType.valueOf(txnCoordinatorPB.getSourceType().getNumber()); + txnCoordinator.ip = txnCoordinatorPB.getIp(); + return txnCoordinator; + } + + public static TransactionState transactionStateFromPb(TxnInfoPB txnInfo) { + LOG.debug("txnInfo={}", txnInfo); + long dbId = txnInfo.getDbId(); + List tableIdList = txnInfo.getTableIdsList(); + long transactionId = txnInfo.getTxnId(); + String label = txnInfo.getLabel(); + + TUniqueId requestId = null; + if (txnInfo.hasRequestId()) { + requestId = new TUniqueId(txnInfo.getRequestId().getHi(), txnInfo.getRequestId().getLo()); + } + + LoadJobSourceType loadJobSourceType = null; + if (txnInfo.hasLoadJobSourceType()) { + loadJobSourceType = LoadJobSourceType.valueOf(txnInfo.getLoadJobSourceType().getNumber()); + } + TxnCoordinator txnCoordinator = null; + if (txnInfo.hasCoordinator()) { + txnCoordinator = TxnUtil.txnCoordinatorFromPb(txnInfo.getCoordinator()); + } + + TransactionStatus transactionStatus = TransactionStatus.valueOf(txnInfo.getStatus().getNumber()); + String reason = txnInfo.getReason(); + long callbackId = txnInfo.getListenerId(); + long timeoutMs = txnInfo.getTimeoutMs(); + + TxnCommitAttachment commitAttachment = null; + if (txnInfo.hasCommitAttachment()) { + if (txnInfo.getCommitAttachment().getType() == TxnCommitAttachmentPB.Type.LODD_JOB_FINAL_OPERATION) { + commitAttachment = + TxnUtil.loadJobFinalOperationFromPb(txnInfo.getCommitAttachment()); + } + + if (txnInfo.getCommitAttachment().getType() == TxnCommitAttachmentPB.Type.RT_TASK_TXN_COMMIT_ATTACHMENT) { + commitAttachment = + TxnUtil.rtTaskTxnCommitAttachmentFromPb(txnInfo.getCommitAttachment()); + } + } + + long prepareTime = txnInfo.hasPrepareTime() ? txnInfo.getPrepareTime() : -1; + long preCommitTime = txnInfo.hasPrecommitTime() ? txnInfo.getPrecommitTime() : -1; + long commitTime = txnInfo.hasCommitTime() ? txnInfo.getCommitTime() : -1; + long finishTime = txnInfo.hasFinishTime() ? txnInfo.getFinishTime() : -1; + + TransactionState transactionState = new TransactionState( + dbId, + tableIdList, + transactionId, + label, + requestId, + loadJobSourceType, + txnCoordinator, + transactionStatus, + reason, + callbackId, + timeoutMs, + commitAttachment, + prepareTime, + preCommitTime, + commitTime, + finishTime + ); + LOG.debug("transactionState={}", transactionState); + return transactionState; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TransDbProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TransDbProcDir.java index 74aa71116d9a67..909243f1547bdb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TransDbProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TransDbProcDir.java @@ -19,7 +19,7 @@ import org.apache.doris.catalog.Env; import org.apache.doris.common.AnalysisException; -import org.apache.doris.transaction.GlobalTransactionMgr; +import org.apache.doris.transaction.GlobalTransactionMgrIface; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; @@ -38,7 +38,7 @@ public TransDbProcDir() { public ProcResult fetchResult() throws AnalysisException { BaseProcResult result = new BaseProcResult(); result.setNames(TITLE_NAMES); - GlobalTransactionMgr transactionMgr = Env.getCurrentGlobalTransactionMgr(); + GlobalTransactionMgrIface transactionMgr = Env.getCurrentGlobalTransactionMgr(); result.addRows(transactionMgr.getDbInfo()); return result; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TransPartitionProcNode.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TransPartitionProcNode.java index 2c22667944ecbb..c2ae3a4b1c5ef8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TransPartitionProcNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TransPartitionProcNode.java @@ -19,7 +19,7 @@ import org.apache.doris.catalog.Env; import org.apache.doris.common.AnalysisException; -import org.apache.doris.transaction.GlobalTransactionMgr; +import org.apache.doris.transaction.GlobalTransactionMgrIface; import com.google.common.collect.ImmutableList; @@ -44,7 +44,7 @@ public TransPartitionProcNode(long dbId, long tid, long tableId) { @Override public ProcResult fetchResult() throws AnalysisException { - GlobalTransactionMgr transactionMgr = Env.getCurrentGlobalTransactionMgr(); + GlobalTransactionMgrIface transactionMgr = Env.getCurrentGlobalTransactionMgr(); List> partitionInfos = transactionMgr.getPartitionTransInfo(dbId, tid, tableId); // set result BaseProcResult result = new BaseProcResult(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TransProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TransProcDir.java index 91ccf53d0db50d..372aadded20e47 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TransProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TransProcDir.java @@ -19,7 +19,7 @@ import org.apache.doris.catalog.Env; import org.apache.doris.common.AnalysisException; -import org.apache.doris.transaction.GlobalTransactionMgr; +import org.apache.doris.transaction.GlobalTransactionMgrIface; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; @@ -59,7 +59,7 @@ public TransProcDir(long dbId, String state) { public ProcResult fetchResult() throws AnalysisException { BaseProcResult result = new BaseProcResult(); result.setNames(TITLE_NAMES); - GlobalTransactionMgr transactionMgr = Env.getCurrentGlobalTransactionMgr(); + GlobalTransactionMgrIface transactionMgr = Env.getCurrentGlobalTransactionMgr(); List> infos = transactionMgr.getDbTransInfo(dbId, state.equals("running"), MAX_SHOW_ENTRIES); result.setRows(infos); return result; diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TransStateProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TransStateProcDir.java index 0bc9f62acd5803..eab42cfffd0030 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TransStateProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TransStateProcDir.java @@ -19,7 +19,7 @@ import org.apache.doris.catalog.Env; import org.apache.doris.common.AnalysisException; -import org.apache.doris.transaction.GlobalTransactionMgr; +import org.apache.doris.transaction.GlobalTransactionMgrIface; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; @@ -39,7 +39,7 @@ public TransStateProcDir(Long dbId) { public ProcResult fetchResult() throws AnalysisException { BaseProcResult result = new BaseProcResult(); result.setNames(TITLE_NAMES); - GlobalTransactionMgr transactionMgr = Env.getCurrentGlobalTransactionMgr(); + GlobalTransactionMgrIface transactionMgr = Env.getCurrentGlobalTransactionMgr(); result.setRows(transactionMgr.getDbTransStateInfo(dbId)); return result; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TransTablesProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TransTablesProcDir.java index 554bc4dc2031df..48d7bb699fa02c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TransTablesProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TransTablesProcDir.java @@ -20,7 +20,7 @@ import org.apache.doris.catalog.Env; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.util.ListComparator; -import org.apache.doris.transaction.GlobalTransactionMgr; +import org.apache.doris.transaction.GlobalTransactionMgrIface; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; @@ -52,7 +52,7 @@ public boolean register(String name, ProcNodeInterface node) { @Override public ProcResult fetchResult() throws AnalysisException { // get info - GlobalTransactionMgr transactionMgr = Env.getCurrentGlobalTransactionMgr(); + GlobalTransactionMgrIface transactionMgr = Env.getCurrentGlobalTransactionMgr(); List> tableInfos = transactionMgr.getTableTransInfo(dbId, txnId); // sort by table id ListComparator> comparator = new ListComparator>(0); diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetStreamLoadState.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetStreamLoadState.java index 641c46668c1262..95ac8266043f10 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetStreamLoadState.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetStreamLoadState.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Env; -import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.httpv2.entity.ResponseEntityBuilder; import com.google.common.base.Strings; @@ -54,11 +53,10 @@ public Object execute(@PathVariable(value = DB_KEY) final String dbName, Database db; try { db = Env.getCurrentInternalCatalog().getDbOrMetaException(fullDbName); - } catch (MetaNotFoundException e) { + String state = Env.getCurrentGlobalTransactionMgr().getLabelState(db.getId(), label).toString(); + return ResponseEntityBuilder.ok(state); + } catch (Exception e) { return ResponseEntityBuilder.okWithCommonError(e.getMessage()); } - - String state = Env.getCurrentGlobalTransactionMgr().getLabelState(db.getId(), label).toString(); - return ResponseEntityBuilder.ok(state); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteJob.java index a764b42773b0c1..aeb32877009ddd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteJob.java @@ -59,7 +59,7 @@ import org.apache.doris.thrift.TPushType; import org.apache.doris.thrift.TTaskType; import org.apache.doris.transaction.AbstractTxnStateChangeCallback; -import org.apache.doris.transaction.GlobalTransactionMgr; +import org.apache.doris.transaction.GlobalTransactionMgrIface; import org.apache.doris.transaction.TabletCommitInfo; import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TransactionStatus; @@ -424,7 +424,7 @@ public String commit() throws Exception { @Override public void cancel(String reason) { - GlobalTransactionMgr globalTransactionMgr = Env.getCurrentGlobalTransactionMgr(); + GlobalTransactionMgrIface globalTransactionMgr = Env.getCurrentGlobalTransactionMgr(); try { globalTransactionMgr.abortTransaction(deleteInfo.getDbId(), signature, reason); } catch (Exception e) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java index 1840494dcb2a24..77bcff215413e8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java @@ -826,7 +826,7 @@ private void cleanLabelInternal(long dbId, String label, boolean isReplay) { // 2. Remove from DatabaseTransactionMgr try { Env.getCurrentGlobalTransactionMgr().cleanLabel(dbId, label, isReplay); - } catch (AnalysisException e) { + } catch (Exception e) { // just ignore, because we don't want to throw any exception here. LOG.warn("Exception:", e); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java index 69e1420ff34dc5..1bbd98edc64da3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java @@ -490,7 +490,7 @@ private Set submitPushTasks() throws UserException { || !tabletToSentReplicaPushTask.get(tabletId).containsKey(replicaId)) { long backendId = replica.getBackendId(); long taskSignature = Env.getCurrentGlobalTransactionMgr() - .getTransactionIDGenerator().getNextTransactionId(); + .getNextTransactionId(); PushBrokerReaderParams params = getPushBrokerReaderParams(olapTable, indexId); // deep copy TBrokerScanRange because filePath and fileSize will be updated diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaProgress.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaProgress.java index bb31d3ce5460e8..09b7fc9213c805 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaProgress.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaProgress.java @@ -65,6 +65,11 @@ public KafkaProgress(TKafkaRLTaskProgress tKafkaRLTaskProgress) { this.partitionIdToOffset = tKafkaRLTaskProgress.getPartitionCmtOffset(); } + public KafkaProgress(Map partitionIdToOffset) { + super(LoadDataSourceType.KAFKA); + this.partitionIdToOffset = partitionIdToOffset; + } + public Map getPartitionIdToOffset(List partitionIds) { Map result = Maps.newHashMap(); for (Map.Entry entry : partitionIdToOffset.entrySet()) { @@ -85,6 +90,10 @@ public Long getOffsetByPartition(int kafkaPartition) { return partitionIdToOffset.get(kafkaPartition); } + public Map getOffsetByPartition() { + return partitionIdToOffset; + } + public boolean containsPartition(Integer kafkaPartition) { return partitionIdToOffset.containsKey(kafkaPartition); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RLTaskTxnCommitAttachment.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RLTaskTxnCommitAttachment.java index 73417ae63efb54..fd4ceb93f31780 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RLTaskTxnCommitAttachment.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RLTaskTxnCommitAttachment.java @@ -17,6 +17,7 @@ package org.apache.doris.load.routineload; +import org.apache.doris.cloud.proto.Cloud.RLTaskTxnCommitAttachmentPB; import org.apache.doris.thrift.TRLTaskTxnCommitAttachment; import org.apache.doris.thrift.TUniqueId; import org.apache.doris.transaction.TransactionState; @@ -67,6 +68,23 @@ public RLTaskTxnCommitAttachment(TRLTaskTxnCommitAttachment rlTaskTxnCommitAttac } } + public RLTaskTxnCommitAttachment(RLTaskTxnCommitAttachmentPB rlTaskTxnCommitAttachment) { + super(TransactionState.LoadJobSourceType.ROUTINE_LOAD_TASK); + this.jobId = rlTaskTxnCommitAttachment.getJobId(); + this.taskId = new TUniqueId(rlTaskTxnCommitAttachment.getTaskId().getHi(), + rlTaskTxnCommitAttachment.getTaskId().getLo()); + this.filteredRows = rlTaskTxnCommitAttachment.getFilteredRows(); + this.loadedRows = rlTaskTxnCommitAttachment.getLoadedRows(); + this.unselectedRows = rlTaskTxnCommitAttachment.getUnselectedRows(); + this.receivedBytes = rlTaskTxnCommitAttachment.getReceivedBytes(); + this.taskExecutionTimeMs = rlTaskTxnCommitAttachment.getTaskExecutionTimeMs(); + + KafkaProgress progress = new KafkaProgress(rlTaskTxnCommitAttachment.getProgress().getPartitionToOffset()); + + this.progress = progress; + this.errorLogUrl = rlTaskTxnCommitAttachment.getErrorLogUrl(); + } + public long getJobId() { return jobId; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/sync/canal/CanalSyncChannel.java b/fe/fe-core/src/main/java/org/apache/doris/load/sync/canal/CanalSyncChannel.java index 73cfc77f0cb217..3126984d33a3af 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/sync/canal/CanalSyncChannel.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/sync/canal/CanalSyncChannel.java @@ -42,7 +42,7 @@ import org.apache.doris.thrift.TTxnParams; import org.apache.doris.thrift.TUniqueId; import org.apache.doris.transaction.BeginTransactionException; -import org.apache.doris.transaction.GlobalTransactionMgr; +import org.apache.doris.transaction.GlobalTransactionMgrIface; import org.apache.doris.transaction.TransactionEntry; import org.apache.doris.transaction.TransactionState; @@ -121,7 +121,7 @@ public void beginTxn(long batchId) throws UserException, TException, TimeoutExce String label = "label_job" + + jobId + "_channel" + id + "_db" + db.getId() + "_tbl" + tbl.getId() + "_batch" + batchId + "_" + currentTime; String targetColumn = Joiner.on(",").join(columns) + "," + DELETE_COLUMN; - GlobalTransactionMgr globalTransactionMgr = Env.getCurrentGlobalTransactionMgr(); + GlobalTransactionMgrIface globalTransactionMgr = Env.getCurrentGlobalTransactionMgr(); long txnLimit = db.getTransactionQuotaSize(); long runningTxnNums = globalTransactionMgr.getRunningTxnNums(db.getId()); if (runningTxnNums < txnLimit) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java index e006f3761f450f..dc40c9e5d791d1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java @@ -211,7 +211,7 @@ import org.apache.doris.thrift.TCheckStorageFormatResult; import org.apache.doris.thrift.TTaskType; import org.apache.doris.thrift.TUnit; -import org.apache.doris.transaction.GlobalTransactionMgr; +import org.apache.doris.transaction.GlobalTransactionMgrIface; import org.apache.doris.transaction.TransactionStatus; import com.google.common.base.Preconditions; @@ -2235,7 +2235,7 @@ private void handleShowTransaction() throws AnalysisException { DatabaseIf db = ctx.getEnv().getInternalCatalog().getDbOrAnalysisException(showStmt.getDbName()); TransactionStatus status = showStmt.getStatus(); - GlobalTransactionMgr transactionMgr = Env.getCurrentGlobalTransactionMgr(); + GlobalTransactionMgrIface transactionMgr = Env.getCurrentGlobalTransactionMgr(); if (status != TransactionStatus.UNKNOWN) { resultSet = new ShowResultSet(showStmt.getMetaData(), transactionMgr.getDbTransInfoByStatus(db.getId(), status)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/DbUsedDataQuotaInfoCollector.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/DbUsedDataQuotaInfoCollector.java index 3a713bdc63f190..f03da231b9b114 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/DbUsedDataQuotaInfoCollector.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/DbUsedDataQuotaInfoCollector.java @@ -44,7 +44,7 @@ protected void runAfterCatalogReady() { private void updateAllDatabaseUsedDataQuota() { Env env = Env.getCurrentEnv(); List dbIdList = env.getInternalCatalog().getDbIds(); - GlobalTransactionMgr globalTransactionMgr = env.getGlobalTransactionMgr(); + GlobalTransactionMgrIface globalTransactionMgr = env.getGlobalTransactionMgr(); for (Long dbId : dbIdList) { Database db = env.getInternalCatalog().getDbNullable(dbId); if (db == null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java index 4832d66bedb030..10e877f512167d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java @@ -29,7 +29,6 @@ import org.apache.doris.common.Pair; import org.apache.doris.common.QuotaExceedException; import org.apache.doris.common.UserException; -import org.apache.doris.common.io.Writable; import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.metric.AutoMappedMetric; import org.apache.doris.metric.GaugeMetricImpl; @@ -72,25 +71,34 @@ * Attention: all api in txn manager should get db lock or load lock first, then get txn manager's lock, * or there will be dead lock */ -public class GlobalTransactionMgr implements Writable { +public class GlobalTransactionMgr implements GlobalTransactionMgrIface { private static final Logger LOG = LogManager.getLogger(GlobalTransactionMgr.class); - private Map dbIdToDatabaseTransactionMgrs = Maps.newConcurrentMap(); + private Map dbIdToDatabaseTransactionMgrs; - private TransactionIdGenerator idGenerator = new TransactionIdGenerator(); - private TxnStateCallbackFactory callbackFactory = new TxnStateCallbackFactory(); + private TransactionIdGenerator idGenerator; + private TxnStateCallbackFactory callbackFactory; private Env env; public GlobalTransactionMgr(Env env) { this.env = env; + this.dbIdToDatabaseTransactionMgrs = Maps.newConcurrentMap(); + this.idGenerator = new TransactionIdGenerator(); + this.callbackFactory = new TxnStateCallbackFactory(); } + @Override + public void setEditLog(EditLog editLog) { + this.idGenerator.setEditLog(editLog); + } + + @Override public TxnStateCallbackFactory getCallbackFactory() { return callbackFactory; } - public DatabaseTransactionMgr getDatabaseTransactionMgr(long dbId) throws AnalysisException { + protected DatabaseTransactionMgr getDatabaseTransactionMgr(long dbId) throws AnalysisException { DatabaseTransactionMgr dbTransactionMgr = dbIdToDatabaseTransactionMgrs.get(dbId); if (dbTransactionMgr == null) { throw new AnalysisException("databaseTransactionMgr[" + dbId + "] does not exist"); @@ -98,6 +106,7 @@ public DatabaseTransactionMgr getDatabaseTransactionMgr(long dbId) throws Analys return dbTransactionMgr; } + @Override public void addDatabaseTransactionMgr(Long dbId) { if (dbIdToDatabaseTransactionMgrs.putIfAbsent(dbId, new DatabaseTransactionMgr(dbId, env, idGenerator)) == null) { @@ -105,12 +114,14 @@ public void addDatabaseTransactionMgr(Long dbId) { } } + @Override public void removeDatabaseTransactionMgr(Long dbId) { if (dbIdToDatabaseTransactionMgrs.remove(dbId) != null) { LOG.debug("remove database transaction manager for db {}", dbId); } } + @Override public long beginTransaction(long dbId, List tableIdList, String label, TxnCoordinator coordinator, LoadJobSourceType sourceType, long timeoutSecond) throws AnalysisException, LabelAlreadyUsedException, BeginTransactionException, DuplicatedRequestException, @@ -130,6 +141,7 @@ public long beginTransaction(long dbId, List tableIdList, String label, Tx * @throws BeginTransactionException * @throws DuplicatedRequestException */ + @Override public long beginTransaction(long dbId, List tableIdList, String label, TUniqueId requestId, TxnCoordinator coordinator, LoadJobSourceType sourceType, long listenerId, long timeoutSecond) throws AnalysisException, LabelAlreadyUsedException, BeginTransactionException, DuplicatedRequestException, @@ -162,36 +174,7 @@ public long beginTransaction(long dbId, List tableIdList, String label, TU } } - private void checkValidTimeoutSecond(long timeoutSecond, int maxLoadTimeoutSecond, - int minLoadTimeOutSecond) throws AnalysisException { - if (timeoutSecond > maxLoadTimeoutSecond || timeoutSecond < minLoadTimeOutSecond) { - throw new AnalysisException("Invalid timeout: " + timeoutSecond + ". Timeout should between " - + minLoadTimeOutSecond + " and " + maxLoadTimeoutSecond - + " seconds"); - } - } - - public TransactionStatus getLabelState(long dbId, String label) { - try { - DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(dbId); - return dbTransactionMgr.getLabelState(label); - } catch (AnalysisException e) { - LOG.warn("Get transaction status by label " + label + " failed", e); - return TransactionStatus.UNKNOWN; - } - - } - - public Long getTransactionId(long dbId, String label) { - try { - DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(dbId); - return dbTransactionMgr.getTransactionIdByLabel(label); - } catch (AnalysisException e) { - LOG.warn("Get transaction id by label " + label + " failed", e); - return null; - } - } - + @Override public void preCommitTransaction2PC(Database db, List
tableList, long transactionId, List tabletCommitInfos, long timeoutMillis, TxnCommitAttachment txnCommitAttachment) @@ -307,7 +290,8 @@ public void commitTransaction2PC(Database db, List
tableList, long transa + " data will be visable later.", transactionId, stopWatch.getTime()); } - public void abortTransaction(long dbId, long transactionId, String reason) throws UserException { + @Override + public void abortTransaction(Long dbId, Long transactionId, String reason) throws UserException { Database db = Env.getCurrentInternalCatalog().getDbNullable(dbId); TransactionState transactionState = getDatabaseTransactionMgr(dbId).getTransactionState(transactionId); if (transactionState == null) { @@ -318,11 +302,7 @@ public void abortTransaction(long dbId, long transactionId, String reason) throw abortTransaction(dbId, transactionId, reason, null, tableList); } - public void abortTransaction(long dbId, long transactionId, String reason, List
tableList) - throws UserException { - abortTransaction(dbId, transactionId, reason, null, tableList); - } - + @Override public void abortTransaction(Long dbId, Long txnId, String reason, TxnCommitAttachment txnCommitAttachment, List
tableList) throws UserException { DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(dbId); @@ -338,11 +318,13 @@ public void abortTransaction(Long dbId, Long txnId, String reason, } // for http cancel stream load api + @Override public void abortTransaction(Long dbId, String label, String reason) throws UserException { DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(dbId); dbTransactionMgr.abortTransaction(label, reason); } + @Override public void abortTransaction2PC(Long dbId, long transactionId, List
tableList) throws UserException { DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(dbId); if (!MetaLockUtils.tryWriteLockTablesOrMetaException(tableList, 5000, TimeUnit.MILLISECONDS)) { @@ -418,6 +400,7 @@ public void finishTransaction(long dbId, long transactionId) throws UserExceptio * * @throws AnalysisException is database does not exist anymore */ + @Override public boolean isPreviousTransactionsFinished(long endTransactionId, long dbId, List tableIdList) throws AnalysisException { try { @@ -441,6 +424,7 @@ public boolean isPreviousTransactionsFinished(long endTransactionId, long dbId, * * @throws AnalysisException is database does not exist anymore */ + @Override public boolean isPreviousTransactionsFinished(long endTransactionId, long dbId, long tableId, long partitionId) throws AnalysisException { DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(dbId); @@ -452,6 +436,7 @@ public boolean isPreviousTransactionsFinished(long endTransactionId, long dbId, * expired: txn is in VISIBLE or ABORTED, and is expired. * timeout: txn is in PREPARE, but timeout */ + @Override public void removeExpiredAndTimeoutTxns() { long currentMillis = System.currentTimeMillis(); for (DatabaseTransactionMgr dbTransactionMgr : dbIdToDatabaseTransactionMgrs.values()) { @@ -459,63 +444,145 @@ public void removeExpiredAndTimeoutTxns() { } } - public TransactionState getTransactionState(long dbId, long transactionId) { + @Override + public void cleanLabel(Long dbId, String label, boolean isReplay) throws Exception { + getDatabaseTransactionMgr(dbId).cleanLabel(label, isReplay); + } + + @Override + public void updateMultiTableRunningTransactionTableIds(Long dbId, Long transactionId, List tableIds) + throws UserException { + getDatabaseTransactionMgr(dbId).updateMultiTableRunningTransactionTableIds(transactionId, tableIds); + } + + @Override + public void putTransactionTableNames(Long dbId, Long transactionId, List tableIds) + throws Exception { + getDatabaseTransactionMgr(dbId).putTransactionTableNames(transactionId, tableIds); + } + + @Override + public TWaitingTxnStatusResult getWaitingTxnStatus(TWaitingTxnStatusRequest request) + throws AnalysisException, TimeoutException { + long dbId = request.getDbId(); + int commitTimeoutSec = Config.commit_timeout_second; + for (int i = 0; i < commitTimeoutSec; ++i) { + Env.getCurrentInternalCatalog().getDbOrAnalysisException(dbId); + TWaitingTxnStatusResult statusResult = new TWaitingTxnStatusResult(); + statusResult.status = new TStatus(); + TransactionStatus txnStatus = null; + if (request.isSetTxnId()) { + long txnId = request.getTxnId(); + TransactionState txnState = Env.getCurrentGlobalTransactionMgr().getTransactionState(dbId, txnId); + if (txnState == null) { + throw new AnalysisException("txn does not exist: " + txnId); + } + txnStatus = txnState.getTransactionStatus(); + if (!txnState.getReason().trim().isEmpty()) { + statusResult.status.setErrorMsgsIsSet(true); + statusResult.status.addToErrorMsgs(txnState.getReason()); + } + } else { + txnStatus = getLabelState(dbId, request.getLabel()); + } + if (txnStatus == TransactionStatus.UNKNOWN || txnStatus.isFinalStatus()) { + statusResult.setTxnStatusId(txnStatus.value()); + return statusResult; + } + try { + Thread.sleep(1000L); + } catch (InterruptedException e) { + LOG.info("commit sleep exception.", e); + } + } + throw new TimeoutException("Operation is timeout"); + } + + @Override + public void updateDatabaseUsedQuotaData(long dbId, long usedQuotaDataBytes) throws AnalysisException { + DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(dbId); + dbTransactionMgr.updateDatabaseUsedQuotaData(usedQuotaDataBytes); + } + + /** + * If a Coordinate BE is down when running txn, the txn will remain in FE until killed by timeout + * So when FE identify the Coordinate BE is down, FE should cancel it initiative + */ + @Override + public void abortTxnWhenCoordinateBeDown(String coordinateHost, int limit) { + List> transactionIdByCoordinateBe = getTransactionIdByCoordinateBe(coordinateHost, limit); + for (Pair txnInfo : transactionIdByCoordinateBe) { + try { + DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(txnInfo.first); + TransactionState transactionState = dbTransactionMgr.getTransactionState(txnInfo.second); + if (transactionState.getTransactionStatus() == TransactionStatus.PRECOMMITTED) { + continue; + } + dbTransactionMgr.abortTransaction(txnInfo.second, "coordinate BE is down", null); + } catch (UserException e) { + LOG.warn("Abort txn on coordinate BE {} failed, msg={}", coordinateHost, e.getMessage()); + } + } + } + + @Override + public TransactionStatus getLabelState(long dbId, String label) { try { DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(dbId); - return dbTransactionMgr.getTransactionState(transactionId); + return dbTransactionMgr.getLabelState(label); } catch (AnalysisException e) { - LOG.warn("Get transaction {} in db {} failed. msg: {}", transactionId, dbId, e.getMessage()); - return null; + LOG.warn("Get transaction status by label " + label + " failed", e); + return TransactionStatus.UNKNOWN; } - } - public void setEditLog(EditLog editLog) { - this.idGenerator.setEditLog(editLog); } - // for replay idToTransactionState - // check point also run transaction cleaner, the cleaner maybe concurrently modify id to - public void replayUpsertTransactionState(TransactionState transactionState) throws MetaNotFoundException { + @Override + public Long getTransactionId(Long dbId, String label) { try { - DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(transactionState.getDbId()); - dbTransactionMgr.replayUpsertTransactionState(transactionState); + DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(dbId); + return dbTransactionMgr.getTransactionIdByLabel(label); } catch (AnalysisException e) { - throw new MetaNotFoundException(e); + LOG.warn("Get transaction id by label " + label + " failed", e); + return null; } } - @Deprecated - // Use replayBatchDeleteTransactions instead - public void replayDeleteTransactionState(TransactionState transactionState) throws MetaNotFoundException { + public TransactionState getTransactionState(long dbId, long transactionId) { try { - DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(transactionState.getDbId()); - dbTransactionMgr.replayDeleteTransaction(transactionState); + DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(dbId); + return dbTransactionMgr.getTransactionState(transactionId); } catch (AnalysisException e) { - throw new MetaNotFoundException(e); + LOG.warn("Get transaction {} in db {} failed. msg: {}", transactionId, dbId, e.getMessage()); + return null; } } - public void replayBatchRemoveTransactions(BatchRemoveTransactionsOperation operation) { - Map> dbTxnIds = operation.getDbTxnIds(); - for (Long dbId : dbTxnIds.keySet()) { - try { - DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(dbId); - dbTransactionMgr.replayBatchRemoveTransaction(dbTxnIds.get(dbId)); - } catch (AnalysisException e) { - LOG.warn("replay batch remove transactions failed. db " + dbId, e); - } - } + @Override + public List getPreCommittedTxnList(Long dbId) throws AnalysisException { + return getDatabaseTransactionMgr(dbId).getPreCommittedTxnList(); } - public void replayBatchRemoveTransactionV2(BatchRemoveTransactionsOperationV2 operation) { - try { - DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(operation.getDbId()); - dbTransactionMgr.replayBatchRemoveTransaction(operation); - } catch (AnalysisException e) { - LOG.warn("replay batch remove transactions failed. db " + operation.getDbId(), e); + @Override + public Long getTransactionIdByLabel(Long dbId, String label, List statusList) + throws UserException { + return getDatabaseTransactionMgr(dbId).getTransactionIdByLabel(label, statusList); + } + + /** + * It is a non thread safe method, only invoked by checkpoint thread + * without any lock or image dump thread with db lock + */ + @Override + public int getTransactionNum() { + int txnNum = 0; + for (DatabaseTransactionMgr dbTransactionMgr : dbIdToDatabaseTransactionMgrs.values()) { + txnNum += dbTransactionMgr.getTransactionNum(); } + return txnNum; } + @Override public List> getDbInfo() { List> infos = new ArrayList<>(); long totalRunningNum = 0; @@ -545,7 +612,8 @@ public List> getDbInfo() { return infos; } - public List> getDbTransStateInfo(long dbId) { + @Override + public List> getDbTransStateInfo(Long dbId) { try { DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(dbId); return dbTransactionMgr.getDbTransStateInfo(); @@ -555,16 +623,19 @@ public List> getDbTransStateInfo(long dbId) { } } - public List> getDbTransInfo(long dbId, boolean running, int limit) throws AnalysisException { + @Override + public List> getDbTransInfo(Long dbId, boolean running, int limit) throws AnalysisException { DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(dbId); return dbTransactionMgr.getTxnStateInfoList(running, limit); } - public List> getDbTransInfoByStatus(long dbId, TransactionStatus status) throws AnalysisException { + @Override + public List> getDbTransInfoByStatus(Long dbId, TransactionStatus status) throws AnalysisException { DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(dbId); return dbTransactionMgr.getTxnStateInfoList(status); } + @Override public long getTxnNumByStatus(TransactionStatus status) { long counter = 0; for (DatabaseTransactionMgr dbMgr : dbIdToDatabaseTransactionMgrs.values()) { @@ -574,6 +645,7 @@ public long getTxnNumByStatus(TransactionStatus status) { } // get show info of a specified txnId + @Override public List> getSingleTranInfo(long dbId, long txnId) throws AnalysisException { DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(dbId); return dbTransactionMgr.getSingleTranInfo(dbId, txnId); @@ -590,49 +662,13 @@ public List> getPartitionTransInfo(long dbId, long tid, long ta return dbTransactionMgr.getPartitionTransInfo(tid, tableId); } - /** - * It is a non thread safe method, only invoked by checkpoint thread - * without any lock or image dump thread with db lock - */ - public int getTransactionNum() { - int txnNum = 0; - for (DatabaseTransactionMgr dbTransactionMgr : dbIdToDatabaseTransactionMgrs.values()) { - txnNum += dbTransactionMgr.getTransactionNum(); - } - return txnNum; - } - + @Override public TransactionIdGenerator getTransactionIDGenerator() { return this.idGenerator; } - @Override - public void write(DataOutput out) throws IOException { - int numTransactions = getTransactionNum(); - out.writeInt(numTransactions); - for (DatabaseTransactionMgr dbTransactionMgr : dbIdToDatabaseTransactionMgrs.values()) { - dbTransactionMgr.unprotectWriteAllTransactionStates(out); - } - idGenerator.write(out); - } - - public void readFields(DataInput in) throws IOException { - int numTransactions = in.readInt(); - for (int i = 0; i < numTransactions; ++i) { - TransactionState transactionState = new TransactionState(); - transactionState.readFields(in); - try { - DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(transactionState.getDbId()); - dbTransactionMgr.unprotectUpsertTransactionState(transactionState, true); - } catch (AnalysisException e) { - LOG.warn("failed to get db transaction manager for txn: {}", transactionState); - throw new IOException("Read transaction states failed", e); - } - } - idGenerator.readFields(in); - } - - public TransactionState getTransactionStateByCallbackIdAndStatus( + @Deprecated + private TransactionState getTransactionStateByCallbackIdAndStatus( long dbId, long callbackId, Set status) { try { DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(dbId); @@ -643,7 +679,8 @@ public TransactionState getTransactionStateByCallbackIdAndStatus( } } - public TransactionState getTransactionStateByCallbackId(long dbId, long callbackId) { + @Deprecated + private TransactionState getTransactionStateByCallbackId(long dbId, long callbackId) { try { DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(dbId); return dbTransactionMgr.getTransactionStateByCallbackId(callbackId); @@ -653,7 +690,8 @@ public TransactionState getTransactionStateByCallbackId(long dbId, long callback } } - public List> getTransactionIdByCoordinateBe(String coordinateHost, int limit) { + @Deprecated + protected List> getTransactionIdByCoordinateBe(String coordinateHost, int limit) { ArrayList> txnInfos = new ArrayList<>(); for (DatabaseTransactionMgr databaseTransactionMgr : dbIdToDatabaseTransactionMgrs.values()) { txnInfos.addAll(databaseTransactionMgr.getTransactionIdByCoordinateBe(coordinateHost, limit)); @@ -664,72 +702,13 @@ public List> getTransactionIdByCoordinateBe(String coordinateHo return txnInfos.size() > limit ? new ArrayList<>(txnInfos.subList(0, limit)) : txnInfos; } - /** - * If a Coordinate BE is down when running txn, the txn will remain in FE until killed by timeout - * So when FE identify the Coordinate BE is down, FE should cancel it initiative - */ - public void abortTxnWhenCoordinateBeDown(String coordinateHost, int limit) { - List> transactionIdByCoordinateBe = getTransactionIdByCoordinateBe(coordinateHost, limit); - for (Pair txnInfo : transactionIdByCoordinateBe) { - try { - DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(txnInfo.first); - TransactionState transactionState = dbTransactionMgr.getTransactionState(txnInfo.second); - if (transactionState.getTransactionStatus() == TransactionStatus.PRECOMMITTED) { - continue; - } - dbTransactionMgr.abortTransaction(txnInfo.second, "coordinate BE is down", null); - } catch (UserException e) { - LOG.warn("Abort txn on coordinate BE {} failed, msg={}", coordinateHost, e.getMessage()); - } - } - } - - public void updateDatabaseUsedQuotaData(long dbId, long usedQuotaDataBytes) throws AnalysisException { - DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(dbId); - dbTransactionMgr.updateDatabaseUsedQuotaData(usedQuotaDataBytes); - } - - public TWaitingTxnStatusResult getWaitingTxnStatus(TWaitingTxnStatusRequest request) - throws AnalysisException, TimeoutException { - long dbId = request.getDbId(); - int commitTimeoutSec = Config.commit_timeout_second; - for (int i = 0; i < commitTimeoutSec; ++i) { - Env.getCurrentInternalCatalog().getDbOrAnalysisException(dbId); - TWaitingTxnStatusResult statusResult = new TWaitingTxnStatusResult(); - statusResult.status = new TStatus(); - TransactionStatus txnStatus = null; - if (request.isSetTxnId()) { - long txnId = request.getTxnId(); - TransactionState txnState = Env.getCurrentGlobalTransactionMgr().getTransactionState(dbId, txnId); - if (txnState == null) { - throw new AnalysisException("txn does not exist: " + txnId); - } - txnStatus = txnState.getTransactionStatus(); - if (!txnState.getReason().trim().isEmpty()) { - statusResult.status.setErrorMsgsIsSet(true); - statusResult.status.addToErrorMsgs(txnState.getReason()); - } - } else { - txnStatus = getLabelState(dbId, request.getLabel()); - } - if (txnStatus == TransactionStatus.UNKNOWN || txnStatus.isFinalStatus()) { - statusResult.setTxnStatusId(txnStatus.value()); - return statusResult; - } - try { - Thread.sleep(1000L); - } catch (InterruptedException e) { - LOG.info("commit sleep exception.", e); - } - } - throw new TimeoutException("Operation is timeout"); - } - + @Override public long getAllRunningTxnNum() { return updateTxnMetric(databaseTransactionMgr -> (long) databaseTransactionMgr.getRunningTxnNum(), MetricRepo.DB_GAUGE_TXN_NUM); } + @Override public long getAllPublishTxnNum() { return updateTxnMetric( databaseTransactionMgr -> (long) databaseTransactionMgr.getCommittedTxnList().size(), @@ -750,30 +729,87 @@ private long updateTxnMetric(Function metricSuppli return total; } - public List getPreCommittedTxnList(Long dbId) throws AnalysisException { - return getDatabaseTransactionMgr(dbId).getPreCommittedTxnList(); + @Override + public int getRunningTxnNums(Long dbId) throws AnalysisException { + return getDatabaseTransactionMgr(dbId).getRunningTxnNums(); } - public void cleanLabel(Long dbId, String label, boolean isReplay) throws AnalysisException { - getDatabaseTransactionMgr(dbId).cleanLabel(label, isReplay); + @Override + public Long getNextTransactionId() { + return this.idGenerator.getNextTransactionId(); } - public Long getTransactionIdByLabel(Long dbId, String label, List statusList) - throws UserException { - return getDatabaseTransactionMgr(dbId).getTransactionIdByLabel(label, statusList); + @Override + public void write(DataOutput out) throws IOException { + int numTransactions = getTransactionNum(); + out.writeInt(numTransactions); + for (DatabaseTransactionMgr dbTransactionMgr : dbIdToDatabaseTransactionMgrs.values()) { + dbTransactionMgr.unprotectWriteAllTransactionStates(out); + } + idGenerator.write(out); } - public int getRunningTxnNums(Long dbId) throws AnalysisException { - return getDatabaseTransactionMgr(dbId).getRunningTxnNums(); + @Override + public void readFields(DataInput in) throws IOException { + int numTransactions = in.readInt(); + for (int i = 0; i < numTransactions; ++i) { + TransactionState transactionState = new TransactionState(); + transactionState.readFields(in); + try { + DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(transactionState.getDbId()); + dbTransactionMgr.unprotectUpsertTransactionState(transactionState, true); + } catch (AnalysisException e) { + LOG.warn("failed to get db transaction manager for txn: {}", transactionState); + throw new IOException("Read transaction states failed", e); + } + } + idGenerator.readFields(in); } - public void updateMultiTableRunningTransactionTableIds(Long dbId, Long transactionId, List tableIds) - throws AnalysisException { - getDatabaseTransactionMgr(dbId).updateMultiTableRunningTransactionTableIds(transactionId, tableIds); + // for replay idToTransactionState + // check point also run transaction cleaner, the cleaner maybe concurrently modify id to + @Override + public void replayUpsertTransactionState(TransactionState transactionState) throws MetaNotFoundException { + try { + DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(transactionState.getDbId()); + dbTransactionMgr.replayUpsertTransactionState(transactionState); + } catch (AnalysisException e) { + throw new MetaNotFoundException(e); + } } - public void putTransactionTableNames(Long dbId, Long transactionId, List tableIds) - throws AnalysisException { - getDatabaseTransactionMgr(dbId).putTransactionTableNames(transactionId, tableIds); + @Override + @Deprecated + // Use replayBatchDeleteTransactions instead + public void replayDeleteTransactionState(TransactionState transactionState) throws MetaNotFoundException { + try { + DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(transactionState.getDbId()); + dbTransactionMgr.replayDeleteTransaction(transactionState); + } catch (AnalysisException e) { + throw new MetaNotFoundException(e); + } + } + + @Override + public void replayBatchRemoveTransactions(BatchRemoveTransactionsOperation operation) { + Map> dbTxnIds = operation.getDbTxnIds(); + for (Long dbId : dbTxnIds.keySet()) { + try { + DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(dbId); + dbTransactionMgr.replayBatchRemoveTransaction(dbTxnIds.get(dbId)); + } catch (AnalysisException e) { + LOG.warn("replay batch remove transactions failed. db " + dbId, e); + } + } + } + + @Override + public void replayBatchRemoveTransactionV2(BatchRemoveTransactionsOperationV2 operation) { + try { + DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(operation.getDbId()); + dbTransactionMgr.replayBatchRemoveTransaction(operation); + } catch (AnalysisException e) { + LOG.warn("replay batch remove transactions failed. db " + operation.getDbId(), e); + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgrIface.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgrIface.java new file mode 100644 index 00000000000000..6f407fb72683b0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgrIface.java @@ -0,0 +1,184 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.transaction; + +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.Table; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DuplicatedRequestException; +import org.apache.doris.common.LabelAlreadyUsedException; +import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.common.QuotaExceedException; +import org.apache.doris.common.UserException; +import org.apache.doris.common.io.Writable; +import org.apache.doris.persist.BatchRemoveTransactionsOperation; +import org.apache.doris.persist.BatchRemoveTransactionsOperationV2; +import org.apache.doris.persist.EditLog; +import org.apache.doris.thrift.TUniqueId; +import org.apache.doris.thrift.TWaitingTxnStatusRequest; +import org.apache.doris.thrift.TWaitingTxnStatusResult; +import org.apache.doris.transaction.TransactionState.LoadJobSourceType; +import org.apache.doris.transaction.TransactionState.TxnCoordinator; + +import java.io.DataInput; +import java.io.IOException; +import java.util.List; +import java.util.concurrent.TimeoutException; + +public interface GlobalTransactionMgrIface extends Writable { + default void checkValidTimeoutSecond(long timeoutSecond, int maxLoadTimeoutSecond, + int minLoadTimeOutSecond) throws AnalysisException { + if (timeoutSecond > maxLoadTimeoutSecond || timeoutSecond < minLoadTimeOutSecond) { + throw new AnalysisException("Invalid timeout: " + timeoutSecond + ". Timeout should between " + + minLoadTimeOutSecond + " and " + maxLoadTimeoutSecond + + " seconds"); + } + } + + public void setEditLog(EditLog editLog); + + public TxnStateCallbackFactory getCallbackFactory(); + + public void addDatabaseTransactionMgr(Long dbId); + + public void removeDatabaseTransactionMgr(Long dbId); + + public long beginTransaction(long dbId, List tableIdList, String label, TxnCoordinator coordinator, + LoadJobSourceType sourceType, long timeoutSecond) + throws AnalysisException, LabelAlreadyUsedException, BeginTransactionException, DuplicatedRequestException, + QuotaExceedException, MetaNotFoundException; + + public long beginTransaction(long dbId, List tableIdList, String label, TUniqueId requestId, + TxnCoordinator coordinator, LoadJobSourceType sourceType, long listenerId, long timeoutSecond) + throws AnalysisException, LabelAlreadyUsedException, BeginTransactionException, DuplicatedRequestException, + QuotaExceedException, MetaNotFoundException; + + public void preCommitTransaction2PC(Database db, List
tableList, long transactionId, + List tabletCommitInfos, long timeoutMillis, + TxnCommitAttachment txnCommitAttachment) + throws UserException; + + public void commitTransaction(long dbId, List
tableList, + long transactionId, List tabletCommitInfos) + throws UserException; + + public void commitTransaction(long dbId, List
tableList, long transactionId, + List tabletCommitInfos, TxnCommitAttachment txnCommitAttachment) + throws UserException; + + public boolean commitAndPublishTransaction(DatabaseIf db, List
tableList, long transactionId, + List tabletCommitInfos, long timeoutMillis) + throws UserException; + + public boolean commitAndPublishTransaction(DatabaseIf db, List
tableList, long transactionId, + List tabletCommitInfos, long timeoutMillis, + TxnCommitAttachment txnCommitAttachment) + throws UserException; + + public void commitTransaction2PC(Database db, List
tableList, long transactionId, long timeoutMillis) + throws UserException; + + public void abortTransaction(Long dbId, Long transactionId, String reason) throws UserException; + + public void abortTransaction(Long dbId, Long txnId, String reason, + TxnCommitAttachment txnCommitAttachment, List
tableList) throws UserException; + + public void abortTransaction(Long dbId, String label, String reason) throws UserException; + + public void abortTransaction2PC(Long dbId, long transactionId, List
tableList) throws UserException; + + public List getReadyToPublishTransactions(); + + public boolean existCommittedTxns(Long dbId, Long tableId, Long partitionId); + + public void finishTransaction(long dbId, long transactionId) throws UserException; + + public boolean isPreviousTransactionsFinished(long endTransactionId, long dbId, List tableIdList) + throws AnalysisException; + + public boolean isPreviousTransactionsFinished(long endTransactionId, long dbId, long tableId, + long partitionId) throws AnalysisException; + + public void removeExpiredAndTimeoutTxns(); + + public void cleanLabel(Long dbId, String label, boolean isReplay) throws Exception; + + public void updateMultiTableRunningTransactionTableIds(Long dbId, Long transactionId, List tableIds) + throws UserException; + + public void putTransactionTableNames(Long dbId, Long transactionId, List tableIds) + throws Exception; + + public TWaitingTxnStatusResult getWaitingTxnStatus(TWaitingTxnStatusRequest request) + throws AnalysisException, TimeoutException; + + public void updateDatabaseUsedQuotaData(long dbId, long usedQuotaDataBytes) throws AnalysisException; + + public void abortTxnWhenCoordinateBeDown(String coordinateHost, int limit); + + public TransactionStatus getLabelState(long dbId, String label) throws AnalysisException; + + public Long getTransactionId(Long dbId, String label) throws AnalysisException; + + public TransactionState getTransactionState(long dbId, long transactionId); + + public Long getTransactionIdByLabel(Long dbId, String label, List statusList) + throws UserException; + + public List getPreCommittedTxnList(Long dbId) throws AnalysisException; + + public int getTransactionNum(); + + public TransactionIdGenerator getTransactionIDGenerator() throws Exception; + + public List> getDbTransInfoByStatus(Long dbId, TransactionStatus status) throws AnalysisException; + + public List> getDbTransStateInfo(Long dbId) throws AnalysisException; + + public List> getDbTransInfo(Long dbId, boolean running, int limit) throws AnalysisException; + + public List> getTableTransInfo(long dbId, long txnId) throws AnalysisException; + + public List> getPartitionTransInfo(long dbId, long tid, long tableId) + throws AnalysisException; + + public List> getDbInfo() throws AnalysisException; + + public long getTxnNumByStatus(TransactionStatus status); + + public List> getSingleTranInfo(long dbId, long txnId) throws AnalysisException; + + public long getAllRunningTxnNum(); + + public int getRunningTxnNums(Long dbId) throws AnalysisException; + + public long getAllPublishTxnNum(); + + public Long getNextTransactionId() throws UserException; + + public void readFields(DataInput in) throws IOException; + + public void replayUpsertTransactionState(TransactionState transactionState) throws Exception; + + public void replayDeleteTransactionState(TransactionState transactionState) throws Exception; + + public void replayBatchRemoveTransactions(BatchRemoveTransactionsOperation operation) throws Exception; + + public void replayBatchRemoveTransactionV2(BatchRemoveTransactionsOperationV2 operation) throws Exception; +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java index 250f2c56665b76..f88c0e83fd1264 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java @@ -63,7 +63,7 @@ private void publishVersion() { if (DebugPointUtil.isEnable("PublishVersionDaemon.stop_publish")) { return; } - GlobalTransactionMgr globalTransactionMgr = Env.getCurrentGlobalTransactionMgr(); + GlobalTransactionMgrIface globalTransactionMgr = Env.getCurrentGlobalTransactionMgr(); List readyTransactionStates = globalTransactionMgr.getReadyToPublishTransactions(); if (readyTransactionStates.isEmpty()) { return; diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionState.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionState.java index c4fc6de761d6c1..59e3d7c25b3550 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionState.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionState.java @@ -338,6 +338,22 @@ public TransactionState(long dbId, List tableIdList, long transactionId, S this.timeoutMs = timeoutMs; } + //for TxnInfoPB convert to TransactionState + public TransactionState(long dbId, List tableIdList, long transactionId, String label, TUniqueId requestId, + LoadJobSourceType sourceType, TxnCoordinator txnCoordinator, TransactionStatus transactionStatus, + String reason, long callbackId, long timeoutMs, TxnCommitAttachment txnCommitAttachment, long prepareTime, + long preCommitTime, long commitTime, long finishTime) { + this(dbId, tableIdList, transactionId, label, requestId, sourceType, txnCoordinator, callbackId, timeoutMs); + + this.transactionStatus = transactionStatus; + this.prepareTime = prepareTime; + this.preCommitTime = preCommitTime; + this.commitTime = commitTime; + this.finishTime = finishTime; + this.reason = reason; + this.txnCommitAttachment = txnCommitAttachment; + } + public void setErrorReplicas(Set newErrorReplicas) { this.errorReplicas = newErrorReplicas; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/RollupJobV2Test.java b/fe/fe-core/src/test/java/org/apache/doris/alter/RollupJobV2Test.java index e93f524a9199da..d4cdadf53bde2e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/alter/RollupJobV2Test.java +++ b/fe/fe-core/src/test/java/org/apache/doris/alter/RollupJobV2Test.java @@ -59,7 +59,7 @@ import org.apache.doris.thrift.TStorageFormat; import org.apache.doris.thrift.TTaskType; import org.apache.doris.transaction.FakeTransactionIDGenerator; -import org.apache.doris.transaction.GlobalTransactionMgr; +import org.apache.doris.transaction.GlobalTransactionMgrIface; import com.google.common.collect.Lists; import mockit.Expectations; @@ -86,8 +86,8 @@ public class RollupJobV2Test { private static String fileName = "./RollupJobV2Test"; private static FakeTransactionIDGenerator fakeTransactionIDGenerator; - private static GlobalTransactionMgr masterTransMgr; - private static GlobalTransactionMgr slaveTransMgr; + private static GlobalTransactionMgrIface masterTransMgr; + private static GlobalTransactionMgrIface slaveTransMgr; private static Env masterEnv; private static Env slaveEnv; diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeJobV2Test.java b/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeJobV2Test.java index 60020d08a8c892..51b3bd3a8aaee0 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeJobV2Test.java +++ b/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeJobV2Test.java @@ -117,9 +117,9 @@ public void setUp() fakeTransactionIDGenerator = new FakeTransactionIDGenerator(); masterEnv = CatalogTestUtil.createTestCatalog(); slaveEnv = CatalogTestUtil.createTestCatalog(); - masterTransMgr = masterEnv.getGlobalTransactionMgr(); + masterTransMgr = (GlobalTransactionMgr) masterEnv.getGlobalTransactionMgr(); masterTransMgr.setEditLog(masterEnv.getEditLog()); - slaveTransMgr = slaveEnv.getGlobalTransactionMgr(); + slaveTransMgr = (GlobalTransactionMgr) slaveEnv.getGlobalTransactionMgr(); slaveTransMgr.setEditLog(slaveEnv.getEditLog()); analyzer = AccessTestUtil.fetchAdminAnalyzer(false); addColumnClause.analyze(analyzer); diff --git a/fe/fe-core/src/test/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgrTest.java new file mode 100644 index 00000000000000..a1aa78d2595cf7 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgrTest.java @@ -0,0 +1,407 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 org.apache.doris.cloud.transaction; + +import org.apache.doris.catalog.CatalogTestUtil; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.FakeEditLog; +import org.apache.doris.catalog.FakeEnv; +import org.apache.doris.catalog.Table; +import org.apache.doris.cloud.proto.Cloud; +import org.apache.doris.cloud.proto.Cloud.AbortTxnResponse; +import org.apache.doris.cloud.proto.Cloud.BeginTxnResponse; +import org.apache.doris.cloud.proto.Cloud.CheckTxnConflictResponse; +import org.apache.doris.cloud.proto.Cloud.CommitTxnResponse; +import org.apache.doris.cloud.proto.Cloud.GetCurrentMaxTxnResponse; +import org.apache.doris.cloud.proto.Cloud.MetaServiceCode; +import org.apache.doris.cloud.proto.Cloud.TxnInfoPB; +import org.apache.doris.cloud.rpc.MetaServiceProxy; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; +import org.apache.doris.common.DuplicatedRequestException; +import org.apache.doris.common.FeMetaVersion; +import org.apache.doris.common.LabelAlreadyUsedException; +import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.common.QuotaExceedException; +import org.apache.doris.common.UserException; +import org.apache.doris.transaction.BeginTransactionException; +import org.apache.doris.transaction.GlobalTransactionMgrIface; +import org.apache.doris.transaction.TransactionState; + +import com.google.common.collect.Lists; +import mockit.Mock; +import mockit.MockUp; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.jupiter.api.Assertions; + +import java.lang.reflect.InvocationTargetException; +import java.util.concurrent.atomic.AtomicLong; + +public class CloudGlobalTransactionMgrTest { + + private static FakeEditLog fakeEditLog; + private static FakeEnv fakeEnv; + private static GlobalTransactionMgrIface masterTransMgr; + private static Env masterEnv; + + private TransactionState.TxnCoordinator transactionSource = new TransactionState.TxnCoordinator(TransactionState.TxnSourceType.FE, "localfe"); + + @Before + public void setUp() throws InstantiationException, IllegalAccessException, IllegalArgumentException, + InvocationTargetException, NoSuchMethodException, SecurityException { + + Config.cloud_unique_id = "cloud_unique_id"; + Config.meta_service_endpoint = "127.0.0.1:20121"; + fakeEditLog = new FakeEditLog(); + fakeEnv = new FakeEnv(); + masterEnv = CatalogTestUtil.createTestCatalog(); + FakeEnv.setMetaVersion(FeMetaVersion.VERSION_CURRENT); + masterTransMgr = masterEnv.getGlobalTransactionMgr(); + } + + @Test + public void testBeginTransaction() throws LabelAlreadyUsedException, AnalysisException, + BeginTransactionException, DuplicatedRequestException, QuotaExceedException, MetaNotFoundException { + AtomicLong id = new AtomicLong(1000); + new MockUp(MetaServiceProxy.class) { + @Mock + public Cloud.BeginTxnResponse beginTxn(Cloud.BeginTxnRequest request) { + BeginTxnResponse.Builder beginTxnResponseBuilder = BeginTxnResponse.newBuilder(); + beginTxnResponseBuilder.setTxnId(id.getAndIncrement()) + .setStatus(Cloud.MetaServiceResponseStatus.newBuilder().setCode(MetaServiceCode.OK).setMsg("OK")); + + return beginTxnResponseBuilder.build(); + } + }; + + long transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(CatalogTestUtil.testTableId1), + CatalogTestUtil.testTxnLabel1, + transactionSource, + TransactionState.LoadJobSourceType.FRONTEND, Config.stream_load_default_timeout_second); + + Assert.assertEquals(transactionId + 1, id.get()); + } + + @Test + public void testBeginTransactionConflict() throws LabelAlreadyUsedException, AnalysisException, + BeginTransactionException, DuplicatedRequestException, QuotaExceedException, MetaNotFoundException { + AtomicLong id = new AtomicLong(1000); + new MockUp(MetaServiceProxy.class) { + int times = 1; + @Mock + public Cloud.BeginTxnResponse beginTxn(Cloud.BeginTxnRequest request) { + BeginTxnResponse.Builder beginTxnResponseBuilder = BeginTxnResponse.newBuilder(); + if (times > 5) { + beginTxnResponseBuilder.setTxnId(id.getAndIncrement()) + .setStatus(Cloud.MetaServiceResponseStatus.newBuilder().setCode(MetaServiceCode.OK).setMsg("OK")); + } else { + beginTxnResponseBuilder.setStatus(Cloud.MetaServiceResponseStatus.newBuilder() + .setCode(MetaServiceCode.KV_TXN_CONFLICT).setMsg("kv txn conflict")); + } + times++; + return beginTxnResponseBuilder.build(); + } + }; + + long transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(CatalogTestUtil.testTableId1), + CatalogTestUtil.testTxnLabel1, + transactionSource, + TransactionState.LoadJobSourceType.FRONTEND, Config.stream_load_default_timeout_second); + + Assert.assertEquals(transactionId + 1, id.get()); + } + + @Test + public void testBeginTransactionLabelAlreadyUsedException() throws LabelAlreadyUsedException, AnalysisException, + BeginTransactionException, DuplicatedRequestException, QuotaExceedException, MetaNotFoundException { + new MockUp(MetaServiceProxy.class) { + @Mock + public Cloud.BeginTxnResponse beginTxn(Cloud.BeginTxnRequest request) { + BeginTxnResponse.Builder beginTxnResponseBuilder = BeginTxnResponse.newBuilder(); + beginTxnResponseBuilder.setStatus(Cloud.MetaServiceResponseStatus.newBuilder() + .setCode(MetaServiceCode.TXN_LABEL_ALREADY_USED).setMsg("label already used")); + return beginTxnResponseBuilder.build(); + } + }; + + Assertions.assertThrows(LabelAlreadyUsedException.class, + () -> { + masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(CatalogTestUtil.testTableId1), + CatalogTestUtil.testTxnLabel1, + transactionSource, + TransactionState.LoadJobSourceType.FRONTEND, Config.stream_load_default_timeout_second); + }); + } + + @Test + public void testBeginTransactionDuplicatedRequestException() throws LabelAlreadyUsedException, AnalysisException, + BeginTransactionException, DuplicatedRequestException, QuotaExceedException, MetaNotFoundException { + new MockUp(MetaServiceProxy.class) { + @Mock + public Cloud.BeginTxnResponse beginTxn(Cloud.BeginTxnRequest request) { + BeginTxnResponse.Builder beginTxnResponseBuilder = BeginTxnResponse.newBuilder(); + beginTxnResponseBuilder.setStatus(Cloud.MetaServiceResponseStatus.newBuilder() + .setCode(MetaServiceCode.TXN_DUPLICATED_REQ).setMsg("duplicated request")); + return beginTxnResponseBuilder.build(); + } + }; + + Assertions.assertThrows(DuplicatedRequestException.class, + () -> { + masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(CatalogTestUtil.testTableId1), + CatalogTestUtil.testTxnLabel1, + transactionSource, + TransactionState.LoadJobSourceType.FRONTEND, Config.stream_load_default_timeout_second); + }); + } + + @Test + public void testCommitTransaction() throws UserException { + new MockUp(MetaServiceProxy.class) { + @Mock + public Cloud.CommitTxnResponse commitTxn(Cloud.CommitTxnRequest request) { + TxnInfoPB.Builder txnInfoBuilder = TxnInfoPB.newBuilder(); + txnInfoBuilder.setDbId(CatalogTestUtil.testTableId1); + txnInfoBuilder.addAllTableIds(Lists.newArrayList(CatalogTestUtil.testTableId1)); + txnInfoBuilder.setLabel(CatalogTestUtil.testTxnLabel1); + txnInfoBuilder.setListenerId(-1); + CommitTxnResponse.Builder commitTxnResponseBuilder = CommitTxnResponse.newBuilder(); + commitTxnResponseBuilder.setStatus(Cloud.MetaServiceResponseStatus.newBuilder() + .setCode(MetaServiceCode.OK).setMsg("OK")) + .setTxnInfo(txnInfoBuilder.build()); + return commitTxnResponseBuilder.build(); + } + }; + + long transactionId = 123533; + Table testTable1 = masterEnv.getInternalCatalog().getDbOrMetaException(CatalogTestUtil.testDbId1) + .getTableOrMetaException(CatalogTestUtil.testTableId1); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), + transactionId, null); + } + + @Test + public void testCommitTransactionAlreadyVisible() throws UserException { + new MockUp(MetaServiceProxy.class) { + @Mock + public Cloud.CommitTxnResponse commitTxn(Cloud.CommitTxnRequest request) { + TxnInfoPB.Builder txnInfoBuilder = TxnInfoPB.newBuilder(); + txnInfoBuilder.setDbId(CatalogTestUtil.testTableId1); + txnInfoBuilder.addAllTableIds(Lists.newArrayList(CatalogTestUtil.testTableId1)); + txnInfoBuilder.setLabel(CatalogTestUtil.testTxnLabel1); + txnInfoBuilder.setListenerId(-1); + CommitTxnResponse.Builder commitTxnResponseBuilder = CommitTxnResponse.newBuilder(); + commitTxnResponseBuilder.setStatus(Cloud.MetaServiceResponseStatus.newBuilder() + .setCode(MetaServiceCode.TXN_ALREADY_VISIBLE).setMsg("txn already visible")) + .setTxnInfo(txnInfoBuilder.build()); + return commitTxnResponseBuilder.build(); + } + }; + + long transactionId = 123533; + Table testTable1 = masterEnv.getInternalCatalog().getDbOrMetaException(CatalogTestUtil.testDbId1) + .getTableOrMetaException(CatalogTestUtil.testTableId1); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), + transactionId, null); + } + + @Test + public void testCommitTransactionAlreadyAborted() throws UserException { + new MockUp(MetaServiceProxy.class) { + @Mock + public Cloud.CommitTxnResponse commitTxn(Cloud.CommitTxnRequest request) { + TxnInfoPB.Builder txnInfoBuilder = TxnInfoPB.newBuilder(); + txnInfoBuilder.setDbId(CatalogTestUtil.testTableId1); + txnInfoBuilder.addAllTableIds(Lists.newArrayList(CatalogTestUtil.testTableId1)); + txnInfoBuilder.setLabel(CatalogTestUtil.testTxnLabel1); + txnInfoBuilder.setListenerId(-1); + CommitTxnResponse.Builder commitTxnResponseBuilder = CommitTxnResponse.newBuilder(); + commitTxnResponseBuilder.setStatus(Cloud.MetaServiceResponseStatus.newBuilder() + .setCode(MetaServiceCode.TXN_ALREADY_ABORTED).setMsg("txn already aborted")) + .setTxnInfo(txnInfoBuilder.build()); + return commitTxnResponseBuilder.build(); + } + }; + + Assertions.assertThrows(UserException.class, + () -> { + long transactionId = 123533; + Table testTable1 = masterEnv.getInternalCatalog().getDbOrMetaException(CatalogTestUtil.testDbId1) + .getTableOrMetaException(CatalogTestUtil.testTableId1); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), + transactionId, null); + }); + } + + @Test + public void testCommitTransactionConflict() throws UserException { + new MockUp(MetaServiceProxy.class) { + int times = 1; + @Mock + public Cloud.CommitTxnResponse commitTxn(Cloud.CommitTxnRequest request) { + CommitTxnResponse.Builder commitTxnResponseBuilder = CommitTxnResponse.newBuilder(); + if (times > 5) { + TxnInfoPB.Builder txnInfoBuilder = TxnInfoPB.newBuilder(); + txnInfoBuilder.setDbId(CatalogTestUtil.testTableId1); + txnInfoBuilder.addAllTableIds(Lists.newArrayList(CatalogTestUtil.testTableId1)); + txnInfoBuilder.setLabel(CatalogTestUtil.testTxnLabel1); + txnInfoBuilder.setListenerId(-1); + commitTxnResponseBuilder.setStatus(Cloud.MetaServiceResponseStatus.newBuilder() + .setCode(MetaServiceCode.TXN_ALREADY_VISIBLE).setMsg("txn already visible")) + .setTxnInfo(txnInfoBuilder.build()); + } else { + commitTxnResponseBuilder.setStatus(Cloud.MetaServiceResponseStatus.newBuilder() + .setCode(MetaServiceCode.KV_TXN_CONFLICT).setMsg("kv txn conflict")); + } + times++; + return commitTxnResponseBuilder.build(); + } + }; + long transactionId = 123533; + Table testTable1 = masterEnv.getInternalCatalog().getDbOrMetaException(CatalogTestUtil.testDbId1) + .getTableOrMetaException(CatalogTestUtil.testTableId1); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), + transactionId, null); + } + + @Test + public void testAbortTransaction() throws UserException { + new MockUp(MetaServiceProxy.class) { + @Mock + public Cloud.AbortTxnResponse abortTxn(Cloud.AbortTxnRequest request) { + AbortTxnResponse.Builder abortTxnResponseBuilder = AbortTxnResponse.newBuilder(); + abortTxnResponseBuilder.setStatus(Cloud.MetaServiceResponseStatus.newBuilder() + .setCode(MetaServiceCode.OK).setMsg("OK")); + return abortTxnResponseBuilder.build(); + } + }; + long transactionId = 123533; + masterTransMgr.abortTransaction(CatalogTestUtil.testDbId1, transactionId, "User Cancelled"); + } + + @Test + public void testAbortTransactionByLabel() throws UserException { + new MockUp(MetaServiceProxy.class) { + @Mock + public Cloud.AbortTxnResponse abortTxn(Cloud.AbortTxnRequest request) { + AbortTxnResponse.Builder abortTxnResponseBuilder = AbortTxnResponse.newBuilder(); + abortTxnResponseBuilder.setStatus(Cloud.MetaServiceResponseStatus.newBuilder() + .setCode(MetaServiceCode.OK).setMsg("OK")); + return abortTxnResponseBuilder.build(); + } + }; + masterTransMgr.abortTransaction(CatalogTestUtil.testDbId1, CatalogTestUtil.testTxnLabel1, "User Cancelled"); + } + + @Test + public void testAbortTransactionConflict() throws UserException { + new MockUp(MetaServiceProxy.class) { + int times = 1; + @Mock + public Cloud.AbortTxnResponse abortTxn(Cloud.AbortTxnRequest request) { + AbortTxnResponse.Builder abortTxnResponseBuilder = AbortTxnResponse.newBuilder(); + if (times > 5) { + abortTxnResponseBuilder.setStatus(Cloud.MetaServiceResponseStatus.newBuilder() + .setCode(MetaServiceCode.OK).setMsg("OK")); + return abortTxnResponseBuilder.build(); + } else { + abortTxnResponseBuilder.setStatus(Cloud.MetaServiceResponseStatus.newBuilder() + .setCode(MetaServiceCode.KV_TXN_CONFLICT).setMsg("kv txn conflict")); + } + times++; + return abortTxnResponseBuilder.build(); + } + }; + long transactionId = 123533; + masterTransMgr.abortTransaction(CatalogTestUtil.testDbId1, transactionId, "User Cancelled"); + } + + @Test + public void testAbortTransactionByLabelConflict() throws UserException { + new MockUp(MetaServiceProxy.class) { + int times = 1; + @Mock + public Cloud.AbortTxnResponse abortTxn(Cloud.AbortTxnRequest request) { + AbortTxnResponse.Builder abortTxnResponseBuilder = AbortTxnResponse.newBuilder(); + if (times > 5) { + abortTxnResponseBuilder.setStatus(Cloud.MetaServiceResponseStatus.newBuilder() + .setCode(MetaServiceCode.OK).setMsg("OK")); + return abortTxnResponseBuilder.build(); + } else { + abortTxnResponseBuilder.setStatus(Cloud.MetaServiceResponseStatus.newBuilder() + .setCode(MetaServiceCode.KV_TXN_CONFLICT).setMsg("kv txn conflict")); + } + times++; + return abortTxnResponseBuilder.build(); + } + }; + masterTransMgr.abortTransaction(CatalogTestUtil.testDbId1, CatalogTestUtil.testTxnLabel1, "User Cancelled"); + } + + @Test + public void testIsPreviousTransactionsFinished() throws UserException { + new MockUp(MetaServiceProxy.class) { + @Mock + public Cloud.CheckTxnConflictResponse checkTxnConflict(Cloud.CheckTxnConflictRequest request) { + CheckTxnConflictResponse.Builder checkTxnConflictResponseBuilder = CheckTxnConflictResponse.newBuilder(); + checkTxnConflictResponseBuilder.setStatus(Cloud.MetaServiceResponseStatus.newBuilder() + .setCode(MetaServiceCode.OK).setMsg("OK")) + .setFinished(true); + return checkTxnConflictResponseBuilder.build(); + } + }; + boolean result = masterTransMgr.isPreviousTransactionsFinished(12131231, + CatalogTestUtil.testDbId1, Lists.newArrayList(CatalogTestUtil.testTableId1)); + Assert.assertEquals(result, true); + } + + @Test + public void testIsPreviousTransactionsFinishedException() throws UserException { + new MockUp(MetaServiceProxy.class) { + @Mock + public Cloud.CheckTxnConflictResponse checkTxnConflict(Cloud.CheckTxnConflictRequest request) { + CheckTxnConflictResponse.Builder checkTxnConflictResponseBuilder = CheckTxnConflictResponse.newBuilder(); + checkTxnConflictResponseBuilder.setStatus(Cloud.MetaServiceResponseStatus.newBuilder() + .setCode(MetaServiceCode.OK).setMsg("OK")) + .setFinished(false); + return checkTxnConflictResponseBuilder.build(); + } + }; + boolean result = masterTransMgr.isPreviousTransactionsFinished(12131231, + CatalogTestUtil.testDbId1, Lists.newArrayList(CatalogTestUtil.testTableId1)); + Assert.assertEquals(result, false); + } + + @Test + public void testGetNextTransactionId() throws UserException { + new MockUp(MetaServiceProxy.class) { + @Mock + public Cloud.GetCurrentMaxTxnResponse getCurrentMaxTxnId(Cloud.GetCurrentMaxTxnRequest request) { + GetCurrentMaxTxnResponse.Builder getCurrentMaxTxnResponseBuilder = GetCurrentMaxTxnResponse.newBuilder(); + getCurrentMaxTxnResponseBuilder.setStatus(Cloud.MetaServiceResponseStatus.newBuilder() + .setCode(MetaServiceCode.OK).setMsg("OK")) + .setCurrentMaxTxnId(1000); + return getCurrentMaxTxnResponseBuilder.build(); + } + }; + long result = masterTransMgr.getNextTransactionId(); + Assert.assertEquals(1000, result); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/transaction/DatabaseTransactionMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/transaction/DatabaseTransactionMgrTest.java index 9108570e5e4e09..1030fac8071dd3 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/transaction/DatabaseTransactionMgrTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/transaction/DatabaseTransactionMgrTest.java @@ -81,10 +81,10 @@ public void setUp() throws InstantiationException, IllegalAccessException, Illeg metaContext.setMetaVersion(FeMetaVersion.VERSION_CURRENT); metaContext.setThreadLocalInfo(); - masterTransMgr = masterEnv.getGlobalTransactionMgr(); + masterTransMgr = (GlobalTransactionMgr) masterEnv.getGlobalTransactionMgr(); masterTransMgr.setEditLog(masterEnv.getEditLog()); - slaveTransMgr = slaveEnv.getGlobalTransactionMgr(); + slaveTransMgr = (GlobalTransactionMgr) slaveEnv.getGlobalTransactionMgr(); slaveTransMgr.setEditLog(slaveEnv.getEditLog()); LabelToTxnId = addTransactionToTransactionMgr(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java index 0203d614e00ef9..c989230c1b6cb1 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java @@ -91,10 +91,10 @@ public void setUp() throws InstantiationException, IllegalAccessException, Illeg metaContext.setMetaVersion(FeMetaVersion.VERSION_CURRENT); metaContext.setThreadLocalInfo(); - masterTransMgr = masterEnv.getGlobalTransactionMgr(); + masterTransMgr = (GlobalTransactionMgr) masterEnv.getGlobalTransactionMgr(); masterTransMgr.setEditLog(masterEnv.getEditLog()); - slaveTransMgr = slaveEnv.getGlobalTransactionMgr(); + slaveTransMgr = (GlobalTransactionMgr) slaveEnv.getGlobalTransactionMgr(); slaveTransMgr.setEditLog(slaveEnv.getEditLog()); } From d924a978beaa322551dc1700b44b45797fede4e2 Mon Sep 17 00:00:00 2001 From: Petrichor <31833513+vinlee19@users.noreply.github.com> Date: Mon, 22 Jan 2024 20:24:33 +0800 Subject: [PATCH 139/200] [improvement](docs) delete unsupported data type for jdbc catalog MSSQL (#30217) --- docs/en/docs/lakehouse/multi-catalog/jdbc.md | 1 - docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md | 1 - 2 files changed, 2 deletions(-) diff --git a/docs/en/docs/lakehouse/multi-catalog/jdbc.md b/docs/en/docs/lakehouse/multi-catalog/jdbc.md index dc45c247d54ab0..4a1caf0f6e12e5 100644 --- a/docs/en/docs/lakehouse/multi-catalog/jdbc.md +++ b/docs/en/docs/lakehouse/multi-catalog/jdbc.md @@ -477,7 +477,6 @@ As for data mapping from SQLServer to Doris, one Database in Doris corresponds t | date | DATE | | | datetime/datetime2/smalldatetime | DATETIMEV2 | | | char/varchar/text/nchar/nvarchar/ntext | STRING | | -| binary/varbinary | STRING | | | time/datetimeoffset | STRING | | | Other | UNSUPPORTED | | diff --git a/docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md b/docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md index 52aece0db2b5c5..d93f123b6189f0 100644 --- a/docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md +++ b/docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md @@ -478,7 +478,6 @@ CREATE CATALOG jdbc_sqlserve PROPERTIES ( | date | DATE | | | datetime/datetime2/smalldatetime | DATETIMEV2 | | | char/varchar/text/nchar/nvarchar/ntext | STRING | | -| binary/varbinary | STRING | | | time/datetimeoffset | STRING | | | Other | UNSUPPORTED | | From 2b1ec70afd045ed421fcb346c1597d3e4931840b Mon Sep 17 00:00:00 2001 From: wangbo Date: Mon, 22 Jan 2024 20:25:28 +0800 Subject: [PATCH 140/200] [Feature](executor)Workload Group support Non-Pipeline Execution (#30164) --- be/src/agent/workload_group_listener.cpp | 6 +- be/src/common/config.cpp | 2 +- be/src/pipeline/pipeline_fragment_context.cpp | 7 +-- .../pipeline_x_fragment_context.cpp | 5 +- be/src/runtime/fragment_mgr.cpp | 61 ++++++++----------- be/src/runtime/query_context.cpp | 25 ++++++++ be/src/runtime/query_context.h | 13 ++-- .../runtime/task_group/task_group_manager.cpp | 57 ++++++++++++++--- .../runtime/task_group/task_group_manager.h | 9 ++- .../workload_sched_policy_mgr.cpp | 1 - be/src/vec/exec/scan/scanner_context.h | 2 +- .../java/org/apache/doris/qe/Coordinator.java | 3 + .../org/apache/doris/qe/StmtExecutor.java | 2 +- gensrc/thrift/PaloInternalService.thrift | 16 ++--- 14 files changed, 132 insertions(+), 77 deletions(-) diff --git a/be/src/agent/workload_group_listener.cpp b/be/src/agent/workload_group_listener.cpp index 6ea7c28669c839..237d6c77274d69 100644 --- a/be/src/agent/workload_group_listener.cpp +++ b/be/src/agent/workload_group_listener.cpp @@ -26,6 +26,7 @@ namespace doris { void WorkloadGroupListener::handle_topic_info(const std::vector& topic_info_list) { std::set current_wg_ids; + bool is_set_cgroup_path = config::doris_cgroup_cpu_path != ""; for (const TopicInfo& topic_info : topic_info_list) { if (!topic_info.__isset.workload_group_info) { continue; @@ -52,7 +53,7 @@ void WorkloadGroupListener::handle_topic_info(const std::vector& topi // 4 create and update task scheduler Status ret2 = _exec_env->task_group_manager()->upsert_cg_task_scheduler(&task_group_info, _exec_env); - if (!ret2.ok()) { + if (is_set_cgroup_path && !ret2.ok()) { LOG(INFO) << "upsert task sche failed, tg_id=" << task_group_info.id << ", reason=" << ret2.to_string(); } @@ -63,7 +64,8 @@ void WorkloadGroupListener::handle_topic_info(const std::vector& topi << ", cgroup cpu_shares=" << task_group_info.cgroup_cpu_shares << ", cgroup cpu_hard_limit=" << task_group_info.cgroup_cpu_hard_limit << ", enable_cgroup_cpu_soft_limit=" - << (config::enable_cgroup_cpu_soft_limit ? "true" : "false"); + << (config::enable_cgroup_cpu_soft_limit ? "true" : "false") + << ", is set cgroup path=" << (is_set_cgroup_path ? "true" : "flase"); } _exec_env->task_group_manager()->delete_task_group_by_ids(current_wg_ids); diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 5468ac29a02385..271ad72410ded8 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -1133,7 +1133,7 @@ DEFINE_Bool(enable_flush_file_cache_async, "true"); // cgroup DEFINE_mString(doris_cgroup_cpu_path, ""); -DEFINE_mBool(enable_cgroup_cpu_soft_limit, "false"); +DEFINE_mBool(enable_cgroup_cpu_soft_limit, "true"); DEFINE_Bool(ignore_always_true_predicate_for_segment, "true"); diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp index 538a2ce1bdb546..909039b23fbfcc 100644 --- a/be/src/pipeline/pipeline_fragment_context.cpp +++ b/be/src/pipeline/pipeline_fragment_context.cpp @@ -718,12 +718,7 @@ Status PipelineFragmentContext::submit() { int submit_tasks = 0; Status st; - auto* scheduler = _exec_env->pipeline_task_scheduler(); - if (_query_ctx->get_task_scheduler()) { - scheduler = _query_ctx->get_task_scheduler(); - } else if (_task_group_entity && _query_ctx->use_task_group_for_cpu_limit.load()) { - scheduler = _exec_env->pipeline_task_group_scheduler(); - } + auto* scheduler = _query_ctx->get_pipe_exec_scheduler(); for (auto& task : _tasks) { st = scheduler->schedule_task(task.get()); if (!st) { diff --git a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp index a44db667450de9..4a16b97b2f3ff2 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp +++ b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp @@ -1219,10 +1219,7 @@ Status PipelineXFragmentContext::submit() { int submit_tasks = 0; Status st; - auto* scheduler = _exec_env->pipeline_task_scheduler(); - if (_task_group_entity) { - scheduler = _exec_env->pipeline_task_group_scheduler(); - } + auto* scheduler = _query_ctx->get_pipe_exec_scheduler(); for (auto& task : _tasks) { for (auto& t : task) { st = scheduler->schedule_task(t.get()); diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index 0fbaacb4fa22c8..a76c7687f02af5 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -677,41 +677,27 @@ Status FragmentMgr::_get_query_ctx(const Params& params, TUniqueId query_id, boo query_ctx->register_memory_statistics(); query_ctx->register_cpu_statistics(); + bool is_pipeline = false; if constexpr (std::is_same_v) { - if (params.__isset.workload_groups && !params.workload_groups.empty()) { - uint64_t tg_id = params.workload_groups[0].id; - auto* tg_mgr = _exec_env->task_group_manager(); - if (auto task_group_ptr = tg_mgr->get_task_group_by_id(tg_id)) { - task_group_ptr->add_mem_tracker_limiter(query_ctx->query_mem_tracker); - // set task group to queryctx for memory tracker can be removed, see QueryContext's destructor - query_ctx->set_task_group(task_group_ptr); - stringstream ss; - ss << "Query/load id: " << print_id(query_ctx->query_id()) - << ", use task group:" << task_group_ptr->debug_string() - << ", enable cpu hard limit:" - << (tg_mgr->enable_cpu_hard_limit() ? "true" : "false"); - bool ret = false; - if (tg_mgr->enable_cgroup()) { - ret = tg_mgr->set_cg_task_sche_for_query_ctx(tg_id, query_ctx.get()); - if (ret) { - ss << ", use cgroup for cpu limit."; - } else { - ss << ", not found cgroup sche, no limit for cpu."; - } - } else { - ss << ", use doris sche for cpu limit."; - query_ctx->use_task_group_for_cpu_limit.store(true); - } - LOG(INFO) << ss.str(); - _exec_env->runtime_query_statistics_mgr()->set_workload_group_id( - print_id(query_id), tg_id); - } else { - VLOG_DEBUG << "Query/load id: " << print_id(query_ctx->query_id()) - << " no task group found, does not use task group."; - } - } else { - VLOG_DEBUG << "Query/load id: " << print_id(query_ctx->query_id()) - << " does not use task group."; + is_pipeline = true; + } + + if (params.__isset.workload_groups && !params.workload_groups.empty()) { + uint64_t tg_id = params.workload_groups[0].id; + auto* tg_mgr = _exec_env->task_group_manager(); + if (auto task_group_ptr = tg_mgr->get_task_group_by_id(tg_id)) { + task_group_ptr->add_mem_tracker_limiter(query_ctx->query_mem_tracker); + // set task group to queryctx for memory tracker can be removed, see QueryContext's destructor + query_ctx->set_task_group(task_group_ptr); + _exec_env->runtime_query_statistics_mgr()->set_workload_group_id(print_id(query_id), + tg_id); + query_ctx->set_query_scheduler(tg_id); + + LOG(INFO) << "Query/load id: " << print_id(query_ctx->query_id()) + << ", use task group: " << task_group_ptr->debug_string() + << ", is pipeline: " << ((int)is_pipeline) + << ", enable cgroup soft limit: " + << ((int)config::enable_cgroup_cpu_soft_limit); } } @@ -795,7 +781,12 @@ Status FragmentMgr::exec_plan_fragment(const TExecPlanFragmentParams& params, std::make_pair(params.params.fragment_instance_id, fragment_executor)); _cv.notify_all(); } - auto st = _thread_pool->submit_func( + + auto* current_thread_pool = query_ctx->get_non_pipe_exec_thread_pool(); + if (!current_thread_pool) { + current_thread_pool = _thread_pool.get(); + } + auto st = current_thread_pool->submit_func( [this, fragment_executor, cb] { _exec_actual(fragment_executor, cb); }); if (!st.ok()) { { diff --git a/be/src/runtime/query_context.cpp b/be/src/runtime/query_context.cpp index fffb5ad57a9815..a70bf6695acd96 100644 --- a/be/src/runtime/query_context.cpp +++ b/be/src/runtime/query_context.cpp @@ -20,6 +20,7 @@ #include "pipeline/pipeline_fragment_context.h" #include "pipeline/pipeline_x/dependency.h" #include "runtime/runtime_query_statistics_mgr.h" +#include "runtime/task_group/task_group_manager.h" namespace doris { @@ -152,4 +153,28 @@ void QueryContext::register_cpu_statistics() { } } +void QueryContext::set_query_scheduler(uint64_t tg_id) { + auto* tg_mgr = _exec_env->task_group_manager(); + tg_mgr->get_query_scheduler(tg_id, &_task_scheduler, &_scan_task_scheduler, + &_non_pipe_thread_pool); +} + +doris::pipeline::TaskScheduler* QueryContext::get_pipe_exec_scheduler() { + if (!config::enable_cgroup_cpu_soft_limit) { + return _exec_env->pipeline_task_group_scheduler(); + } else if (_task_scheduler) { + return _task_scheduler; + } else { + return _exec_env->pipeline_task_scheduler(); + } +} + +ThreadPool* QueryContext::get_non_pipe_exec_thread_pool() { + if (_task_group) { + return _non_pipe_thread_pool; + } else { + return nullptr; + } +} + } // namespace doris diff --git a/be/src/runtime/query_context.h b/be/src/runtime/query_context.h index dd24206c415fe4..d5a8f12cee1a79 100644 --- a/be/src/runtime/query_context.h +++ b/be/src/runtime/query_context.h @@ -218,6 +218,12 @@ class QueryContext { std::shared_ptr get_cpu_statistics() { return _cpu_statistics; } + void set_query_scheduler(uint64_t wg_id); + + doris::pipeline::TaskScheduler* get_pipe_exec_scheduler(); + + ThreadPool* get_non_pipe_exec_thread_pool(); + public: DescriptorTbl* desc_tbl = nullptr; bool set_rsc_info = false; @@ -247,8 +253,6 @@ class QueryContext { // only for file scan node std::map file_scan_range_params_map; - std::atomic use_task_group_for_cpu_limit = false; - private: TUniqueId _query_id; ExecEnv* _exec_env = nullptr; @@ -272,7 +276,7 @@ class QueryContext { std::shared_ptr _shared_scanner_controller; vectorized::RuntimePredicate _runtime_predicate; - taskgroup::TaskGroupPtr _task_group; + taskgroup::TaskGroupPtr _task_group = nullptr; std::unique_ptr _runtime_filter_mgr; const TQueryOptions _query_options; @@ -281,8 +285,9 @@ class QueryContext { // to report the real message if failed. Status _exec_status = Status::OK(); - pipeline::TaskScheduler* _task_scheduler = nullptr; + doris::pipeline::TaskScheduler* _task_scheduler = nullptr; vectorized::SimplifiedScanScheduler* _scan_task_scheduler = nullptr; + ThreadPool* _non_pipe_thread_pool = nullptr; std::unique_ptr _execution_dependency; std::shared_ptr _cpu_statistics = nullptr; diff --git a/be/src/runtime/task_group/task_group_manager.cpp b/be/src/runtime/task_group/task_group_manager.cpp index cbd7b5f73f6e8d..74694baa9fc221 100644 --- a/be/src/runtime/task_group/task_group_manager.cpp +++ b/be/src/runtime/task_group/task_group_manager.cpp @@ -67,18 +67,25 @@ TaskGroupPtr TaskGroupManager::get_task_group_by_id(uint64_t tg_id) { return nullptr; } -bool TaskGroupManager::set_cg_task_sche_for_query_ctx(uint64_t tg_id, QueryContext* query_ctx_ptr) { - std::lock_guard write_lock(_task_scheduler_lock); +void TaskGroupManager::get_query_scheduler(uint64_t tg_id, + doris::pipeline::TaskScheduler** exec_sched, + vectorized::SimplifiedScanScheduler** scan_sched, + ThreadPool** non_pipe_thread_pool) { + std::shared_lock r_lock(_task_scheduler_lock); auto tg_sche_it = _tg_sche_map.find(tg_id); if (tg_sche_it != _tg_sche_map.end()) { - query_ctx_ptr->set_task_scheduler(tg_sche_it->second.get()); - auto _tg_scan_sche_it = _tg_scan_sche_map.find(tg_id); - if (_tg_scan_sche_it != _tg_scan_sche_map.end()) { - query_ctx_ptr->set_scan_task_scheduler(_tg_scan_sche_it->second.get()); - return true; - } + *exec_sched = tg_sche_it->second.get(); + } + + auto tg_scan_sche_it = _tg_scan_sche_map.find(tg_id); + if (tg_scan_sche_it != _tg_scan_sche_map.end()) { + *scan_sched = tg_scan_sche_it->second.get(); + } + + auto non_pipe_thread_pool_iter = _non_pipe_thread_pool_map.find(tg_id); + if (non_pipe_thread_pool_iter != _non_pipe_thread_pool_map.end()) { + *non_pipe_thread_pool = non_pipe_thread_pool_iter->second.get(); } - return false; } Status TaskGroupManager::upsert_cg_task_scheduler(taskgroup::TaskGroupInfo* tg_info, @@ -135,7 +142,23 @@ Status TaskGroupManager::upsert_cg_task_scheduler(taskgroup::TaskGroupInfo* tg_i } } - // step 4 update cgroup cpu if needed + // step 4: init non-pipe scheduler + if (_non_pipe_thread_pool_map.find(tg_id) == _non_pipe_thread_pool_map.end()) { + std::unique_ptr thread_pool = nullptr; + auto ret = ThreadPoolBuilder("nonPip_" + tg_name) + .set_min_threads(1) + .set_max_threads(config::fragment_pool_thread_num_max) + .set_max_queue_size(config::fragment_pool_queue_size) + .set_cgroup_cpu_ctl(cg_cu_ctl_ptr) + .build(&thread_pool); + if (!ret.ok()) { + LOG(INFO) << "create non-pipline thread pool failed"; + } else { + _non_pipe_thread_pool_map.emplace(tg_id, std::move(thread_pool)); + } + } + + // step 5: update cgroup cpu if needed if (enable_cpu_hard_limit) { if (cpu_hard_limit > 0) { _cgroup_ctl_map.at(tg_id)->update_cpu_hard_limit(cpu_hard_limit); @@ -160,6 +183,7 @@ void TaskGroupManager::delete_task_group_by_ids(std::set used_wg_id) { // stop task sche may cost some time, so it should not be locked std::set task_sche_to_del; std::set scan_task_sche_to_del; + std::set non_pip_thread_pool_to_del; std::set deleted_tg_ids; { std::shared_lock read_lock(_task_scheduler_lock); @@ -177,6 +201,13 @@ void TaskGroupManager::delete_task_group_by_ids(std::set used_wg_id) { scan_task_sche_to_del.insert(_tg_scan_sche_map[tg_id].get()); } } + for (auto iter = _non_pipe_thread_pool_map.begin(); iter != _non_pipe_thread_pool_map.end(); + iter++) { + uint64_t tg_id = iter->first; + if (used_wg_id.find(tg_id) == used_wg_id.end()) { + non_pip_thread_pool_to_del.insert(_non_pipe_thread_pool_map[tg_id].get()); + } + } } // 1 stop all threads for (auto* ptr1 : task_sche_to_del) { @@ -185,6 +216,9 @@ void TaskGroupManager::delete_task_group_by_ids(std::set used_wg_id) { for (auto* ptr2 : scan_task_sche_to_del) { ptr2->stop(); } + for (auto& ptr3 : non_pip_thread_pool_to_del) { + ptr3->shutdown(); + } // 2 release resource in memory { std::lock_guard write_lock(_task_scheduler_lock); @@ -242,6 +276,9 @@ void TaskGroupManager::stop() { for (auto& task_sche : _tg_scan_sche_map) { task_sche.second->stop(); } + for (auto& no_pip_sche : _non_pipe_thread_pool_map) { + no_pip_sche.second->shutdown(); + } } } // namespace doris::taskgroup diff --git a/be/src/runtime/task_group/task_group_manager.h b/be/src/runtime/task_group/task_group_manager.h index 08968b6fe999f4..a7ccb52f00e1e5 100644 --- a/be/src/runtime/task_group/task_group_manager.h +++ b/be/src/runtime/task_group/task_group_manager.h @@ -65,11 +65,9 @@ class TaskGroupManager { bool enable_cpu_hard_limit() { return _enable_cpu_hard_limit.load(); } - bool set_cg_task_sche_for_query_ctx(uint64_t tg_id, QueryContext* query_ctx_ptr); - - // currently cgroup both support cpu soft limit and cpu hard limit - // doris task group only support cpu soft limit - bool enable_cgroup() { return enable_cpu_hard_limit() || config::enable_cgroup_cpu_soft_limit; } + void get_query_scheduler(uint64_t tg_id, doris::pipeline::TaskScheduler** exec_sched, + vectorized::SimplifiedScanScheduler** scan_sched, + ThreadPool** non_pipe_thread_pool); private: std::shared_mutex _group_mutex; @@ -81,6 +79,7 @@ class TaskGroupManager { std::map> _tg_sche_map; std::map> _tg_scan_sche_map; std::map> _cgroup_ctl_map; + std::map> _non_pipe_thread_pool_map; std::shared_mutex _init_cg_ctl_lock; std::unique_ptr _cg_cpu_ctl; diff --git a/be/src/runtime/workload_management/workload_sched_policy_mgr.cpp b/be/src/runtime/workload_management/workload_sched_policy_mgr.cpp index 2398bff465c82c..731dd0c8661637 100644 --- a/be/src/runtime/workload_management/workload_sched_policy_mgr.cpp +++ b/be/src/runtime/workload_management/workload_sched_policy_mgr.cpp @@ -81,7 +81,6 @@ void WorkloadSchedPolicyMgr::_schedule_workload() { if (list.size() == 0) { continue; } - LOG(INFO) << "[workload_schedule] get query list size=" << list.size(); for (int i = 0; i < list.size(); i++) { WorkloadQueryInfo* query_info_ptr = &(list[i]); diff --git a/be/src/vec/exec/scan/scanner_context.h b/be/src/vec/exec/scan/scanner_context.h index 1dd9966b162641..28aec83d6a2bc5 100644 --- a/be/src/vec/exec/scan/scanner_context.h +++ b/be/src/vec/exec/scan/scanner_context.h @@ -253,7 +253,7 @@ class ScannerContext : public std::enable_shared_from_this, const int64_t _max_bytes_in_queue; doris::vectorized::ScannerScheduler* _scanner_scheduler; - SimplifiedScanScheduler* _simple_scan_scheduler = nullptr; // used for cpu hard limit + SimplifiedScanScheduler* _simple_scan_scheduler = nullptr; // List "scanners" saves all "unfinished" scanners. // The scanner scheduler will pop scanners from this list, run scanner, // and then if the scanner is not finished, will be pushed back to this list. diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index 586a27ee1033d1..8581980da2f1bf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -3639,6 +3639,9 @@ List toThrift(int backendNum) { params.params.setPerNodeScanRanges(scanRanges); params.params.setPerExchNumSenders(perExchNumSenders); + if (tWorkloadGroups != null) { + params.setWorkloadGroups(tWorkloadGroups); + } params.params.setDestinations(destinations); params.params.setSenderId(i); params.params.setNumSenders(instanceExecParams.size()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java index 767d03f43d7f44..8b72ce3eb2eca0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -1519,7 +1519,7 @@ private void sendResult(boolean isOutfileQuery, boolean isSendFields, Queriable coordBase = new PointQueryExec(planner, analyzer); } else { coord = new Coordinator(context, analyzer, planner, context.getStatsErrorEstimator()); - if (Config.enable_workload_group && context.sessionVariable.getEnablePipelineEngine()) { + if (Config.enable_workload_group) { coord.setTWorkloadGroups(context.getEnv().getWorkloadGroupMgr().getWorkloadGroup(context)); } else { context.setWorkloadGroupName(""); diff --git a/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift index fb9a1888f6ce7a..10f70bd7f86f12 100644 --- a/gensrc/thrift/PaloInternalService.thrift +++ b/gensrc/thrift/PaloInternalService.thrift @@ -400,6 +400,13 @@ struct TGlobalDict { 2: optional map slot_dicts // map from slot id to column dict id, because 2 or more column may share the dict } +struct TPipelineWorkloadGroup { + 1: optional i64 id + 2: optional string name + 3: optional map properties + 4: optional i64 version +} + // ExecPlanFragment struct TExecPlanFragmentParams { 1: required PaloInternalServiceVersion protocol_version @@ -483,6 +490,8 @@ struct TExecPlanFragmentParams { 29: optional i64 content_length + 30: optional list workload_groups + // For cloud 1000: optional bool is_mow_table; } @@ -670,13 +679,6 @@ struct TPipelineInstanceParams { 7: optional map per_node_shared_scans } -struct TPipelineWorkloadGroup { - 1: optional i64 id - 2: optional string name - 3: optional map properties - 4: optional i64 version -} - // ExecPlanFragment struct TPipelineFragmentParams { 1: required PaloInternalServiceVersion protocol_version From 7606538484a7cd315e019673584e2983f77ed95a Mon Sep 17 00:00:00 2001 From: meiyi Date: Mon, 22 Jan 2024 20:33:00 +0800 Subject: [PATCH 141/200] [fix](group commit) abort txn should use label if replay wal failed (#30219) --- be/src/olap/wal/wal_manager.cpp | 10 +++---- be/src/olap/wal/wal_table.cpp | 29 +++++++++---------- be/src/olap/wal/wal_table.h | 2 +- be/src/runtime/group_commit_mgr.cpp | 12 ++++---- .../doris/service/FrontendServiceImpl.java | 24 ++++++++++----- gensrc/thrift/FrontendService.thrift | 1 + 6 files changed, 43 insertions(+), 35 deletions(-) diff --git a/be/src/olap/wal/wal_manager.cpp b/be/src/olap/wal/wal_manager.cpp index 621a1aa080607b..45a9b58ff2dd3e 100644 --- a/be/src/olap/wal/wal_manager.cpp +++ b/be/src/olap/wal/wal_manager.cpp @@ -198,9 +198,9 @@ size_t WalManager::get_wal_queue_size(int64_t table_id) { return 0; } } else { - //table_id is -1 meaning get all table wal size - for (auto it = _wal_queues.begin(); it != _wal_queues.end(); it++) { - count += it->second.size(); + // table_id is -1 meaning get all table wal size + for (auto& [_, table_wals] : _wal_queues) { + count += table_wals.size(); } } return count; @@ -372,8 +372,8 @@ size_t WalManager::get_wal_table_size(int64_t table_id) { void WalManager::_stop_relay_wal() { std::lock_guard wrlock(_table_lock); - for (auto it = _table_map.begin(); it != _table_map.end(); it++) { - it->second->stop(); + for (auto& [_, wal_table] : _table_map) { + wal_table->stop(); } } diff --git a/be/src/olap/wal/wal_table.cpp b/be/src/olap/wal/wal_table.cpp index a6d7a4054c828c..8adc9cf5f50fdb 100644 --- a/be/src/olap/wal/wal_table.cpp +++ b/be/src/olap/wal/wal_table.cpp @@ -58,28 +58,27 @@ void WalTable::_pick_relay_wals() { std::lock_guard lock(_replay_wal_lock); std::vector need_replay_wals; std::vector need_erase_wals; - for (auto it = _replay_wal_map.begin(); it != _replay_wal_map.end(); it++) { - auto wal_info = it->second; + for (const auto& [wal_path, wal_info] : _replay_wal_map) { if (wal_info->get_retry_num() >= config::group_commit_replay_wal_retry_num) { LOG(WARNING) << "All replay wal failed, db=" << _db_id << ", table=" << _table_id - << ", wal=" << it->first << ", retry_num=" << wal_info->get_retry_num(); - auto st = _exec_env->wal_mgr()->rename_to_tmp_path(it->first, _table_id, + << ", wal=" << wal_path << ", retry_num=" << wal_info->get_retry_num(); + auto st = _exec_env->wal_mgr()->rename_to_tmp_path(wal_path, _table_id, wal_info->get_wal_id()); if (!st.ok()) { - LOG(WARNING) << "rename " << it->first << " fail" + LOG(WARNING) << "rename " << wal_path << " fail" << ",st:" << st.to_string(); } if (config::group_commit_wait_replay_wal_finish) { - auto notify_st = _exec_env->wal_mgr()->notify_relay_wal(it->second->get_wal_id()); + auto notify_st = _exec_env->wal_mgr()->notify_relay_wal(wal_info->get_wal_id()); if (!notify_st.ok()) { - LOG(WARNING) << "notify wal " << it->second->get_wal_id() << " fail"; + LOG(WARNING) << "notify wal " << wal_info->get_wal_id() << " fail"; } } - need_erase_wals.push_back(it->first); + need_erase_wals.push_back(wal_path); continue; } if (_need_replay(wal_info)) { - need_replay_wals.push_back(it->first); + need_replay_wals.push_back(wal_path); } } for (const auto& wal : need_erase_wals) { @@ -168,13 +167,13 @@ bool WalTable::_need_replay(std::shared_ptr wal_info) { #endif } -Status WalTable::_try_abort_txn(int64_t db_id, int64_t wal_id) { +Status WalTable::_try_abort_txn(int64_t db_id, std::string& label) { TLoadTxnRollbackRequest request; request.__set_auth_code(0); // this is a fake, fe not check it now request.__set_db_id(db_id); // TODO should we use label, because the replay wal use the same label and different wal_id - request.__set_txnId(wal_id); - std::string reason = "relay wal " + std::to_string(wal_id); + request.__set_label(label); + std::string reason = "relay wal with label " + label; request.__set_reason(reason); TLoadTxnRollbackResult result; TNetworkAddress master_addr = _exec_env->master_info()->network_address; @@ -185,7 +184,7 @@ Status WalTable::_try_abort_txn(int64_t db_id, int64_t wal_id) { }, 10000L); auto result_status = Status::create(result.status); - LOG(INFO) << "abort txn " << wal_id << ",st:" << st << ",result_status:" << result_status; + LOG(INFO) << "abort label " << label << ", st:" << st << ", result_status:" << result_status; return result_status; } @@ -196,9 +195,9 @@ Status WalTable::_replay_wal_internal(const std::string& wal) { RETURN_IF_ERROR(_parse_wal_path(wal, wal_id, label)); #ifndef BE_TEST if (!config::group_commit_wait_replay_wal_finish) { - auto st = _try_abort_txn(_db_id, wal_id); + auto st = _try_abort_txn(_db_id, label); if (!st.ok()) { - LOG(WARNING) << "abort txn " << wal_id << " fail"; + LOG(WARNING) << "failed to abort txn with label " << label; } } #endif diff --git a/be/src/olap/wal/wal_table.h b/be/src/olap/wal/wal_table.h index 07287d8f7e69ff..9b1ead87a2023f 100644 --- a/be/src/olap/wal/wal_table.h +++ b/be/src/olap/wal/wal_table.h @@ -47,7 +47,7 @@ class WalTable { Status _replay_wal_internal(const std::string& wal); Status _parse_wal_path(const std::string& wal, int64_t& wal_id, std::string& label); - Status _try_abort_txn(int64_t db_id, int64_t wal_id); + Status _try_abort_txn(int64_t db_id, std::string& label); Status _get_column_info(int64_t db_id, int64_t tb_id, std::map& column_info_map); diff --git a/be/src/runtime/group_commit_mgr.cpp b/be/src/runtime/group_commit_mgr.cpp index 3931306cd6fd1c..d057dd92b922ea 100644 --- a/be/src/runtime/group_commit_mgr.cpp +++ b/be/src/runtime/group_commit_mgr.cpp @@ -191,14 +191,14 @@ Status GroupCommitTable::get_first_block_load_queue( std::unique_lock l(_lock); for (int i = 0; i < 3; i++) { bool is_schema_version_match = true; - for (auto it = _load_block_queues.begin(); it != _load_block_queues.end(); ++it) { - if (!it->second->need_commit()) { - if (base_schema_version == it->second->schema_version) { - if (it->second->add_load_id(load_id).ok()) { - load_block_queue = it->second; + for (const auto& [_, inner_block_queue] : _load_block_queues) { + if (!inner_block_queue->need_commit()) { + if (base_schema_version == inner_block_queue->schema_version) { + if (inner_block_queue->add_load_id(load_id).ok()) { + load_block_queue = inner_block_queue; return Status::OK(); } - } else if (base_schema_version < it->second->schema_version) { + } else if (base_schema_version < inner_block_queue->schema_version) { is_schema_version_match = false; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index f9e3372bf28076..5b3cfb25ac95b9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -1663,15 +1663,23 @@ private void loadTxnRollbackImpl(TLoadTxnRollbackRequest request) throws UserExc throw new MetaNotFoundException("db " + request.getDb() + " does not exist"); } long dbId = db.getId(); - TransactionState transactionState = Env.getCurrentGlobalTransactionMgr() - .getTransactionState(dbId, request.getTxnId()); - if (transactionState == null) { - throw new UserException("transaction [" + request.getTxnId() + "] not found"); + if (request.getTxnId() != 0) { // txnId is required in thrift + TransactionState transactionState = Env.getCurrentGlobalTransactionMgr() + .getTransactionState(dbId, request.getTxnId()); + if (transactionState == null) { + throw new UserException("transaction [" + request.getTxnId() + "] not found"); + } + List
tableList = db.getTablesOnIdOrderIfExist(transactionState.getTableIdList()); + Env.getCurrentGlobalTransactionMgr().abortTransaction(dbId, request.getTxnId(), + request.isSetReason() ? request.getReason() : "system cancel", + TxnCommitAttachment.fromThrift(request.getTxnCommitAttachment()), tableList); + } else if (request.isSetLabel()) { + Env.getCurrentGlobalTransactionMgr() + .abortTransaction(db.getId(), request.getLabel(), + request.isSetReason() ? request.getReason() : "system cancel"); + } else { + throw new UserException("must set txn_id or label"); } - List
tableList = db.getTablesOnIdOrderIfExist(transactionState.getTableIdList()); - Env.getCurrentGlobalTransactionMgr().abortTransaction(dbId, request.getTxnId(), - request.isSetReason() ? request.getReason() : "system cancel", - TxnCommitAttachment.fromThrift(request.getTxnCommitAttachment()), tableList); } @Override diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index 9868171109ebd3..7b65103c581d23 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -835,6 +835,7 @@ struct TLoadTxnRollbackRequest { 12: optional i64 db_id 13: optional list tbls 14: optional string auth_code_uuid + 15: optional string label } struct TLoadTxnRollbackResult { From 6a4c0d8706b7b132a081029c0ab223fe7d1c8662 Mon Sep 17 00:00:00 2001 From: HHoflittlefish777 <77738092+HHoflittlefish777@users.noreply.github.com> Date: Mon, 22 Jan 2024 20:34:14 +0800 Subject: [PATCH 142/200] [fix](routine-load) pause job when json path is invalid #30197 If jsonpaths is set wrong, routine load job will report error but running all time.For example: CREATE ROUTINE LOAD jobName ON tableName PROPERTIES ( "format" = "json", "max_batch_interval" = "5", "max_batch_rows" = "300000", "max_batch_size" = "209715200", "jsonpaths" = "[\'t\',\'a\']" ) FROM KAFKA ( "kafka_broker_list" = "$IP:PORT", "kafka_topic" = "XXX", "property.kafka_default_offsets" = "OFFSET_BEGINNING" ); Jsonpaths ['t','a'] is invalid, but job will running all time. --- be/src/common/status.h | 2 + .../vec/exec/format/json/new_json_reader.cpp | 6 +- .../load/routineload/RoutineLoadJob.java | 10 ++ .../doris/transaction/TransactionState.java | 3 +- gensrc/thrift/Status.thrift | 1 + .../routine_load/data/invalid_json_path.json | 20 ++++ .../test_routine_load_error.groovy | 110 ++++++++++++++++++ 7 files changed, 148 insertions(+), 4 deletions(-) create mode 100644 regression-test/suites/load_p0/routine_load/data/invalid_json_path.json diff --git a/be/src/common/status.h b/be/src/common/status.h index 2bec1c397e8e89..a4ab9f60b0f8e1 100644 --- a/be/src/common/status.h +++ b/be/src/common/status.h @@ -35,6 +35,7 @@ namespace ErrorCode { TStatusError(MEM_ALLOC_FAILED, true); \ TStatusError(BUFFER_ALLOCATION_FAILED, true); \ TStatusError(INVALID_ARGUMENT, false); \ + TStatusError(INVALID_JSON_PATH, false); \ TStatusError(MINIMUM_RESERVATION_UNAVAILABLE, true); \ TStatusError(CORRUPTION, true); \ TStatusError(IO_ERROR, true); \ @@ -405,6 +406,7 @@ class [[nodiscard]] Status { ERROR_CTOR(MemoryAllocFailed, MEM_ALLOC_FAILED) ERROR_CTOR(BufferAllocFailed, BUFFER_ALLOCATION_FAILED) ERROR_CTOR(InvalidArgument, INVALID_ARGUMENT) + ERROR_CTOR(InvalidJsonPath, INVALID_JSON_PATH) ERROR_CTOR(MinimumReservationUnavailable, MINIMUM_RESERVATION_UNAVAILABLE) ERROR_CTOR(Corruption, CORRUPTION) ERROR_CTOR(IOError, IO_ERROR) diff --git a/be/src/vec/exec/format/json/new_json_reader.cpp b/be/src/vec/exec/format/json/new_json_reader.cpp index 21b99ff9720547..97affdcd0bb554 100644 --- a/be/src/vec/exec/format/json/new_json_reader.cpp +++ b/be/src/vec/exec/format/json/new_json_reader.cpp @@ -414,12 +414,12 @@ Status NewJsonReader::_parse_jsonpath_and_json_root() { rapidjson::Document jsonpaths_doc; if (!jsonpaths_doc.Parse(_jsonpaths.c_str(), _jsonpaths.length()).HasParseError()) { if (!jsonpaths_doc.IsArray()) { - return Status::InvalidArgument("Invalid json path: {}", _jsonpaths); + return Status::InvalidJsonPath("Invalid json path: {}", _jsonpaths); } for (int i = 0; i < jsonpaths_doc.Size(); i++) { const rapidjson::Value& path = jsonpaths_doc[i]; if (!path.IsString()) { - return Status::InvalidArgument("Invalid json path: {}", _jsonpaths); + return Status::InvalidJsonPath("Invalid json path: {}", _jsonpaths); } std::vector parsed_paths; JsonFunctions::parse_json_paths(path.GetString(), &parsed_paths); @@ -427,7 +427,7 @@ Status NewJsonReader::_parse_jsonpath_and_json_root() { } } else { - return Status::InvalidArgument("Invalid json path: {}", _jsonpaths); + return Status::InvalidJsonPath("Invalid json path: {}", _jsonpaths); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java index 889d240ce291d6..9ce8bb2e5ac892 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java @@ -1145,6 +1145,16 @@ public void afterAborted(TransactionState txnState, boolean txnOperated, String String msg; if (txnStatusChangeReason != null) { switch (txnStatusChangeReason) { + case INVALID_JSON_PATH: + msg = "be " + taskBeId + " abort task," + + " task id: " + routineLoadTaskInfo.getId() + + " job id: " + routineLoadTaskInfo.getJobId() + + " with reason: " + txnStatusChangeReasonString + + " please check the jsonpaths"; + updateState(JobState.PAUSED, + new ErrorReason(InternalErrorCode.TASKS_ABORT_ERR, msg), + false /* not replay */); + return; case OFFSET_OUT_OF_RANGE: msg = "be " + taskBeId + " abort task," + " task id: " + routineLoadTaskInfo.getId() diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionState.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionState.java index 59e3d7c25b3550..d1a591868628d9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionState.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionState.java @@ -108,7 +108,8 @@ public enum TxnStatusChangeReason { TIMEOUT, OFFSET_OUT_OF_RANGE, PAUSE, - NO_PARTITIONS; + NO_PARTITIONS, + INVALID_JSON_PATH; public static TxnStatusChangeReason fromString(String reasonString) { for (TxnStatusChangeReason txnStatusChangeReason : TxnStatusChangeReason.values()) { diff --git a/gensrc/thrift/Status.thrift b/gensrc/thrift/Status.thrift index 039fd8abe1bd06..5805430eeada0f 100644 --- a/gensrc/thrift/Status.thrift +++ b/gensrc/thrift/Status.thrift @@ -75,6 +75,7 @@ enum TStatusCode { INCOMPLETE = 44, OLAP_ERR_VERSION_ALREADY_MERGED = 45, DATA_QUALITY_ERROR = 46, + INVALID_JSON_PATH = 47, VEC_EXCEPTION = 50, VEC_LOGIC_ERROR = 51, diff --git a/regression-test/suites/load_p0/routine_load/data/invalid_json_path.json b/regression-test/suites/load_p0/routine_load/data/invalid_json_path.json new file mode 100644 index 00000000000000..6ef920fcad9136 --- /dev/null +++ b/regression-test/suites/load_p0/routine_load/data/invalid_json_path.json @@ -0,0 +1,20 @@ +{"k00": "21", "k01": "2023-08-18", "k02": "0", "k03": "63", "k04": "-27847", "k05": "-35409596", "k06": "8638201997392767650", "k07": "4919963231735304178", "k08": "-23382.541", "k09": "-1803403621.4263129", "k10": "-22009767", "k11": "\\N", "k12": "2023-03-31 10:56:14", "k13": "2023-01-20", "k14": "2023-02-18 13:37:52", "k15": "N", "k16": "T", "k17": "PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi", "k18": "\\N"} +{"k00": "8", "k01": "2023-08-14", "k02": "1", "k03": "109", "k04": "-31573", "k05": "-1362465190", "k06": "3990845741226497177", "k07": "2732763251146840270", "k08": "-25698.553", "k09": "1312831962.5678179", "k10": "\\N", "k11": "\\N", "k12": "2023-03-07 14:13:19", "k13": "2022-10-18", "k14": "2023-07-16 05:03:13", "k15": "D", "k16": "", "k17": "PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme", "k18": "\\N"} +{"k00": "91", "k01": "2023-08-27", "k02": "1", "k03": "90", "k04": "2465", "k05": "702240964", "k06": "6373830997821598984", "k07": "305860046137409400", "k08": "15991.356", "k09": "1599972327.386147", "k10": "\\N", "k11": "\\N", "k12": "2023-04-26 19:31:10", "k13": "2023-07-21", "k14": "\\N", "k15": "2", "k16": "", "k17": "B7YKYBYT8w0YC926bZ8Yz1VzyiWw2NWDAiTlEoPVyz9AXGti2Npg1FxWqWk4hEaALw0ZBSuiAIPj41lq36g5QRpPmAjNPK", "k18": "\\N"} +{"k00": "80", "k01": "2023-08-18", "k02": "0", "k03": "-18", "k04": "-8971", "k05": "679027874", "k06": "6535956962935330265", "k07": "3960889045799757165", "k08": "-13219.76", "k09": "1187161924.505394", "k10": "\\N", "k11": "\\N", "k12": "2023-03-11 07:40:00", "k13": "2022-11-29", "k14": "2023-01-14 07:24:07", "k15": "N", "k16": "D", "k17": "3Nhx6xX1qdwaq7lxwLRSKMtJFbC03swWv12mpySSVysH3igGZTiGPuKMsYW7HAkf6CWc7c0nzqDsjuH3FYVMNCWRmfxMrmY8rykQCC4Ve", "k18": "\\N"} +{"k00": "20", "k01": "2023-08-17", "k02": "0", "k03": "-5", "k04": "18158", "k05": "784479801", "k06": "1485484354598941738", "k07": "-6632681928222776815", "k08": "9708.4307", "k09": "-330432620.706069", "k10": "\\N", "k11": "\\N", "k12": "2022-09-15 21:40:55", "k13": "2023-02-23", "k14": "2023-08-13 21:31:54", "k15": "O", "k16": "X", "k17": "2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn", "k18": "\\N"} +{"k00": "66", "k01": "2023-08-15", "k02": "1", "k03": "-91", "k04": "28378", "k05": "609923317", "k06": "4872185586197131212", "k07": "1207709464099378591", "k08": "\\N", "k09": "-1863683325.9851229", "k10": "\\N", "k11": "\\N", "k12": "2022-09-24 10:39:23", "k13": "2022-09-24", "k14": "2022-10-16 18:36:43", "k15": "Y", "k16": "z", "k17": "AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6", "k18": "\\N"} +{"k00": "49", "k01": "2023-08-08", "k02": "0", "k03": "\\N", "k04": "16275", "k05": "-2144851675", "k06": "-2303421957908954634", "k07": "-46526938720058765", "k08": "-13141.143", "k09": "-686632233.2302", "k10": "\\N", "k11": "\\N", "k12": "2022-09-01 00:16:01", "k13": "2023-03-25", "k14": "2022-09-07 14:59:03", "k15": "s", "k16": "", "k17": "yvuILR2iNxfe8RRml", "k18": "\\N"} +{"k00": "57", "k01": "2023-08-19", "k02": "1", "k03": "2", "k04": "-25462", "k05": "-74112029", "k06": "6458082754318544493", "k07": "-7910671781690629051", "k08": "-15205.859", "k09": "-306870797.484914", "k10": "\\N", "k11": "\\N", "k12": "2023-07-10 18:39:10", "k13": "2023-02-12", "k14": "2023-01-27 07:26:06", "k15": "y", "k16": "", "k17": "Xi9nDVrLv8m6AwEpUxmtzFAuK48sQ", "k18": "\\N"} +{"k00": "31", "k01": "2023-08-27", "k02": "0", "k03": "17", "k04": "-18849", "k05": "1728109133", "k06": "3266501886640700374", "k07": "527195452623418935", "k08": "-24062.328", "k09": "-1514348021.262435", "k10": "\\N", "k11": "\\N", "k12": "2022-10-07 03:24:23", "k13": "2022-09-25", "k14": "\\N", "k15": "0", "k16": "8", "k17": "yKMiAntORoRa8svnMfcxlOPwwND1m5s2fdS26Xu6cfs6HK5SAibqIp9h8sZcpjHy4", "k18": "\\N"} +{"k00": "81", "k01": "2023-08-23", "k02": "0", "k03": "106", "k04": "11492", "k05": "-667795397", "k06": "4480250461471356146", "k07": "-5346660566234294101", "k08": "9082.75", "k09": "385167225.902608", "k10": "\\N", "k11": "\\N", "k12": "2023-03-20 03:33:16", "k13": "2022-11-24", "k14": "2023-02-16 18:29:41", "k15": "G", "k16": "9", "k17": "Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag", "k18": "\\N"} +{"k00": "58", "k01": "2023-08-22", "k02": "\\N", "k03": "0", "k04": "-18231", "k05": "1832867360", "k06": "6997858407575297145", "k07": "2480714305422728023", "k08": "-5450.4888", "k09": "1475901032.138386", "k10": "\\N", "k11": "\\N", "k12": "2023-02-02 05:13:24", "k13": "2022-09-18", "k14": "2023-04-23 10:51:15", "k15": "k", "k16": "", "k17": "LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja", "k18": "\\N"} +{"k00": "85", "k01": "2023-08-11", "k02": "1", "k03": "-7", "k04": "24304", "k05": "-2043877415", "k06": "-2024144417867729183", "k07": "\\N", "k08": "5363.0244", "k09": "-578615669.042831", "k10": "\\N", "k11": "\\N", "k12": "2023-07-15 01:07:41", "k13": "2023-08-13", "k14": "2023-01-20 11:57:48", "k15": "i", "k16": "", "k17": "WQ9dh9ajPu0y", "k18": "\\N"} +{"k00": "60", "k01": "2023-08-27", "k02": "0", "k03": "-52", "k04": "-2338", "k05": "-757056972", "k06": "1047567408607120856", "k07": "6541476642780646552", "k08": "6614.0894", "k09": "-1204448798.5178549", "k10": "\\N", "k11": "\\N", "k12": "2022-12-29 14:47:30", "k13": "2022-09-24", "k14": "2023-08-01 12:41:59", "k15": "O", "k16": "F", "k17": "RM4F1Ke7lkcnuxF2nK0j9VBW3MDcgyHR4pseBjtFnqS6GUkVFuzF6u3Cp9Nv7ab0O6UYrpP4DhU", "k18": "\\N"} +{"k00": "41", "k01": "2023-08-27", "k02": "1", "k03": "-104", "k04": "22750", "k05": "\\N", "k06": "8527773271030840740", "k07": "5554497317268279215", "k08": "-5296.8281", "k09": "-1715646888.01304", "k10": "\\N", "k11": "\\N", "k12": "2022-12-02 17:56:44", "k13": "2022-10-12", "k14": "2023-02-19 07:02:54", "k15": "V", "k16": "", "k17": "E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V", "k18": "\\N"} +{"k00": "62", "k01": "2023-08-21", "k02": "0", "k03": "81", "k04": "20302", "k05": "-200761532", "k06": "6365479976421007608", "k07": "\\N", "k08": "-29916.533", "k09": "1709141750.8284781", "k10": "\\N", "k11": "\\N", "k12": "2023-05-04 01:14:51", "k13": "2022-09-17", "k14": "2022-12-04 19:30:09", "k15": "d", "k16": "v", "k17": "BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD", "k18": "\\N"} +{"k00": "50", "k01": "2023-08-24", "k02": "1", "k03": "15", "k04": "14403", "k05": "\\N", "k06": "-6418906115745394180", "k07": "9205303779366462513", "k08": "-4331.5488", "k09": "-615112179.557648", "k10": "\\N", "k11": "\\N", "k12": "2022-12-29 02:27:20", "k13": "2023-06-01", "k14": "2023-08-12 04:50:04", "k15": "a", "k16": "", "k17": "eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM", "k18": "\\N"} +{"k00": "50", "k01": "2023-08-06", "k02": "1", "k03": "109", "k04": "-6330", "k05": "1479023892", "k06": "-8630800697573159428", "k07": "-1645095773540208759", "k08": "17880.961", "k09": "-1453844792.0139489", "k10": "\\N", "k11": "\\N", "k12": "2022-09-22 02:03:21", "k13": "2023-05-14", "k14": "2023-03-25 02:18:34", "k15": "m", "k16": "", "k17": "JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl", "k18": "\\N"} +{"k00": "68", "k01": "2023-08-23", "k02": "1", "k03": "-73", "k04": "20117", "k05": "1737338128", "k06": "795638676048937749", "k07": "-5551546237562433901", "k08": "-30627.039", "k09": "68589475.684545", "k10": "\\N", "k11": "\\N", "k12": "2022-12-28 20:26:51", "k13": "2022-10-04", "k14": "2023-07-30 00:20:06", "k15": "y", "k16": "", "k17": "keZ3JlWWpdnPBejf0cuiCQCVBBTd5gjvO08NVdcAFewqL7nRT4N9lnvSU6pWmletA5VbPQCeQapJdcnQCHfZUDCf4ulCnczyqr7SGrbGRT0XYcd7iktKM", "k18": "\\N"} +{"k00": "90", "k01": "2023-08-27", "k02": "1", "k03": "22", "k04": "16456", "k05": "-1476824962", "k06": "-3279894870153540825", "k07": "8990195191470116763", "k08": "26651.906", "k09": "206860148.942546", "k10": "\\N", "k11": "\\N", "k12": "2022-10-07 03:11:03", "k13": "2023-03-18", "k14": "2023-04-15 00:38:33", "k15": "T", "k16": "L", "k17": "QW0GQ3GoMtHgxPQOWGfVaveynahNpsNs09siMFA1OtO6QEDBQTdivmGyq7bFzejAqwbbVQQpREAmeLjcFSXLnQuou2KbwYD", "k18": "\\N"} +{"k00": "65", "k01": "2023-08-09", "k02": "0", "k03": "94", "k04": "31514", "k05": "814994517", "k06": "-297697460695940343", "k07": "734910652450318597", "k08": "-13061.892", "k09": "62750847.041706", "k10": "-9808654", "k11": "\\N", "k12": "2023-08-14 22:01:27", "k13": "2023-05-19", "k14": "2022-11-13 13:44:28", "k15": "V", "k16": "", "k17": "aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf", "k18": "\\N"} diff --git a/regression-test/suites/load_p0/routine_load/test_routine_load_error.groovy b/regression-test/suites/load_p0/routine_load/test_routine_load_error.groovy index c4180e35f4df40..191ea4381fd1af 100644 --- a/regression-test/suites/load_p0/routine_load/test_routine_load_error.groovy +++ b/regression-test/suites/load_p0/routine_load/test_routine_load_error.groovy @@ -25,6 +25,10 @@ suite("test_routine_load_error","p0") { "multi_table_load_invalid_table", ] + def kafkaJsonTopics = [ + "invalid_json_path", + ] + String enabled = context.config.otherConfigs.get("enableKafkaTest") String kafka_port = context.config.otherConfigs.get("kafka_port") String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") @@ -48,6 +52,15 @@ suite("test_routine_load_error","p0") { producer.send(record) } } + for (String kafkaJsonTopic in kafkaJsonTopics) { + def kafkaJson = new File("""${context.file.parent}/data/${kafkaJsonTopic}.json""").text + def lines = kafkaJson.readLines() + lines.each { line -> + logger.info("=====${line}========") + def record = new ProducerRecord<>(kafkaJsonTopic, null, line) + producer.send(record) + } + } } def i = 0 @@ -138,4 +151,101 @@ suite("test_routine_load_error","p0") { sql "stop routine load for ${jobName}" } } + + // test json path is invalid + if (enabled != null && enabled.equalsIgnoreCase("true")) { + def tableName = "test_invalid_json_path" + def jobName = "invalid_json_path" + try { + sql """ DROP TABLE IF EXISTS ${tableName} """ + sql """ + CREATE TABLE IF NOT EXISTS ${tableName} + ( + k00 INT NOT NULL, + k01 array NULL, + k02 array NULL, + k03 array NULL, + k04 array NULL, + k05 array NULL, + k06 array NULL, + k07 array NULL, + k08 array NULL, + k09 array NULL, + k10 array NULL, + k11 array NULL, + k12 array NULL, + k13 array NULL, + k14 array NULL, + k15 array NULL, + k16 array NULL, + k17 array NULL, + kd01 array NOT NULL DEFAULT "[]", + kd02 array NOT NULL DEFAULT "[]", + kd03 array NOT NULL DEFAULT "[]", + kd04 array NOT NULL DEFAULT "[]", + kd05 array NOT NULL DEFAULT "[]", + kd06 array NOT NULL DEFAULT "[]", + kd07 array NOT NULL DEFAULT "[]", + kd08 array NOT NULL DEFAULT "[]", + kd09 array NOT NULL DEFAULT "[]", + kd10 array NOT NULL DEFAULT "[]", + kd11 array NOT NULL DEFAULT "[]", + kd12 array NOT NULL DEFAULT "[]", + kd13 array NOT NULL DEFAULT "[]", + kd14 array NOT NULL DEFAULT "[]", + kd15 array NOT NULL DEFAULT "[]", + kd16 array NOT NULL DEFAULT "[]", + kd17 array NOT NULL DEFAULT "[]" + ) + UNIQUE KEY(k00) + DISTRIBUTED BY HASH(k00) BUCKETS 32 + PROPERTIES ( + "replication_num" = "1", + "enable_unique_key_merge_on_write" = "true" + ); + """ + + sql """ + CREATE ROUTINE LOAD ${jobName} ON ${tableName} + COLUMNS(k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17) + PROPERTIES + ( + "format" = "json", + "max_batch_interval" = "5", + "max_batch_rows" = "300000", + "max_batch_size" = "209715200", + "jsonpaths" = "[\'t\',\'a\']" + ) + FROM KAFKA + ( + "kafka_broker_list" = "${externalEnvIp}:${kafka_port}", + "kafka_topic" = "${jobName}", + "property.kafka_default_offsets" = "OFFSET_BEGINNING" + ); + """ + + def count = 0 + while (true) { + sleep(1000) + def res = sql "show routine load for ${jobName}" + def state = res[0][8].toString() + log.info("routine load state: ${res[0][8].toString()}".toString()) + log.info("routine load statistic: ${res[0][14].toString()}".toString()) + log.info("reason of state changed: ${res[0][17].toString()}".toString()) + if (state != "PAUSED") { + count++ + if (count > 60) { + assertEquals(1, 2) + } + continue; + } + log.info("reason of state changed: ${res[0][17].toString()}".toString()) + assertTrue(res[0][17].toString().contains("Invalid json path")) + break; + } + } finally { + sql "stop routine load for ${jobName}" + sql """ DROP TABLE IF EXISTS ${tableName} """ + } + } } \ No newline at end of file From c4291bcbacf0d59132b41e80e07d4f2460dbf51a Mon Sep 17 00:00:00 2001 From: Kaijie Chen Date: Mon, 22 Jan 2024 20:35:18 +0800 Subject: [PATCH 143/200] [fix](move-memtable) make sure index descriptor is set when creating delta writer (#30157) --------- Co-authored-by: Yongqiang YANG <98214048+dataroaring@users.noreply.github.com> Co-authored-by: Xin Liao --- be/src/vec/sink/writer/vtablet_writer_v2.cpp | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/be/src/vec/sink/writer/vtablet_writer_v2.cpp b/be/src/vec/sink/writer/vtablet_writer_v2.cpp index e23fe761ecf1d6..cfeeb782bed43c 100644 --- a/be/src/vec/sink/writer/vtablet_writer_v2.cpp +++ b/be/src/vec/sink/writer/vtablet_writer_v2.cpp @@ -431,15 +431,27 @@ Status VTabletWriterV2::_write_memtable(std::shared_ptr block .is_high_priority = _is_high_priority, .write_file_cache = _write_file_cache, }; + bool index_not_found = true; for (const auto& index : _schema->indexes()) { if (index->index_id == rows.index_id) { req.slots = &index->slots; req.schema_hash = index->schema_hash; + index_not_found = false; break; } } + if (index_not_found) { + LOG(WARNING) << "index " << rows.index_id + << " not found in schema, load_id=" << print_id(_load_id); + return std::unique_ptr(nullptr); + } return DeltaWriterV2::open(&req, streams, _state); }); + if (delta_writer == nullptr) { + LOG(WARNING) << "failed to open DeltaWriter for tablet " << tablet_id + << ", load_id=" << print_id(_load_id); + return Status::InternalError("failed to open DeltaWriter for tablet {}", tablet_id); + } { SCOPED_TIMER(_wait_mem_limit_timer); ExecEnv::GetInstance()->memtable_memory_limiter()->handle_memtable_flush(); From c21ddbeeff6565cf4dcdd8c59968280a534af167 Mon Sep 17 00:00:00 2001 From: zhangdong <493738387@qq.com> Date: Mon, 22 Jan 2024 20:42:25 +0800 Subject: [PATCH 144/200] [fix](mtmv)return MTMV with at least one available partition #30156 --- .../doris/mtmv/MTMVRelationManager.java | 21 ++++++++++++++----- .../mv/InitMaterializationContextHook.java | 2 +- 2 files changed, 17 insertions(+), 6 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java index 35bf777acb5625..77414e4fc88d72 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java @@ -33,6 +33,7 @@ import org.apache.doris.nereids.trees.plans.commands.info.ResumeMTMVInfo; import org.apache.doris.nereids.trees.plans.commands.info.TableNameInfo; import org.apache.doris.persist.AlterMTMV; +import org.apache.doris.qe.ConnectContext; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Maps; @@ -53,16 +54,26 @@ public class MTMVRelationManager implements MTMVHookService { private static final Logger LOG = LogManager.getLogger(MTMVRelationManager.class); private Map> tableMTMVs = Maps.newConcurrentMap(); - public Set getMtmvsByBaseTable(BaseTableInfo table) { + private Set getMtmvsByBaseTable(BaseTableInfo table) { return tableMTMVs.getOrDefault(table, ImmutableSet.of()); } - public Set getAvailableMTMVs(List tableInfos) { + /** + * if At least one partition is available, return this mtmv + * + * @param tableInfos + * @param ctx + * @return + */ + public Set getAvailableMTMVs(List tableInfos, ConnectContext ctx) { Set res = Sets.newHashSet(); - Set mvInfos = getAvailableMTMVInfos(tableInfos); + Set mvInfos = getMTMVInfos(tableInfos); for (BaseTableInfo tableInfo : mvInfos) { try { - res.add((MTMV) MTMVUtil.getTable(tableInfo)); + MTMV mtmv = (MTMV) MTMVUtil.getTable(tableInfo); + if (!CollectionUtils.isEmpty(MTMVUtil.getMTMVCanRewritePartitions(mtmv, ctx))) { + res.add(mtmv); + } } catch (AnalysisException e) { // not throw exception to client, just ignore it LOG.warn("getTable failed: {}", tableInfo.toString(), e); @@ -71,7 +82,7 @@ public Set getAvailableMTMVs(List tableInfos) { return res; } - public Set getAvailableMTMVInfos(List tableInfos) { + private Set getMTMVInfos(List tableInfos) { Set mvInfos = Sets.newHashSet(); for (BaseTableInfo tableInfo : tableInfos) { mvInfos.addAll(getMtmvsByBaseTable(tableInfo)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java index 07a4ff208bca18..baf714fe280763 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java @@ -70,7 +70,7 @@ private void initMaterializationContext(CascadesContext cascadesContext) { List usedBaseTables = collectedTables.stream().map(BaseTableInfo::new).collect(Collectors.toList()); Set availableMTMVs = Env.getCurrentEnv().getMtmvService().getRelationManager() - .getAvailableMTMVs(usedBaseTables); + .getAvailableMTMVs(usedBaseTables, cascadesContext.getConnectContext()); if (availableMTMVs.isEmpty()) { return; } From 7db104e88f090041ce7e54c882012329c502e7fa Mon Sep 17 00:00:00 2001 From: Lei Zhang <27994433+SWJTU-ZhangLei@users.noreply.github.com> Date: Tue, 23 Jan 2024 01:56:52 +0800 Subject: [PATCH 145/200] [fix](fe) Upgrade mariadb client version from 3.0.4 to 3.0.9 (#30242) mariadb-java-client 3.0.4 has two serious problems: * https://jira.mariadb.org/browse/CONJ-972 * https://jira.mariadb.org/browse/CONJ-973 --- fe/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/pom.xml b/fe/pom.xml index 4b205c4caebe54..e0018b8c1816c8 100644 --- a/fe/pom.xml +++ b/fe/pom.xml @@ -322,7 +322,7 @@ under the License. 4.5.13 4.4.15 1.12.625 - 3.0.4 + 3.0.9 0.2.14 3.3.6 2.8.1 From b2e84d5158d37fe80e93990628e15cc37ca8e304 Mon Sep 17 00:00:00 2001 From: Petrichor <31833513+vinlee19@users.noreply.github.com> Date: Tue, 23 Jan 2024 10:17:25 +0800 Subject: [PATCH 146/200] [improvement](docs) add faq for flink-connector-doris (#30043) --- docs/en/docs/ecosystem/flink-doris-connector.md | 2 +- docs/zh-CN/docs/ecosystem/flink-doris-connector.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/docs/ecosystem/flink-doris-connector.md b/docs/en/docs/ecosystem/flink-doris-connector.md index f1ef66b1cb529d..98ec7fdbe6c238 100644 --- a/docs/en/docs/ecosystem/flink-doris-connector.md +++ b/docs/en/docs/ecosystem/flink-doris-connector.md @@ -779,7 +779,7 @@ You can search for the log `abort transaction response` in TaskManager and deter 14. **org.apache.flink.table.api.SqlParserException when using doris.filter.query: SQL parsing failed. "xx" encountered at row x, column xx** -This problem is mainly caused by the conditional varchar/string type, which needs to be quoted. The correct way to write it is xxx = ''xxx''. In this way, the Flink SQL parser will interpret two consecutive single quotes as one single quote character instead of The end of the string, and the concatenated string is used as the value of the attribute. +This problem is mainly caused by the conditional varchar/string type, which needs to be quoted. The correct way to write it is xxx = ''xxx''. In this way, the Flink SQL parser will interpret two consecutive single quotes as one single quote character instead of The end of the string, and the concatenated string is used as the value of the attribute. For example: `t1 >= '2024-01-01'` can be written as `'doris.filter.query' = 't1 >=''2024-01-01'''`. 15. **Failed to connect to backend: http://host:webserver_port, and BE is still alive** diff --git a/docs/zh-CN/docs/ecosystem/flink-doris-connector.md b/docs/zh-CN/docs/ecosystem/flink-doris-connector.md index 321c7da429829b..9c367fbcd3f8ac 100644 --- a/docs/zh-CN/docs/ecosystem/flink-doris-connector.md +++ b/docs/zh-CN/docs/ecosystem/flink-doris-connector.md @@ -779,8 +779,8 @@ Flink在数据导入时,如果有脏数据,比如字段格式、长度等问 14. **使用doris.filter.query出现org.apache.flink.table.api.SqlParserException: SQL parse failed. Encountered "xx" at line x, column xx** -出现这个问题主要是条件varchar/string类型,需要加引号导致的,正确写法是 xxx = ''xxx'',这样Flink SQL 解析器会将两个连续的单引号解释为一个单引号字符,而不是字符串的结束,并将拼接后的字符串作为属性的值。 +出现这个问题主要是条件varchar/string类型,需要加引号导致的,正确写法是 xxx = ''xxx'',这样Flink SQL 解析器会将两个连续的单引号解释为一个单引号字符,而不是字符串的结束,并将拼接后的字符串作为属性的值。比如说:`t1 >= '2024-01-01'`,可以写成`'doris.filter.query' = 't1 >=''2024-01-01'''`。 15. **如果出现Failed to connect to backend: http://host:webserver_port, 并且Be还是活着的** -可能是因为你配置的be的ip,外部的Flink集群无法访问。这主要是因为当连接fe时,会通过fe解析出be的地址。例如,当你添加的be 地址为`127.0.0.1`,那么flink通过fe获取的be地址就为`127.0.0.1:webserver_port`, 此时Flink就会去访问这个地址。当出现这个问题时,可以通过在with属性中增加实际对应的be外部ip地`'benodes'="be_ip:webserver_port,be_ip:webserver_port..."`,整库同步则可增加`--sink-conf benodes=be_ip:webserver,be_ip:webserver...`。 +可能是因为你配置的be的ip,外部的Flink集群无法访问。这主要是因为当连接fe时,会通过fe解析出be的地址。例如,当你添加的be 地址为`127.0.0.1`,那么Flink通过fe获取的be地址就为`127.0.0.1:webserver_port`, 此时Flink就会去访问这个地址。当出现这个问题时,可以通过在with属性中增加实际对应的be外部ip地`'benodes' = "be_ip:webserver_port, be_ip:webserver_port..."`,整库同步则可增加`--sink-conf benodes=be_ip:webserver,be_ip:webserver...`。 From e2b5c2a3d69b08f00c338b37bdb0b417be4eee46 Mon Sep 17 00:00:00 2001 From: Gabriel Date: Tue, 23 Jan 2024 10:32:06 +0800 Subject: [PATCH 147/200] [pipelineX](fix) Fix use-after-free MultiCastSourceDependency (#30199) --- be/src/pipeline/exec/analytic_sink_operator.h | 8 +- .../pipeline/exec/analytic_source_operator.h | 7 +- .../exec/multi_cast_data_stream_source.cpp | 7 ++ .../exec/multi_cast_data_stream_source.h | 2 +- .../exec/multi_cast_data_streamer.cpp | 32 +++++-- .../pipeline/exec/multi_cast_data_streamer.h | 13 ++- .../pipeline/exec/set_probe_sink_operator.cpp | 2 +- be/src/pipeline/exec/set_sink_operator.cpp | 2 +- be/src/pipeline/pipeline_x/dependency.cpp | 9 ++ be/src/pipeline/pipeline_x/dependency.h | 88 +++++++++++++++++-- .../local_exchange_source_operator.cpp | 8 +- .../local_exchange_source_operator.h | 1 + be/src/pipeline/pipeline_x/operator.cpp | 11 ++- .../pipeline_x_fragment_context.cpp | 4 +- .../java/org/apache/doris/qe/Coordinator.java | 4 +- 15 files changed, 170 insertions(+), 28 deletions(-) diff --git a/be/src/pipeline/exec/analytic_sink_operator.h b/be/src/pipeline/exec/analytic_sink_operator.h index e99a0731b343a1..0214b22c006b28 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.h +++ b/be/src/pipeline/exec/analytic_sink_operator.h @@ -63,6 +63,10 @@ class AnalyticSinkLocalState : public PipelineXSinkLocalState(parent, state) {} Status init(RuntimeState* state, LocalSinkStateInfo& info) override; + Status close(RuntimeState* state, Status exec_status) override { + _shared_state->release_sink_dep(); + return PipelineXSinkLocalState::close(state, exec_status); + } private: friend class AnalyticSinkOperatorX; @@ -70,11 +74,11 @@ class AnalyticSinkLocalState : public PipelineXSinkLocalStatefound_partition_end); if (need_more_input) { - _shared_state->source_dep->block(); + _dependency->set_block_to_read(); _dependency->set_ready(); } else { _dependency->block(); - _shared_state->source_dep->set_ready(); + _dependency->set_ready_to_read(); } return need_more_input; } diff --git a/be/src/pipeline/exec/analytic_source_operator.h b/be/src/pipeline/exec/analytic_source_operator.h index ba7b00be90611b..dc86bc9506279c 100644 --- a/be/src/pipeline/exec/analytic_source_operator.h +++ b/be/src/pipeline/exec/analytic_source_operator.h @@ -83,9 +83,12 @@ class AnalyticLocalState final : public PipelineXLocalStatefound_partition_end); if (need_more_input) { _dependency->block(); - _shared_state->sink_dep->set_ready(); + _dependency->set_ready_to_write(); + if (!_shared_state->sink_released_flag) { + _shared_state->sink_dep->set_ready(); + } } else { - _shared_state->sink_dep->block(); + _dependency->set_block_to_write(); _dependency->set_ready(); } return need_more_input; diff --git a/be/src/pipeline/exec/multi_cast_data_stream_source.cpp b/be/src/pipeline/exec/multi_cast_data_stream_source.cpp index 6ac06ee5f10fb2..d360e2eb5ddc12 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_source.cpp +++ b/be/src/pipeline/exec/multi_cast_data_stream_source.cpp @@ -152,6 +152,13 @@ Status MultiCastDataStreamSourceLocalState::init(RuntimeState* state, LocalState return Status::OK(); } +Status MultiCastDataStreamSourceLocalState::close(RuntimeState* state) { + _shared_state->multi_cast_data_streamer.released_dependency( + _parent->cast()._consumer_id); + RETURN_IF_ERROR(Base::close(state)); + return Status::OK(); +} + Status MultiCastDataStreamerSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* block, SourceState& source_state) { diff --git a/be/src/pipeline/exec/multi_cast_data_stream_source.h b/be/src/pipeline/exec/multi_cast_data_stream_source.h index baeca2ca7b1a93..73c506f9cb8a96 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_source.h +++ b/be/src/pipeline/exec/multi_cast_data_stream_source.h @@ -117,7 +117,7 @@ class MultiCastDataStreamSourceLocalState final RETURN_IF_ERROR(_acquire_runtime_filter()); return Status::OK(); } - + Status close(RuntimeState* state) override; friend class MultiCastDataStreamerSourceOperatorX; RuntimeFilterDependency* filterdependency() override { return _filter_dependency.get(); } diff --git a/be/src/pipeline/exec/multi_cast_data_streamer.cpp b/be/src/pipeline/exec/multi_cast_data_streamer.cpp index 175a21469b87e3..f3e44731aef274 100644 --- a/be/src/pipeline/exec/multi_cast_data_streamer.cpp +++ b/be/src/pipeline/exec/multi_cast_data_streamer.cpp @@ -104,15 +104,37 @@ void MultiCastDataStreamer::_set_ready_for_read(int sender_idx) { if (_dependencies.empty()) { return; } - auto* dep = _dependencies[sender_idx]; - DCHECK(dep); - dep->set_ready(); + if (_dependencies_release_flag[sender_idx]) { + return; + } + { + std::unique_lock lc(_release_lock); + if (_dependencies_release_flag[sender_idx]) { + return; + } + auto* dep = _dependencies[sender_idx]; + DCHECK(dep); + dep->set_ready(); + } } void MultiCastDataStreamer::_set_ready_for_read() { + size_t i = 0; for (auto* dep : _dependencies) { - DCHECK(dep); - dep->set_ready(); + if (_dependencies_release_flag[i]) { + i++; + continue; + } + { + std::unique_lock lc(_release_lock); + if (_dependencies_release_flag[i]) { + i++; + continue; + } + DCHECK(dep); + dep->set_ready(); + i++; + } } } diff --git a/be/src/pipeline/exec/multi_cast_data_streamer.h b/be/src/pipeline/exec/multi_cast_data_streamer.h index 5e4179e0cadabb..7f221d622c0624 100644 --- a/be/src/pipeline/exec/multi_cast_data_streamer.h +++ b/be/src/pipeline/exec/multi_cast_data_streamer.h @@ -38,10 +38,14 @@ class MultiCastDataStreamer { bool with_dependencies = false) : _row_desc(row_desc), _profile(pool->add(new RuntimeProfile("MultiCastDataStreamSink"))), - _cast_sender_count(cast_sender_count) { + _cast_sender_count(cast_sender_count), + _dependencies_release_flag(cast_sender_count) { _sender_pos_to_read.resize(cast_sender_count, _multi_cast_blocks.end()); if (with_dependencies) { _dependencies.resize(cast_sender_count, nullptr); + for (size_t i = 0; i < cast_sender_count; i++) { + _dependencies_release_flag[i] = false; + } } _peak_mem_usage = ADD_COUNTER(profile(), "PeakMemUsage", TUnit::BYTES); @@ -79,6 +83,11 @@ class MultiCastDataStreamer { _block_reading(sender_idx); } + void released_dependency(int sender_idx) { + std::unique_lock lc(_release_lock); + _dependencies_release_flag[sender_idx] = true; + } + private: void _set_ready_for_read(int sender_idx); void _set_ready_for_read(); @@ -97,6 +106,8 @@ class MultiCastDataStreamer { RuntimeProfile::Counter* _process_rows = nullptr; RuntimeProfile::Counter* _peak_mem_usage = nullptr; + std::mutex _release_lock; + std::vector> _dependencies_release_flag; std::vector _dependencies; }; } // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/set_probe_sink_operator.cpp b/be/src/pipeline/exec/set_probe_sink_operator.cpp index 5ff2c3df2d2076..09e5b4b0045125 100644 --- a/be/src/pipeline/exec/set_probe_sink_operator.cpp +++ b/be/src/pipeline/exec/set_probe_sink_operator.cpp @@ -219,7 +219,7 @@ void SetProbeSinkOperatorX::_finalize_probe( local_state._shared_state->probe_finished_children_dependency[_cur_child_id + 1] ->set_ready(); } else { - local_state._shared_state->source_dep->set_ready(); + local_state._dependency->set_ready_to_read(); } } diff --git a/be/src/pipeline/exec/set_sink_operator.cpp b/be/src/pipeline/exec/set_sink_operator.cpp index cb106d76edb497..862ad411f5c180 100644 --- a/be/src/pipeline/exec/set_sink_operator.cpp +++ b/be/src/pipeline/exec/set_sink_operator.cpp @@ -94,7 +94,7 @@ Status SetSinkOperatorX::sink(RuntimeState* state, vectorized::Blo local_state._shared_state->probe_finished_children_dependency[_cur_child_id + 1] ->set_ready(); if (_child_quantity == 1) { - local_state._shared_state->source_dep->set_ready(); + local_state._dependency->set_ready_to_read(); } } } diff --git a/be/src/pipeline/pipeline_x/dependency.cpp b/be/src/pipeline/pipeline_x/dependency.cpp index 86fe982343a427..bb27a688820601 100644 --- a/be/src/pipeline/pipeline_x/dependency.cpp +++ b/be/src/pipeline/pipeline_x/dependency.cpp @@ -189,4 +189,13 @@ void LocalExchangeSharedState::sub_running_sink_operators() { } } +LocalExchangeSharedState::LocalExchangeSharedState(int num_instances) + : dependencies_release_flag(num_instances) { + source_dependencies.resize(num_instances, nullptr); + mem_trackers.resize(num_instances, nullptr); + for (size_t i = 0; i < num_instances; i++) { + dependencies_release_flag[i] = false; + } +} + } // namespace doris::pipeline diff --git a/be/src/pipeline/pipeline_x/dependency.h b/be/src/pipeline/pipeline_x/dependency.h index 61c251e00b2a2a..172f7383f3ece2 100644 --- a/be/src/pipeline/pipeline_x/dependency.h +++ b/be/src/pipeline/pipeline_x/dependency.h @@ -57,9 +57,22 @@ static constexpr auto TIME_UNIT_DEPENDENCY_LOG = 30 * 1000L * 1000L * 1000L; static_assert(TIME_UNIT_DEPENDENCY_LOG < SLOW_DEPENDENCY_THRESHOLD); struct BasicSharedState { - DependencySPtr source_dep = nullptr; - DependencySPtr sink_dep = nullptr; + Dependency* source_dep = nullptr; + Dependency* sink_dep = nullptr; + std::atomic_bool source_released_flag {false}; + std::atomic_bool sink_released_flag {false}; + std::mutex source_release_lock; + std::mutex sink_release_lock; + + void release_source_dep() { + std::unique_lock lc(source_release_lock); + source_released_flag = true; + } + void release_sink_dep() { + std::unique_lock lc(sink_release_lock); + sink_released_flag = true; + } virtual ~BasicSharedState() = default; }; @@ -108,9 +121,50 @@ class Dependency : public std::enable_shared_from_this { virtual void set_ready(); void set_ready_to_read() { DCHECK(_is_write_dependency) << debug_string(); + if (_shared_state->source_released_flag) { + return; + } + std::unique_lock lc(_shared_state->source_release_lock); + if (_shared_state->source_released_flag) { + return; + } DCHECK(_shared_state->source_dep != nullptr) << debug_string(); _shared_state->source_dep->set_ready(); } + void set_block_to_read() { + DCHECK(_is_write_dependency) << debug_string(); + if (_shared_state->source_released_flag) { + return; + } + std::unique_lock lc(_shared_state->source_release_lock); + if (_shared_state->source_released_flag) { + return; + } + DCHECK(_shared_state->source_dep != nullptr) << debug_string(); + _shared_state->source_dep->block(); + } + void set_ready_to_write() { + if (_shared_state->sink_released_flag) { + return; + } + std::unique_lock lc(_shared_state->sink_release_lock); + if (_shared_state->sink_released_flag) { + return; + } + DCHECK(_shared_state->sink_dep != nullptr) << debug_string(); + _shared_state->sink_dep->set_ready(); + } + void set_block_to_write() { + if (_shared_state->sink_released_flag) { + return; + } + std::unique_lock lc(_shared_state->sink_release_lock); + if (_shared_state->sink_released_flag) { + return; + } + DCHECK(_shared_state->sink_dep != nullptr) << debug_string(); + _shared_state->sink_dep->block(); + } // Notify downstream pipeline tasks this dependency is blocked. virtual void block() { _ready = false; } @@ -610,25 +664,47 @@ class Exchanger; struct LocalExchangeSharedState : public BasicSharedState { public: ENABLE_FACTORY_CREATOR(LocalExchangeSharedState); + LocalExchangeSharedState(int num_instances); std::unique_ptr exchanger {}; - std::vector source_dependencies; + std::vector source_dependencies; + std::vector dependencies_release_flag; Dependency* sink_dependency; std::vector mem_trackers; std::atomic mem_usage = 0; std::mutex le_lock; void sub_running_sink_operators(); void _set_ready_for_read() { + size_t i = 0; for (auto& dep : source_dependencies) { - DCHECK(dep); - dep->set_ready(); + if (dependencies_release_flag[i]) { + i++; + continue; + } + { + std::unique_lock lc(source_release_lock); + if (dependencies_release_flag[i]) { + i++; + continue; + } + DCHECK(dep); + dep->set_ready(); + i++; + } } } - void set_dep_by_channel_id(DependencySPtr dep, int channel_id) { + void set_dep_by_channel_id(Dependency* dep, int channel_id) { source_dependencies[channel_id] = dep; } void set_ready_to_read(int channel_id) { + if (dependencies_release_flag[channel_id]) { + return; + } + std::unique_lock lc(source_release_lock); + if (dependencies_release_flag[channel_id]) { + return; + } auto& dep = source_dependencies[channel_id]; DCHECK(dep) << channel_id; dep->set_ready(); diff --git a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.cpp b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.cpp index 029dcb15a480c5..9e98e3b6e8f18b 100644 --- a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.cpp +++ b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.cpp @@ -37,7 +37,7 @@ Status LocalExchangeSourceLocalState::init(RuntimeState* state, LocalStateInfo& SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); _channel_id = info.task_idx; - _shared_state->set_dep_by_channel_id(info.dependency, _channel_id); + _shared_state->set_dep_by_channel_id(_dependency, _channel_id); _shared_state->mem_trackers[_channel_id] = _mem_tracker.get(); _exchanger = _shared_state->exchanger.get(); DCHECK(_exchanger != nullptr); @@ -61,6 +61,12 @@ std::string LocalExchangeSourceLocalState::debug_string(int indentation_level) c return fmt::to_string(debug_string_buffer); } +Status LocalExchangeSourceLocalState::close(RuntimeState* state) { + _shared_state->dependencies_release_flag[_channel_id] = true; + RETURN_IF_ERROR(Base::close(state)); + return Status::OK(); +} + Status LocalExchangeSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* block, SourceState& source_state) { auto& local_state = get_local_state(state); diff --git a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.h b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.h index 63d71bbe08be17..4c95a84b533ce9 100644 --- a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.h +++ b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.h @@ -48,6 +48,7 @@ class LocalExchangeSourceLocalState final Status init(RuntimeState* state, LocalStateInfo& info) override; std::string debug_string(int indentation_level) const override; + Status close(RuntimeState* state) override; private: friend class LocalExchangeSourceOperatorX; diff --git a/be/src/pipeline/pipeline_x/operator.cpp b/be/src/pipeline/pipeline_x/operator.cpp index bbb7473f86878b..e00b1632eb4d67 100644 --- a/be/src/pipeline/pipeline_x/operator.cpp +++ b/be/src/pipeline/pipeline_x/operator.cpp @@ -338,15 +338,15 @@ Status PipelineXLocalState::init(RuntimeState* state, LocalState _shared_state = (typename DependencyType::SharedState*)_dependency->shared_state().get(); - _shared_state->source_dep = info.dependency; - _shared_state->sink_dep = deps.front(); + _shared_state->source_dep = info.dependency.get(); + _shared_state->sink_dep = deps.front().get(); } else if constexpr (!is_fake_shared) { _dependency->set_shared_state(deps.front()->shared_state()); _shared_state = (typename DependencyType::SharedState*)_dependency->shared_state().get(); - _shared_state->source_dep = info.dependency; - _shared_state->sink_dep = deps.front(); + _shared_state->source_dep = info.dependency.get(); + _shared_state->sink_dep = deps.front().get(); } } @@ -378,6 +378,9 @@ Status PipelineXLocalState::close(RuntimeState* state) { if (_closed) { return Status::OK(); } + if (_shared_state) { + _shared_state->release_source_dep(); + } if constexpr (!std::is_same_v) { COUNTER_SET(_wait_for_dependency_timer, _dependency->watcher_elapse_time()); _dependency->clear_shared_state(); diff --git a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp index 4a16b97b2f3ff2..cf4c312926e7fd 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp +++ b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp @@ -739,7 +739,7 @@ Status PipelineXFragmentContext::_add_local_exchange_impl( is_shuffled_hash_join, shuffle_idx_to_instance_idx)); // 2. Create and initialize LocalExchangeSharedState. - auto shared_state = LocalExchangeSharedState::create_shared(); + auto shared_state = LocalExchangeSharedState::create_shared(_num_instances); switch (data_distribution.distribution_type) { case ExchangeType::HASH_SHUFFLE: shared_state->exchanger = ShuffleExchanger::create_unique( @@ -771,8 +771,6 @@ Status PipelineXFragmentContext::_add_local_exchange_impl( return Status::InternalError("Unsupported local exchange type : " + std::to_string((int)data_distribution.distribution_type)); } - shared_state->source_dependencies.resize(_num_instances, nullptr); - shared_state->mem_trackers.resize(_num_instances, nullptr); auto sink_dep = std::make_shared(sink_id, local_exchange_id, _runtime_state->get_query_ctx()); sink_dep->set_shared_state(shared_state); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index 8581980da2f1bf..222fc37a42915a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -3985,7 +3985,9 @@ private void attachInstanceProfileToFragmentProfile() { if (enablePipelineEngine) { for (PipelineExecContext ctx : pipelineExecContexts.values()) { if (enablePipelineXEngine) { - ctx.attachPipelineProfileToFragmentProfile(); + synchronized (this) { + ctx.attachPipelineProfileToFragmentProfile(); + } } else { ctx.profileStream() .forEach(p -> executionProfile.addInstanceProfile(ctx.profileFragmentId, p)); From 23dfb28beffc2c78bd1dc4b8e97fea52b17235d1 Mon Sep 17 00:00:00 2001 From: Pxl Date: Tue, 23 Jan 2024 10:39:18 +0800 Subject: [PATCH 148/200] [Feature](materialized-view) support predicate apprear both on key and value mv column (#30215) support predicate apprear both on key and value mv column --- .../AbstractSelectMaterializedIndexRule.java | 3 +- .../SelectMaterializedIndexWithAggregate.java | 96 +++++-------------- .../testAggQueryOnAggMV1.out | 6 ++ .../testAggQueryOnAggMV1.groovy | 14 ++- 4 files changed, 46 insertions(+), 73 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java index 02c033a036206c..7cd63642b39273 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java @@ -35,6 +35,7 @@ import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.VirtualSlotReference; import org.apache.doris.nereids.trees.expressions.WhenClause; import org.apache.doris.nereids.trees.expressions.functions.ExpressionTrait; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; @@ -160,7 +161,7 @@ protected static boolean containsAllColumn(Expression expression, Set mv return true; } if (expression.children().isEmpty()) { - return false; + return expression instanceof VirtualSlotReference; } for (Expression child : expression.children()) { if (child instanceof Literal) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithAggregate.java index c0f70caf8510ad..a254e8d67d1afc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithAggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithAggregate.java @@ -204,7 +204,7 @@ public List buildRules() { Optional.of(project)), ExpressionUtils.replace(agg.getGroupByExpressions(), project.getAliasToProducer()), - collectRequireExprWithAggAndProject(agg.getExpressions(), project.getProjects()) + collectRequireExprWithAggAndProject(agg.getExpressions(), Optional.of(project)) ); LogicalOlapScan mvPlan = createLogicalOlapScan(scan, result); @@ -251,7 +251,7 @@ public List buildRules() { .collect(Collectors.toSet()); ImmutableSet requiredExpr = ImmutableSet.builder() .addAll(collectRequireExprWithAggAndProject( - agg.getExpressions(), project.getProjects())) + agg.getExpressions(), Optional.of(project))) .addAll(filter.getExpressions()) .build(); SelectResult result = select( @@ -320,9 +320,9 @@ public List buildRules() { LogicalOlapScan scan = project.child(); ImmutableSet requiredExpr = ImmutableSet.builder() .addAll(collectRequireExprWithAggAndProject( - agg.getExpressions(), project.getProjects())) + agg.getExpressions(), Optional.of(project))) .addAll(collectRequireExprWithAggAndProject( - filter.getExpressions(), project.getProjects())) + filter.getExpressions(), Optional.of(project))) .build(); SelectResult result = select( scan, @@ -482,7 +482,7 @@ public List buildRules() { Optional.of(project)), ExpressionUtils.replace(nonVirtualGroupByExprs(agg), project.getAliasToProducer()), - collectRequireExprWithAggAndProject(agg.getExpressions(), project.getProjects()) + collectRequireExprWithAggAndProject(agg.getExpressions(), Optional.of(project)) ); LogicalOlapScan mvPlan = createLogicalOlapScan(scan, result); @@ -536,7 +536,7 @@ public List buildRules() { .collect(Collectors.toSet()); ImmutableSet requiredExpr = ImmutableSet.builder() .addAll(collectRequireExprWithAggAndProject( - agg.getExpressions(), project.getProjects())) + agg.getExpressions(), Optional.of(project))) .addAll(filter.getExpressions()) .build(); SelectResult result = select( @@ -600,9 +600,9 @@ public List buildRules() { LogicalOlapScan scan = project.child(); ImmutableSet requiredExpr = ImmutableSet.builder() .addAll(collectRequireExprWithAggAndProject( - agg.getExpressions(), project.getProjects())) + agg.getExpressions(), Optional.of(project))) .addAll(collectRequireExprWithAggAndProject( - filter.getExpressions(), project.getProjects())) + filter.getExpressions(), Optional.of(project))) .build(); SelectResult result = select( scan, @@ -1084,41 +1084,10 @@ private AggRewriteResult rewriteAgg(MaterializedIndex index, ExprRewriteMap exprRewriteMap = new ExprRewriteMap(); RewriteContext context = new RewriteContext(new CheckContext(scan, index.getId()), exprRewriteMap); aggregateFunctions.forEach(aggFun -> AggFuncRewriter.rewrite(aggFun, context)); - - // has rewritten agg functions - Map slotMap = exprRewriteMap.slotMap; - // Note that the slots in the rewritten agg functions shouldn't appear in filters or grouping expressions. - // For example: we have a duplicated-type table t(c1, c2) and a materialized index that has - // a bitmap_union column `mv_bitmap_union_c2` for the column c2. - // The query `select c1, count(distinct c2) from t where c2 > 0 group by c1` can't use the materialized - // index because we have a filter `c2 > 0` for the aggregated column c2. - Set slotsToReplace = slotMap.keySet(); - Set indexConjuncts; - try { - indexConjuncts = PlanNode - .splitAndCompoundPredicateToConjuncts(context.checkContext.getMeta().getWhereClause()).stream() - .map(e -> new NereidsParser().parseExpression(e.toSql()).toSql()).collect(Collectors.toSet()); - } catch (Exception e) { - return new AggRewriteResult(index, false, null, null); - } - if (isInputSlotsContainsNone( - predicates.stream().filter(e -> !indexConjuncts.contains(e.toSql())).collect(Collectors.toList()), - slotsToReplace)) { - ImmutableSet newRequiredSlots = requiredScanOutput.stream() - .map(slot -> (Slot) ExpressionUtils.replace(slot, slotMap)).collect(ImmutableSet.toImmutableSet()); - return new AggRewriteResult(index, true, newRequiredSlots, exprRewriteMap); - } - - return new AggRewriteResult(index, false, null, null); + return new AggRewriteResult(index, true, requiredScanOutput, exprRewriteMap); } private static class ExprRewriteMap { - - /** - * Replace map for scan output slot. - */ - public final Map slotMap; - /** * Replace map for expressions in project. */ @@ -1131,13 +1100,12 @@ private static class ExprRewriteMap { private Map aggFuncStrMap; public ExprRewriteMap() { - this.slotMap = Maps.newHashMap(); this.projectExprMap = Maps.newHashMap(); this.aggFuncMap = Maps.newHashMap(); } public boolean isEmpty() { - return slotMap.isEmpty(); + return aggFuncMap.isEmpty(); } private void buildStrMap() { @@ -1238,7 +1206,6 @@ public Expression visitCount(Count count, RewriteContext context) { .orElseThrow(() -> new AnalysisException( "cannot find bitmap union slot when select mv")); - context.exprRewriteMap.slotMap.put(slotOpt.get(), bitmapUnionSlot); context.exprRewriteMap.projectExprMap.put(slotOpt.get(), bitmapUnionSlot); BitmapUnionCount bitmapUnionCount = new BitmapUnionCount(bitmapUnionSlot); context.exprRewriteMap.aggFuncMap.put(count, bitmapUnionCount); @@ -1269,9 +1236,6 @@ public Expression visitCount(Count count, RewriteContext context) { .filter(s -> countColumn.equalsIgnoreCase(normalizeName(s.getName()))).findFirst() .orElseThrow(() -> new AnalysisException("cannot find count slot when select mv")); - if (child instanceof Slot) { - context.exprRewriteMap.slotMap.put((Slot) child, countSlot); - } context.exprRewriteMap.projectExprMap.put(child, countSlot); Sum sum = new Sum(countSlot); context.exprRewriteMap.aggFuncMap.put(count, sum); @@ -1308,7 +1272,6 @@ public Expression visitBitmapUnion(BitmapUnion bitmapUnion, RewriteContext conte .findFirst().orElseThrow( () -> new AnalysisException("cannot find bitmap union slot when select mv")); - context.exprRewriteMap.slotMap.put(slotOpt.get(), bitmapUnionSlot); context.exprRewriteMap.projectExprMap.put(toBitmap, bitmapUnionSlot); BitmapUnion newBitmapUnion = new BitmapUnion(bitmapUnionSlot); context.exprRewriteMap.aggFuncMap.put(bitmapUnion, newBitmapUnion); @@ -1328,9 +1291,6 @@ public Expression visitBitmapUnion(BitmapUnion bitmapUnion, RewriteContext conte .stream().filter(s -> bitmapUnionColumn.equalsIgnoreCase(normalizeName(s.getName()))) .findFirst() .orElseThrow(() -> new AnalysisException("cannot find bitmap union slot when select mv")); - if (child instanceof Slot) { - context.exprRewriteMap.slotMap.put((Slot) child, bitmapUnionSlot); - } context.exprRewriteMap.projectExprMap.put(child, bitmapUnionSlot); BitmapUnion newBitmapUnion = new BitmapUnion(bitmapUnionSlot); context.exprRewriteMap.aggFuncMap.put(bitmapUnion, newBitmapUnion); @@ -1370,7 +1330,6 @@ public Expression visitBitmapUnionCount(BitmapUnionCount bitmapUnionCount, Rewri .orElseThrow(() -> new AnalysisException( "cannot find bitmap union count slot when select mv")); - context.exprRewriteMap.slotMap.put(slotOpt.get(), bitmapUnionCountSlot); context.exprRewriteMap.projectExprMap.put(toBitmap, bitmapUnionCountSlot); BitmapUnionCount newBitmapUnionCount = new BitmapUnionCount(bitmapUnionCountSlot); context.exprRewriteMap.aggFuncMap.put(bitmapUnionCount, newBitmapUnionCount); @@ -1390,9 +1349,6 @@ public Expression visitBitmapUnionCount(BitmapUnionCount bitmapUnionCount, Rewri .stream().filter(s -> bitmapUnionCountColumn.equalsIgnoreCase(normalizeName(s.getName()))) .findFirst().orElseThrow( () -> new AnalysisException("cannot find bitmap union count slot when select mv")); - if (child instanceof Slot) { - context.exprRewriteMap.slotMap.put((Slot) child, bitmapUnionCountSlot); - } context.exprRewriteMap.projectExprMap.put(child, bitmapUnionCountSlot); BitmapUnionCount newBitmapUnionCount = new BitmapUnionCount(bitmapUnionCountSlot); context.exprRewriteMap.aggFuncMap.put(bitmapUnionCount, newBitmapUnionCount); @@ -1429,7 +1385,6 @@ public Expression visitHllUnion(HllUnion hllUnion, RewriteContext context) { .orElseThrow(() -> new AnalysisException( "cannot find hll union slot when select mv")); - context.exprRewriteMap.slotMap.put(slotOpt.get(), hllUnionSlot); context.exprRewriteMap.projectExprMap.put(hllHash, hllUnionSlot); HllUnion newHllUnion = new HllUnion(hllUnionSlot); context.exprRewriteMap.aggFuncMap.put(hllUnion, newHllUnion); @@ -1467,7 +1422,6 @@ public Expression visitHllUnionAgg(HllUnionAgg hllUnionAgg, RewriteContext conte .orElseThrow(() -> new AnalysisException( "cannot find hll union slot when select mv")); - context.exprRewriteMap.slotMap.put(slotOpt.get(), hllUnionSlot); context.exprRewriteMap.projectExprMap.put(hllHash, hllUnionSlot); HllUnionAgg newHllUnionAgg = new HllUnionAgg(hllUnionSlot); context.exprRewriteMap.aggFuncMap.put(hllUnionAgg, newHllUnionAgg); @@ -1506,7 +1460,6 @@ public Expression visitNdv(Ndv ndv, RewriteContext context) { .orElseThrow(() -> new AnalysisException( "cannot find hll union slot when select mv")); - context.exprRewriteMap.slotMap.put(slotOpt.get(), hllUnionSlot); context.exprRewriteMap.projectExprMap.put(slotOpt.get(), hllUnionSlot); HllUnionAgg hllUnionAgg = new HllUnionAgg(hllUnionSlot); context.exprRewriteMap.aggFuncMap.put(ndv, hllUnionAgg); @@ -1531,9 +1484,6 @@ public Expression visitSum(Sum sum, RewriteContext context) { Slot sumSlot = context.checkContext.scan.getOutputByIndex(context.checkContext.index).stream() .filter(s -> sumColumn.equalsIgnoreCase(normalizeName(s.getName()))).findFirst() .orElseThrow(() -> new AnalysisException("cannot find sum slot when select mv")); - for (Slot slot : sum.child().getInputSlots()) { - context.exprRewriteMap.slotMap.put(slot, sumSlot); - } context.exprRewriteMap.projectExprMap.put(sum.child(), sumSlot); Sum newSum = new Sum(sumSlot); context.exprRewriteMap.aggFuncMap.put(sum, newSum); @@ -1560,7 +1510,6 @@ public Expression visitAggregateFunction(AggregateFunction aggregateFunction, Re Set slots = aggregateFunction.collect(SlotReference.class::isInstance); for (Slot slot : slots) { - context.exprRewriteMap.slotMap.put(slot, aggStateSlot); context.exprRewriteMap.projectExprMap.put(slot, aggStateSlot); } @@ -1675,8 +1624,13 @@ private List generateNewOutputsWithMvOutputs( * +--LogicalOlapScan() * t -> abs(k1#0) + 1 */ - private Set collectRequireExprWithAggAndProject( - List aggExpressions, List projectExpressions) { + private Set collectRequireExprWithAggAndProject(List aggExpressions, + Optional> project) { + List projectExpressions = project.isPresent() ? project.get().getProjects() : null; + if (projectExpressions == null) { + return aggExpressions.stream().collect(ImmutableSet.toImmutableSet()); + } + Optional> slotToProducerOpt = project.map(Project::getAliasToProducer); Map exprIdToExpression = projectExpressions.stream() .collect(Collectors.toMap(NamedExpression::getExprId, e -> { if (e instanceof Alias) { @@ -1684,13 +1638,13 @@ private Set collectRequireExprWithAggAndProject( } return e; })); - return aggExpressions.stream() - .map(e -> { - if ((e instanceof NamedExpression) - && exprIdToExpression.containsKey(((NamedExpression) e).getExprId())) { - return exprIdToExpression.get(((NamedExpression) e).getExprId()); - } - return e; - }).collect(ImmutableSet.toImmutableSet()); + return aggExpressions.stream().map(e -> { + if ((e instanceof NamedExpression) && exprIdToExpression.containsKey(((NamedExpression) e).getExprId())) { + return exprIdToExpression.get(((NamedExpression) e).getExprId()); + } + return e; + }).map(e -> { + return slotToProducerOpt.map(slotToExpressions -> ExpressionUtils.replace(e, slotToExpressions)).orElse(e); + }).collect(ImmutableSet.toImmutableSet()); } } diff --git a/regression-test/data/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.out b/regression-test/data/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.out index cd7da97e1a793c..f37d3ef54f31f6 100644 --- a/regression-test/data/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.out +++ b/regression-test/data/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.out @@ -23,3 +23,9 @@ 2 2 3 3 +-- !select_mv -- +1 2 + +-- !select_mv -- +1 2 1 + diff --git a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.groovy b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.groovy index 1b7bc10a39c68f..9b3e8ba371e328 100644 --- a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.groovy +++ b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.groovy @@ -37,7 +37,7 @@ suite ("testAggQueryOnAggMV1") { sql """insert into emps values("2020-01-03",3,"c",3,3,3);""" - createMV("create materialized view emps_mv as select deptno, sum(salary), max(commission) from emps group by deptno ;") + createMV("create materialized view emps_mv as select deptno, sum(salary), max(commission) from emps group by deptno;") createMV("create materialized view emps_mv_count_key as select deptno, count(deptno) from emps group by deptno;") createMV("create materialized view emps_mv_if as select deptno, sum(if(empid = 1, empid, salary)) from emps group by deptno;") @@ -73,4 +73,16 @@ suite ("testAggQueryOnAggMV1") { contains "(emps_mv_if)" } qt_select_mv "select deptno, sum(if(empid = 1, empid, salary)) from emps group by deptno order by deptno;" + + explain { + sql("select deptno, count(deptno) from emps where deptno=1 group by deptno order by deptno;") + contains "(emps_mv_count_key)" + } + qt_select_mv "select deptno, count(deptno) from emps where deptno=1 group by deptno order by deptno;" + + explain { + sql("select deptno, sum(salary), max(commission) from emps where salary=1 group by deptno order by deptno;") + contains "(emps)" + } + qt_select_mv "select deptno, sum(salary), max(commission) from emps where salary=1 group by deptno order by deptno;" } \ No newline at end of file From ae5b38f95e2d1b7d6a721fbf737099f03e1337b4 Mon Sep 17 00:00:00 2001 From: wuwenchi Date: Tue, 23 Jan 2024 10:42:42 +0800 Subject: [PATCH 149/200] [fix](paimon)set timestamp's scale for parquet which has no logical type (#30119) --- .../format/parquet/parquet_column_convert.h | 28 ++++++++++ .../paimon/paimon_timestamp_types.out | 13 +++++ .../paimon/paimon_timestamp_types.groovy | 55 +++++++++++++++++++ 3 files changed, 96 insertions(+) create mode 100644 regression-test/data/external_table_p2/paimon/paimon_timestamp_types.out create mode 100644 regression-test/suites/external_table_p2/paimon/paimon_timestamp_types.groovy diff --git a/be/src/vec/exec/format/parquet/parquet_column_convert.h b/be/src/vec/exec/format/parquet/parquet_column_convert.h index b5c3ffb7c8870f..39ee29f663f9ed 100644 --- a/be/src/vec/exec/format/parquet/parquet_column_convert.h +++ b/be/src/vec/exec/format/parquet/parquet_column_convert.h @@ -127,6 +127,30 @@ struct ConvertParams { DecimalScaleParams decimal_scale; FieldSchema* field_schema = nullptr; + /** + * Some frameworks like paimon maybe writes non-standard parquet files. Timestamp field doesn't have + * logicalType or converted_type to indicates its precision. We have to reset the time mask. + */ + void reset_time_scale_if_missing(int scale) { + const auto& schema = field_schema->parquet_schema; + if (!schema.__isset.logicalType && !schema.__isset.converted_type) { + int ts_scale = 9; + if (scale <= 3) { + ts_scale = 3; + } else if (scale <= 6) { + ts_scale = 6; + } + second_mask = common::exp10_i64(ts_scale); + scale_to_nano_factor = common::exp10_i64(9 - ts_scale); + + // The missing parque metadata makes it impossible for us to know the time zone information, + // so we default to UTC here. + if (ctz == nullptr) { + ctz = const_cast(&utc0); + } + } + } + void init(FieldSchema* field_schema_, cctz::time_zone* ctz_) { field_schema = field_schema_; if (ctz_ != nullptr) { @@ -671,8 +695,12 @@ inline Status get_converter(tparquet::Type::type parquet_physical_type, Primitiv break; case TypeIndex::DateTimeV2: if (tparquet::Type::INT96 == parquet_physical_type) { + // int96 only stores nanoseconds in standard parquet file + convert_params->reset_time_scale_if_missing(9); *converter = std::make_unique(); } else if (tparquet::Type::INT64 == parquet_physical_type) { + convert_params->reset_time_scale_if_missing( + remove_nullable(dst_data_type)->get_scale()); *converter = std::make_unique(); } break; diff --git a/regression-test/data/external_table_p2/paimon/paimon_timestamp_types.out b/regression-test/data/external_table_p2/paimon/paimon_timestamp_types.out new file mode 100644 index 00000000000000..641424b160e67c --- /dev/null +++ b/regression-test/data/external_table_p2/paimon/paimon_timestamp_types.out @@ -0,0 +1,13 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !c1 -- +1 5432-08-30T05:43:21.100 5432-08-30T05:43:21.120 5432-08-30T05:43:21.123 5432-08-30T05:43:21.123400 5432-08-30T05:43:21.123450 5432-08-30T05:43:21.123456 5432-08-30T05:43:21.123456 5432-08-30T05:43:21.123456 5432-08-30T05:43:21.123456 + +-- !c2 -- +1 5432-08-30T05:43:21.100 5432-08-30T05:43:21.120 5432-08-30T05:43:21.123 5432-08-30T05:43:21.123400 5432-08-30T05:43:21.123450 5432-08-30T05:43:21.123456 5432-08-30T05:43:21.123456 5432-08-30T05:43:21.123456 5432-08-30T05:43:21.123456 + +-- !c3 -- +1 5432-08-30T05:43:21.100 5432-08-30T05:43:21.120 5432-08-30T05:43:21.123 5432-08-30T05:43:21.123400 5432-08-30T05:43:21.123450 5432-08-30T05:43:21.123456 5432-08-30T05:43:21.123456 5432-08-30T05:43:21.123456 5432-08-30T05:43:21.123456 + +-- !c4 -- +1 5432-08-30T05:43:21.100 5432-08-30T05:43:21.120 5432-08-30T05:43:21.123 5432-08-30T05:43:21.123400 5432-08-30T05:43:21.123450 5432-08-30T05:43:21.123456 5432-08-30T05:43:21.123456 5432-08-30T05:43:21.123456 5432-08-30T05:43:21.123456 + diff --git a/regression-test/suites/external_table_p2/paimon/paimon_timestamp_types.groovy b/regression-test/suites/external_table_p2/paimon/paimon_timestamp_types.groovy new file mode 100644 index 00000000000000..dbb1f1d038c28a --- /dev/null +++ b/regression-test/suites/external_table_p2/paimon/paimon_timestamp_types.groovy @@ -0,0 +1,55 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +suite("paimon_timestamp_types", "p2,external,paimon,external_remote,external_remote_paimon") { + + def ts_orc = """select * from ts_orc""" + def ts_parquet = """select * from ts_parquet""" + + String enabled = context.config.otherConfigs.get("enableExternalPaimonTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String catalog_name = "paimon_timestamp_catalog" + String user_name = context.config.otherConfigs.get("extHiveHmsUser") + String hiveHost = context.config.otherConfigs.get("extHiveHmsHost") + String hivePort = context.config.otherConfigs.get("extHdfsPort") + + sql """drop catalog if exists ${catalog_name};""" + sql """ + create catalog if not exists ${catalog_name} properties ( + "type" = "paimon", + "paimon.catalog.type" = "filesystem", + "warehouse" = "hdfs://${hiveHost}/${hivePort}/paimon/paimon1", + "hadoop.username" = "${user_name}" + ); + """ + logger.info("catalog " + catalog_name + " created") + sql """switch ${catalog_name};""" + logger.info("switched to catalog " + catalog_name) + sql """use db1;""" + logger.info("use db1") + + sql """set force_jni_scanner=true""" + qt_c1 ts_orc + qt_c2 ts_parquet + + sql """set force_jni_scanner=false""" + qt_c3 ts_orc + qt_c4 ts_parquet + + } +} + From 5fe0e6c215f096ad044e8a2450816ebac1b58502 Mon Sep 17 00:00:00 2001 From: zclllyybb Date: Tue, 23 Jan 2024 10:46:51 +0800 Subject: [PATCH 150/200] [chore](docs) fix some docs wrong & add important comment & fe start config for old machine (#29742) fix some docs wrong & add important comment & fe start config for old machine --- be/src/vec/sink/writer/vtablet_writer.h | 12 +++++++----- conf/fe.conf | 2 +- .../docs/install/source-install/compilation-arm.md | 3 +++ .../sql-reference/Data-Types/DATETIME.md | 7 ++++--- .../docs/install/source-install/compilation-arm.md | 3 +++ .../sql-reference/Data-Types/DATETIME.md | 14 +++++++------- 6 files changed, 25 insertions(+), 16 deletions(-) diff --git a/be/src/vec/sink/writer/vtablet_writer.h b/be/src/vec/sink/writer/vtablet_writer.h index 68534cd3a16ec5..e9ae83ae484e08 100644 --- a/be/src/vec/sink/writer/vtablet_writer.h +++ b/be/src/vec/sink/writer/vtablet_writer.h @@ -233,14 +233,15 @@ class VNodeChannel { _slave_tablet_nodes[tablet_id] = slave_nodes; } - // build a request and build corresponding connect to BE. + // this function is NON_REENTRANT + Status init(RuntimeState* state); + /// these two functions will call open_internal. should keep that clear --- REENTRANT + // build corresponding connect to BE. NON-REENTRANT void open(); - // for auto partition, we use this to open more tablet. + // for auto partition, we use this to open more tablet. KEEP IT REENTRANT void incremental_open(); - - Status init(RuntimeState* state); - // this will block until all request transmission which were opened or incremental opened finished. + // this function will called multi times. NON_REENTRANT Status open_wait(); Status add_block(vectorized::Block* block, const Payload* payload, bool is_append = false); @@ -407,6 +408,7 @@ class VNodeChannel { using AddBlockReq = std::pair, std::shared_ptr>; std::queue _pending_blocks; + // send block to slave BE rely on this. dont reconstruct it. std::shared_ptr> _send_block_callback = nullptr; bool _is_incremental; diff --git a/conf/fe.conf b/conf/fe.conf index b1151585db7464..a8d30963890979 100644 --- a/conf/fe.conf +++ b/conf/fe.conf @@ -30,7 +30,7 @@ LOG_DIR = ${DORIS_HOME}/log # JAVA_OPTS="-Dsun.security.krb5.debug=true -Djavax.security.auth.useSubjectCredsOnly=false -Xss4m -Xmx8192m -XX:+UseMembar -XX:SurvivorRatio=8 -XX:MaxTenuringThreshold=7 -XX:+PrintGCDateStamps -XX:+PrintGCDetails -XX:+UseConcMarkSweepGC -XX:+UseParNewGC -XX:+CMSClassUnloadingEnabled -XX:-CMSParallelRemarkEnabled -XX:CMSInitiatingOccupancyFraction=80 -XX:SoftRefLRUPolicyMSPerMB=0 -Xloggc:$DORIS_HOME/log/fe.gc.log.$CUR_DATE" # G1 JAVA OPTS -JAVA_OPTS="-Djavax.security.auth.useSubjectCredsOnly=false -Xss4m -Xmx8192m -XX:+UseG1GC -XX:MaxGCPauseMillis=200 -XX:+PrintGCDateStamps -XX:+PrintGCDetails -Xloggc:$DORIS_HOME/log/fe.gc.log.$CUR_DATE -Dlog4j2.formatMsgNoLookups=true" +JAVA_OPTS="-Djavax.security.auth.useSubjectCredsOnly=false -Xss4m -Xmx8192m -XX:+UnlockExperimentalVMOptions -XX:+UseG1GC -XX:MaxGCPauseMillis=200 -XX:+PrintGCDateStamps -XX:+PrintGCDetails -Xloggc:$DORIS_HOME/log/fe.gc.log.$CUR_DATE -Dlog4j2.formatMsgNoLookups=true" # For jdk 9+, this JAVA_OPTS_FOR_JDK_9 will be used as default CMS JVM options # JAVA_OPTS_FOR_JDK_9="-Dsun.security.krb5.debug=true -Djavax.security.auth.useSubjectCredsOnly=false -Xss4m -Xmx8192m -XX:SurvivorRatio=8 -XX:MaxTenuringThreshold=7 -XX:+CMSClassUnloadingEnabled -XX:-CMSParallelRemarkEnabled -XX:CMSInitiatingOccupancyFraction=80 -XX:SoftRefLRUPolicyMSPerMB=0 -Xlog:gc*:$DORIS_HOME/log/fe.gc.log.$CUR_DATE:time" diff --git a/docs/en/docs/install/source-install/compilation-arm.md b/docs/en/docs/install/source-install/compilation-arm.md index 6a828752192fe6..51c7118e2c7fd2 100644 --- a/docs/en/docs/install/source-install/compilation-arm.md +++ b/docs/en/docs/install/source-install/compilation-arm.md @@ -24,6 +24,9 @@ specific language governing permissions and limitations under the License. --> +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + # Compile with ARM This topic is about how to compile Doris on the ARM64 platform. diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Types/DATETIME.md b/docs/en/docs/sql-manual/sql-reference/Data-Types/DATETIME.md index 554c9e634a0ee6..609ca416f1dcd0 100644 --- a/docs/en/docs/sql-manual/sql-reference/Data-Types/DATETIME.md +++ b/docs/en/docs/sql-manual/sql-reference/Data-Types/DATETIME.md @@ -33,6 +33,7 @@ DATETIME ### Description + DATETIME([P]) Date and time type. The optional parameter P indicates the time precision and the value range is [0, 6], that is, it supports up to 6 decimal places (microseconds). 0 when not set. @@ -41,13 +42,13 @@ The form of printing is 'yyyy-MM-dd HH:mm:ss.SSSSSS' ### note - DATETIME supports temporal precision up to microseconds. When parsing imported DATETIME type data using the BE side (e.g. using Stream load, Spark load, etc.), or using the FE side with the [Nereids](../../../query-acceleration/nereids) on, decimals exceeding the current precision will be **rounded**. - DATETIME reads support resolving the time zone in the format of the original DATETIME literal followed by the time zone: +DATETIME supports temporal precision up to microseconds. When parsing imported DATETIME type data using the BE side (e.g. using Stream load, Spark load, etc.), or using the FE side with the [Nereids](../../../query-acceleration/nereids) on, decimals exceeding the current precision will be **rounded**. +DATETIME reads support resolving the time zone in the format of the original DATETIME literal followed by the time zone: ```sql