1
0

[HUDI-1621] Gets the parallelism from context when init StreamWriteOperatorCoordinator (#2579)

This commit is contained in:
lamber-ken
2021-02-17 20:04:38 +08:00
committed by GitHub
parent 37972071ff
commit c4bbcb7f0e
5 changed files with 10 additions and 13 deletions

View File

@@ -398,20 +398,20 @@ public class StreamWriteOperatorCoordinator
public static class Provider implements OperatorCoordinator.Provider {
private final OperatorID operatorId;
private final Configuration conf;
private final int numTasks;
public Provider(OperatorID operatorId, Configuration conf, int numTasks) {
public Provider(OperatorID operatorId, Configuration conf) {
this.operatorId = operatorId;
this.conf = conf;
this.numTasks = numTasks;
}
@Override
public OperatorID getOperatorId() {
return this.operatorId;
}
@Override
public OperatorCoordinator create(Context context) {
return new StreamWriteOperatorCoordinator(this.conf, this.numTasks);
return new StreamWriteOperatorCoordinator(this.conf, context.currentParallelism());
}
}
}

View File

@@ -39,15 +39,12 @@ public class StreamWriteOperatorFactory<I>
private final StreamWriteOperator<I> operator;
private final Configuration conf;
private final int numTasks;
public StreamWriteOperatorFactory(
Configuration conf,
int numTasks) {
Configuration conf) {
super(new StreamWriteOperator<>(conf));
this.operator = (StreamWriteOperator<I>) getOperator();
this.conf = conf;
this.numTasks = numTasks;
}
@Override
@@ -65,7 +62,7 @@ public class StreamWriteOperatorFactory<I>
@Override
public OperatorCoordinator.Provider getCoordinatorProvider(String s, OperatorID operatorID) {
return new StreamWriteOperatorCoordinator.Provider(operatorID, this.conf, this.numTasks);
return new StreamWriteOperatorCoordinator.Provider(operatorID, this.conf);
}
@Override

View File

@@ -74,7 +74,7 @@ public class HoodieFlinkStreamerV2 {
Configuration conf = FlinkOptions.fromStreamerConfig(cfg);
int numWriteTask = conf.getInteger(FlinkOptions.WRITE_TASK_PARALLELISM);
StreamWriteOperatorFactory<HoodieRecord> operatorFactory =
new StreamWriteOperatorFactory<>(conf, numWriteTask);
new StreamWriteOperatorFactory<>(conf);
DataStream<Object> dataStream = env.addSource(new FlinkKafkaConsumer<>(
cfg.kafkaTopic,

View File

@@ -93,7 +93,7 @@ public class StreamWriteITCase extends TestLogger {
(RowType) AvroSchemaConverter.convertToDataType(StreamerUtil.getSourceSchema(conf))
.getLogicalType();
StreamWriteOperatorFactory<HoodieRecord> operatorFactory =
new StreamWriteOperatorFactory<>(conf, 4);
new StreamWriteOperatorFactory<>(conf);
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
rowType,

View File

@@ -15,7 +15,7 @@
# See the License for the specific language governing permissions and
# limitations under the License.
###
log4j.rootLogger=WARN, CONSOLE
log4j.rootLogger=INFO, CONSOLE
log4j.logger.org.apache=INFO
log4j.logger.org.apache.hudi=DEBUG
log4j.logger.org.apache.hadoop.hbase=ERROR
@@ -27,5 +27,5 @@ log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter
log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true
log4j.appender.CONSOLE.filter.a.LevelMin=WARN
log4j.appender.CONSOLE.filter.a.LevelMin=INFO
log4j.appender.CONSOLE.filter.a.LevelMax=FATAL