Skip to content

Commit

Permalink
add buffer queue
Browse files Browse the repository at this point in the history
  • Loading branch information
JingsongLi committed Aug 5, 2024
1 parent c368c81 commit 3e0af0a
Show file tree
Hide file tree
Showing 4 changed files with 167 additions and 14 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,9 @@

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;
Expand All @@ -35,9 +38,13 @@ 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;
Expand All @@ -46,10 +53,18 @@ public class AsyncPositionOutputStream extends PositionOutputStream {

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() {
Expand All @@ -73,7 +88,8 @@ private void doWork() throws InterruptedException, IOException {
}
if (event instanceof DataEvent) {
DataEvent dataEvent = (DataEvent) event;
out.write(dataEvent.data);
out.write(dataEvent.data, dataEvent.offset, dataEvent.length);
bufferQueue.add(dataEvent.data);
}
if (event instanceof FlushEvent) {
out.flush();
Expand All @@ -91,30 +107,52 @@ public long getPos() throws IOException {
return position;
}

private void flushBuffer() {
if (buffer.getCount() == 0) {
return;
}
putEvent(new DataEvent(buffer.getBuffer(), 0, 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++;
putEvent(new DataEvent(new byte[] {(byte) b}, 0, 1));
while (buffer.write((byte) b) != 1) {
flushBuffer();
}
}

@Override
public void write(byte[] b) throws IOException {
checkException();
position += b.length;
putEvent(new DataEvent(b, 0, b.length));
write(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));
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) {
Expand All @@ -134,6 +172,7 @@ public void flush() throws IOException {
@Override
public void close() throws IOException {
checkException();
flushBuffer();
putEvent(new EndEvent());
try {
this.future.get();
Expand All @@ -160,6 +199,9 @@ private void checkException() throws IOException {
if (throwable instanceof IOException) {
throw (IOException) throwable;
}
if (throwable instanceof RuntimeException) {
throw (RuntimeException) throwable;
}
throw new IOException(throwable);
}
}
Expand All @@ -169,11 +211,13 @@ private interface AsyncEvent {}
private static class DataEvent implements AsyncEvent {

private final byte[] data;
private final int offset;
private final int length;

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

Expand Down
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;
}
}
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 {

public ReuseByteArrayOutputStream(int size) {
super(size);
}

public void setBuffer(byte[] buffer) {
this.buf = buffer;
}

public byte[] getBuffer() {
return buf;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,15 @@ public void write(byte[] b) throws IOException {
out.write(new byte[] {5, 6, 7});
out.flush();
assertThat(byteOut.getPos()).isEqualTo(6);

out.write(new byte[] {8, 9});

// test repeat flush
out.flush();
out.flush();
assertThat(byteOut.getPos()).isEqualTo(8);

assertThat(out.getBufferQueue().size()).isEqualTo(1);
}

@Test
Expand All @@ -76,13 +85,13 @@ public void testFlushWithException() throws IOException {
ByteArrayPositionOutputStream byteOut =
new ByteArrayPositionOutputStream() {
@Override
public void write(byte[] b) throws IOException {
public void write(byte[] b, int off, int len) {
try {
Thread.sleep(100);
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
throw new IOException(msg);
throw new RuntimeException(msg);
}
};
AsyncPositionOutputStream out = new AsyncPositionOutputStream(byteOut);
Expand Down Expand Up @@ -117,14 +126,14 @@ public void testThrowException() throws IOException {
ByteArrayPositionOutputStream out =
new ByteArrayPositionOutputStream() {
@Override
public void write(byte[] b) throws IOException {
throw new IOException(msg);
public void write(byte[] b, int off, int len) {
throw new RuntimeException(msg);
}
};

AsyncPositionOutputStream asyncOut = new AsyncPositionOutputStream(out);
asyncOut.write(new byte[] {1, 2, 3});
assertThatThrownBy(asyncOut::close).hasMessage(msg);
assertThatThrownBy(asyncOut::close).hasMessageContaining(msg);
assertThat(out.closed).isTrue();
}

Expand Down

0 comments on commit 3e0af0a

Please sign in to comment.