advancedxy commented on code in PR #2492:
URL: https://github.com/apache/uniffle/pull/2492#discussion_r2128632391


##########
server/src/main/java/org/apache/uniffle/server/buffer/lab/ChunkCreator.java:
##########
@@ -0,0 +1,316 @@
+/*
+ * 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.uniffle.server.buffer.lab;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.LongAdder;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Does the management of LAB chunk creations. A monotonically incrementing id 
is associated with
+ * every chunk
+ */
+public class ChunkCreator {
+  private static final Logger LOG = 
LoggerFactory.getLogger(ChunkCreator.class);
+  // monotonically increasing chunkid. Starts at 1.
+  private final AtomicInteger chunkID = new AtomicInteger(1);
+
+  // mapping from chunk IDs to chunks
+  private final Map<Integer, Chunk> chunkIdMap = new 
ConcurrentHashMap<Integer, Chunk>();
+  static ChunkCreator instance;
+  private final int maxAlloc;
+  private ChunkPool chunksPool;
+  private final int chunkSize;
+
+  ChunkCreator(int chunkSize, long bufferCapacity, int maxAlloc) {
+    this.chunkSize = chunkSize;
+    this.maxAlloc = maxAlloc;
+    initializePools(chunkSize, bufferCapacity);
+  }
+
+  private void initializePools(int chunkSize, long bufferCapacity) {
+    this.chunksPool = initializePool(bufferCapacity, chunkSize);
+  }
+
+  /**
+   * Initializes the instance of ChunkCreator
+   *
+   * @param chunkSize the chunkSize
+   * @param bufferCapacity the buffer capacity
+   * @return singleton ChunkCreator
+   */
+  public static synchronized void initialize(int chunkSize, long 
bufferCapacity, int maxAlloc) {
+    if (instance != null) {
+      return;
+    }
+    instance = new ChunkCreator(chunkSize, bufferCapacity, maxAlloc);
+  }
+
+  public static ChunkCreator getInstance() {
+    return instance;
+  }
+
+  /**
+   * Creates and inits a chunk with specific index type and type.
+   *
+   * @return the chunk that was initialized
+   */
+  Chunk getChunk() {
+    return getChunk(chunksPool.getChunkSize());
+  }
+
+  /**
+   * Creates and inits a chunk.
+   *
+   * @return the chunk that was initialized
+   * @param size the size of the chunk to be allocated, in bytes
+   */
+  Chunk getChunk(int size) {
+    Chunk chunk = null;
+    ChunkPool pool = null;
+
+    // if the size is suitable for one of the pools
+    if (chunksPool != null && size == chunksPool.getChunkSize()) {
+      pool = chunksPool;
+    }
+
+    if (pool != null) {
+      chunk = pool.getChunk();
+      if (chunk == null) {
+        LOG.warn(
+            "The chunk pool is full. Reached maxCount= "
+                + pool.getMaxCount()
+                + ". Creating chunk outside of the pool.");
+      }
+    }
+
+    if (chunk == null) {
+      chunk = createChunk(false, size);
+    }
+    chunk.init();
+    return chunk;
+  }
+
+  /**
+   * Creates the chunk
+   *
+   * @param pool indicates if the chunks have to be created which will be used 
by the Pool
+   * @param size the size of the chunk to be allocated, in bytes
+   * @return the chunk
+   */
+  private Chunk createChunk(boolean pool, int size) {
+    Chunk chunk;
+    int id = chunkID.getAndIncrement();
+    assert id > 0;

Review Comment:
   this seem redundant? 



##########
server/src/main/java/org/apache/uniffle/server/ShuffleServerConf.java:
##########
@@ -499,6 +499,46 @@ public class ShuffleServerConf extends RssBaseConf {
               "Threshold when flushing shuffle data to persistent storage, 
recommend value would be 256K, "
                   + "512K, or even 1M");
 
+  public static final ConfigOption<Boolean> SERVER_SHUFFLE_BUFFER_LAB_ENABLE =
+      ConfigOptions.key("rss.server.buffer.lab.enable")

Review Comment:
   nit: lab.enabled?



##########
server/src/main/java/org/apache/uniffle/server/buffer/lab/LAB.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.uniffle.server.buffer.lab;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.uniffle.common.ShufflePartitionedBlock;
+
+/**
+ * Local allocation buffer.
+ *
+ * <p>The LAB is basically a bump-the-pointer allocator that allocates big 
(100K) chunks from and
+ * then doles it out to threads that request slices into the array. These 
chunks can get pooled as
+ * well. See {@link ChunkCreator}.
+ *
+ * <p>The purpose of this is to combat heap fragmentation in the shuffle 
server. By ensuring that
+ * all blocks in a given partition refer only to large chunks of contiguous 
memory, we ensure that
+ * large blocks get freed up when the partition is flushed.
+ *
+ * <p>Without the LAB, the byte array allocated during insertion end up 
interleaved throughout the
+ * heap, and the old generation gets progressively more fragmented until a 
stop-the-world compacting
+ * collection occurs.
+ *
+ * <p>This manages the large sized chunks. When blocks are to be added to 
partition, LAB's {@link
+ * #tryCopyBlockToChunk(ShufflePartitionedBlock)} gets called. This allocates 
enough size in the
+ * chunk to hold this block's data and copies into this area and then recreate 
a
+ * LABShufflePartitionedBlock over this copied data.
+ *
+ * <p>
+ *
+ * @see ChunkCreator
+ */
+public class LAB {
+  private Chunk currChunk;
+
+  List<Integer> chunks = new LinkedList<>();
+  private final int maxAlloc;

Review Comment:
   nit: this field name is a bit of vague, could this be called as capacity or 
something similar?



##########
common/src/main/java/org/apache/uniffle/common/ShufflePartitionedBlock.java:
##########
@@ -76,15 +80,12 @@ public boolean equals(Object o) {
       return false;
     }
     ShufflePartitionedBlock that = (ShufflePartitionedBlock) o;
-    return dataLength == that.dataLength
-        && crc == that.crc
-        && blockId == that.blockId
-        && data.equals(that.data);

Review Comment:
   Why this change?  I think at least we should keep dataLength and crc check?



##########
server/src/main/java/org/apache/uniffle/server/buffer/lab/ChunkCreator.java:
##########
@@ -0,0 +1,316 @@
+/*
+ * 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.uniffle.server.buffer.lab;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.LongAdder;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Does the management of LAB chunk creations. A monotonically incrementing id 
is associated with
+ * every chunk
+ */
+public class ChunkCreator {
+  private static final Logger LOG = 
LoggerFactory.getLogger(ChunkCreator.class);
+  // monotonically increasing chunkid. Starts at 1.
+  private final AtomicInteger chunkID = new AtomicInteger(1);
+
+  // mapping from chunk IDs to chunks
+  private final Map<Integer, Chunk> chunkIdMap = new 
ConcurrentHashMap<Integer, Chunk>();
+  static ChunkCreator instance;
+  private final int maxAlloc;
+  private ChunkPool chunksPool;
+  private final int chunkSize;
+
+  ChunkCreator(int chunkSize, long bufferCapacity, int maxAlloc) {
+    this.chunkSize = chunkSize;
+    this.maxAlloc = maxAlloc;
+    initializePools(chunkSize, bufferCapacity);
+  }
+
+  private void initializePools(int chunkSize, long bufferCapacity) {
+    this.chunksPool = initializePool(bufferCapacity, chunkSize);
+  }

Review Comment:
   it seems that chunksPool should be private final and initialized in the 
constructor?



##########
server/src/main/java/org/apache/uniffle/server/buffer/ShuffleBufferManager.java:
##########
@@ -66,6 +69,7 @@ public class ShuffleBufferManager {
   private static final Logger LOG = 
LoggerFactory.getLogger(ShuffleBufferManager.class);
 
   private final ShuffleBufferType shuffleBufferType;
+  private final Boolean enableLAB;

Review Comment:
   nit: -> `isLABEnabled?`



##########
server/src/main/java/org/apache/uniffle/server/buffer/lab/ChunkCreator.java:
##########
@@ -0,0 +1,316 @@
+/*
+ * 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.uniffle.server.buffer.lab;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.LongAdder;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Does the management of LAB chunk creations. A monotonically incrementing id 
is associated with
+ * every chunk
+ */
+public class ChunkCreator {
+  private static final Logger LOG = 
LoggerFactory.getLogger(ChunkCreator.class);
+  // monotonically increasing chunkid. Starts at 1.
+  private final AtomicInteger chunkID = new AtomicInteger(1);
+
+  // mapping from chunk IDs to chunks
+  private final Map<Integer, Chunk> chunkIdMap = new 
ConcurrentHashMap<Integer, Chunk>();
+  static ChunkCreator instance;
+  private final int maxAlloc;
+  private ChunkPool chunksPool;
+  private final int chunkSize;
+
+  ChunkCreator(int chunkSize, long bufferCapacity, int maxAlloc) {
+    this.chunkSize = chunkSize;
+    this.maxAlloc = maxAlloc;
+    initializePools(chunkSize, bufferCapacity);
+  }
+
+  private void initializePools(int chunkSize, long bufferCapacity) {
+    this.chunksPool = initializePool(bufferCapacity, chunkSize);
+  }
+
+  /**
+   * Initializes the instance of ChunkCreator
+   *
+   * @param chunkSize the chunkSize
+   * @param bufferCapacity the buffer capacity
+   * @return singleton ChunkCreator
+   */
+  public static synchronized void initialize(int chunkSize, long 
bufferCapacity, int maxAlloc) {
+    if (instance != null) {
+      return;

Review Comment:
   let's add a log here to indicate the chunk creator is already initialized 
with xx configurations.



##########
server/src/main/java/org/apache/uniffle/server/buffer/lab/Chunk.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.uniffle.server.buffer.lab;
+
+import io.netty.buffer.ByteBuf;
+
+/** A chunk of memory out of which allocations are sliced. */
+public abstract class Chunk {
+  /** Actual underlying data */
+  protected ByteBuf data;
+  /** Size of chunk in bytes */
+  protected final int size;
+  // The unique id associated with the chunk.
+  private final int id;
+  private final boolean fromPool;
+
+  /**
+   * Create an uninitialized chunk. Note that memory is not allocated yet, so 
this is cheap.
+   *
+   * @param size in bytes
+   * @param id the chunk id
+   */
+  public Chunk(int size, int id) {
+    this(size, id, false);
+  }
+
+  /**
+   * Create an uninitialized chunk. Note that memory is not allocated yet, so 
this is cheap.
+   *
+   * @param size in bytes
+   * @param id the chunk id
+   * @param fromPool if the chunk is formed by pool
+   */
+  public Chunk(int size, int id, boolean fromPool) {
+    this.size = size;
+    this.id = id;
+    this.fromPool = fromPool;
+  }
+
+  int getId() {
+    return this.id;
+  }
+
+  boolean isFromPool() {
+    return this.fromPool;
+  }
+
+  /**
+   * Actually claim the memory for this chunk. This should only be called from 
the thread that
+   * constructed the chunk. It is thread-safe against other threads calling 
alloc(), who will block
+   * until the allocation is complete.
+   */
+  public void init() {
+    allocateDataBuffer();
+  }
+
+  abstract void allocateDataBuffer();
+
+  public int alloc(int size) {

Review Comment:
   the implementation and the method name doesn't seem aligned? Could you add 
some java doc and/or change the method name here?



##########
common/src/main/java/org/apache/uniffle/common/ShufflePartitionedBlock.java:
##########
@@ -61,6 +62,10 @@ public ShufflePartitionedBlock(
     this.data = data;
   }
 
+  public boolean isInLAB() {

Review Comment:
   let's add some javadoc for this method?
   
   And nit: I think a more concise name might be `isOnLAB`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to