Skip to content
Draft
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
@@ -0,0 +1,33 @@
/*
* 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;

import org.apache.seatunnel.api.table.event.CloseTableEvent;

import java.io.IOException;

/** Sink writers that can release per-table resources before the task is fully closed. */
public interface SupportCloseTableSinkWriter {

/**
* Handle a close-table event emitted by a bounded multi-table source.
*
* @param event close-table event with the finished table identifier
*/
void handleCloseTableEvent(CloseTableEvent event) throws IOException;
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,10 @@

import org.apache.seatunnel.api.sink.MultiTableResourceManager;
import org.apache.seatunnel.api.sink.SinkWriter;
import org.apache.seatunnel.api.sink.SupportCloseTableSinkWriter;
import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter;
import org.apache.seatunnel.api.sink.SupportSchemaEvolutionSinkWriter;
import org.apache.seatunnel.api.table.event.CloseTableEvent;
import org.apache.seatunnel.api.table.schema.event.SchemaChangeEvent;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.tracing.MDCTracer;
Expand All @@ -34,6 +36,7 @@
import java.util.Map;
import java.util.Optional;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
Expand Down Expand Up @@ -61,7 +64,8 @@
@Slf4j
public class MultiTableSinkWriter
implements SinkWriter<SeaTunnelRow, MultiTableCommitInfo, MultiTableState>,
SupportSchemaEvolutionSinkWriter {
SupportSchemaEvolutionSinkWriter,
SupportCloseTableSinkWriter {

private final Map<SinkIdentifier, SinkWriter<SeaTunnelRow, ?, ?>> sinkWriters;
private final Map<SinkIdentifier, SinkWriter.Context> sinkWritersContext;
Expand All @@ -72,6 +76,14 @@ public class MultiTableSinkWriter
private final Random random = new Random();
private final List<BlockingQueue<SeaTunnelRow>> blockingQueues = new ArrayList<>();
private final ExecutorService executorService;
private final Set<String> closedTableIds = ConcurrentHashMap.newKeySet();
/** Tracks which upstream subtasks have already acknowledged a table as finished. */
private final ConcurrentMap<String, Set<Integer>> closeTableEventSources =
new ConcurrentHashMap<>();
/** Stores how many upstream close-table events must arrive before a table can be closed. */
private final ConcurrentMap<String, Integer> expectedCloseTableEventCounts =
new ConcurrentHashMap<>();

private MultiTableResourceManager resourceManager;
private volatile boolean submitted = false;

Expand Down Expand Up @@ -223,6 +235,38 @@ public void applySchemaChange(SchemaChangeEvent event) throws IOException {
}
}

@Override
public void handleCloseTableEvent(CloseTableEvent event) throws IOException {
if (event == null || event.tableId() == null) {
log.debug("Ignore empty close table event: {}", event);
return;
}
Integer sourceSubtaskId = event.getSourceSubtaskId();
Integer expectedSourceEventCount = event.getExpectedSourceEventCount();
if (sourceSubtaskId == null
|| expectedSourceEventCount == null
|| expectedSourceEventCount <= 1) {
closeTable(event.tableId());
return;
}
Set<Integer> receivedSourceSubtasks =
closeTableEventSources.computeIfAbsent(
event.tableId(), key -> ConcurrentHashMap.newKeySet());
receivedSourceSubtasks.add(sourceSubtaskId);
expectedCloseTableEventCounts.merge(event.tableId(), expectedSourceEventCount, Math::max);
int currentCount = receivedSourceSubtasks.size();
int requiredCount = expectedCloseTableEventCounts.get(event.tableId());
if (currentCount < requiredCount) {
log.debug(
"Received {}/{} close table events for table {}, waiting for all upstream readers",
currentCount,
requiredCount,
event.tableId());
return;
}
closeTable(event.tableId());
}

/**
* Routes a row to the appropriate blocking queue for async writing.
*
Expand Down Expand Up @@ -252,6 +296,9 @@ public void applySchemaChange(SchemaChangeEvent event) throws IOException {
*/
@Override
public void write(SeaTunnelRow element) throws IOException {
if (element == null) {
return;
}
if (element != null && element.getOptions() != null) {
if (element.getOptions().containsKey("flush_event")
|| element.getOptions().containsKey("schema_change_event")) {
Expand All @@ -260,6 +307,13 @@ public void write(SeaTunnelRow element) throws IOException {
}
}

if (element.getTableId() != null && closedTableIds.contains(element.getTableId())) {
throw new IOException(
String.format(
"Received row for table %s after close table event was handled",
element.getTableId()));
}

if (!submitted) {
submitted = true;
runnable.forEach(executorService::submit);
Expand Down Expand Up @@ -293,6 +347,84 @@ public void write(SeaTunnelRow element) throws IOException {
}
}

private void closeTable(String tableId) throws IOException {
if (!closedTableIds.add(tableId)) {
log.debug("Table {} is already closed in multi table sink writer", tableId);
return;
}
closeTableEventSources.remove(tableId);
expectedCloseTableEventCounts.remove(tableId);
waitUntilTableQueueDrained(tableId);

boolean matched = false;
Throwable firstError = null;
for (int i = 0; i < sinkWritersWithIndex.size(); i++) {
synchronized (runnable.get(i)) {
Map<SinkIdentifier, SinkWriter<SeaTunnelRow, ?, ?>> writerMap =
sinkWritersWithIndex.get(i);
List<SinkIdentifier> matchedIdentifiers = new ArrayList<>();
for (Map.Entry<SinkIdentifier, SinkWriter<SeaTunnelRow, ?, ?>> entry :
writerMap.entrySet()) {
if (tableId.equals(entry.getKey().getTableIdentifier())) {
matchedIdentifiers.add(entry.getKey());
}
}
if (matchedIdentifiers.isEmpty()) {
continue;
}
matched = true;
for (SinkIdentifier identifier : matchedIdentifiers) {
SinkWriter<SeaTunnelRow, ?, ?> sinkWriter = writerMap.remove(identifier);
sinkWriters.remove(identifier);
if (sinkWriter == null) {
continue;
}
try {
sinkWriter.close();
} catch (Throwable e) {
if (firstError == null) {
firstError = e;
}
log.error("Failed to close sink writer for table {}", tableId, e);
}
}
runnable.get(i).removeTableWriter(tableId);
}
}
sinkPrimaryKeys.remove(tableId);
if (!matched) {
log.debug("Ignore close table event for unknown table {}", tableId);
} else {
log.info("Closed sink writers for table {} after close table event", tableId);
}
if (firstError != null) {
throw new IOException("Failed to close sink writers for table " + tableId, firstError);
}
}

private void waitUntilTableQueueDrained(String tableId) {
try {
while (hasQueuedRows(tableId)) {
Thread.sleep(100L);
subSinkErrorCheck();
}
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException(e);
}
}

private boolean hasQueuedRows(String tableId) {
for (BlockingQueue<SeaTunnelRow> blockingQueue : blockingQueues) {
for (SeaTunnelRow row : blockingQueue) {
if (tableId.equals(row.getTableId())) {
return true;
}
}
}
return false;
}

/**
* Captures the state of all sub-writers for the given checkpoint.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,4 +95,8 @@ public Throwable getThrowable() {
public String getCurrentTableId() {
return currentTableId;
}

public void removeTableWriter(String tableId) {
tableIdWriterMap.remove(tableId);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.seatunnel.api.source;

import org.apache.seatunnel.api.table.event.CloseTableEvent;
import org.apache.seatunnel.api.table.schema.event.SchemaChangeEvent;

/**
Expand All @@ -32,6 +33,8 @@ default void markSchemaChangeBeforeCheckpoint() {}

default void collect(SchemaChangeEvent event) {}

default void collect(CloseTableEvent event) {}

default void markSchemaChangeAfterCheckpoint() {}

/**
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
/*
* 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.table.event;

import org.apache.seatunnel.api.table.catalog.TablePath;

import lombok.AllArgsConstructor;
import lombok.EqualsAndHashCode;
import lombok.Getter;
import lombok.ToString;

import java.io.Serializable;

/**
* Signals that a bounded source has finished producing records for a table, so downstream sink
* writers can release table-scoped resources early.
*/
@Getter
@ToString
@EqualsAndHashCode
@AllArgsConstructor
public class CloseTableEvent implements Serializable {

private static final long serialVersionUID = 1L;

/** The downstream table whose resources can be reclaimed once all upstream signals arrive. */
private final TablePath tablePath;

/** The upstream subtask that emitted this event on its own output channel. */
private final Integer sourceSubtaskId;

/** Number of distinct upstream subtasks that must emit this event before closing the table. */
private final Integer expectedSourceEventCount;

public CloseTableEvent(TablePath tablePath) {
this(tablePath, null, null);
}

public String tableId() {
return tablePath == null ? null : tablePath.getFullName();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.apache.seatunnel.api.common.PluginIdentifierInterface;
import org.apache.seatunnel.api.source.SeaTunnelJobAware;
import org.apache.seatunnel.api.table.catalog.CatalogTable;
import org.apache.seatunnel.api.table.event.CloseTableEvent;
import org.apache.seatunnel.api.table.schema.event.SchemaChangeEvent;
import org.apache.seatunnel.api.table.type.SeaTunnelDataType;

Expand Down Expand Up @@ -52,6 +53,10 @@ default SchemaChangeEvent mapSchemaChangeEvent(SchemaChangeEvent schemaChangeEve
return schemaChangeEvent;
}

default CloseTableEvent mapCloseTableEvent(CloseTableEvent closeTableEvent) {
return closeTableEvent;
}

/** call it when Transformer completed */
default void close() {}
}
Loading
Loading