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

[core] Support parallel close writers #4297

Closed
wants to merge 9 commits into from
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
6 changes: 6 additions & 0 deletions docs/layouts/shortcodes/generated/core_configuration.html
Original file line number Diff line number Diff line change
Expand Up @@ -942,5 +942,11 @@
<td>Integer</td>
<td>The bytes of types (CHAR, VARCHAR, BINARY, VARBINARY) devote to the zorder sort.</td>
</tr>
<tr>
<td><h5>table.close-writers-thread-number</h5></td>
<td style="word-wrap: break-word;">4</td>
<td>Integer</td>
<td>The number of threads used to flush data and close files during checkpoint.</td>
</tr>
</tbody>
</table>
11 changes: 11 additions & 0 deletions paimon-common/src/main/java/org/apache/paimon/CoreOptions.java
Original file line number Diff line number Diff line change
Expand Up @@ -1381,6 +1381,13 @@ public class CoreOptions implements Serializable {
.withDescription(
"Whether to enable asynchronous IO writing when writing files.");

public static final ConfigOption<Integer> TABLE_CLOSE_WRITERS_THREAD_NUMBER =
key("table.close-writers-thread-number")
.intType()
.defaultValue(4)
.withDescription(
"The number of threads used to flush data and close files during checkpoint.");

@ExcludeFromDocumentation("Only used internally to support materialized table")
public static final ConfigOption<Long> MATERIALIZED_TABLE_SNAPSHOT =
key("materialized-table.snapshot")
Expand Down Expand Up @@ -1829,6 +1836,10 @@ public int numSortedRunCompactionTrigger() {
return options.get(NUM_SORTED_RUNS_COMPACTION_TRIGGER);
}

public int tableCloseWritersThreadNumber() {
return options.get(TABLE_CLOSE_WRITERS_THREAD_NUMBER);
}

@Nullable
public Duration optimizedCompactionInterval() {
return options.get(COMPACTION_OPTIMIZATION_INTERVAL);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,8 +37,8 @@ public AbstractMemorySegmentPool(long maxMemory, int pageSize) {
}

@Override
public MemorySegment nextSegment() {
if (this.segments.size() > 0) {
public synchronized MemorySegment nextSegment() {
if (!this.segments.isEmpty()) {
return this.segments.poll();
} else if (numPage < maxPages) {
numPage++;
Expand All @@ -56,7 +56,7 @@ public int pageSize() {
}

@Override
public void returnAll(List<MemorySegment> memory) {
public synchronized void returnAll(List<MemorySegment> memory) {
segments.addAll(memory);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,12 +48,17 @@
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentSkipListSet;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;
import java.util.stream.Collectors;

import static org.apache.paimon.CoreOptions.PARTITION_DEFAULT_NAME;
import static org.apache.paimon.io.DataFileMeta.getMaxSequenceNumber;
Expand Down Expand Up @@ -88,6 +93,7 @@ public abstract class AbstractFileStoreWrite<T> implements FileStoreWrite<T> {
protected CompactionMetrics compactionMetrics = null;
protected final String tableName;
private boolean isInsertOnly;
private final ExecutorService closeWritersExecutor;

protected AbstractFileStoreWrite(
SnapshotManager snapshotManager,
Expand All @@ -97,7 +103,8 @@ protected AbstractFileStoreWrite(
String tableName,
int totalBuckets,
RowType partitionType,
int writerNumberMax) {
int writerNumberMax,
int tableCloseWritersThreadNumber) {
this.snapshotManager = snapshotManager;
this.scan = scan;
this.indexFactory = indexFactory;
Expand All @@ -107,6 +114,10 @@ protected AbstractFileStoreWrite(
this.writers = new HashMap<>();
this.tableName = tableName;
this.writerNumberMax = writerNumberMax;
this.closeWritersExecutor =
Executors.newFixedThreadPool(
tableCloseWritersThreadNumber,
new ExecutorThreadFactory("table-close-writers-thread-" + tableName));
}

@Override
Expand Down Expand Up @@ -190,68 +201,108 @@ public List<CommitMessage> prepareCommit(boolean waitCompaction, long commitIden
writerCleanChecker = createWriterCleanChecker();
}

List<CommitMessage> result = new ArrayList<>();

Iterator<Map.Entry<BinaryRow, Map<Integer, WriterContainer<T>>>> partIter =
writers.entrySet().iterator();
while (partIter.hasNext()) {
Map.Entry<BinaryRow, Map<Integer, WriterContainer<T>>> partEntry = partIter.next();
BinaryRow partition = partEntry.getKey();
Iterator<Map.Entry<Integer, WriterContainer<T>>> bucketIter =
partEntry.getValue().entrySet().iterator();
while (bucketIter.hasNext()) {
Map.Entry<Integer, WriterContainer<T>> entry = bucketIter.next();
int bucket = entry.getKey();
WriterContainer<T> writerContainer = entry.getValue();

CommitIncrement increment = writerContainer.writer.prepareCommit(waitCompaction);
List<IndexFileMeta> newIndexFiles = new ArrayList<>();
if (writerContainer.indexMaintainer != null) {
newIndexFiles.addAll(writerContainer.indexMaintainer.prepareCommit());
}
CompactDeletionFile compactDeletionFile = increment.compactDeletionFile();
if (compactDeletionFile != null) {
compactDeletionFile.getOrCompute().ifPresent(newIndexFiles::add);
}
CommitMessageImpl committable =
new CommitMessageImpl(
partition,
bucket,
increment.newFilesIncrement(),
increment.compactIncrement(),
new IndexIncrement(newIndexFiles));
result.add(committable);

if (committable.isEmpty()) {
if (writerCleanChecker.apply(writerContainer)) {
// Clear writer if no update, and if its latest modification has committed.
//
// We need a mechanism to clear writers, otherwise there will be more and
// more such as yesterday's partition that no longer needs to be written.
if (LOG.isDebugEnabled()) {
LOG.debug(
"Closing writer for partition {}, bucket {}. "
+ "Writer's last modified identifier is {}, "
+ "while current commit identifier is {}.",
partition,
bucket,
writerContainer.lastModifiedCommitIdentifier,
commitIdentifier);
List<CompletableFuture<CommitMessage>> futures = new ArrayList<>();
Set<BinaryRow> partitionsToRemove = ConcurrentHashMap.newKeySet();
Map<BinaryRow, Set<Integer>> bucketsToRemovePerPartition = new ConcurrentHashMap<>();

writers.forEach(
(partition, bucketMap) -> {
Set<Integer> bucketsToRemove = new ConcurrentSkipListSet<>();
bucketMap.forEach(
(bucket, writerContainer) -> {
CompletableFuture<CommitMessage> future =
CompletableFuture.supplyAsync(
() -> {
try {
return closeWriterContainer(
partition,
bucket,
writerContainer,
waitCompaction,
writerCleanChecker,
commitIdentifier,
bucketsToRemove);
} catch (Exception e) {
throw new RuntimeException(e);
}
},
closeWritersExecutor);
futures.add(future);
});
bucketsToRemovePerPartition.put(partition, bucketsToRemove);
});

CompletableFuture<Void> allTasksDone =
CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]));
allTasksDone.get();

List<CommitMessage> results =
futures.stream().map(CompletableFuture::join).collect(Collectors.toList());

bucketsToRemovePerPartition.forEach(
(partition, buckets) -> {
if (!buckets.isEmpty()) {
buckets.forEach(bucket -> writers.get(partition).remove(bucket));
if (writers.get(partition).isEmpty()) {
partitionsToRemove.add(partition);
}
writerContainer.writer.close();
bucketIter.remove();
}
} else {
writerContainer.lastModifiedCommitIdentifier = commitIdentifier;
}
}
});

if (partEntry.getValue().isEmpty()) {
partIter.remove();
partitionsToRemove.forEach(writers::remove);
return results;
}

@VisibleForTesting
public CommitMessage closeWriterContainer(
BinaryRow partition,
int bucket,
WriterContainer<T> writerContainer,
boolean waitCompaction,
Function<WriterContainer<T>, Boolean> writerCleanChecker,
long commitIdentifier,
Set<Integer> bucketsToRemove)
throws Exception {
CommitIncrement increment = writerContainer.writer.prepareCommit(waitCompaction);
List<IndexFileMeta> newIndexFiles = new ArrayList<>();
if (writerContainer.indexMaintainer != null) {
newIndexFiles.addAll(writerContainer.indexMaintainer.prepareCommit());
}
CompactDeletionFile compactDeletionFile = increment.compactDeletionFile();
if (compactDeletionFile != null) {
compactDeletionFile.getOrCompute().ifPresent(newIndexFiles::add);
}
CommitMessageImpl committable =
new CommitMessageImpl(
partition,
bucket,
increment.newFilesIncrement(),
increment.compactIncrement(),
new IndexIncrement(newIndexFiles));

if (committable.isEmpty()) {
if (writerCleanChecker.apply(writerContainer)) {
// Clear writer if no update, and if its latest modification has committed.
//
// We need a mechanism to clear writers, otherwise there will be more and
// more such as yesterday's partition that no longer needs to be written.
if (LOG.isDebugEnabled()) {
LOG.debug(
"Closing writer for partition {}, bucket {}. "
+ "Writer's last modified identifier is {}, "
+ "while current commit identifier is {}.",
partition,
bucket,
writerContainer.lastModifiedCommitIdentifier,
commitIdentifier);
}
writerContainer.writer.close();
bucketsToRemove.add(bucket);
}
} else {
writerContainer.lastModifiedCommitIdentifier = commitIdentifier;
}

return result;
return committable;
}

// This abstract function returns a whole function (instead of just a boolean value),
Expand Down Expand Up @@ -291,11 +342,32 @@ Function<WriterContainer<T>, Boolean> createNoConflictAwareWriterCleanChecker()

@Override
public void close() throws Exception {
for (Map<Integer, WriterContainer<T>> bucketWriters : writers.values()) {
for (WriterContainer<T> writerContainer : bucketWriters.values()) {
writerContainer.writer.close();
List<CompletableFuture<Void>> futures = new ArrayList<>();
// Close each writer in parallel
for (Map<Integer, WriterContainer<T>> bucketWriter : writers.values()) {
for (WriterContainer<?> writerContainer : bucketWriter.values()) {
futures.add(
CompletableFuture.runAsync(
() -> {
try {
writerContainer.writer.close();
} catch (Exception e) {
LOG.error("Failed to close writer: ", e);
}
},
closeWritersExecutor));
}
}
CompletableFuture<Void> allTasksDone =
CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]));
allTasksDone.get();
if (closeWritersExecutor != null) {
closeWritersExecutor.shutdown();
if (!closeWritersExecutor.awaitTermination(1, TimeUnit.MINUTES)) {
closeWritersExecutor.shutdownNow();
}
}

writers.clear();
if (lazyCompactExecutor != null && closeCompactExecutorWhenLeaving) {
lazyCompactExecutor.shutdownNow();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,8 @@ public MemoryFileStoreWrite(
tableName,
options.bucket(),
partitionType,
options.writeMaxWritersToSpill());
options.writeMaxWritersToSpill(),
options.tableCloseWritersThreadNumber());
this.options = options;
this.cacheManager = new CacheManager(options.lookupCacheMaxMemory());
}
Expand Down
Loading
Loading