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] Introduce AsyncPositionOutputStream #3875

Merged
merged 10 commits into from
Aug 6, 2024
Merged
Show file tree
Hide file tree
Changes from 5 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 @@ -26,6 +26,12 @@
</tr>
</thead>
<tbody>
<tr>
<td><h5>async-file-write</h5></td>
<td style="word-wrap: break-word;">true</td>
<td>Boolean</td>
<td>Whether to enable asynchronous IO writing when writing files.</td>
</tr>
<tr>
<td><h5>auto-create</h5></td>
<td style="word-wrap: break-word;">false</td>
Expand Down
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 @@ -1283,6 +1283,13 @@ public class CoreOptions implements Serializable {
"When a batch job queries from a table, if a partition does not exist in the current branch, "
+ "the reader will try to get this partition from this fallback branch.");

public static final ConfigOption<Boolean> ASYNC_FILE_WRITE =
key("async-file-write")
.booleanType()
.defaultValue(true)
.withDescription(
"Whether to enable asynchronous IO writing when writing files.");

private final Options options;

public CoreOptions(Map<String, String> options) {
Expand Down Expand Up @@ -2020,6 +2027,10 @@ public boolean prepareCommitWaitCompaction() {
return options.get(LOOKUP_WAIT);
}

public boolean asyncFileWrite() {
return options.get(ASYNC_FILE_WRITE);
}

public boolean metadataIcebergCompatible() {
return options.get(METADATA_ICEBERG_COMPATIBLE);
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,189 @@
/*
* 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.paimon.fs;

import java.io.IOException;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;

import static org.apache.paimon.utils.ThreadUtils.newDaemonThreadFactory;

/** A {@link PositionOutputStream} which uses a async thread to write data. */
public class AsyncPositionOutputStream extends PositionOutputStream {

public static final ExecutorService EXECUTOR_SERVICE =
Executors.newCachedThreadPool(newDaemonThreadFactory("AsyncOutputStream"));
public static final int AWAIT_TIMEOUT_SECONDS = 10;

private final PositionOutputStream out;
private final LinkedBlockingQueue<AsyncEvent> eventQueue;
private final AtomicReference<Throwable> exception;
private final Future<?> future;

private volatile boolean isClosed;
private long position;

public AsyncPositionOutputStream(PositionOutputStream out) {
this.out = out;
this.eventQueue = new LinkedBlockingQueue<>();
this.exception = new AtomicReference<>();
this.isClosed = false;
this.position = 0;
this.future = EXECUTOR_SERVICE.submit(this::execute);
}

private void execute() {
try {
doWork();
} catch (Throwable e) {
exception.set(e);
throw new RuntimeException(e);
}
}

private void doWork() throws InterruptedException, IOException {
try {
while (!isClosed) {
AsyncEvent event = eventQueue.poll(AWAIT_TIMEOUT_SECONDS, TimeUnit.SECONDS);
if (event == null) {
continue;
}
if (event instanceof EndEvent) {
return;
}
if (event instanceof DataEvent) {
DataEvent dataEvent = (DataEvent) event;
out.write(dataEvent.data);
}
if (event instanceof FlushEvent) {
out.flush();
((FlushEvent) event).latch.countDown();
}
}
} finally {
out.close();
}
}

@Override
public long getPos() throws IOException {
checkException();
return position;
}

@Override
public void write(int b) throws IOException {
checkException();
position++;
putEvent(new DataEvent(new byte[] {(byte) b}, 0, 1));
}

@Override
public void write(byte[] b) throws IOException {
checkException();
position += b.length;
putEvent(new DataEvent(b, 0, b.length));
}

@Override
public void write(byte[] b, int off, int len) throws IOException {
checkException();
position += len;
putEvent(new DataEvent(b, off, len));
}

@Override
public void flush() throws IOException {
checkException();
FlushEvent event = new FlushEvent();
putEvent(event);
while (true) {
try {
boolean await = event.latch.await(AWAIT_TIMEOUT_SECONDS, TimeUnit.SECONDS);
if (await) {
return;
}
checkException();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException(e);
}
}
}

@Override
public void close() throws IOException {
checkException();
putEvent(new EndEvent());
try {
this.future.get();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException(e);
} catch (ExecutionException e) {
throw new RuntimeException(e);
} finally {
isClosed = true;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What's the usage of this isClosed? The thread needs an EndEvent or isClosed == true to exit. However you use future.get() to wait for thread and then set isClosed = true, meaning that only EndEvent can stop the thread.

}
}

private void putEvent(AsyncEvent event) {
try {
eventQueue.put(event);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException(e);
}
}

private void checkException() throws IOException {
Throwable throwable = exception.get();
if (throwable != null) {
if (throwable instanceof IOException) {
throw (IOException) throwable;
}
throw new IOException(throwable);
}
}

private interface AsyncEvent {}

private static class DataEvent implements AsyncEvent {

private final byte[] data;

public DataEvent(byte[] input, int offset, int length) {
byte[] data = new byte[length];
System.arraycopy(input, offset, data, 0, length);
this.data = data;
}
}

private static class FlushEvent implements AsyncEvent {
private final CountDownLatch latch = new CountDownLatch(1);
}

private static class EndEvent implements AsyncEvent {}
}
Loading
Loading