-
Notifications
You must be signed in to change notification settings - Fork 999
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
Changes from all commits
Commits
Show all changes
10 commits
Select commit
Hold shift + click to select a range
c0b65ce
fix
JingsongLi 8b1660e
fix
JingsongLi f5548a5
enable by default
JingsongLi 8a7ec3c
fix
JingsongLi 4c796a5
fix
JingsongLi c368c81
fix comment
JingsongLi 3e0af0a
add buffer queue
JingsongLi 4048b6d
add tests
JingsongLi 5e686d9
fix
JingsongLi bc780c3
fix
JingsongLi File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
227 changes: 227 additions & 0 deletions
227
paimon-common/src/main/java/org/apache/paimon/fs/AsyncPositionOutputStream.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,227 @@ | ||
/* | ||
* 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 org.apache.paimon.annotation.VisibleForTesting; | ||
import org.apache.paimon.utils.FixLenByteArrayOutputStream; | ||
|
||
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; | ||
public static final int BUFFER_SIZE = 1024 * 32; | ||
|
||
private final PositionOutputStream out; | ||
private final FixLenByteArrayOutputStream buffer; | ||
private final LinkedBlockingQueue<byte[]> bufferQueue; | ||
private final LinkedBlockingQueue<AsyncEvent> eventQueue; | ||
private final AtomicReference<Throwable> exception; | ||
private final Future<?> future; | ||
|
||
private long position; | ||
|
||
public AsyncPositionOutputStream(PositionOutputStream out) { | ||
this.out = out; | ||
this.bufferQueue = new LinkedBlockingQueue<>(); | ||
this.eventQueue = new LinkedBlockingQueue<>(); | ||
this.exception = new AtomicReference<>(); | ||
this.position = 0; | ||
this.future = EXECUTOR_SERVICE.submit(this::execute); | ||
this.buffer = new FixLenByteArrayOutputStream(); | ||
this.buffer.setBuffer(new byte[BUFFER_SIZE]); | ||
} | ||
|
||
@VisibleForTesting | ||
LinkedBlockingQueue<byte[]> getBufferQueue() { | ||
return bufferQueue; | ||
} | ||
|
||
private void execute() { | ||
try { | ||
doWork(); | ||
} catch (Throwable e) { | ||
exception.set(e); | ||
throw new RuntimeException(e); | ||
} | ||
} | ||
|
||
private void doWork() throws InterruptedException, IOException { | ||
try { | ||
while (true) { | ||
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, 0, dataEvent.length); | ||
bufferQueue.add(dataEvent.data); | ||
} | ||
if (event instanceof FlushEvent) { | ||
out.flush(); | ||
((FlushEvent) event).latch.countDown(); | ||
} | ||
} | ||
} finally { | ||
out.close(); | ||
} | ||
} | ||
|
||
@Override | ||
public long getPos() throws IOException { | ||
checkException(); | ||
return position; | ||
} | ||
|
||
private void flushBuffer() { | ||
if (buffer.getCount() == 0) { | ||
return; | ||
} | ||
putEvent(new DataEvent(buffer.getBuffer(), buffer.getCount())); | ||
byte[] byteArray = bufferQueue.poll(); | ||
if (byteArray == null) { | ||
byteArray = new byte[BUFFER_SIZE]; | ||
} | ||
buffer.setBuffer(byteArray); | ||
buffer.setCount(0); | ||
} | ||
|
||
@Override | ||
public void write(int b) throws IOException { | ||
checkException(); | ||
position++; | ||
while (buffer.write((byte) b) != 1) { | ||
flushBuffer(); | ||
} | ||
} | ||
|
||
@Override | ||
public void write(byte[] b) throws IOException { | ||
write(b, 0, b.length); | ||
} | ||
|
||
@Override | ||
public void write(byte[] b, int off, int len) throws IOException { | ||
checkException(); | ||
position += len; | ||
while (true) { | ||
int written = buffer.write(b, off, len); | ||
off += written; | ||
len -= written; | ||
if (len == 0) { | ||
return; | ||
} | ||
flushBuffer(); | ||
} | ||
} | ||
|
||
@Override | ||
public void flush() throws IOException { | ||
checkException(); | ||
flushBuffer(); | ||
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(); | ||
flushBuffer(); | ||
putEvent(new EndEvent()); | ||
try { | ||
this.future.get(); | ||
} catch (InterruptedException e) { | ||
Thread.currentThread().interrupt(); | ||
throw new RuntimeException(e); | ||
} catch (ExecutionException e) { | ||
throw new RuntimeException(e); | ||
} | ||
} | ||
|
||
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; | ||
} | ||
if (throwable instanceof RuntimeException) { | ||
throw (RuntimeException) throwable; | ||
} | ||
throw new IOException(throwable); | ||
} | ||
} | ||
|
||
private interface AsyncEvent {} | ||
|
||
private static class DataEvent implements AsyncEvent { | ||
|
||
private final byte[] data; | ||
private final int length; | ||
|
||
public DataEvent(byte[] data, int length) { | ||
this.data = data; | ||
this.length = length; | ||
} | ||
} | ||
|
||
private static class FlushEvent implements AsyncEvent { | ||
private final CountDownLatch latch = new CountDownLatch(1); | ||
} | ||
|
||
private static class EndEvent implements AsyncEvent {} | ||
} |
63 changes: 63 additions & 0 deletions
63
paimon-common/src/main/java/org/apache/paimon/utils/FixLenByteArrayOutputStream.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,63 @@ | ||
/* | ||
* 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.utils; | ||
|
||
import java.io.ByteArrayOutputStream; | ||
|
||
/** A {@link ByteArrayOutputStream} which can reuse byte array. */ | ||
public class FixLenByteArrayOutputStream { | ||
|
||
private byte[] buf; | ||
private int count; | ||
|
||
public void setBuffer(byte[] buffer) { | ||
this.buf = buffer; | ||
} | ||
|
||
public byte[] getBuffer() { | ||
return buf; | ||
} | ||
|
||
public int write(byte[] b, int off, int len) { | ||
if ((off < 0) || (off > b.length) || (len < 0) || ((off + len) - b.length > 0)) { | ||
throw new IndexOutOfBoundsException(); | ||
} | ||
int writeLen = Math.min(len, buf.length - count); | ||
System.arraycopy(b, off, buf, count, writeLen); | ||
count += writeLen; | ||
return writeLen; | ||
} | ||
|
||
public int getCount() { | ||
return count; | ||
} | ||
|
||
public int write(byte b) { | ||
if (count < buf.length) { | ||
buf[count] = b; | ||
count += 1; | ||
return 1; | ||
} | ||
return 0; | ||
} | ||
|
||
public void setCount(int count) { | ||
this.count = count; | ||
} | ||
} |
37 changes: 37 additions & 0 deletions
37
paimon-common/src/main/java/org/apache/paimon/utils/ReuseByteArrayOutputStream.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,37 @@ | ||
/* | ||
* 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.utils; | ||
|
||
import java.io.ByteArrayOutputStream; | ||
|
||
/** A {@link ByteArrayOutputStream} which can reuse byte array. */ | ||
public class ReuseByteArrayOutputStream extends ByteArrayOutputStream { | ||
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. What's the usage of this class? I don't see it is used anywhere. 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. Just for util usage. |
||
|
||
public ReuseByteArrayOutputStream(int size) { | ||
super(size); | ||
} | ||
|
||
public void setBuffer(byte[] buffer) { | ||
this.buf = buffer; | ||
} | ||
|
||
public byte[] getBuffer() { | ||
return buf; | ||
} | ||
} |
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
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.
It is flushBuffer, close will also do inner flush too.