hailin0 commented on code in PR #5238:
URL: https://github.com/apache/seatunnel/pull/5238#discussion_r1286902507
##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java:
##########
@@ -184,10 +186,14 @@ public void received(Record<?> record) {
runningTask
.getExecutionContext()
.sendToMember(
- new SinkPrepareCommitOperation(
+ new
SinkPrepareCommitOperation<CommitInfoT>(
barrier,
committerTaskLocation,
-
SerializationUtils.serialize(commitInfoT)),
+
commitInfoSerializer.isPresent()
+ ? commitInfoSerializer
+ .get()
+
.serialize(commitInfoT)
+ : null),
Review Comment:
Use connector CommitInfoSerializer
##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/context/SeaTunnelSplitEnumeratorContext.java:
##########
@@ -67,22 +69,26 @@ public void assignSplit(int subtaskIndex, List<SplitT>
splits) {
log.warn("No reader is obtained, skip this assign!");
return;
}
+
+ List<byte[]> splitBytes =
+ splits.stream()
+ .map(split -> sneaky(() ->
task.getSplitSerializer().serialize(split)))
Review Comment:
Use connector SplitSerializer
##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/RestoredSplitOperation.java:
##########
@@ -82,27 +86,31 @@ public void run() throws Exception {
TaskExecutionService taskExecutionService =
server.getTaskExecutionService();
RetryUtils.retryWithException(
() -> {
- ClassLoader classLoader =
+ SourceSplitEnumeratorTask<SourceSplit> task =
+ taskExecutionService.getTask(taskLocation);
+ ClassLoader taskClassLoader =
taskExecutionService
.getExecutionContext(taskLocation.getTaskGroupLocation())
.getClassLoader();
+ ClassLoader mainClassLoader =
Thread.currentThread().getContextClassLoader();
+
+ List<SourceSplit> deserializeSplits = new ArrayList<>();
+ try {
+
Thread.currentThread().setContextClassLoader(taskClassLoader);
+ for (byte[] split : splits) {
+
deserializeSplits.add(task.getSplitSerializer().deserialize(split));
Review Comment:
Use connector SplitSerializer
##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SourceFlowLifeCycle.java:
##########
@@ -338,21 +336,17 @@ public void restoreState(List<ActionSubtaskState>
actionStateList) throws Except
if (actionStateList.isEmpty()) {
return;
}
- List<SplitT> splits =
+ List<byte[]> splits =
actionStateList.stream()
.map(ActionSubtaskState::getState)
.flatMap(Collection::stream)
.filter(Objects::nonNull)
- .map(bytes -> sneaky(() ->
splitSerializer.deserialize(bytes)))
.collect(Collectors.toList());
try {
runningTask
.getExecutionContext()
.sendToMember(
- new RestoredSplitOperation(
- enumeratorTaskLocation,
-
SerializationUtils.serialize(splits.toArray()),
- indexID),
+ new RestoredSplitOperation(enumeratorTaskLocation,
splits, indexID),
Review Comment:
Use connector SplitSerializer
##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/AssignSplitOperation.java:
##########
@@ -56,13 +55,22 @@ public void run() throws Exception {
() -> {
SourceSeaTunnelTask<?, SplitT> task =
server.getTaskExecutionService().getTask(taskID);
- ClassLoader classLoader =
+ ClassLoader taskClassLoader =
server.getTaskExecutionService()
.getExecutionContext(taskID.getTaskGroupLocation())
.getClassLoader();
- Object[] o = SerializationUtils.deserialize(splits,
classLoader);
- task.receivedSourceSplit(
- Arrays.stream(o).map(i -> (SplitT)
i).collect(Collectors.toList()));
+ ClassLoader mainClassLoader =
Thread.currentThread().getContextClassLoader();
+ List<SplitT> deserializeSplits = new ArrayList<>();
+ try {
+
Thread.currentThread().setContextClassLoader(taskClassLoader);
+ for (byte[] split : this.splits) {
+
deserializeSplits.add(task.getSplitSerializer().deserialize(split));
Review Comment:
Use connector SplitSerializer
##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/sink/SinkPrepareCommitOperation.java:
##########
@@ -73,15 +72,24 @@ public int getClassId() {
public void run() throws Exception {
TaskExecutionService taskExecutionService =
((SeaTunnelServer) getService()).getTaskExecutionService();
- SinkAggregatedCommitterTask<?, ?> committerTask =
+ SinkAggregatedCommitterTask<CommitInfoT, ?> committerTask =
taskExecutionService.getTask(taskLocation);
- ClassLoader classLoader =
+ ClassLoader taskClassLoader =
taskExecutionService
.getExecutionContext(taskLocation.getTaskGroupLocation())
.getClassLoader();
+ ClassLoader mainClassLoader =
Thread.currentThread().getContextClassLoader();
+
if (commitInfos != null) {
- committerTask.receivedWriterCommitInfo(
- barrier.getId(),
SerializationUtils.deserialize(commitInfos, classLoader));
+ CommitInfoT deserializeCommitInfo = null;
+ try {
+ Thread.currentThread().setContextClassLoader(taskClassLoader);
+ deserializeCommitInfo =
+
committerTask.getCommitInfoSerializer().deserialize(commitInfos);
Review Comment:
Use connector SplitSerializer
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]