Skip to content
Closed
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 @@ -120,8 +120,8 @@ public static DataStreamSink<Object> bulkInsert(Configuration conf, RowType rowT
conf.getInteger(FlinkOptions.WRITE_SORT_MEMORY) * 1024L * 1024L);
}
return dataStream
.transform(opIdentifier("bucket_bulk_insert", conf), TypeInformation.of(Object.class), operatorFactory)
.uid("uid_bucket_bulk_insert" + conf.getString(FlinkOptions.TABLE_NAME))
.transform(opName("bucket_bulk_insert", conf), TypeInformation.of(Object.class), operatorFactory)
.uid(opUID("bucket_bulk_insert", conf))
.setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS))
.addSink(DummySink.INSTANCE)
.name("dummy");
Expand Down Expand Up @@ -152,7 +152,7 @@ public static DataStreamSink<Object> bulkInsert(Configuration conf, RowType rowT
}
}
return dataStream
.transform(opIdentifier("hoodie_bulk_insert_write", conf),
.transform(opName("hoodie_bulk_insert_write", conf),
TypeInformation.of(Object.class),
operatorFactory)
// follow the parallelism of upstream operators to avoid shuffle
Expand Down Expand Up @@ -196,8 +196,8 @@ public static DataStream<Object> append(
WriteOperatorFactory<RowData> operatorFactory = AppendWriteOperator.getFactory(conf, rowType);

return dataStream
.transform(opIdentifier("hoodie_append_write", conf), TypeInformation.of(Object.class), operatorFactory)
.uid("uid_hoodie_stream_write" + conf.getString(FlinkOptions.TABLE_NAME))
.transform(opName("hoodie_append_write", conf), TypeInformation.of(Object.class), operatorFactory)
.uid(opUID("hoodie_stream_write", conf))
.setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS));
}

Expand Down Expand Up @@ -254,7 +254,7 @@ private static DataStream<HoodieRecord> streamBootstrap(
TypeInformation.of(HoodieRecord.class),
new BootstrapOperator<>(conf))
.setParallelism(conf.getOptional(FlinkOptions.INDEX_BOOTSTRAP_TASKS).orElse(dataStream1.getParallelism()))
.uid("uid_index_bootstrap_" + conf.getString(FlinkOptions.TABLE_NAME));
.uid(opUID("index_bootstrap", conf));
}

return dataStream1;
Expand All @@ -280,7 +280,7 @@ private static DataStream<HoodieRecord> boundedBootstrap(
TypeInformation.of(HoodieRecord.class),
new BatchBootstrapOperator<>(conf))
.setParallelism(conf.getOptional(FlinkOptions.INDEX_BOOTSTRAP_TASKS).orElse(dataStream.getParallelism()))
.uid("uid_batch_index_bootstrap_" + conf.getString(FlinkOptions.TABLE_NAME));
.uid(opUID("batch_index_bootstrap", conf));
}

/**
Expand Down Expand Up @@ -320,8 +320,8 @@ public static DataStream<Object> hoodieStreamWrite(Configuration conf, DataStrea
String indexKeyFields = conf.getString(FlinkOptions.INDEX_KEY_FIELD);
BucketIndexPartitioner<HoodieKey> partitioner = new BucketIndexPartitioner<>(bucketNum, indexKeyFields);
return dataStream.partitionCustom(partitioner, HoodieRecord::getKey)
.transform(opIdentifier("bucket_write", conf), TypeInformation.of(Object.class), operatorFactory)
.uid("uid_bucket_write" + conf.getString(FlinkOptions.TABLE_NAME))
.transform(opName("bucket_write", conf), TypeInformation.of(Object.class), operatorFactory)
.uid(opUID("bucket_write", conf))
.setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS));
} else {
WriteOperatorFactory<HoodieRecord> operatorFactory = StreamWriteOperator.getFactory(conf);
Expand All @@ -332,12 +332,12 @@ public static DataStream<Object> hoodieStreamWrite(Configuration conf, DataStrea
"bucket_assigner",
TypeInformation.of(HoodieRecord.class),
new KeyedProcessOperator<>(new BucketAssignFunction<>(conf)))
.uid("uid_bucket_assigner_" + conf.getString(FlinkOptions.TABLE_NAME))
.uid(opUID("bucket_assigner", conf))
.setParallelism(conf.getInteger(FlinkOptions.BUCKET_ASSIGN_TASKS))
// shuffle by fileId(bucket id)
.keyBy(record -> record.getCurrentLocation().getFileId())
.transform(opIdentifier("stream_write", conf), TypeInformation.of(Object.class), operatorFactory)
.uid("uid_stream_write" + conf.getString(FlinkOptions.TABLE_NAME))
.transform(opName("stream_write", conf), TypeInformation.of(Object.class), operatorFactory)
.uid(opUID("stream_write", conf))
.setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS));
}
}
Expand Down Expand Up @@ -435,10 +435,14 @@ public static DataStreamSink<Object> dummySink(DataStream<Object> dataStream) {
.name("dummy");
}

public static String opIdentifier(String operatorN, Configuration conf) {
public static String opName(String operatorN, Configuration conf) {
return operatorN + ": " + conf.getString(FlinkOptions.TABLE_NAME);
}

public static String opUID(String operatorN, Configuration conf) {
return "uid_" + operatorN + "_" + conf.getString(FlinkOptions.TABLE_NAME);
}

/**
* Dummy sink that does nothing.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import org.apache.hudi.configuration.OptionsInference;
import org.apache.hudi.configuration.OptionsResolver;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.sink.utils.Pipelines;
import org.apache.hudi.source.FileIndex;
import org.apache.hudi.source.IncrementalInputSplits;
import org.apache.hudi.source.StreamReadMonitoringFunction;
Expand Down Expand Up @@ -188,9 +189,11 @@ public DataStream<RowData> produceDataStream(StreamExecutionEnvironment execEnv)
InputFormat<RowData, ?> inputFormat = getInputFormat(true);
OneInputStreamOperatorFactory<MergeOnReadInputSplit, RowData> factory = StreamReadOperator.factory((MergeOnReadInputFormat) inputFormat);
SingleOutputStreamOperator<RowData> source = execEnv.addSource(monitoringFunction, getSourceOperatorName("split_monitor"))
.uid(Pipelines.opUID("split_monitor", conf))
.setParallelism(1)
.keyBy(MergeOnReadInputSplit::getFileId)
.transform("split_reader", typeInfo, factory)
.uid(Pipelines.opUID("split_reader", conf))
.setParallelism(conf.getInteger(FlinkOptions.READ_TASKS));
return new DataStreamSource<>(source);
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we reuse the util method: opItentifier ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Of course, I have optimized it.

Copy link
Contributor

Choose a reason for hiding this comment

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

Can you apply the review patch though ~

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done.

Copy link
Contributor

Choose a reason for hiding this comment

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

Did you notice that i have pasted a patch under the comments, you can apply the patch with command:

git apply xxx.patch

Copy link
Contributor Author

Choose a reason for hiding this comment

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

OK

} else {
Expand Down