Skip to content
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] [Connector-V2] support TTL for multiple table sink writer #8398

Open
wants to merge 7 commits into
base: dev
Choose a base branch
from
Open
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
2 changes: 2 additions & 0 deletions docs/en/concept/schema-evolution.md
Original file line number Diff line number Diff line change
Expand Up @@ -285,6 +285,8 @@ sink {

# Validate ddl update for sink writer multi replica
multi_table_sink_replica = 2
# multi sink writer ttl time (s)
multi_table_sink_ttl_sec = 30
}
}
```
2 changes: 2 additions & 0 deletions docs/zh/concept/schema-evolution.md
Original file line number Diff line number Diff line change
Expand Up @@ -286,6 +286,8 @@ sink {

# Validate ddl update for sink writer multi replica
multi_table_sink_replica = 2
# multi sink writer ttl time (s)
multi_table_sink_ttl_sec = 30
}
}
```
Original file line number Diff line number Diff line change
Expand Up @@ -29,4 +29,10 @@ public class SinkCommonOptions {
.intType()
.defaultValue(1)
.withDescription("The replica number of multi table sink writer");

public static Option<Integer> MULTI_TABLE_SINK_TTL_SEC =
Options.key("multi_table_sink_ttl_sec")
.intType()
.defaultValue(-1)
.withDescription("TTL for multi table writer");
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,4 +27,6 @@
@AllArgsConstructor
public class MultiTableAggregatedCommitInfo implements Serializable {
private Map<String, Object> commitInfo;

private Map<String, Boolean> hasWriteData;
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,137 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.seatunnel.api.sink.multitablesink;

import org.apache.seatunnel.api.sink.MultiTableResourceManager;
import org.apache.seatunnel.api.sink.SeaTunnelSink;
import org.apache.seatunnel.api.sink.SinkAggregatedCommitter;
import org.apache.seatunnel.api.sink.SupportMultiTableSinkAggregatedCommitter;
import org.apache.seatunnel.api.sink.SupportResourceShare;

import lombok.Getter;

import java.io.IOException;
import java.util.List;
import java.util.Optional;

public class MultiTablePreparedSinkAggregatedCommitter
implements SinkAggregatedCommitter, SupportResourceShare {
private SeaTunnelSink sink;
private int queueIndex;
@Getter private SinkAggregatedCommitter<?, ?> committer;
private volatile MultiTableResourceManager resourceManager;
@Getter private volatile boolean isPrepared = false;
private volatile boolean isInitialized = false;

public MultiTablePreparedSinkAggregatedCommitter(SeaTunnelSink sink) {
this.sink = sink;
}

public SinkAggregatedCommitter<?, ?> prepare() {
if (committer == null) {
try {
Optional<SinkAggregatedCommitter<?, ?>> sinkOptional =
sink.createAggregatedCommitter();
if (sinkOptional.isPresent()) {
committer = sinkOptional.get();
isPrepared = true;
if (committer instanceof SupportMultiTableSinkAggregatedCommitter
&& resourceManager != null) {
((SupportMultiTableSinkAggregatedCommitter<?>) committer)
.setMultiTableResourceManager(resourceManager, queueIndex);
}
}
} catch (IOException e) {
throw new RuntimeException(e);
}
}
return this.committer;
}

@Override
public void init() {
if (isInitialized) {
return;
}
prepare();
if (committer != null) {
committer.init();
}
isInitialized = true;
}

@Override
public MultiTableResourceManager<?> initMultiTableResourceManager(
int tableSize, int queueSize) {
if (resourceManager != null) {
return resourceManager;
}
prepare();
if (committer instanceof SupportMultiTableSinkAggregatedCommitter) {
resourceManager =
((SupportMultiTableSinkAggregatedCommitter<?>) committer)
.initMultiTableResourceManager(tableSize, queueSize);
return resourceManager;
} else {
return null;
}
}

public void setMultiTableResourceManager(
MultiTableResourceManager resourceManager, int queueIndex) {
if (this.resourceManager == null) {
this.resourceManager = resourceManager;
this.queueIndex = queueIndex;
if (committer != null
&& committer instanceof SupportMultiTableSinkAggregatedCommitter) {
((SupportMultiTableSinkAggregatedCommitter<?>) committer)
.setMultiTableResourceManager(resourceManager, queueIndex);
}
}
}

@Override
public List<?> commit(List aggregatedCommitInfo) throws IOException {
prepare();
init();
return committer.commit(aggregatedCommitInfo);
}

@Override
public Object combine(List commitInfos) {
prepare();
init();
return committer.combine(commitInfos);
}

@Override
public void abort(List aggregatedCommitInfo) throws Exception {
prepare();
init();
committer.abort(aggregatedCommitInfo);
}

@Override
public void close() throws IOException {
if (committer != null) {
committer.close();
isInitialized = false;
isPrepared = false;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -55,10 +55,13 @@ public class MultiTableSink

@Getter private final Map<TablePath, SeaTunnelSink> sinks;
private final int replicaNum;
private final int multiTableWriterTtl;

public MultiTableSink(MultiTableFactoryContext context) {
this.sinks = context.getSinks();
this.replicaNum = context.getOptions().get(SinkCommonOptions.MULTI_TABLE_SINK_REPLICA);
this.multiTableWriterTtl =
context.getOptions().get(SinkCommonOptions.MULTI_TABLE_SINK_TTL_SEC);
}

@Override
Expand All @@ -76,9 +79,22 @@ public SinkWriter<SeaTunnelRow, MultiTableCommitInfo, MultiTableState> createWri
SeaTunnelSink sink = sinks.get(tablePath);
int index = context.getIndexOfSubtask() * replicaNum + i;
String tableIdentifier = tablePath.toString();
writers.put(
SinkIdentifier.of(tableIdentifier, index),
sink.createWriter(new SinkContextProxy(index, replicaNum, context)));
if (multiTableWriterTtl < 0) {
writers.put(
SinkIdentifier.of(tableIdentifier, index),
sink.createWriter(new SinkContextProxy(index, replicaNum, context)));
} else {
writers.put(
SinkIdentifier.of(tableIdentifier, index),
new MultiTableTtlWriter(
writers,
tableIdentifier,
index,
replicaNum,
sink,
context,
multiTableWriterTtl));
}
sinkWritersContext.put(SinkIdentifier.of(tableIdentifier, index), context);
}
}
Expand All @@ -104,15 +120,43 @@ public SinkWriter<SeaTunnelRow, MultiTableCommitInfo, MultiTableState> restoreWr
.filter(Objects::nonNull)
.flatMap(Collection::stream)
.collect(Collectors.toList());
if (state.isEmpty()) {
writers.put(
sinkIdentifier,
sink.createWriter(new SinkContextProxy(index, replicaNum, context)));
if (multiTableWriterTtl < 0) {
if (state.isEmpty()) {
writers.put(
sinkIdentifier,
sink.createWriter(
new SinkContextProxy(index, replicaNum, context)));
} else {
writers.put(
sinkIdentifier,
sink.restoreWriter(
new SinkContextProxy(index, replicaNum, context), state));
}
} else {
writers.put(
sinkIdentifier,
sink.restoreWriter(
new SinkContextProxy(index, replicaNum, context), state));
if (state.isEmpty()) {
writers.put(
SinkIdentifier.of(tablePath.toString(), index),
new MultiTableTtlWriter(
writers,
tablePath.toString(),
index,
replicaNum,
sink,
context,
multiTableWriterTtl));
} else {
writers.put(
SinkIdentifier.of(tablePath.toString(), index),
new MultiTableTtlWriter(
writers,
tablePath.toString(),
index,
replicaNum,
sink,
context,
multiTableWriterTtl,
state));
}
}
sinkWritersContext.put(sinkIdentifier, context);
}
Expand Down Expand Up @@ -153,10 +197,16 @@ public Optional<Serializer<MultiTableCommitInfo>> getCommitInfoSerializer() {
Map<String, SinkAggregatedCommitter<?, ?>> aggCommitters = new HashMap<>();
for (TablePath tablePath : sinks.keySet()) {
SeaTunnelSink sink = sinks.get(tablePath);
Optional<SinkAggregatedCommitter<?, ?>> sinkOptional = sink.createAggregatedCommitter();
sinkOptional.ifPresent(
sinkAggregatedCommitter ->
aggCommitters.put(tablePath.toString(), sinkAggregatedCommitter));
if (multiTableWriterTtl < 0) {
Optional<SinkAggregatedCommitter<?, ?>> sinkOptional =
sink.createAggregatedCommitter();
sinkOptional.ifPresent(
sinkAggregatedCommitter ->
aggCommitters.put(tablePath.toString(), sinkAggregatedCommitter));
} else {
aggCommitters.put(
tablePath.toString(), new MultiTablePreparedSinkAggregatedCommitter(sink));
}
}
if (aggCommitters.isEmpty()) {
return Optional.empty();
Expand Down
Loading
Loading