|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | + |
| 19 | +package org.apache.flink.table.planner.plan.nodes.exec.batch; |
| 20 | + |
| 21 | +import org.apache.flink.api.dag.Transformation; |
| 22 | +import org.apache.flink.configuration.ReadableConfig; |
| 23 | +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; |
| 24 | +import org.apache.flink.table.api.TableException; |
| 25 | +import org.apache.flink.table.data.RowData; |
| 26 | +import org.apache.flink.table.planner.adaptive.AdaptiveJoinOperatorGenerator; |
| 27 | +import org.apache.flink.table.planner.delegation.PlannerBase; |
| 28 | +import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge; |
| 29 | +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase; |
| 30 | +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeConfig; |
| 31 | +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; |
| 32 | +import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; |
| 33 | +import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator; |
| 34 | +import org.apache.flink.table.planner.plan.nodes.exec.spec.JoinSpec; |
| 35 | +import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil; |
| 36 | +import org.apache.flink.table.planner.plan.utils.JoinUtil; |
| 37 | +import org.apache.flink.table.planner.plan.utils.OperatorType; |
| 38 | +import org.apache.flink.table.runtime.generated.GeneratedJoinCondition; |
| 39 | +import org.apache.flink.table.runtime.operators.join.adaptive.AdaptiveJoin; |
| 40 | +import org.apache.flink.table.runtime.operators.join.adaptive.AdaptiveJoinOperatorFactory; |
| 41 | +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; |
| 42 | +import org.apache.flink.table.types.logical.RowType; |
| 43 | +import org.apache.flink.util.InstantiationUtil; |
| 44 | + |
| 45 | +import java.io.IOException; |
| 46 | +import java.util.List; |
| 47 | + |
| 48 | +/** {@link BatchExecNode} for adaptive join. */ |
| 49 | +public class BatchExecAdaptiveJoin extends ExecNodeBase<RowData> |
| 50 | + implements BatchExecNode<RowData>, SingleTransformationTranslator<RowData> { |
| 51 | + |
| 52 | + private final JoinSpec joinSpec; |
| 53 | + private final boolean leftIsBuild; |
| 54 | + private final int estimatedLeftAvgRowSize; |
| 55 | + private final int estimatedRightAvgRowSize; |
| 56 | + private final long estimatedLeftRowCount; |
| 57 | + private final long estimatedRightRowCount; |
| 58 | + private final boolean tryDistinctBuildRow; |
| 59 | + private final String description; |
| 60 | + private final OperatorType originalJoin; |
| 61 | + |
| 62 | + public BatchExecAdaptiveJoin( |
| 63 | + ReadableConfig tableConfig, |
| 64 | + JoinSpec joinSpec, |
| 65 | + int estimatedLeftAvgRowSize, |
| 66 | + int estimatedRightAvgRowSize, |
| 67 | + long estimatedLeftRowCount, |
| 68 | + long estimatedRightRowCount, |
| 69 | + boolean leftIsBuild, |
| 70 | + boolean tryDistinctBuildRow, |
| 71 | + List<InputProperty> inputProperties, |
| 72 | + RowType outputType, |
| 73 | + String description, |
| 74 | + OperatorType originalJoin) { |
| 75 | + super( |
| 76 | + ExecNodeContext.newNodeId(), |
| 77 | + ExecNodeContext.newContext(BatchExecAdaptiveJoin.class), |
| 78 | + ExecNodeContext.newPersistedConfig(BatchExecAdaptiveJoin.class, tableConfig), |
| 79 | + inputProperties, |
| 80 | + outputType, |
| 81 | + description); |
| 82 | + this.joinSpec = joinSpec; |
| 83 | + this.estimatedLeftAvgRowSize = estimatedLeftAvgRowSize; |
| 84 | + this.estimatedRightAvgRowSize = estimatedRightAvgRowSize; |
| 85 | + this.estimatedLeftRowCount = estimatedLeftRowCount; |
| 86 | + this.estimatedRightRowCount = estimatedRightRowCount; |
| 87 | + this.leftIsBuild = leftIsBuild; |
| 88 | + this.tryDistinctBuildRow = tryDistinctBuildRow; |
| 89 | + this.description = description; |
| 90 | + this.originalJoin = originalJoin; |
| 91 | + } |
| 92 | + |
| 93 | + @Override |
| 94 | + @SuppressWarnings("unchecked") |
| 95 | + protected Transformation<RowData> translateToPlanInternal( |
| 96 | + PlannerBase planner, ExecNodeConfig config) { |
| 97 | + ExecEdge leftInputEdge = getInputEdges().get(0); |
| 98 | + ExecEdge rightInputEdge = getInputEdges().get(1); |
| 99 | + |
| 100 | + Transformation<RowData> leftInputTransform = |
| 101 | + (Transformation<RowData>) leftInputEdge.translateToPlan(planner); |
| 102 | + Transformation<RowData> rightInputTransform = |
| 103 | + (Transformation<RowData>) rightInputEdge.translateToPlan(planner); |
| 104 | + // get input types |
| 105 | + RowType leftType = (RowType) leftInputEdge.getOutputType(); |
| 106 | + RowType rightType = (RowType) rightInputEdge.getOutputType(); |
| 107 | + long managedMemory = JoinUtil.getManagedMemory(joinSpec.getJoinType(), config); |
| 108 | + GeneratedJoinCondition condFunc = |
| 109 | + JoinUtil.generateConditionFunction( |
| 110 | + config, |
| 111 | + planner.getFlinkContext().getClassLoader(), |
| 112 | + joinSpec.getNonEquiCondition().orElse(null), |
| 113 | + leftType, |
| 114 | + rightType); |
| 115 | + |
| 116 | + AdaptiveJoinOperatorGenerator adaptiveJoin = |
| 117 | + new AdaptiveJoinOperatorGenerator( |
| 118 | + joinSpec.getLeftKeys(), |
| 119 | + joinSpec.getRightKeys(), |
| 120 | + joinSpec.getJoinType(), |
| 121 | + joinSpec.getFilterNulls(), |
| 122 | + leftType, |
| 123 | + rightType, |
| 124 | + condFunc, |
| 125 | + estimatedLeftAvgRowSize, |
| 126 | + estimatedRightAvgRowSize, |
| 127 | + estimatedLeftRowCount, |
| 128 | + estimatedRightRowCount, |
| 129 | + tryDistinctBuildRow, |
| 130 | + managedMemory, |
| 131 | + leftIsBuild, |
| 132 | + originalJoin); |
| 133 | + |
| 134 | + return ExecNodeUtil.createTwoInputTransformation( |
| 135 | + leftInputTransform, |
| 136 | + rightInputTransform, |
| 137 | + createTransformationName(config), |
| 138 | + createTransformationDescription(config), |
| 139 | + getAdaptiveJoinOperatorFactory(adaptiveJoin), |
| 140 | + InternalTypeInfo.of(getOutputType()), |
| 141 | + // Given that the probe side might be decided at runtime, we choose the larger |
| 142 | + // parallelism here. |
| 143 | + Math.max(leftInputTransform.getParallelism(), rightInputTransform.getParallelism()), |
| 144 | + managedMemory, |
| 145 | + false); |
| 146 | + } |
| 147 | + |
| 148 | + private StreamOperatorFactory<RowData> getAdaptiveJoinOperatorFactory( |
| 149 | + AdaptiveJoin adaptiveJoin) { |
| 150 | + try { |
| 151 | + byte[] adaptiveJoinSerialized = InstantiationUtil.serializeObject(adaptiveJoin); |
| 152 | + return new AdaptiveJoinOperatorFactory<>(adaptiveJoinSerialized); |
| 153 | + } catch (IOException e) { |
| 154 | + throw new TableException("The adaptive join operator failed to serialize.", e); |
| 155 | + } |
| 156 | + } |
| 157 | + |
| 158 | + @Override |
| 159 | + public String getDescription() { |
| 160 | + return "AdaptiveJoin(" |
| 161 | + + "originalJoin=[" |
| 162 | + + originalJoin |
| 163 | + + "], " |
| 164 | + + description.substring(description.indexOf('(') + 1); |
| 165 | + } |
| 166 | +} |
0 commit comments