[feature](partial update) support MOW partial update for insert statement (#21597)

This commit is contained in:
bobhan1
2023-09-16 17:11:59 +08:00
committed by GitHub
parent 643d09de06
commit ed8db3727c
24 changed files with 1078 additions and 33 deletions

View File

@ -692,6 +692,12 @@ Translated with www.DeepL.com/Translator (free version)
Build MemTable on DataSink node, and send segments to other backends through brpc streaming.
It reduces duplicate work among replicas, and saves time in data serialization & deserialization.
* `enable_unique_key_partial_update`
<version since="2.0.2">
Whether to enable partial columns update semantics for native insert into statement, default is false.
</version>
***
#### Supplementary instructions on statement execution timeout control

View File

@ -64,6 +64,9 @@ INSERT [IGNORE] INTO table_name
> 2. SHUFFLE: When the target table is a partition table, enabling this hint will do repartiiton.
> 3. NOSHUFFLE: Even if the target table is a partition table, repartiiton will not be performed, but some other operations will be performed to ensure that the data is correctly dropped into each partition.
For a Unique table with merge-on-write enabled, you can also perform partial columns updates using the insert statement. To perform partial column updates with the insert statement, you need to set the session variable enable_unique_key_partial_update to true (the default value for this variable is false, meaning partial columns updates with the insert statement are not allowed by default). When performing partial columns updates, the columns being inserted must contain at least all the Key columns and specify the columns you want to update. If the Key column values for the inserted row already exist in the original table, the data in the row with the same key column values will be updated. If the Key column values for the inserted row do not exist in the original table, a new row will be inserted into the table. In this case, columns not specified in the insert statement must either have default values or be nullable. These missing columns will first attempt to be populated with default values, and if a column has no default value, it will be filled with null. If a column cannot be null, the insert operation will fail.
Notice:
When executing the `INSERT` statement, the default behavior is to filter the data that does not conform to the target table format, such as the string is too long. However, for business scenarios that require data not to be filtered, you can set the session variable `enable_insert_strict` to `true` to ensure that `INSERT` will not be executed successfully when data is filtered out.

View File

@ -679,6 +679,12 @@ try (Connection conn = DriverManager.getConnection("jdbc:mysql://127.0.0.1:9030/
在 DataSink 节点上构建 MemTable,并通过 brpc streaming 发送 segment 到其他 BE。
该方法减少了多副本之间的重复工作,并且节省了数据序列化和反序列化的时间。
* `enable_unique_key_partial_update`
<version since="2.0.2">
是否在对insert into语句启用部分列更新的语义,默认为 false
</version>
***
#### 关于语句执行超时控制的补充说明

View File

@ -65,6 +65,8 @@ INSERT [IGNORE] INTO table_name
> 2. SHUFFLE:当目标表是分区表,开启这个 hint 会进行 repartiiton。
> 3. NOSHUFFLE:即使目标表是分区表,也不会进行 repartiiton,但会做一些其他操作以保证数据正确落到各个分区中。
对于开启了merge-on-write的Unique表,还可以使用insert语句进行部分列更新的操作。要使用insert语句进行部分列更新,需要将会话变量enable_uniuqe_key_partial_update的值设置为true(该变量默认值为false,即默认无法通过insert语句进行部分列更新)。进行部分列更新时,插入的列必须至少包含所有的Key列,同时指定需要更新的列。如果插入行Key列的值在原表中存在,则将更新具有相同key列值那一行的数据。如果插入行Key列的值在原表中不存在,则将向表中插入一条新的数据,此时insert语句中没有指定的列必须有默认值或可以为null,这些缺失列会首先尝试用默认值填充,如果该列没有默认值,则尝试使用null值填充,如果该列不能为null,则本次插入失败。
注意:
当前执行 `INSERT` 语句时,对于有不符合目标表格式的数据,默认的行为是过滤,比如字符串超长等。但是对于有要求数据不能够被过滤的业务场景,可以通过设置会话变量 `enable_insert_strict``true` 来确保当有数据被过滤掉的时候,`INSERT` 不会被执行成功。

View File

@ -203,6 +203,7 @@ public class DeleteStmt extends DdlStmt {
null,
isPartialUpdate,
false);
((NativeInsertStmt) insertStmt).setIsFromDeleteOrUpdateStmt(true);
}
private void analyzeTargetTable(Analyzer analyzer) throws UserException {

View File

@ -167,6 +167,7 @@ public class NativeInsertStmt extends InsertStmt {
public boolean isInnerGroupCommit = false;
private boolean isInsertIgnore = false;
private boolean isFromDeleteOrUpdateStmt = false;
public NativeInsertStmt(InsertTarget target, String label, List<String> cols, InsertSource source,
List<String> hints) {
@ -405,8 +406,10 @@ public class NativeInsertStmt extends InsertStmt {
OlapTableSink sink = (OlapTableSink) dataSink;
TUniqueId loadId = analyzer.getContext().queryId();
int sendBatchParallelism = analyzer.getContext().getSessionVariable().getSendBatchParallelism();
boolean isInsertStrict = analyzer.getContext().getSessionVariable().getEnableInsertStrict()
&& !isFromDeleteOrUpdateStmt;
sink.init(loadId, transactionId, db.getId(), timeoutSecond,
sendBatchParallelism, false, false, isInsertIgnore);
sendBatchParallelism, false, isInsertStrict, isInsertIgnore);
}
}
@ -618,6 +621,10 @@ public class NativeInsertStmt extends InsertStmt {
ErrorReport.reportAnalysisException(ErrorCode.ERR_WRONG_VALUE_COUNT);
}
if (analyzer.getContext().getSessionVariable().isEnableUniqueKeyPartialUpdate()) {
trySetPartialUpdate();
}
// Check if all columns mentioned is enough
checkColumnCoverage(mentionedColumns, targetTable.getBaseSchema());
@ -1098,4 +1105,54 @@ public class NativeInsertStmt extends InsertStmt {
public ByteString getRangeBytes() {
return rangeBytes;
}
public void setIsFromDeleteOrUpdateStmt(boolean isFromDeleteOrUpdateStmt) {
this.isFromDeleteOrUpdateStmt = isFromDeleteOrUpdateStmt;
}
private void trySetPartialUpdate() throws UserException {
if (isFromDeleteOrUpdateStmt || isPartialUpdate || !(targetTable instanceof OlapTable)) {
return;
}
OlapTable olapTable = (OlapTable) targetTable;
if (!olapTable.getEnableUniqueKeyMergeOnWrite()) {
throw new UserException("Partial update is only allowed in unique table with merge-on-write enabled.");
}
for (Column col : olapTable.getFullSchema()) {
boolean exists = false;
for (Column insertCol : targetColumns) {
if (insertCol.getName() != null && insertCol.getName().equals(col.getName())) {
if (!col.isVisible() && !Column.DELETE_SIGN.equals(col.getName())) {
throw new UserException("Partial update should not include invisible column except"
+ " delete sign column: " + col.getName());
}
exists = true;
break;
}
}
if (col.isKey() && !exists) {
throw new UserException("Partial update should include all key columns, missing: " + col.getName());
}
}
isPartialUpdate = true;
partialUpdateCols.addAll(targetColumnNames);
if (isPartialUpdate && olapTable.hasSequenceCol() && olapTable.getSequenceMapCol() != null
&& partialUpdateCols.contains(olapTable.getSequenceMapCol())) {
partialUpdateCols.add(Column.SEQUENCE_COL);
}
// we should re-generate olapTuple
DescriptorTable descTable = analyzer.getDescTbl();
olapTuple = descTable.createTupleDescriptor();
for (Column col : olapTable.getFullSchema()) {
if (!partialUpdateCols.contains(col.getName())) {
continue;
}
SlotDescriptor slotDesc = descTable.addSlotDescriptor(olapTuple);
slotDesc.setIsMaterialized(true);
slotDesc.setType(col.getType());
slotDesc.setColumn(col);
slotDesc.setIsNullable(col.isAllowNull());
}
}
}

View File

@ -127,6 +127,7 @@ public class UpdateStmt extends DdlStmt {
null,
isPartialUpdate,
false);
((NativeInsertStmt) insertStmt).setIsFromDeleteOrUpdateStmt(true);
}
private void analyzeTargetTable(Analyzer analyzer) throws UserException {

View File

@ -47,29 +47,39 @@ public class UnboundOlapTableSink<CHILD_TYPE extends Plan> extends LogicalSink<C
private final List<String> hints;
private final List<String> partitions;
private final boolean isPartialUpdate;
private final boolean isFromNativeInsertStmt;
public UnboundOlapTableSink(List<String> nameParts, List<String> colNames, List<String> hints,
List<String> partitions, CHILD_TYPE child) {
this(nameParts, colNames, hints, partitions, false, Optional.empty(), Optional.empty(), child);
this(nameParts, colNames, hints, partitions, false, false, Optional.empty(), Optional.empty(), child);
}
public UnboundOlapTableSink(List<String> nameParts, List<String> colNames, List<String> hints,
List<String> partitions, boolean isPartialUpdate, CHILD_TYPE child) {
this(nameParts, colNames, hints, partitions, isPartialUpdate, Optional.empty(), Optional.empty(), child);
this(nameParts, colNames, hints, partitions, isPartialUpdate, false,
Optional.empty(), Optional.empty(), child);
}
public UnboundOlapTableSink(List<String> nameParts, List<String> colNames, List<String> hints,
List<String> partitions, boolean isPartialUpdate, boolean isFromNativeInsertStmt, CHILD_TYPE child) {
this(nameParts, colNames, hints, partitions, isPartialUpdate, isFromNativeInsertStmt,
Optional.empty(), Optional.empty(), child);
}
/**
* constructor
*/
public UnboundOlapTableSink(List<String> nameParts, List<String> colNames, List<String> hints,
List<String> partitions, boolean isPartialUpdate, Optional<GroupExpression> groupExpression,
Optional<LogicalProperties> logicalProperties, CHILD_TYPE child) {
List<String> partitions, boolean isPartialUpdate, boolean isFromNativeInsertStmt,
Optional<GroupExpression> groupExpression, Optional<LogicalProperties> logicalProperties,
CHILD_TYPE child) {
super(PlanType.LOGICAL_UNBOUND_OLAP_TABLE_SINK, ImmutableList.of(), groupExpression, logicalProperties, child);
this.nameParts = Utils.copyRequiredList(nameParts);
this.colNames = Utils.copyRequiredList(colNames);
this.hints = Utils.copyRequiredList(hints);
this.partitions = Utils.copyRequiredList(partitions);
this.isPartialUpdate = isPartialUpdate;
this.isFromNativeInsertStmt = isFromNativeInsertStmt;
}
public List<String> getColNames() {
@ -92,11 +102,15 @@ public class UnboundOlapTableSink<CHILD_TYPE extends Plan> extends LogicalSink<C
return isPartialUpdate;
}
public boolean isFromNativeInsertStmt() {
return isFromNativeInsertStmt;
}
@Override
public Plan withChildren(List<Plan> children) {
Preconditions.checkArgument(children.size() == 1, "UnboundOlapTableSink only accepts one child");
return new UnboundOlapTableSink<>(nameParts, colNames, hints, partitions, isPartialUpdate,
groupExpression, Optional.of(getLogicalProperties()), children.get(0));
isFromNativeInsertStmt, groupExpression, Optional.of(getLogicalProperties()), children.get(0));
}
@Override
@ -131,15 +145,15 @@ public class UnboundOlapTableSink<CHILD_TYPE extends Plan> extends LogicalSink<C
@Override
public Plan withGroupExpression(Optional<GroupExpression> groupExpression) {
return new UnboundOlapTableSink<>(nameParts, colNames, hints, partitions,
isPartialUpdate, groupExpression, Optional.of(getLogicalProperties()), child());
return new UnboundOlapTableSink<>(nameParts, colNames, hints, partitions, isPartialUpdate,
isFromNativeInsertStmt, groupExpression, Optional.of(getLogicalProperties()), child());
}
@Override
public Plan withGroupExprLogicalPropChildren(Optional<GroupExpression> groupExpression,
Optional<LogicalProperties> logicalProperties, List<Plan> children) {
return new UnboundOlapTableSink<>(nameParts, colNames, hints, partitions,
isPartialUpdate, groupExpression, logicalProperties, children.get(0));
isPartialUpdate, isFromNativeInsertStmt, groupExpression, logicalProperties, children.get(0));
}
@Override

View File

@ -367,11 +367,34 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor<PlanFragment, Pla
olapTableSink.getPartitionIds().isEmpty() ? null : olapTableSink.getPartitionIds(),
olapTableSink.isSingleReplicaLoad()
);
if (olapTableSink.isPartialUpdate()) {
if (olapTableSink.isPartialUpdate() || (olapTableSink.isFromNativeInsertStmt()
&& ConnectContext.get().getSessionVariable().isEnableUniqueKeyPartialUpdate())) {
OlapTable olapTable = (OlapTable) olapTableSink.getTargetTable();
if (!olapTable.getEnableUniqueKeyMergeOnWrite()) {
throw new AnalysisException("Partial update is only allowed in"
+ "unique table with merge-on-write enabled.");
}
HashSet<String> partialUpdateCols = new HashSet<>();
for (Column col : olapTable.getFullSchema()) {
boolean exists = false;
for (Column insertCol : olapTableSink.getCols()) {
if (insertCol.getName() != null && insertCol.getName().equals(col.getName())) {
exists = true;
break;
}
}
if (col.isKey() && !exists) {
throw new AnalysisException("Partial update should include all key columns, missing: "
+ col.getName());
}
}
for (Column col : olapTableSink.getCols()) {
partialUpdateCols.add(col.getName());
}
if (olapTable.hasSequenceCol() && olapTable.getSequenceMapCol() != null
&& partialUpdateCols.contains(olapTable.getSequenceMapCol())) {
partialUpdateCols.add(Column.SEQUENCE_COL);
}
sink.setPartialUpdateInputColumns(true, partialUpdateCols);
}
rootFragment.setSink(sink);

View File

@ -411,6 +411,8 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
colNames,
ImmutableList.of(),
partitions,
false,
true,
visitQuery(ctx.query()));
if (ctx.explain() != null) {
return withExplain(sink, ctx.explain());

View File

@ -84,6 +84,7 @@ public class BindSink implements AnalysisRuleFactory {
.map(NamedExpression.class::cast)
.collect(ImmutableList.toImmutableList()),
sink.isPartialUpdate(),
sink.isFromNativeInsertStmt(),
sink.child());
// we need to insert all the columns of the target table

View File

@ -40,6 +40,7 @@ public class LogicalOlapTableSinkToPhysicalOlapTableSink extends OneImplementati
sink.getCols(),
ctx.connectContext.getSessionVariable().isEnableSingleReplicaInsert(),
sink.isPartialUpdate(),
sink.isFromNativeInsertStmt(),
Optional.empty(),
sink.getLogicalProperties(),
sink.child());

