NOTE: If there are no inputs, returns an empty list, not null.
*/
+ @JsonIgnore
List The input concept is not corresponding to the execution edge, but rather to the {@link Input}.
*/
@Internal
+@JsonIgnoreProperties(ignoreUnknown = true)
public class InputProperty {
+ /** The input does not require any specific data distribution. */
+ public static final RequiredDistribution ANY_DISTRIBUTION =
+ new RequiredDistribution(DistributionType.ANY) {};
+
+ /**
+ * The input will read all records for each parallelism of the target node. All records appear
+ * in each parallelism.
+ */
+ public static final RequiredDistribution BROADCAST_DISTRIBUTION =
+ new RequiredDistribution(DistributionType.BROADCAST) {};
+
+ /** The input will read all records, and the parallelism of the target node must be 1. */
+ public static final RequiredDistribution SINGLETON_DISTRIBUTION =
+ new RequiredDistribution(DistributionType.SINGLETON) {};
+
+ /**
+ * Returns a place-holder required distribution.
+ *
+ * Currently {@link InputProperty} is only used for deadlock breakup and multi-input in batch
+ * mode, so for {@link ExecNode}s not affecting the algorithm we use this place-holder.
+ *
+ * We should fill out the detailed {@link InputProperty} for each sub-class of {@link
+ * ExecNode} in the future.
+ */
+ public static final RequiredDistribution UNKNOWN_DISTRIBUTION =
+ new RequiredDistribution(DistributionType.UNKNOWN) {};
+
public static final InputProperty DEFAULT = InputProperty.builder().build();
+ public static final String FIELD_NAME_REQUIRED_DISTRIBUTION = "requiredDistribution";
+ public static final String FIELD_NAME_DAM_BEHAVIOR = "damBehavior";
+ public static final String FIELD_NAME_PRIORITY = "priority";
+
/**
* The required input data distribution when the target {@link ExecNode} read data in from the
* corresponding input.
*/
+ @JsonProperty(FIELD_NAME_REQUIRED_DISTRIBUTION)
+ @JsonSerialize(using = RequiredDistributionJsonSerializer.class)
+ @JsonDeserialize(using = RequiredDistributionJsonDeserializer.class)
private final RequiredDistribution requiredDistribution;
/** How does the input record trigger the output behavior of the target {@link ExecNode}. */
+ @JsonProperty(FIELD_NAME_DAM_BEHAVIOR)
private final DamBehavior damBehavior;
/**
@@ -51,27 +97,54 @@ public class InputProperty {
* The smaller the integer, the higher the priority. Same integer indicates the same
* priority.
*/
+ @JsonProperty(FIELD_NAME_PRIORITY)
private final int priority;
- private InputProperty(
- RequiredDistribution requiredDistribution, DamBehavior damBehavior, int priority) {
- this.requiredDistribution = requiredDistribution;
- this.damBehavior = damBehavior;
+ @JsonCreator
+ public InputProperty(
+ @JsonProperty(FIELD_NAME_REQUIRED_DISTRIBUTION)
+ RequiredDistribution requiredDistribution,
+ @JsonProperty(FIELD_NAME_DAM_BEHAVIOR) DamBehavior damBehavior,
+ @JsonProperty(FIELD_NAME_PRIORITY) int priority) {
+ this.requiredDistribution = checkNotNull(requiredDistribution);
+ this.damBehavior = checkNotNull(damBehavior);
this.priority = priority;
}
+ @JsonIgnore
public RequiredDistribution getRequiredDistribution() {
return requiredDistribution;
}
+ @JsonIgnore
public DamBehavior getDamBehavior() {
return damBehavior;
}
+ @JsonIgnore
public int getPriority() {
return priority;
}
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ InputProperty inputProperty = (InputProperty) o;
+ return priority == inputProperty.priority
+ && requiredDistribution.equals(inputProperty.requiredDistribution)
+ && damBehavior == inputProperty.damBehavior;
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(requiredDistribution, damBehavior, priority);
+ }
+
@Override
public String toString() {
return "InputProperty{"
@@ -132,6 +205,23 @@ public DistributionType getType() {
return type;
}
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ RequiredDistribution that = (RequiredDistribution) o;
+ return type == that.type;
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(type);
+ }
+
@Override
public String toString() {
return type.name();
@@ -155,39 +245,34 @@ public int[] getKeys() {
return keys;
}
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ if (!super.equals(o)) {
+ return false;
+ }
+ HashDistribution that = (HashDistribution) o;
+ return Arrays.equals(keys, that.keys);
+ }
+
+ @Override
+ public int hashCode() {
+ int result = super.hashCode();
+ result = 31 * result + Arrays.hashCode(keys);
+ return result;
+ }
+
@Override
public String toString() {
return "HASH" + Arrays.toString(keys);
}
}
- /** The input does not require any specific data distribution. */
- public static final RequiredDistribution ANY_DISTRIBUTION =
- new RequiredDistribution(DistributionType.ANY) {};
-
- /**
- * The input will read all records for each parallelism of the target node. All records appear
- * in each parallelism.
- */
- public static final RequiredDistribution BROADCAST_DISTRIBUTION =
- new RequiredDistribution(DistributionType.BROADCAST) {};
-
- /** The input will read all records, and the parallelism of the target node must be 1. */
- public static final RequiredDistribution SINGLETON_DISTRIBUTION =
- new RequiredDistribution(DistributionType.SINGLETON) {};
-
- /**
- * Returns a place-holder required distribution.
- *
- * Currently {@link InputProperty} is only used for deadlock breakup and multi-input in batch
- * mode, so for {@link ExecNode}s not affecting the algorithm we use this place-holder.
- *
- * We should fill out the detailed {@link InputProperty} for each sub-class of {@link
- * ExecNode} in the future.
- */
- public static final RequiredDistribution UNKNOWN_DISTRIBUTION =
- new RequiredDistribution(DistributionType.UNKNOWN) {};
-
/**
* The input will read the records whose keys hash to a particular hash value.
*
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecTableSourceScan.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecTableSourceScan.java
index 7b49c2814ca00..d34947f992769 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecTableSourceScan.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecTableSourceScan.java
@@ -26,6 +26,7 @@
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSourceSpec;
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.types.logical.RowType;
@@ -37,8 +38,8 @@ public class BatchExecTableSourceScan extends CommonExecTableSourceScan
implements BatchExecNode