Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

[FLINK-37543][table-planner] Support sink reuse in batch mode #26379

Open
wants to merge 2 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -105,6 +105,17 @@ public class OptimizerConfigOptions {
+ TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED.key()
+ " is true.");

@Documentation.TableOption(execMode = Documentation.ExecMode.BATCH_STREAMING)
public static final ConfigOption<Boolean> TABLE_OPTIMIZER_REUSE_SINK_ENABLED =
key("table.optimizer.reuse-sink-enabled")
.booleanType()
.defaultValue(false)
.withDescription(
"When it is true, the optimizer will try to find out duplicated table sinks and "
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: find out duplicated -> find duplicate

+ "reuse them. This works only when "
+ TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED.key()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am curious when would you have TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED set and not want to reuse duplicates? i.e. is it feasible for us to extend what TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED means rather than introduce a new flag?

+ " is true.");

@Documentation.TableOption(execMode = Documentation.ExecMode.BATCH_STREAMING)
public static final ConfigOption<Boolean> TABLE_OPTIMIZER_SOURCE_REPORT_STATISTICS_ENABLED =
key("table.optimizer.source.report-statistics-enabled")
Expand Down
Original file line number Diff line number Diff line change
@@ -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.
*/

package org.apache.flink.table.connector.sink.abilities;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.table.connector.sink.DynamicTableSink;

/**
* Interface for {@link DynamicTableSink}s that support target column writing.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: support -> supports

*
* <p>The planner will parse target columns from the DML clause and call {@link
* #applyTargetColumns(int[][])} to pass an array of column index paths to sink.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: to sink -> to the sink

*
* <p>The array indices are 0-based and support composite columns within (possibly nested)
* structures. This information comes from the column list of the DML clause, e.g., for a sink table
* t1 which schema is: {@code a STRING, b ROW < b1 INT, b2 STRING>, c BIGINT}
*
* <ul>
* <li>insert: 'insert into t1(a, b.b2) ...', the column list will be 'a, b.b2', and will provide
* {@code [[0], [1, 1]]}. The statement 'insert into t1 select ...' will not apply this
* ability.
* <li>update: 'update t1 set a=1, b.b1=2 where ...', the column list will be 'a, b.b1', and will
* provide {@code [[0], [1, 0]]}.
* </ul>
*
* <p>Note: Planner will not apply this ability for the delete statement because it has no column
* list.
*
* <p>A sink can use this information to perform target columns writing.
*
* <p>If this interface is implemented and {@link #applyTargetColumns(int[][])} returns true. The
* planner will use this information for plan optimization such as sink reuse.
*/
@PublicEvolving
public interface SupportsTargetColumnWriting {

/**
* Provides an array of column index paths related to user specified target column list.
*
* <p>See the documentation of {@link SupportsTargetColumnWriting} for more information.
*
* @param targetColumns column index paths
* @return true if the target columns are applied successfully, false otherwise.
*/
boolean applyTargetColumns(int[][] targetColumns);
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning;
import org.apache.flink.table.connector.sink.abilities.SupportsRowLevelDelete;
import org.apache.flink.table.connector.sink.abilities.SupportsRowLevelUpdate;
import org.apache.flink.table.connector.sink.abilities.SupportsTargetColumnWriting;
import org.apache.flink.table.connector.sink.abilities.SupportsWritingMetadata;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
Expand All @@ -63,6 +64,7 @@
import org.apache.flink.table.planner.plan.abilities.sink.RowLevelDeleteSpec;
import org.apache.flink.table.planner.plan.abilities.sink.RowLevelUpdateSpec;
import org.apache.flink.table.planner.plan.abilities.sink.SinkAbilitySpec;
import org.apache.flink.table.planner.plan.abilities.sink.TargetColumnWritingSpec;
import org.apache.flink.table.planner.plan.abilities.sink.WritingMetadataSpec;
import org.apache.flink.table.planner.plan.nodes.calcite.LogicalSink;
import org.apache.flink.table.planner.plan.schema.TableSourceTable;
Expand Down Expand Up @@ -279,7 +281,8 @@ private static RelNode convertSinkToRel(
isOverwrite,
sink,
contextResolvedTable.getResolvedTable(),
sinkAbilitySpecs);
sinkAbilitySpecs,
targetColumns);