View File

@ -250,6 +250,8 @@ public class InsertIntoTableCommand extends Command implements ForwardWithSync,
sink.getColNames(),
sink.getHints(),
tempPartitionNames,
sink.isPartialUpdate(),
sink.isFromNativeInsertStmt(),
(LogicalPlan) (sink.child(0)));
new InsertIntoTableCommand(copySink, labelName, false).run(ctx, executor);
if (ctx.getState().getStateType() == MysqlStateType.ERR) {

View File

@ -46,10 +46,12 @@ public class LogicalOlapTableSink<CHILD_TYPE extends Plan> extends LogicalSink<C
private final List<Column> cols;
private final List<Long> partitionIds;
private final boolean isPartialUpdate;
private final boolean isFromNativeInsertStmt;
public LogicalOlapTableSink(Database database, OlapTable targetTable, List<Column> cols, List<Long> partitionIds,
List<NamedExpression> outputExprs, boolean isPartialUpdate, CHILD_TYPE child) {
this(database, targetTable, cols, partitionIds, outputExprs, isPartialUpdate,
List<NamedExpression> outputExprs, boolean isPartialUpdate, boolean isFromNativeInsertStmt,
CHILD_TYPE child) {
this(database, targetTable, cols, partitionIds, outputExprs, isPartialUpdate, isFromNativeInsertStmt,
Optional.empty(), Optional.empty(), child);
}
@ -58,13 +60,14 @@ public class LogicalOlapTableSink<CHILD_TYPE extends Plan> extends LogicalSink<C
*/
public LogicalOlapTableSink(Database database, OlapTable targetTable, List<Column> cols,
List<Long> partitionIds, List<NamedExpression> outputExprs, boolean isPartialUpdate,
Optional<GroupExpression> groupExpression, Optional<LogicalProperties> logicalProperties,
CHILD_TYPE child) {
boolean isFromNativeInsertStmt, Optional<GroupExpression> groupExpression,
Optional<LogicalProperties> logicalProperties, CHILD_TYPE child) {
super(PlanType.LOGICAL_OLAP_TABLE_SINK, outputExprs, groupExpression, logicalProperties, child);
this.database = Objects.requireNonNull(database, "database != null in LogicalOlapTableSink");
this.targetTable = Objects.requireNonNull(targetTable, "targetTable != null in LogicalOlapTableSink");
this.cols = Utils.copyRequiredList(cols);
this.isPartialUpdate = isPartialUpdate;
this.isFromNativeInsertStmt = isFromNativeInsertStmt;
this.partitionIds = Utils.copyRequiredList(partitionIds);
}
@ -73,14 +76,14 @@ public class LogicalOlapTableSink<CHILD_TYPE extends Plan> extends LogicalSink<C
.map(NamedExpression.class::cast)
.collect(ImmutableList.toImmutableList());
return new LogicalOlapTableSink<>(database, targetTable, cols, partitionIds, output, isPartialUpdate,
Optional.empty(), Optional.empty(), child);
isFromNativeInsertStmt, Optional.empty(), Optional.empty(), child);
}
@Override
public Plan withChildren(List<Plan> children) {
Preconditions.checkArgument(children.size() == 1, "LogicalOlapTableSink only accepts one child");
return new LogicalOlapTableSink<>(database, targetTable, cols, partitionIds, outputExprs, isPartialUpdate,
Optional.empty(), Optional.empty(), children.get(0));
isFromNativeInsertStmt, Optional.empty(), Optional.empty(), children.get(0));
}
public Database getDatabase() {
@ -103,6 +106,10 @@ public class LogicalOlapTableSink<CHILD_TYPE extends Plan> extends LogicalSink<C
return isPartialUpdate;
}
public boolean isFromNativeInsertStmt() {
return isFromNativeInsertStmt;
}
@Override
public boolean equals(Object o) {
if (this == o) {
@ -115,14 +122,16 @@ public class LogicalOlapTableSink<CHILD_TYPE extends Plan> extends LogicalSink<C
return false;
}
LogicalOlapTableSink<?> that = (LogicalOlapTableSink<?>) o;
return isPartialUpdate == that.isPartialUpdate && Objects.equals(database, that.database)
return isPartialUpdate == that.isPartialUpdate && isFromNativeInsertStmt == that.isFromNativeInsertStmt
&& Objects.equals(database, that.database)
&& Objects.equals(targetTable, that.targetTable) && Objects.equals(cols, that.cols)
&& Objects.equals(partitionIds, that.partitionIds);
}
@Override
public int hashCode() {
return Objects.hash(super.hashCode(), database, targetTable, cols, partitionIds, isPartialUpdate);
return Objects.hash(super.hashCode(), database, targetTable, cols, partitionIds,
isPartialUpdate, isFromNativeInsertStmt);
}
@Override
@ -133,13 +142,13 @@ public class LogicalOlapTableSink<CHILD_TYPE extends Plan> extends LogicalSink<C
@Override
public Plan withGroupExpression(Optional<GroupExpression> groupExpression) {
return new LogicalOlapTableSink<>(database, targetTable, cols, partitionIds, outputExprs, isPartialUpdate,
groupExpression, Optional.of(getLogicalProperties()), child());
isFromNativeInsertStmt, groupExpression, Optional.of(getLogicalProperties()), child());
}
@Override
public Plan withGroupExprLogicalPropChildren(Optional<GroupExpression> groupExpression,
Optional<LogicalProperties> logicalProperties, List<Plan> children) {
return new LogicalOlapTableSink<>(database, targetTable, cols, partitionIds, outputExprs, isPartialUpdate,
groupExpression, logicalProperties, children.get(0));
isFromNativeInsertStmt, groupExpression, logicalProperties, children.get(0));
}
}

