You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@celeborn.apache.org by zh...@apache.org on 2023/01/13 08:47:07 UTC
[incubator-celeborn] branch main updated: [CELEBORN-202][Flink] Flink plugin BuffPacker adds unpack implements for shu… (#1150)
This is an automated email from the ASF dual-hosted git repository.
zhouky pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-celeborn.git
The following commit(s) were added to refs/heads/main by this push:
new cbeed65e [CELEBORN-202][Flink] Flink plugin BuffPacker adds unpack implements for shu… (#1150)
cbeed65e is described below
commit cbeed65ecd6adde5b0840a09966c64b700e6a610
Author: zhongqiangczq <96...@users.noreply.github.com>
AuthorDate: Fri Jan 13 16:47:03 2023 +0800
[CELEBORN-202][Flink] Flink plugin BuffPacker adds unpack implements for shu… (#1150)
---
.../celeborn/plugin/flink/buffer/BufferPacker.java | 188 +++++++++++++++++++++
.../celeborn/plugin/flink/utils/BufferUtils.java | 6 +-
.../apache/celeborn/plugin/flink/UnpackSuiteJ.java | 81 +++++++++
3 files changed, 272 insertions(+), 3 deletions(-)
diff --git a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/buffer/BufferPacker.java b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/buffer/BufferPacker.java
index dd40b2d7..fc96e805 100644
--- a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/buffer/BufferPacker.java
+++ b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/buffer/BufferPacker.java
@@ -17,10 +17,18 @@
package org.apache.celeborn.plugin.flink.buffer;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.Queue;
+
+import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator;
import org.apache.celeborn.plugin.flink.utils.BufferUtils;
+import org.apache.celeborn.plugin.flink.utils.Utils;
/** Harness used to pack multiple partial buffers together as a full one. */
public class BufferPacker {
@@ -102,4 +110,184 @@ public class BufferPacker {
}
currentSubIdx = -1;
}
+
+ public static Queue<Buffer> unpack(ByteBuf byteBuf) {
+ Queue<Buffer> buffers = new ArrayDeque<>();
+ try {
+ Utils.checkState(byteBuf instanceof Buffer, "Illegal buffer type.");
+
+ Buffer buffer = (Buffer) byteBuf;
+ int position = 0;
+ int totalBytes = buffer.readableBytes();
+ boolean isFirst = true;
+ while (position < totalBytes) {
+ BufferHeader bufferHeader;
+ if (isFirst) {
+ bufferHeader = BufferUtils.getBufferHeader(buffer, position, isFirst);
+ position += BufferUtils.HEADER_LENGTH;
+ } else {
+ // in the remaining datas, the headlength is BufferUtils.HEADER_LENGTH -
+ // BufferUtils.HEADER_LENGTH_PREFIX
+ bufferHeader = BufferUtils.getBufferHeader(buffer, position);
+ position += BufferUtils.HEADER_LENGTH - BufferUtils.HEADER_LENGTH_PREFIX;
+ }
+
+ Buffer slice = buffer.readOnlySlice(position, bufferHeader.getSize());
+ position += bufferHeader.getSize();
+
+ buffers.add(
+ new UnpackSlicedBuffer(
+ slice,
+ bufferHeader.getDataType(),
+ bufferHeader.isCompressed(),
+ bufferHeader.getSize()));
+ slice.retainBuffer();
+ isFirst = false;
+ }
+ return buffers;
+ } catch (Throwable throwable) {
+ buffers.forEach(Buffer::recycleBuffer);
+ throw throwable;
+ } finally {
+ byteBuf.release();
+ }
+ }
+
+ private static class UnpackSlicedBuffer implements Buffer {
+
+ private final Buffer buffer;
+
+ private DataType dataType;
+
+ private boolean isCompressed;
+
+ private final int size;
+
+ UnpackSlicedBuffer(Buffer buffer, DataType dataType, boolean isCompressed, int size) {
+ this.buffer = buffer;
+ this.dataType = dataType;
+ this.isCompressed = isCompressed;
+ this.size = size;
+ }
+
+ @Override
+ public boolean isBuffer() {
+ return dataType.isBuffer();
+ }
+
+ @Override
+ public MemorySegment getMemorySegment() {
+ return buffer.getMemorySegment();
+ }
+
+ @Override
+ public int getMemorySegmentOffset() {
+ return buffer.getMemorySegmentOffset();
+ }
+
+ @Override
+ public BufferRecycler getRecycler() {
+ return buffer.getRecycler();
+ }
+
+ @Override
+ public void recycleBuffer() {
+ buffer.recycleBuffer();
+ }
+
+ @Override
+ public boolean isRecycled() {
+ return buffer.isRecycled();
+ }
+
+ @Override
+ public Buffer retainBuffer() {
+ return buffer.retainBuffer();
+ }
+
+ @Override
+ public Buffer readOnlySlice() {
+ return buffer.readOnlySlice();
+ }
+
+ @Override
+ public Buffer readOnlySlice(int i, int i1) {
+ return buffer.readOnlySlice(i, i1);
+ }
+
+ @Override
+ public int getMaxCapacity() {
+ return buffer.getMaxCapacity();
+ }
+
+ @Override
+ public int getReaderIndex() {
+ return buffer.getReaderIndex();
+ }
+
+ @Override
+ public void setReaderIndex(int i) throws IndexOutOfBoundsException {
+ buffer.setReaderIndex(i);
+ }
+
+ @Override
+ public int getSize() {
+ return size;
+ }
+
+ @Override
+ public void setSize(int i) {
+ buffer.setSize(i);
+ }
+
+ @Override
+ public int readableBytes() {
+ return buffer.readableBytes();
+ }
+
+ @Override
+ public ByteBuffer getNioBufferReadable() {
+ return buffer.getNioBufferReadable();
+ }
+
+ @Override
+ public ByteBuffer getNioBuffer(int i, int i1) throws IndexOutOfBoundsException {
+ return buffer.getNioBuffer(i, i1);
+ }
+
+ @Override
+ public void setAllocator(ByteBufAllocator byteBufAllocator) {
+ buffer.setAllocator(byteBufAllocator);
+ }
+
+ @Override
+ public ByteBuf asByteBuf() {
+ return buffer.asByteBuf();
+ }
+
+ @Override
+ public boolean isCompressed() {
+ return isCompressed;
+ }
+
+ @Override
+ public void setCompressed(boolean b) {
+ isCompressed = b;
+ }
+
+ @Override
+ public DataType getDataType() {
+ return dataType;
+ }
+
+ @Override
+ public void setDataType(DataType dataType) {
+ this.dataType = dataType;
+ }
+
+ @Override
+ public int refCnt() {
+ return buffer.refCnt();
+ }
+ }
}
diff --git a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/utils/BufferUtils.java b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/utils/BufferUtils.java
index 92ec0695..3f1ea15a 100644
--- a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/utils/BufferUtils.java
+++ b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/utils/BufferUtils.java
@@ -35,8 +35,8 @@ public class BufferUtils {
// subpartitionid(4) + attemptId(4) + nextBatchId(4) + compressedsize
public static final int HEADER_LENGTH_PREFIX = 4 * 4;
- // dataType(1) + size(4)
- public static final int HEADER_LENGTH = HEADER_LENGTH_PREFIX + 1 + 4;
+ // dataType(1) + isCompress(1) + size(4)
+ public static final int HEADER_LENGTH = HEADER_LENGTH_PREFIX + 1 + 1 + 4;
/**
* Copies the data of the compressed buffer and the corresponding buffer header to the origin
@@ -77,7 +77,7 @@ public class BufferUtils {
public static BufferHeader getBufferHeader(Buffer buffer, int position, boolean isFirst) {
ByteBuf byteBuf = buffer.asByteBuf();
byteBuf.readerIndex(position);
- if (isFirst) {
+ if (!isFirst) {
return new BufferHeader(
Buffer.DataType.values()[byteBuf.readByte()], byteBuf.readBoolean(), byteBuf.readInt());
} else {
diff --git a/client-flink/common/src/test/java/org/apache/celeborn/plugin/flink/UnpackSuiteJ.java b/client-flink/common/src/test/java/org/apache/celeborn/plugin/flink/UnpackSuiteJ.java
new file mode 100644
index 00000000..87dc77a3
--- /dev/null
+++ b/client-flink/common/src/test/java/org/apache/celeborn/plugin/flink/UnpackSuiteJ.java
@@ -0,0 +1,81 @@
+/*
+ * 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.celeborn.plugin.flink;
+
+import java.io.IOException;
+import java.util.Queue;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.celeborn.plugin.flink.buffer.BufferPacker;
+
+public class UnpackSuiteJ {
+
+ private NetworkBufferPool networkBufferPool;
+ private BufferPool bufferPool;
+
+ @Before
+ public void setup() throws IOException {
+ networkBufferPool = new NetworkBufferPool(10, 128);
+ bufferPool = networkBufferPool.createBufferPool(10, 10);
+ }
+
+ @After
+ public void tearDown() {
+ bufferPool.lazyDestroy();
+ networkBufferPool.destroy();
+ }
+
+ @Test
+ public void testUnpack() throws IOException, InterruptedException {
+ Buffer buffer = bufferPool.requestBuffer();
+ ByteBuf byteBuf = buffer.asByteBuf();
+ byteBuf.writerIndex(0);
+ byteBuf.writeInt(0);
+ byteBuf.writeInt(1);
+ byteBuf.writeInt(2);
+ byteBuf.writeInt(3);
+ byteBuf.writeByte((byte) Buffer.DataType.DATA_BUFFER.ordinal());
+ byteBuf.writeBoolean(true);
+ byteBuf.writeInt(3);
+ byteBuf.writeBytes(new byte[] {1, 2, 3});
+ byteBuf.writeByte((byte) Buffer.DataType.EVENT_BUFFER.ordinal());
+ byteBuf.writeBoolean(false);
+ byteBuf.writeInt(5);
+ byteBuf.writeBytes(new byte[] {1, 2, 3, 4, 5});
+
+ Queue<Buffer> bufferQueue = BufferPacker.unpack(byteBuf);
+ Assert.assertEquals(2, bufferQueue.size());
+ Buffer buffer1 = bufferQueue.poll();
+ Assert.assertEquals(buffer1.getDataType(), Buffer.DataType.DATA_BUFFER);
+ Assert.assertEquals(buffer1.isCompressed(), true);
+ Assert.assertEquals(buffer1.getSize(), 3);
+
+ Buffer buffer2 = bufferQueue.poll();
+ Assert.assertEquals(buffer2.getDataType(), Buffer.DataType.EVENT_BUFFER);
+ Assert.assertEquals(buffer2.isCompressed(), false);
+ Assert.assertEquals(buffer2.getSize(), 5);
+ }
+}