// rewrite rel node for delete
if (isDelete) {
Expand Down Expand Up @@ -995,7 +998,8 @@ private static void prepareDynamicSink(
boolean isOverwrite,
DynamicTableSink sink,
ResolvedCatalogTable table,
List<SinkAbilitySpec> sinkAbilitySpecs) {
List<SinkAbilitySpec> sinkAbilitySpecs,
int[][] targetColumns) {
table.getDistribution()
.ifPresent(
distribution ->
Expand All @@ -1007,6 +1011,8 @@ private static void prepareDynamicSink(
validateAndApplyOverwrite(tableDebugName, isOverwrite, sink, sinkAbilitySpecs);

validateAndApplyMetadata(tableDebugName, sink, table.getResolvedSchema(), sinkAbilitySpecs);

validateAndApplyTargetColumns(sink, targetColumns, sinkAbilitySpecs);
}

/**
Expand Down Expand Up @@ -1285,6 +1291,20 @@ private static void validateAndApplyMetadata(
createConsumedType(schema, sink)));
}

private static void validateAndApplyTargetColumns(
DynamicTableSink sink, int[][] targetColumns, List<SinkAbilitySpec> sinkAbilitySpecs) {
if (targetColumns == null || targetColumns.length == 0) {
return;
}

if (!(sink instanceof SupportsTargetColumnWriting)) {
// Ignore target columns if the sink doesn't support it.
return;
}

sinkAbilitySpecs.add(new TargetColumnWritingSpec(targetColumns));
}

/**
* Returns the {@link DataType} that a sink should consume as the output from the runtime.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,8 @@
@JsonSubTypes.Type(value = PartitioningSpec.class),
@JsonSubTypes.Type(value = WritingMetadataSpec.class),
@JsonSubTypes.Type(value = RowLevelDeleteSpec.class),
@JsonSubTypes.Type(value = RowLevelUpdateSpec.class)
@JsonSubTypes.Type(value = RowLevelUpdateSpec.class),
@JsonSubTypes.Type(value = TargetColumnWritingSpec.class)
})
@Internal
public interface SinkAbilitySpec {
Expand Down
Original file line number Diff line number Diff line change
@@ -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.flink.table.planner.plan.abilities.sink;

import org.apache.flink.table.api.TableException;
import org.apache.flink.table.connector.sink.DynamicTableSink;
import org.apache.flink.table.connector.sink.abilities.SupportsTargetColumnWriting;

import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeName;

import java.util.Arrays;
import java.util.Objects;

/**
* A sub-class of {@link SinkAbilitySpec} that can not only serialize/deserialize the writing target
* column indices to/from JSON, but also can write the target columns for {@link
* SupportsTargetColumnWriting}.
*/
@JsonIgnoreProperties(ignoreUnknown = true)
@JsonTypeName("TargetColumnWriting")
public class TargetColumnWritingSpec implements SinkAbilitySpec {
public static final String FIELD_NAME_TARGET_COLUMNS = "targetColumns";

@JsonProperty(FIELD_NAME_TARGET_COLUMNS)
private final int[][] targetColumns;

@JsonCreator
public TargetColumnWritingSpec(@JsonProperty(FIELD_NAME_TARGET_COLUMNS) int[][] targetColumns) {
this.targetColumns = targetColumns;
}

@Override
public void apply(DynamicTableSink tableSink) {
if (tableSink instanceof SupportsTargetColumnWriting) {
((SupportsTargetColumnWriting) tableSink).applyTargetColumns(targetColumns);
} else {
throw new TableException(
String.format(
"%s does not support SupportsTargetColumnWriting.",
tableSink.getClass().getName()));
}
}

@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
TargetColumnWritingSpec that = (TargetColumnWritingSpec) o;
return Objects.deepEquals(targetColumns, that.targetColumns);
}

@Override
public int hashCode() {
return Arrays.deepHashCode(targetColumns);
}
}
Loading