View File

@ -59,14 +59,15 @@ public class PhysicalOlapTableSink<CHILD_TYPE extends Plan> extends PhysicalSink
private final List<Long> partitionIds;
private final boolean singleReplicaLoad;
private final boolean isPartialUpdate;
private final boolean isFromNativeInsertStmt;
/**
* Constructor
*/
public PhysicalOlapTableSink(Database database, OlapTable targetTable, List<Long> partitionIds, List<Column> cols,
boolean singleReplicaLoad, boolean isPartialUpdate, Optional<GroupExpression> groupExpression,
LogicalProperties logicalProperties, CHILD_TYPE child) {
this(database, targetTable, partitionIds, cols, singleReplicaLoad, isPartialUpdate,
boolean singleReplicaLoad, boolean isPartialUpdate, boolean isFromNativeInsertStmt,
Optional<GroupExpression> groupExpression, LogicalProperties logicalProperties, CHILD_TYPE child) {
this(database, targetTable, partitionIds, cols, singleReplicaLoad, isPartialUpdate, isFromNativeInsertStmt,
groupExpression, logicalProperties, PhysicalProperties.GATHER, null, child);
}
@ -74,9 +75,9 @@ public class PhysicalOlapTableSink<CHILD_TYPE extends Plan> extends PhysicalSink
* Constructor
*/
public PhysicalOlapTableSink(Database database, OlapTable targetTable, List<Long> partitionIds, List<Column> cols,
boolean singleReplicaLoad, boolean isPartialUpdate, Optional<GroupExpression> groupExpression,
LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics,
CHILD_TYPE child) {
boolean singleReplicaLoad, boolean isPartialUpdate, boolean isFromNativeInsertStmt,
Optional<GroupExpression> groupExpression, LogicalProperties logicalProperties,
PhysicalProperties physicalProperties, Statistics statistics, CHILD_TYPE child) {
super(PlanType.PHYSICAL_OLAP_TABLE_SINK, groupExpression, logicalProperties, physicalProperties,
statistics, child);
this.database = Objects.requireNonNull(database, "database != null in PhysicalOlapTableSink");
@ -85,6 +86,7 @@ public class PhysicalOlapTableSink<CHILD_TYPE extends Plan> extends PhysicalSink
this.partitionIds = Utils.copyRequiredList(partitionIds);
this.singleReplicaLoad = singleReplicaLoad;
this.isPartialUpdate = isPartialUpdate;
this.isFromNativeInsertStmt = isFromNativeInsertStmt;
}
public Database getDatabase() {
@ -111,12 +113,16 @@ public class PhysicalOlapTableSink<CHILD_TYPE extends Plan> extends PhysicalSink
return isPartialUpdate;
}
public boolean isFromNativeInsertStmt() {
return isFromNativeInsertStmt;
}
@Override
public Plan withChildren(List<Plan> children) {
Preconditions.checkArgument(children.size() == 1, "PhysicalOlapTableSink only accepts one child");
return new PhysicalOlapTableSink<>(database, targetTable, partitionIds, cols,
singleReplicaLoad, isPartialUpdate, groupExpression, getLogicalProperties(), physicalProperties,
statistics, children.get(0));
singleReplicaLoad, isPartialUpdate, isFromNativeInsertStmt, groupExpression,
getLogicalProperties(), physicalProperties, statistics, children.get(0));
}
@Override
@ -130,6 +136,7 @@ public class PhysicalOlapTableSink<CHILD_TYPE extends Plan> extends PhysicalSink
PhysicalOlapTableSink<?> that = (PhysicalOlapTableSink<?>) o;
return singleReplicaLoad == that.singleReplicaLoad
&& isPartialUpdate == that.isPartialUpdate
&& isFromNativeInsertStmt == that.isFromNativeInsertStmt
&& Objects.equals(database, that.database)
&& Objects.equals(targetTable, that.targetTable)
&& Objects.equals(cols, that.cols)
@ -138,7 +145,8 @@ public class PhysicalOlapTableSink<CHILD_TYPE extends Plan> extends PhysicalSink
@Override
public int hashCode() {
return Objects.hash(database, targetTable, cols, partitionIds, singleReplicaLoad, isPartialUpdate);
return Objects.hash(database, targetTable, cols, partitionIds, singleReplicaLoad,
isPartialUpdate, isFromNativeInsertStmt);
}
@Override
@ -172,20 +180,21 @@ public class PhysicalOlapTableSink<CHILD_TYPE extends Plan> extends PhysicalSink
@Override
public Plan withGroupExpression(Optional<GroupExpression> groupExpression) {
return new PhysicalOlapTableSink<>(database, targetTable, partitionIds, cols, singleReplicaLoad,
isPartialUpdate, groupExpression, getLogicalProperties(), child());
isPartialUpdate, isFromNativeInsertStmt, groupExpression, getLogicalProperties(), child());
}
@Override
public Plan withGroupExprLogicalPropChildren(Optional<GroupExpression> groupExpression,
Optional<LogicalProperties> logicalProperties, List<Plan> children) {
return new PhysicalOlapTableSink<>(database, targetTable, partitionIds, cols, singleReplicaLoad,
isPartialUpdate, groupExpression, logicalProperties.get(), children.get(0));
isPartialUpdate, isFromNativeInsertStmt, groupExpression, logicalProperties.get(), children.get(0));
}
@Override
public PhysicalPlan withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, Statistics statistics) {
return new PhysicalOlapTableSink<>(database, targetTable, partitionIds, cols, singleReplicaLoad,
isPartialUpdate, groupExpression, getLogicalProperties(), physicalProperties, statistics, child());
isPartialUpdate, isFromNativeInsertStmt, groupExpression, getLogicalProperties(),
physicalProperties, statistics, child());
}
/**
@ -225,6 +234,7 @@ public class PhysicalOlapTableSink<CHILD_TYPE extends Plan> extends PhysicalSink
@Override
public PhysicalOlapTableSink<Plan> resetLogicalProperties() {
return new PhysicalOlapTableSink<>(database, targetTable, partitionIds, cols, singleReplicaLoad,
isPartialUpdate, groupExpression, null, physicalProperties, statistics, child());
isPartialUpdate, isFromNativeInsertStmt, groupExpression,
null, physicalProperties, statistics, child());
}
}

View File

@ -400,6 +400,8 @@ public class SessionVariable implements Serializable, Writable {
public static final String ENABLE_MEMTABLE_ON_SINK_NODE =
"enable_memtable_on_sink_node";
public static final String ENABLE_UNIQUE_KEY_PARTIAL_UPDATE = "enable_unique_key_partial_update";
public static final String INVERTED_INDEX_CONJUNCTION_OPT_THRESHOLD = "inverted_index_conjunction_opt_threshold";
public static final String FULL_AUTO_ANALYZE_START_TIME = "full_auto_analyze_start_time";
@ -1192,6 +1194,9 @@ public class SessionVariable implements Serializable, Writable {
flag = VariableMgr.GLOBAL)
public String fullAutoAnalyzeEndTime = "";
@VariableMgr.VarAttr(name = ENABLE_UNIQUE_KEY_PARTIAL_UPDATE, needForward = false)
public boolean enableUniqueKeyPartialUpdate = false;
// If this fe is in fuzzy mode, then will use initFuzzyModeVariables to generate some variables,
// not the default value set in the code.
public void initFuzzyModeVariables() {
@ -2198,6 +2203,14 @@ public class SessionVariable implements Serializable, Writable {
this.truncateCharOrVarcharColumns = truncateCharOrVarcharColumns;
}
public boolean isEnableUniqueKeyPartialUpdate() {
return enableUniqueKeyPartialUpdate;
}
public void setEnableUniqueKeyPartialUpdate(boolean enableUniqueKeyPartialUpdate) {
this.enableUniqueKeyPartialUpdate = enableUniqueKeyPartialUpdate;
}
/**
* Serialize to thrift object.
* Used for rest api.

View File

@ -0,0 +1,66 @@
-- This file is automatically generated. You should know what you did if you want to edit this
-- !1 --
1 doris 1000 123 1
2 doris2 2000 223 1
-- !1 --
1 doris 200 123 1
2 doris2 400 223 1
4 yixiu 400 \N 4321
-- !2 --
1 doris 1000 123 1 2023-01-01
2 doris2 2000 223 1 2023-01-01
-- !2 --
1 doris 1000 123 1 2023-01-01
2 doris2 2600 223 1 2023-07-20
3 unknown 2500 \N 4321 2022-07-18
-- !3 --
1 doris 1000 123 1
2 doris2 2000 223 1
-- !3 --
1 doris 1000 123 1
2 doris2 2000 223 1
-- !4 --
1 doris 1000 123 1
2 doris2 2000 223 1
3 doris3 5000 34 345
-- !4 --
1 doris 1000 123 1
3 doris3 5000 34 345
-- !5 --
1 kevin 18 shenzhen 400 2023-07-01T12:00
-- !5 --
1 kevin 18 shenzhen 400 2023-07-01T12:00
-- !6 --
1 1 3 4
2 2 4 5
3 3 2 3
4 4 1 2
-- !6 --
1 2 3 4
2 3 4 5
3 4 2 3
4 5 1 2
-- !7 --
1 1 1 3 4
2 2 2 4 5
3 3 3 2 3
4 4 4 1 2
-- !7 --
1 1 1 3 4
2 2 2 4 5
3 3 3 2 3
4 4 4 1 2

View File

@ -0,0 +1,37 @@
-- This file is automatically generated. You should know what you did if you want to edit this
-- !tbl1 --
1 1 1 1.0 2000-01-01
2 2 2 2.0 2000-01-02
3 3 3 3.0 2000-01-03
-- !tbl2 --
1 10 10 10.0 2000-01-10
2 20 20 20.0 2000-01-20
3 30 30 30.0 2000-01-30
4 4 4 4.0 2000-01-04
5 5 5 5.0 2000-01-05
-- !tbl3 --
1
3
5
-- !select_result --
1 10 1000.0
3 30 3000.0
5 5 500.0
-- !complex_update --
1 10 1 1000.0 2000-01-01
2 2 2 2.0 2000-01-02
3 30 3 3000.0 2000-01-03
5 5 \N 500.0 \N
-- !select_result --
1 1
3 1
5 1
-- !complex_delete --
2 2 2 2.0 2000-01-02

View File

@ -0,0 +1,66 @@
-- This file is automatically generated. You should know what you did if you want to edit this
-- !1 --
1 doris 1000 123 1
2 doris2 2000 223 1
-- !1 --
1 doris 200 123 1
2 doris2 400 223 1
4 yixiu 400 \N 4321
-- !2 --
1 doris 1000 123 1 2023-01-01
2 doris2 2000 223 1 2023-01-01
-- !2 --
1 doris 1000 123 1 2023-01-01
2 doris2 2600 223 1 2023-07-20
3 unknown 2500 \N 4321 2022-07-18
-- !3 --
1 doris 1000 123 1
2 doris2 2000 223 1
-- !3 --
1 doris 1000 123 1
2 doris2 2000 223 1
-- !4 --
1 doris 1000 123 1
2 doris2 2000 223 1
3 doris3 5000 34 345
-- !4 --
1 doris 1000 123 1
3 doris3 5000 34 345
-- !5 --
1 kevin 18 shenzhen 400 2023-07-01T12:00
-- !5 --
1 kevin 18 shenzhen 400 2023-07-01T12:00
-- !6 --
1 1 3 4
2 2 4 5
3 3 2 3
4 4 1 2
-- !6 --
1 2 3 4
2 3 4 5
3 4 2 3
4 5 1 2
-- !7 --
1 1 1 3 4
2 2 2 4 5
3 3 3 2 3
4 4 4 1 2
-- !7 --
1 1 1 3 4
2 2 2 4 5
3 3 3 2 3
4 4 4 1 2

View File

@ -0,0 +1,37 @@
-- This file is automatically generated. You should know what you did if you want to edit this
-- !tbl1 --
1 1 1 1.0 2000-01-01
2 2 2 2.0 2000-01-02
3 3 3 3.0 2000-01-03
-- !tbl2 --
1 10 10 10.0 2000-01-10
2 20 20 20.0 2000-01-20
3 30 30 30.0 2000-01-30
4 4 4 4.0 2000-01-04
5 5 5 5.0 2000-01-05
-- !tbl3 --
1
3
5
-- !select_result --
1 10 1000.0
3 30 3000.0
5 5 500.0
-- !complex_update --
1 10 1 1000.0 2000-01-01
2 2 2 2.0 2000-01-02
3 30 3 3000.0 2000-01-03
5 5 \N 500.0 \N
-- !select_result --
1 1
3 1
5 1
-- !complex_delete --
2 2 2 2.0 2000-01-02

View File

@ -0,0 +1,227 @@
// 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("nereids_partial_update_native_insert_stmt", "p0") {
sql "set enable_nereids_dml=true;"
sql "set experimental_enable_nereids_planner=true;"
sql "set enable_fallback_to_original_planner=false;"
sql "sync;"
// sql 'set enable_fallback_to_original_planner=false'
def tableName = "nereids_partial_update_native_insert_stmt"
sql """ DROP TABLE IF EXISTS ${tableName} """
sql """
CREATE TABLE ${tableName} (
`id` int(11) NOT NULL COMMENT "用户 ID",
`name` varchar(65533) NOT NULL DEFAULT "yixiu" COMMENT "用户姓名",
`score` int(11) NOT NULL COMMENT "用户得分",
`test` int(11) NULL COMMENT "null test",
`dft` int(11) DEFAULT "4321")
UNIQUE KEY(`id`) DISTRIBUTED BY HASH(`id`) BUCKETS 1
PROPERTIES("replication_num" = "1", "enable_unique_key_merge_on_write" = "true")
"""
sql """insert into ${tableName} values(2, "doris2", 2000, 223, 1),(1, "doris", 1000, 123, 1)"""
qt_1 """ select * from ${tableName} order by id; """
sql "set enable_unique_key_partial_update=true;"
sql "set enable_insert_strict = false;"
sql "sync;"
// partial update using insert stmt in non-strict mode,
// existing rows should be updated and new rows should be inserted with unmentioned columns filled with default or null value
sql """insert into ${tableName}(id,score) values(2,400),(1,200),(4,400)"""
qt_1 """ select * from ${tableName} order by id; """
sql "set enable_unique_key_partial_update=false;"
sql "sync;"
sql """ DROP TABLE IF EXISTS ${tableName} """
def tableName2 = "nereids_partial_update_native_insert_stmt2"
sql """ DROP TABLE IF EXISTS ${tableName2} """
sql """
CREATE TABLE ${tableName2} (
`id` int(11) NOT NULL COMMENT "用户 ID",
`name` varchar(65533) DEFAULT "unknown" COMMENT "用户姓名",
`score` int(11) NOT NULL COMMENT "用户得分",
`test` int(11) NULL COMMENT "null test",
`dft` int(11) DEFAULT "4321",
`update_time` date NULL)
UNIQUE KEY(`id`) DISTRIBUTED BY HASH(`id`) BUCKETS 1
PROPERTIES(
"replication_num" = "1",
"enable_unique_key_merge_on_write" = "true",
"function_column.sequence_col" = "update_time"
)"""
sql """ insert into ${tableName2} values
(2, "doris2", 2000, 223, 1, '2023-01-01'),
(1, "doris", 1000, 123, 1, '2023-01-01');"""
qt_2 "select * from ${tableName2} order by id;"
sql "set enable_unique_key_partial_update=true;"
sql "set enable_insert_strict = false;"
sql "sync;"
// partial update with seq col
sql """ insert into ${tableName2}(id,score,update_time) values
(2,2500,"2023-07-19"),
(2,2600,"2023-07-20"),
(1,1300,"2022-07-19"),
(3,1500,"2022-07-20"),
(3,2500,"2022-07-18"); """
qt_2 "select * from ${tableName2} order by id;"
sql "set enable_unique_key_partial_update=false;"
sql "sync;"
sql """ DROP TABLE IF EXISTS ${tableName2}; """
def tableName3 = "nereids_partial_update_native_insert_stmt3"
sql """ DROP TABLE IF EXISTS ${tableName3}; """
sql """
CREATE TABLE ${tableName3} (
`id` int(11) NOT NULL COMMENT "用户 ID",
`name` varchar(65533) NOT NULL COMMENT "用户姓名",
`score` int(11) NOT NULL COMMENT "用户得分",
`test` int(11) NULL COMMENT "null test",
`dft` int(11) DEFAULT "4321")
UNIQUE KEY(`id`) DISTRIBUTED BY HASH(`id`) BUCKETS 1
PROPERTIES("replication_num" = "1", "enable_unique_key_merge_on_write" = "true")
"""
sql """insert into ${tableName3} values(2, "doris2", 2000, 223, 1),(1, "doris", 1000, 123, 1);"""
qt_3 """ select * from ${tableName3} order by id; """
sql "set enable_unique_key_partial_update=true;"
sql "sync;"
// in partial update, the unmentioned columns should have default values or be nullable
// but field `name` is not nullable and doesn't have default value
test {
sql """insert into ${tableName3}(id,score) values(2,400),(1,200),(4,400)"""
exception "INTERNAL_ERROR"
}
sql "set enable_unique_key_partial_update=false;"
sql "sync;"
qt_3 """ select * from ${tableName3} order by id; """
sql """ DROP TABLE IF EXISTS ${tableName3} """
def tableName4 = "nereids_partial_update_native_insert_stmt4"
sql """ DROP TABLE IF EXISTS ${tableName4} """
sql """
CREATE TABLE ${tableName4} (
`id` int(11) NOT NULL COMMENT "用户 ID",
`name` varchar(65533) NOT NULL DEFAULT "yixiu" COMMENT "用户姓名",
`score` int(11) NULL COMMENT "用户得分",
`test` int(11) NULL COMMENT "null test",
`dft` int(11) DEFAULT "4321")
UNIQUE KEY(`id`) DISTRIBUTED BY HASH(`id`) BUCKETS 1
PROPERTIES("replication_num" = "1", "enable_unique_key_merge_on_write" = "true")
"""
sql """insert into ${tableName4} values(2, "doris2", 2000, 223, 1),(1, "doris", 1000, 123, 1),(3,"doris3",5000,34,345);"""
qt_4 """ select * from ${tableName4} order by id; """
sql "set enable_unique_key_partial_update=true;"
sql "set enable_insert_strict = false;"
sql "sync;"
// partial update with delete sign
sql "insert into ${tableName4}(id,__DORIS_DELETE_SIGN__) values(2,1);"
qt_4 """ select * from ${tableName4} order by id; """
sql "set enable_unique_key_partial_update=false;"
sql "sync;"
sql """ DROP TABLE IF EXISTS ${tableName4} """
def tableName5 = "nereids_partial_update_native_insert_stmt5"
sql """ DROP TABLE IF EXISTS ${tableName5} """
sql """
CREATE TABLE ${tableName5} (
`id` int(11) NULL,
`name` varchar(10) NULL,
`age` int(11) NULL DEFAULT "20",
`city` varchar(10) NOT NULL DEFAULT "beijing",
`balance` decimalv3(9, 0) NULL,
`last_access_time` datetime NULL
) ENGINE = OLAP UNIQUE KEY(`id`)
COMMENT 'OLAP' DISTRIBUTED BY HASH(`id`)
BUCKETS AUTO PROPERTIES (
"replication_allocation" = "tag.location.default: 1",
"storage_format" = "V2",
"enable_unique_key_merge_on_write" = "true",
"light_schema_change" = "true",
"disable_auto_compaction" = "false",
"enable_single_replica_compaction" = "false"
);
"""
sql """insert into ${tableName5} values(1,"kevin",18,"shenzhen",400,"2023-07-01 12:00:00");"""
qt_5 """select * from ${tableName5} order by id;"""
sql "set enable_insert_strict = true;"
sql "set enable_unique_key_partial_update=true;"
sql "sync;"
// partial update using insert stmt in strict mode, the max_filter_ratio is always 0
test {
sql """ insert into ${tableName5}(id,balance,last_access_time) values(1,500,"2023-07-03 12:00:01"),(3,23,"2023-07-03 12:00:02"),(18,9999999,"2023-07-03 12:00:03"); """
exception "Insert has filtered data in strict mode"
}
qt_5 """select * from ${tableName5} order by id;"""
sql "set enable_unique_key_partial_update=false;"
sql "set enable_insert_strict = false;"
sql "sync;"
sql """ DROP TABLE IF EXISTS ${tableName5}; """
def tableName6 = "nereids_partial_update_native_insert_stmt6"
sql """ DROP TABLE IF EXISTS ${tableName6} """
sql """create table ${tableName6} (
k int null,
v int null,
v2 int null,
v3 int null
) unique key (k) distributed by hash(k) buckets 1
properties("replication_num" = "1",
"enable_unique_key_merge_on_write"="true",
"disable_auto_compaction"="true"); """
sql "insert into ${tableName6} values(1,1,3,4),(2,2,4,5),(3,3,2,3),(4,4,1,2);"
qt_6 "select * from ${tableName6} order by k;"
sql "set enable_unique_key_partial_update=true;"
sql "sync;"
sql "insert into ${tableName6}(k,v) select v2,v3 from ${tableName6};"
qt_6 "select * from ${tableName6} order by k;"
sql "set enable_unique_key_partial_update=false;"
sql "set enable_insert_strict = false;"
sql "sync;"
sql """ DROP TABLE IF EXISTS ${tableName6}; """
def tableName7 = "nereids_partial_update_native_insert_stmt7"
sql """ DROP TABLE IF EXISTS ${tableName7} """
sql """create table ${tableName7} (
k1 int null,
k2 int null,
k3 int null,
v1 int null,
v2 int null
) unique key (k1,k2,k3) distributed by hash(k1,k2) buckets 4
properties("replication_num" = "1",
"enable_unique_key_merge_on_write"="true",
"disable_auto_compaction"="true"); """
sql "insert into ${tableName7} values(1,1,1,3,4),(2,2,2,4,5),(3,3,3,2,3),(4,4,4,1,2);"
qt_7 "select * from ${tableName7} order by k1;"
sql "set enable_unique_key_partial_update=true;"
sql "sync;"
test {
sql "insert into ${tableName7}(k1,k2,v2) select k2,k3,v1 from ${tableName7};"
exception "Partial update should include all key columns, missing: k3"
}
qt_7 "select * from ${tableName7} order by k1;"
sql """ DROP TABLE IF EXISTS ${tableName7}; """
sql "set enable_unique_key_partial_update=false;"
sql "set enable_insert_strict = false;"
sql "set enable_fallback_to_original_planner=true;"
sql "sync;"
}

View File

@ -0,0 +1,117 @@
// 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("nereids_partial_update_native_insert_stmt_complex", "p0") {
sql "set enable_nereids_dml=true;"
sql "set experimental_enable_nereids_planner=true;"
sql "set enable_fallback_to_original_planner=false;"
sql "sync;"
// test complex partial update
def tbName1 = "nereids_partial_update_native_insert_stmt_complex1"
def tbName2 = "nereids_partial_update_native_insert_stmt_complex2"
def tbName3 = "nereids_partial_update_native_insert_stmt_complex3"
sql "DROP TABLE IF EXISTS ${tbName1}"
sql "DROP TABLE IF EXISTS ${tbName2}"
sql "DROP TABLE IF EXISTS ${tbName3}"
sql """create table ${tbName1} (
id int,
c1 bigint,
c2 string,
c3 double,
c4 date) unique key (id) distributed by hash(id)
properties('replication_num'='1', 'enable_unique_key_merge_on_write' = 'true');"""
sql """create table ${tbName2} (
id int,
c1 bigint,
c2 string,
c3 double,
c4 date) unique key (id) distributed by hash(id)
properties('replication_num'='1', 'enable_unique_key_merge_on_write' = 'true');"""
sql """create table ${tbName3} (id int) distributed by hash (id) properties('replication_num'='1');"""
sql "set enable_unique_key_partial_update=false;"
sql "sync;"
sql """insert into ${tbName1} values
(1, 1, '1', 1.0, '2000-01-01'),
(2, 2, '2', 2.0, '2000-01-02'),
(3, 3, '3', 3.0, '2000-01-03');"""
sql """insert into ${tbName2} values
(1, 10, '10', 10.0, '2000-01-10'),
(2, 20, '20', 20.0, '2000-01-20'),
(3, 30, '30', 30.0, '2000-01-30'),
(4, 4, '4', 4.0, '2000-01-04'),
(5, 5, '5', 5.0, '2000-01-05');"""
sql """insert into ${tbName3} values(1), (3), (5);"""
qt_tbl1 "select * from ${tbName1} order by id;"
qt_tbl2 "select * from ${tbName2} order by id;"
qt_tbl3 "select * from ${tbName3} order by id;"
qt_select_result """select ${tbName2}.id, ${tbName2}.c1, ${tbName2}.c3 * 100
from ${tbName2} inner join ${tbName3} on ${tbName2}.id = ${tbName3}.id order by ${tbName2}.id;"""
sql "set enable_unique_key_partial_update=true;"
sql "set enable_insert_strict = false;"
sql "sync;"
sql """insert into ${tbName1}(id, c1, c3)
select ${tbName2}.id, ${tbName2}.c1, ${tbName2}.c3 * 100
from ${tbName2} inner join ${tbName3} on ${tbName2}.id = ${tbName3}.id; """
qt_complex_update """select * from ${tbName1} order by id;"""
sql "truncate table ${tbName1};"
sql "truncate table ${tbName2};"
sql "truncate table ${tbName3};"
sql "set enable_unique_key_partial_update=false;"
sql "sync;"
sql """insert into ${tbName1} values
(1, 1, '1', 1.0, '2000-01-01'),
(2, 2, '2', 2.0, '2000-01-02'),
(3, 3, '3', 3.0, '2000-01-03');"""
sql """insert into ${tbName2} values
(1, 10, '10', 10.0, '2000-01-10'),
(2, 20, '20', 20.0, '2000-01-20'),
(3, 30, '30', 30.0, '2000-01-30'),
(4, 4, '4', 4.0, '2000-01-04'),
(5, 5, '5', 5.0, '2000-01-05');"""
sql """insert into ${tbName3} values(1), (3), (5);"""
qt_select_result "select ${tbName2}.id,1 from ${tbName2} inner join ${tbName3} on ${tbName2}.id = ${tbName3}.id order by ${tbName2}.id;"
sql "set enable_unique_key_partial_update=true;"
sql "set enable_insert_strict = false;"
sql "sync;"
sql """ insert into ${tbName1}(id, __DORIS_DELETE_SIGN__)
select ${tbName2}.id,1 from ${tbName2} inner join ${tbName3} on ${tbName2}.id = ${tbName3}.id;"""
qt_complex_delete """select * from ${tbName1} order by id;"""
sql "DROP TABLE IF EXISTS ${tbName1}"
sql "DROP TABLE IF EXISTS ${tbName2}"
sql "DROP TABLE IF EXISTS ${tbName3}"
sql "set enable_unique_key_partial_update=false;"
sql "set enable_insert_strict = false;"
sql "set enable_fallback_to_original_planner=true;"
sql "sync;"
}

View File

@ -0,0 +1,227 @@
// 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_partial_update_native_insert_stmt", "p0") {
sql "set enable_nereids_dml=false;"
sql "set experimental_enable_nereids_planner=false;"
sql "set enable_fallback_to_original_planner=true;"
sql "sync;"
// sql 'set enable_fallback_to_original_planner=false'
def tableName = "test_partial_update_native_insert_stmt"
sql """ DROP TABLE IF EXISTS ${tableName} """
sql """
CREATE TABLE ${tableName} (
`id` int(11) NOT NULL COMMENT "用户 ID",
`name` varchar(65533) NOT NULL DEFAULT "yixiu" COMMENT "用户姓名",
`score` int(11) NOT NULL COMMENT "用户得分",
`test` int(11) NULL COMMENT "null test",
`dft` int(11) DEFAULT "4321")
UNIQUE KEY(`id`) DISTRIBUTED BY HASH(`id`) BUCKETS 1
PROPERTIES("replication_num" = "1", "enable_unique_key_merge_on_write" = "true")
"""
sql """insert into ${tableName} values(2, "doris2", 2000, 223, 1),(1, "doris", 1000, 123, 1)"""
qt_1 """ select * from ${tableName} order by id; """
sql "set enable_unique_key_partial_update=true;"
sql "set enable_insert_strict = false;"
sql "sync;"
// partial update using insert stmt in non-strict mode,
// existing rows should be updated and new rows should be inserted with unmentioned columns filled with default or null value
sql """insert into ${tableName}(id,score) values(2,400),(1,200),(4,400)"""
qt_1 """ select * from ${tableName} order by id; """
sql "set enable_unique_key_partial_update=false;"
sql "sync;"
sql """ DROP TABLE IF EXISTS ${tableName} """
def tableName2 = "test_partial_update_native_insert_stmt2"
sql """ DROP TABLE IF EXISTS ${tableName2} """
sql """
CREATE TABLE ${tableName2} (
`id` int(11) NOT NULL COMMENT "用户 ID",
`name` varchar(65533) DEFAULT "unknown" COMMENT "用户姓名",
`score` int(11) NOT NULL COMMENT "用户得分",
`test` int(11) NULL COMMENT "null test",
`dft` int(11) DEFAULT "4321",
`update_time` date NULL)
UNIQUE KEY(`id`) DISTRIBUTED BY HASH(`id`) BUCKETS 1
PROPERTIES(
"replication_num" = "1",
"enable_unique_key_merge_on_write" = "true",
"function_column.sequence_col" = "update_time"
)"""
sql """ insert into ${tableName2} values
(2, "doris2", 2000, 223, 1, '2023-01-01'),
(1, "doris", 1000, 123, 1, '2023-01-01');"""
qt_2 "select * from ${tableName2} order by id;"
sql "set enable_unique_key_partial_update=true;"
sql "set enable_insert_strict = false;"
sql "sync;"
// partial update with seq col
sql """ insert into ${tableName2}(id,score,update_time) values
(2,2500,"2023-07-19"),
(2,2600,"2023-07-20"),
(1,1300,"2022-07-19"),
(3,1500,"2022-07-20"),
(3,2500,"2022-07-18"); """
qt_2 "select * from ${tableName2} order by id;"
sql "set enable_unique_key_partial_update=false;"
sql "sync;"
sql """ DROP TABLE IF EXISTS ${tableName2}; """
def tableName3 = "test_partial_update_native_insert_stmt3"
sql """ DROP TABLE IF EXISTS ${tableName3}; """
sql """
CREATE TABLE ${tableName3} (
`id` int(11) NOT NULL COMMENT "用户 ID",
`name` varchar(65533) NOT NULL COMMENT "用户姓名",
`score` int(11) NOT NULL COMMENT "用户得分",
`test` int(11) NULL COMMENT "null test",
`dft` int(11) DEFAULT "4321")
UNIQUE KEY(`id`) DISTRIBUTED BY HASH(`id`) BUCKETS 1
PROPERTIES("replication_num" = "1", "enable_unique_key_merge_on_write" = "true")
"""
sql """insert into ${tableName3} values(2, "doris2", 2000, 223, 1),(1, "doris", 1000, 123, 1);"""
qt_3 """ select * from ${tableName3} order by id; """
sql "set enable_unique_key_partial_update=true;"
sql "sync;"
// in partial update, the unmentioned columns should have default values or be nullable
// but field `name` is not nullable and doesn't have default value
test {
sql """insert into ${tableName3}(id,score) values(2,400),(1,200),(4,400)"""
exception "INTERNAL_ERROR"
}
sql "set enable_unique_key_partial_update=false;"
sql "sync;"
qt_3 """ select * from ${tableName3} order by id; """
sql """ DROP TABLE IF EXISTS ${tableName3} """
def tableName4 = "test_partial_update_native_insert_stmt4"
sql """ DROP TABLE IF EXISTS ${tableName4} """
sql """
CREATE TABLE ${tableName4} (
`id` int(11) NOT NULL COMMENT "用户 ID",
`name` varchar(65533) NOT NULL DEFAULT "yixiu" COMMENT "用户姓名",
`score` int(11) NULL COMMENT "用户得分",
`test` int(11) NULL COMMENT "null test",
`dft` int(11) DEFAULT "4321")
UNIQUE KEY(`id`) DISTRIBUTED BY HASH(`id`) BUCKETS 1
PROPERTIES("replication_num" = "1", "enable_unique_key_merge_on_write" = "true")
"""
sql """insert into ${tableName4} values(2, "doris2", 2000, 223, 1),(1, "doris", 1000, 123, 1),(3,"doris3",5000,34,345);"""
qt_4 """ select * from ${tableName4} order by id; """
sql "set enable_unique_key_partial_update=true;"
sql "set enable_insert_strict = false;"
sql "sync;"
// partial update with delete sign
sql "insert into ${tableName4}(id,__DORIS_DELETE_SIGN__) values(2,1);"
qt_4 """ select * from ${tableName4} order by id; """
sql "set enable_unique_key_partial_update=false;"
sql "sync;"
sql """ DROP TABLE IF EXISTS ${tableName4} """
def tableName5 = "test_partial_update_native_insert_stmt5"
sql """ DROP TABLE IF EXISTS ${tableName5} """
sql """
CREATE TABLE ${tableName5} (
`id` int(11) NULL,
`name` varchar(10) NULL,
`age` int(11) NULL DEFAULT "20",
`city` varchar(10) NOT NULL DEFAULT "beijing",
`balance` decimalv3(9, 0) NULL,
`last_access_time` datetime NULL
) ENGINE = OLAP UNIQUE KEY(`id`)
COMMENT 'OLAP' DISTRIBUTED BY HASH(`id`)
BUCKETS AUTO PROPERTIES (
"replication_allocation" = "tag.location.default: 1",
"storage_format" = "V2",
"enable_unique_key_merge_on_write" = "true",
"light_schema_change" = "true",
"disable_auto_compaction" = "false",
"enable_single_replica_compaction" = "false"
);
"""
sql """insert into ${tableName5} values(1,"kevin",18,"shenzhen",400,"2023-07-01 12:00:00");"""
qt_5 """select * from ${tableName5} order by id;"""
sql "set enable_insert_strict = true;"
sql "set enable_unique_key_partial_update=true;"
sql "sync;"
// partial update using insert stmt in strict mode, the max_filter_ratio is always 0
test {
sql """ insert into ${tableName5}(id,balance,last_access_time) values(1,500,"2023-07-03 12:00:01"),(3,23,"2023-07-03 12:00:02"),(18,9999999,"2023-07-03 12:00:03"); """
exception "Insert has filtered data in strict mode"
}
qt_5 """select * from ${tableName5} order by id;"""
sql "set enable_unique_key_partial_update=false;"
sql "set enable_insert_strict = false;"
sql "sync;"
sql """ DROP TABLE IF EXISTS ${tableName5}; """
def tableName6 = "test_partial_update_native_insert_stmt6"
sql """ DROP TABLE IF EXISTS ${tableName6} """
sql """create table ${tableName6} (
k int null,
v int null,
v2 int null,
v3 int null
) unique key (k) distributed by hash(k) buckets 1
properties("replication_num" = "1",
"enable_unique_key_merge_on_write"="true",
"disable_auto_compaction"="true"); """
sql "insert into ${tableName6} values(1,1,3,4),(2,2,4,5),(3,3,2,3),(4,4,1,2);"
qt_6 "select * from ${tableName6} order by k;"
sql "set enable_unique_key_partial_update=true;"
sql "sync;"
sql "insert into ${tableName6}(k,v) select v2,v3 from ${tableName6};"
qt_6 "select * from ${tableName6} order by k;"
sql "set enable_unique_key_partial_update=false;"
sql "set enable_insert_strict = false;"
sql "sync;"
sql """ DROP TABLE IF EXISTS ${tableName6}; """
def tableName7 = "test_partial_update_native_insert_stmt7"
sql """ DROP TABLE IF EXISTS ${tableName7} """
sql """create table ${tableName7} (
k1 int null,
k2 int null,
k3 int null,
v1 int null,
v2 int null
) unique key (k1,k2,k3) distributed by hash(k1,k2) buckets 4
properties("replication_num" = "1",
"enable_unique_key_merge_on_write"="true",
"disable_auto_compaction"="true"); """
sql "insert into ${tableName7} values(1,1,1,3,4),(2,2,2,4,5),(3,3,3,2,3),(4,4,4,1,2);"
qt_7 "select * from ${tableName7} order by k1;"
sql "set enable_unique_key_partial_update=true;"
sql "sync;"
test {
sql "insert into ${tableName7}(k1,k2,v2) select k2,k3,v1 from ${tableName7};"
exception "Partial update should include all key columns, missing: k3"
}
qt_7 "select * from ${tableName7} order by k1;"
sql """ DROP TABLE IF EXISTS ${tableName7}; """
sql "set enable_unique_key_partial_update=false;"
sql "set enable_insert_strict = false;"
sql "set experimental_enable_nereids_planner=true;"
sql "sync;"
}

View File

@ -0,0 +1,117 @@
// 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_partial_update_native_insert_stmt_complex", "p0") {
sql "set enable_nereids_dml=false;"
sql "set experimental_enable_nereids_planner=false;"
sql "set enable_fallback_to_original_planner=true;"
sql "sync;"
// test complex partial update
def tbName1 = "test_partial_update_native_insert_stmt_complex1"
def tbName2 = "test_partial_update_native_insert_stmt_complex2"
def tbName3 = "test_partial_update_native_insert_stmt_complex3"
sql "DROP TABLE IF EXISTS ${tbName1}"
sql "DROP TABLE IF EXISTS ${tbName2}"
sql "DROP TABLE IF EXISTS ${tbName3}"
sql """create table ${tbName1} (
id int,
c1 bigint,
c2 string,
c3 double,
c4 date) unique key (id) distributed by hash(id)
properties('replication_num'='1', 'enable_unique_key_merge_on_write' = 'true');"""
sql """create table ${tbName2} (
id int,
c1 bigint,
c2 string,
c3 double,
c4 date) unique key (id) distributed by hash(id)
properties('replication_num'='1', 'enable_unique_key_merge_on_write' = 'true');"""
sql """create table ${tbName3} (id int) distributed by hash (id) properties('replication_num'='1');"""
sql "set enable_unique_key_partial_update=false;"
sql "sync;"
sql """insert into ${tbName1} values
(1, 1, '1', 1.0, '2000-01-01'),
(2, 2, '2', 2.0, '2000-01-02'),
(3, 3, '3', 3.0, '2000-01-03');"""
sql """insert into ${tbName2} values
(1, 10, '10', 10.0, '2000-01-10'),
(2, 20, '20', 20.0, '2000-01-20'),
(3, 30, '30', 30.0, '2000-01-30'),
(4, 4, '4', 4.0, '2000-01-04'),
(5, 5, '5', 5.0, '2000-01-05');"""
sql """insert into ${tbName3} values(1), (3), (5);"""
qt_tbl1 "select * from ${tbName1} order by id;"
qt_tbl2 "select * from ${tbName2} order by id;"
qt_tbl3 "select * from ${tbName3} order by id;"
qt_select_result """select ${tbName2}.id, ${tbName2}.c1, ${tbName2}.c3 * 100
from ${tbName2} inner join ${tbName3} on ${tbName2}.id = ${tbName3}.id order by ${tbName2}.id;"""
sql "set enable_unique_key_partial_update=true;"
sql "set enable_insert_strict = false;"
sql "sync;"
sql """insert into ${tbName1}(id, c1, c3)
select ${tbName2}.id, ${tbName2}.c1, ${tbName2}.c3 * 100
from ${tbName2} inner join ${tbName3} on ${tbName2}.id = ${tbName3}.id; """
qt_complex_update """select * from ${tbName1} order by id;"""
sql "truncate table ${tbName1};"
sql "truncate table ${tbName2};"
sql "truncate table ${tbName3};"
sql "set enable_unique_key_partial_update=false;"
sql "sync;"
sql """insert into ${tbName1} values
(1, 1, '1', 1.0, '2000-01-01'),
(2, 2, '2', 2.0, '2000-01-02'),
(3, 3, '3', 3.0, '2000-01-03');"""
sql """insert into ${tbName2} values
(1, 10, '10', 10.0, '2000-01-10'),
(2, 20, '20', 20.0, '2000-01-20'),
(3, 30, '30', 30.0, '2000-01-30'),
(4, 4, '4', 4.0, '2000-01-04'),
(5, 5, '5', 5.0, '2000-01-05');"""
sql """insert into ${tbName3} values(1), (3), (5);"""
qt_select_result "select ${tbName2}.id,1 from ${tbName2} inner join ${tbName3} on ${tbName2}.id = ${tbName3}.id order by ${tbName2}.id;"
sql "set enable_unique_key_partial_update=true;"
sql "set enable_insert_strict = false;"
sql "sync;"
sql """ insert into ${tbName1}(id, __DORIS_DELETE_SIGN__)
select ${tbName2}.id,1 from ${tbName2} inner join ${tbName3} on ${tbName2}.id = ${tbName3}.id;"""
qt_complex_delete """select * from ${tbName1} order by id;"""
sql "DROP TABLE IF EXISTS ${tbName1}"
sql "DROP TABLE IF EXISTS ${tbName2}"
sql "DROP TABLE IF EXISTS ${tbName3}"
sql "set enable_unique_key_partial_update=false;"
sql "set enable_insert_strict = false;"
sql "set experimental_enable_nereids_planner=true;"
sql "sync;"
}