[GitHub] [hadoop] steveloughran commented on a diff in pull request #4263: HADOOP-18105 Implement buffer pooling with weak references

2022-05-30 Thread GitBox


steveloughran commented on code in PR #4263:
URL: https://github.com/apache/hadoop/pull/4263#discussion_r884681833


##
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestMoreWeakReferencedElasticByteBufferPool.java:
##
@@ -0,0 +1,93 @@
+/**
+ * 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.hadoop.io;
+
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.test.HadoopTestBase;
+
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+/**
+ * Non parameterized tests for {@code WeakReferencedElasticByteBufferPool}.
+ */
+public class TestMoreWeakReferencedElasticByteBufferPool

Review Comment:
   as long as it fails in the same method where it is passed in, I'm happy. 
it's the delayed failures which are the pain point



-- 
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: common-issues-unsubscr...@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[GitHub] [hadoop] steveloughran commented on a diff in pull request #4263: HADOOP-18105 Implement buffer pooling with weak references

2022-05-24 Thread GitBox


steveloughran commented on code in PR #4263:
URL: https://github.com/apache/hadoop/pull/4263#discussion_r880947333


##
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WeakReferencedElasticByteBufferPool.java:
##
@@ -0,0 +1,126 @@
+/**
+ * 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.hadoop.io;
+
+import java.lang.ref.WeakReference;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.classification.VisibleForTesting;
+
+/**
+ * Buffer pool implementation which uses weak references to store
+ * buffers in the pool, such that they are garbage collected when
+ * there are no references to the buffer during a gc run. This is
+ * important as direct buffers don't get garbage collected automatically
+ * during a gc run as they are not stored on heap memory.
+ * Also the buffers are stored in a tree map which helps in returning
+ * smallest buffer whose size is just greater than requested length.
+ * This is a thread safe implementation.
+ */
+public final class WeakReferencedElasticByteBufferPool extends 
ElasticByteBufferPool {
+
+  private final TreeMap> directBuffers =

Review Comment:
   1. add javadocs here and below, mention use must be in synchronized blocks
   2. field should be of type Map<>, unless it has to be explicitly a tree map



##
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ByteBufferPool.java:
##
@@ -45,4 +45,6 @@ public interface ByteBufferPool {
* @param buffera direct bytebuffer
*/
   void putBuffer(ByteBuffer buffer);
+
+  default void release() { }

Review Comment:
   javadoc?



##
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WeakReferencedElasticByteBufferPool.java:
##
@@ -0,0 +1,123 @@
+/**
+ * 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.hadoop.io;
+
+import java.lang.ref.WeakReference;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.classification.VisibleForTesting;
+
+/**
+ * Buffer pool implementation which uses weak references to store
+ * buffers in the pool, such that they are garbage collected when
+ * there are no references to the buffer during a gc run. This is
+ * important as direct buffer don't get garbage collected automatically
+ * during a gc run as they are not stored on heap memory.
+ * Also the buffers are stored in a tree map which helps in returning
+ * smallest buffer whose size is just greater than requested length.
+ * This is a thread safe implementation.
+ */
+public final class WeakReferencedElasticByteBufferPool extends 
ElasticByteBufferPool {
+
+  private final TreeMap> directBuffers =
+  new TreeMap<>();
+
+  private final TreeMap> heapBuffers =
+  new TreeMap<>();
+
+  private TreeMap> getBufferTree(boolean 
isDirect) {
+return isDirect ? directBuffers : heapBuffers;
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @param direct whether we want a direct byte buffer or a heap one.
+   * @param length length of requested buffer.
+   * @return returns equal or next greater than capacity buffer from
+   * pool if already available and not garbage collected else creates
+   * a new buffer and return it.
+   */
+  @Override
+  public synchronized ByteBuffer getBuffer(boolean direct, int length) {
+TreeMap> buffersTree = 

[GitHub] [hadoop] steveloughran commented on a diff in pull request #4263: HADOOP-18105 Implement buffer pooling with weak references

2022-05-18 Thread GitBox


steveloughran commented on code in PR #4263:
URL: https://github.com/apache/hadoop/pull/4263#discussion_r875762848


##
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestMoreWeakReferencedElasticByteBufferPool.java:
##
@@ -52,6 +55,32 @@ public void testMixedBuffersInPool() {
 
   }
 
+  @Test
+  public void testUnexpectedBufferSizes() {
+WeakReferencedElasticByteBufferPool pool = new 
WeakReferencedElasticByteBufferPool();
+ByteBuffer buffer1 = pool.getBuffer(true, 0);
+try {

Review Comment:
   use LambdaTestUtils.intercept() here and below



-- 
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: common-issues-unsubscr...@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[GitHub] [hadoop] steveloughran commented on a diff in pull request #4263: HADOOP-18105 Implement buffer pooling with weak references

2022-05-18 Thread GitBox


steveloughran commented on code in PR #4263:
URL: https://github.com/apache/hadoop/pull/4263#discussion_r875762072


##
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestWeakReferencedElasticByteBufferPool.java:
##
@@ -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.hadoop.io;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Unit tests for {@code WeakReferencedElasticByteBufferPool}.
+ */
+@RunWith(Parameterized.class)
+public class TestWeakReferencedElasticByteBufferPool {
+
+  private final boolean isDirect;
+
+  private final String type;
+
+  @Parameterized.Parameters(name = "Buffer type : {0}")
+  public static List params() {
+return Arrays.asList("direct", "array");
+  }
+
+  public TestWeakReferencedElasticByteBufferPool(String type) {
+this.type = type;
+this.isDirect = !"array".equals(type);
+  }
+
+  // Add more tests for different time and same size buffers in the pool. 
+  @Test
+  public void testGetAndPutBasic() {
+WeakReferencedElasticByteBufferPool pool = new 
WeakReferencedElasticByteBufferPool();
+int bufferSize = 5;
+ByteBuffer buffer = pool.getBuffer(isDirect, bufferSize);
+Assertions.assertThat(buffer.isDirect())
+.describedAs("Buffered returned should be of correct type {}", 
type)
+.isEqualTo(isDirect);
+Assertions.assertThat(buffer.capacity())
+.describedAs("Initial capacity of returned buffer from pool")
+.isEqualTo(bufferSize);
+Assertions.assertThat(buffer.position())
+.describedAs("Initial position of returned buffer from pool")
+.isEqualTo(0);
+
+byte[] arr = createByteArray(bufferSize);
+buffer.put(arr, 0, arr.length);
+buffer.flip();
+validateBufferContent(buffer, arr);
+Assertions.assertThat(buffer.position())
+.describedAs("Buffer's position after filling bytes in it")
+.isEqualTo(bufferSize);
+// releasing buffer to the pool.
+pool.putBuffer(buffer);
+Assertions.assertThat(buffer.position())
+.describedAs("Position should be reset to 0 after returning buffer 
to the pool")
+.isEqualTo(0);
+
+  }
+
+  @Test
+  public void testPoolingWithDifferentSizes() {
+WeakReferencedElasticByteBufferPool pool = new 
WeakReferencedElasticByteBufferPool();
+ByteBuffer buffer = pool.getBuffer(isDirect, 5);
+ByteBuffer buffer1 = pool.getBuffer(isDirect, 10);
+ByteBuffer buffer2 = pool.getBuffer(isDirect, 15);
+
+Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+.describedAs("Number of buffers in the pool")
+.isEqualTo(0);
+
+pool.putBuffer(buffer1);
+pool.putBuffer(buffer2);
+Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+.describedAs("Number of buffers in the pool")
+.isEqualTo(2);
+ByteBuffer buffer3 = pool.getBuffer(isDirect, 12);
+Assertions.assertThat(buffer3.capacity())
+.describedAs("Pooled buffer should have older capacity")
+.isEqualTo(15);
+Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+.describedAs("Number of buffers in the pool")
+.isEqualTo(1);
+pool.putBuffer(buffer);
+ByteBuffer buffer4 = pool.getBuffer(isDirect, 6);
+Assertions.assertThat(buffer4.capacity())
+.describedAs("Pooled buffer should have older capacity")
+.isEqualTo(10);
+Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+.describedAs("Number of buffers in the pool")
+.isEqualTo(1);
+
+pool.release();
+Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+.describedAs("Number of buffers in the pool post release")
+.isEqualTo(0);
+  }
+
+  @Test
+  public void testPoolingWithDifferentInsertionTime() {
+WeakReferencedElasticByteBufferPool pool = new 
WeakReferencedElasticByteBufferPool();

[GitHub] [hadoop] steveloughran commented on a diff in pull request #4263: HADOOP-18105 Implement buffer pooling with weak references

2022-05-18 Thread GitBox


steveloughran commented on code in PR #4263:
URL: https://github.com/apache/hadoop/pull/4263#discussion_r875761159


##
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestWeakReferencedElasticByteBufferPool.java:
##
@@ -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.hadoop.io;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Unit tests for {@code WeakReferencedElasticByteBufferPool}.
+ */
+@RunWith(Parameterized.class)
+public class TestWeakReferencedElasticByteBufferPool {
+
+  private final boolean isDirect;
+
+  private final String type;
+
+  @Parameterized.Parameters(name = "Buffer type : {0}")
+  public static List params() {
+return Arrays.asList("direct", "array");
+  }
+
+  public TestWeakReferencedElasticByteBufferPool(String type) {
+this.type = type;
+this.isDirect = !"array".equals(type);
+  }
+
+  // Add more tests for different time and same size buffers in the pool. 
+  @Test
+  public void testGetAndPutBasic() {
+WeakReferencedElasticByteBufferPool pool = new 
WeakReferencedElasticByteBufferPool();
+int bufferSize = 5;
+ByteBuffer buffer = pool.getBuffer(isDirect, bufferSize);
+Assertions.assertThat(buffer.isDirect())
+.describedAs("Buffered returned should be of correct type {}", 
type)
+.isEqualTo(isDirect);
+Assertions.assertThat(buffer.capacity())
+.describedAs("Initial capacity of returned buffer from pool")
+.isEqualTo(bufferSize);
+Assertions.assertThat(buffer.position())
+.describedAs("Initial position of returned buffer from pool")
+.isEqualTo(0);
+
+byte[] arr = createByteArray(bufferSize);
+buffer.put(arr, 0, arr.length);
+buffer.flip();
+validateBufferContent(buffer, arr);
+Assertions.assertThat(buffer.position())
+.describedAs("Buffer's position after filling bytes in it")
+.isEqualTo(bufferSize);
+// releasing buffer to the pool.
+pool.putBuffer(buffer);
+Assertions.assertThat(buffer.position())
+.describedAs("Position should be reset to 0 after returning buffer 
to the pool")
+.isEqualTo(0);
+
+  }
+
+  @Test
+  public void testPoolingWithDifferentSizes() {
+WeakReferencedElasticByteBufferPool pool = new 
WeakReferencedElasticByteBufferPool();
+ByteBuffer buffer = pool.getBuffer(isDirect, 5);
+ByteBuffer buffer1 = pool.getBuffer(isDirect, 10);
+ByteBuffer buffer2 = pool.getBuffer(isDirect, 15);
+
+Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+.describedAs("Number of buffers in the pool")
+.isEqualTo(0);
+
+pool.putBuffer(buffer1);
+pool.putBuffer(buffer2);
+Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+.describedAs("Number of buffers in the pool")
+.isEqualTo(2);
+ByteBuffer buffer3 = pool.getBuffer(isDirect, 12);
+Assertions.assertThat(buffer3.capacity())
+.describedAs("Pooled buffer should have older capacity")
+.isEqualTo(15);
+Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+.describedAs("Number of buffers in the pool")
+.isEqualTo(1);
+pool.putBuffer(buffer);
+ByteBuffer buffer4 = pool.getBuffer(isDirect, 6);
+Assertions.assertThat(buffer4.capacity())
+.describedAs("Pooled buffer should have older capacity")
+.isEqualTo(10);
+Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+.describedAs("Number of buffers in the pool")
+.isEqualTo(1);
+
+pool.release();
+Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+.describedAs("Number of buffers in the pool post release")
+.isEqualTo(0);
+  }
+
+  @Test
+  public void testPoolingWithDifferentInsertionTime() {
+WeakReferencedElasticByteBufferPool pool = new 
WeakReferencedElasticByteBufferPool();

[GitHub] [hadoop] steveloughran commented on a diff in pull request #4263: HADOOP-18105 Implement buffer pooling with weak references

2022-05-11 Thread GitBox


steveloughran commented on code in PR #4263:
URL: https://github.com/apache/hadoop/pull/4263#discussion_r870105560


##
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WeakReferencedElasticByteBufferPool.java:
##
@@ -0,0 +1,123 @@
+/**
+ * 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.hadoop.io;
+
+import java.lang.ref.WeakReference;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.classification.VisibleForTesting;
+
+/**
+ * Buffer pool implementation which uses weak references to store
+ * buffers in the pool, such that they are garbage collected when
+ * there are no references to the buffer during a gc run. This is
+ * important as direct buffer don't get garbage collected automatically

Review Comment:
   "direct buffers"



##
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WeakReferencedElasticByteBufferPool.java:
##
@@ -0,0 +1,123 @@
+/**
+ * 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.hadoop.io;
+
+import java.lang.ref.WeakReference;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.classification.VisibleForTesting;
+
+/**
+ * Buffer pool implementation which uses weak references to store
+ * buffers in the pool, such that they are garbage collected when
+ * there are no references to the buffer during a gc run. This is
+ * important as direct buffer don't get garbage collected automatically
+ * during a gc run as they are not stored on heap memory.
+ * Also the buffers are stored in a tree map which helps in returning
+ * smallest buffer whose size is just greater than requested length.
+ * This is a thread safe implementation.
+ */
+public final class WeakReferencedElasticByteBufferPool extends 
ElasticByteBufferPool {
+
+  private final TreeMap> directBuffers =
+  new TreeMap<>();
+
+  private final TreeMap> heapBuffers =
+  new TreeMap<>();
+
+  private TreeMap> getBufferTree(boolean 
isDirect) {
+return isDirect ? directBuffers : heapBuffers;
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @param direct whether we want a direct byte buffer or a heap one.
+   * @param length length of requested buffer.
+   * @return returns equal or next greater than capacity buffer from
+   * pool if already available and not garbage collected else creates
+   * a new buffer and return it.
+   */
+  @Override
+  public synchronized ByteBuffer getBuffer(boolean direct, int length) {
+TreeMap> buffersTree = 
getBufferTree(direct);
+
+// Scan the entire tree and remove all weak null references.
+buffersTree.entrySet().removeIf(next -> next.getValue().get() == null);
+
+Map.Entry> entry =
+buffersTree.ceilingEntry(new Key(length, 0));
+// If there is no buffer present in the pool with desired size.
+if (entry == null) {
+  return direct ? ByteBuffer.allocateDirect(length) :
+  ByteBuffer.allocate(length);
+}
+// buffer is available in the pool and not garbage collected.
+WeakReference bufferInPool = entry.getValue();
+buffersTree.remove(entry.getKey());
+ByteBuffer buffer = bufferInPool.get();
+if (buffer != null) {
+  return buffer;
+}
+// buffer was in pool but already got garbage collected.
+return direct ? ByteBuffer.allocateDirect(length) :
+