-
Notifications
You must be signed in to change notification settings - Fork 1.9k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[Improve][API & Zeta] Using connector custom serializer encode/decode states #5238
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -25,7 +25,6 @@ | |
import org.apache.seatunnel.api.sink.SinkWriter; | ||
import org.apache.seatunnel.api.table.event.SchemaChangeEvent; | ||
import org.apache.seatunnel.api.table.type.Record; | ||
import org.apache.seatunnel.common.utils.SerializationUtils; | ||
import org.apache.seatunnel.engine.core.checkpoint.InternalCheckpointListener; | ||
import org.apache.seatunnel.engine.core.dag.actions.SinkAction; | ||
import org.apache.seatunnel.engine.server.checkpoint.ActionStateKey; | ||
|
@@ -48,6 +47,7 @@ | |
import java.util.Collection; | ||
import java.util.Collections; | ||
import java.util.List; | ||
import java.util.Objects; | ||
import java.util.Optional; | ||
import java.util.concurrent.CompletableFuture; | ||
import java.util.concurrent.ExecutionException; | ||
|
@@ -66,6 +66,7 @@ public class SinkFlowLifeCycle<T, CommitInfoT extends Serializable, AggregatedCo | |
private final SinkAction<T, StateT, CommitInfoT, AggregatedCommitInfoT> sinkAction; | ||
private SinkWriter<T, CommitInfoT, StateT> writer; | ||
|
||
private transient Optional<Serializer<CommitInfoT>> commitInfoSerializer; | ||
private transient Optional<Serializer<StateT>> writerStateSerializer; | ||
|
||
private final int indexID; | ||
|
@@ -110,6 +111,7 @@ public SinkFlowLifeCycle( | |
|
||
@Override | ||
public void init() throws Exception { | ||
this.commitInfoSerializer = sinkAction.getSink().getCommitInfoSerializer(); | ||
this.writerStateSerializer = sinkAction.getSink().getWriterStateSerializer(); | ||
this.committer = sinkAction.getSink().createCommitter(); | ||
this.lastCommitInfo = Optional.empty(); | ||
|
@@ -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), | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Use connector CommitInfoSerializer |
||
committerTaskAddress) | ||
.join(); | ||
} | ||
|
@@ -247,9 +253,9 @@ public void restoreState(List<ActionSubtaskState> actionStateList) throws Except | |
if (writerStateSerializer.isPresent()) { | ||
states = | ||
actionStateList.stream() | ||
.filter(state -> writerStateSerializer.isPresent()) | ||
.map(ActionSubtaskState::getState) | ||
.flatMap(Collection::stream) | ||
.filter(Objects::nonNull) | ||
.map( | ||
bytes -> | ||
sneaky( | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -23,7 +23,6 @@ | |
import org.apache.seatunnel.api.source.SourceReader; | ||
import org.apache.seatunnel.api.source.SourceSplit; | ||
import org.apache.seatunnel.api.table.type.Record; | ||
import org.apache.seatunnel.common.utils.SerializationUtils; | ||
import org.apache.seatunnel.engine.core.checkpoint.CheckpointType; | ||
import org.apache.seatunnel.engine.core.checkpoint.InternalCheckpointListener; | ||
import org.apache.seatunnel.engine.core.dag.actions.SourceAction; | ||
|
@@ -59,7 +58,6 @@ | |
import java.util.concurrent.atomic.AtomicReference; | ||
import java.util.stream.Collectors; | ||
|
||
import static org.apache.seatunnel.engine.common.utils.ExceptionUtil.sneaky; | ||
import static org.apache.seatunnel.engine.server.task.AbstractTask.serializeStates; | ||
|
||
@Slf4j | ||
|
@@ -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), | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Use connector SplitSerializer |
||
enumeratorTaskAddress) | ||
.get(); | ||
} catch (InterruptedException | ExecutionException e) { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,7 +17,6 @@ | |
|
||
package org.apache.seatunnel.engine.server.task.operation.sink; | ||
|
||
import org.apache.seatunnel.common.utils.SerializationUtils; | ||
import org.apache.seatunnel.engine.server.SeaTunnelServer; | ||
import org.apache.seatunnel.engine.server.TaskExecutionService; | ||
import org.apache.seatunnel.engine.server.execution.TaskLocation; | ||
|
@@ -33,7 +32,7 @@ | |
import java.io.IOException; | ||
|
||
@NoArgsConstructor | ||
public class SinkPrepareCommitOperation extends BarrierFlowOperation { | ||
public class SinkPrepareCommitOperation<CommitInfoT> extends BarrierFlowOperation { | ||
private byte[] commitInfos; | ||
|
||
public SinkPrepareCommitOperation( | ||
|
@@ -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); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Use connector SplitSerializer |
||
} finally { | ||
Thread.currentThread().setContextClassLoader(mainClassLoader); | ||
} | ||
committerTask.receivedWriterCommitInfo(barrier.getId(), deserializeCommitInfo); | ||
} | ||
committerTask.triggerBarrier(barrier); | ||
} | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Use connector SplitSerializer