You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@datasketches.apache.org by "leerho (via GitHub)" <gi...@apache.org> on 2023/01/30 05:59:38 UTC

[GitHub] [datasketches-memory] leerho opened a new pull request, #172: Remove handles

leerho opened a new pull request, #172:
URL: https://github.com/apache/datasketches-memory/pull/172

   This is the next phase of changes required to make the Java8 code compatible with the Java 17 Panama Foreign Function and Memory Access API.  In this phase I had to redesign the hierarchy to eliminate the 8 types of Handles and HandleImpls, because Panama uses a completely different approach.  This touched many of the classes.
   
   Also, in this phase I converted nearly all the "asserts" into "checks" (i.e., thrown exceptions).  And to improve clarity with these exceptions I converted many of the common IllegalArgumentExceptions into dedicated exceptions that more clearly communicate the cause, e.g., MemoryBoundsException,  MemoryCloseException, MemoryInvalidException, ReadOnlyException, etc.  Many of these changes impacted the test code, which had to be updated.  And in that process I also cleaned up the test code, removing some duplicate or meaningless tests.
   
   There is still more work to be done, specifically I have to remove some of the capabilities that Panama does not support, such as getAndAddLong(..), compareAndSwapLong(..), getAndSetLong(..), putCharsToUtf8(..), etc.
   
   Please bear with me as I complete this overall task.
   


-- 
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: commits-unsubscribe@datasketches.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@datasketches.apache.org
For additional commands, e-mail: commits-help@datasketches.apache.org


[GitHub] [datasketches-memory] AlexanderSaydakov commented on a diff in pull request #172: Remove handles

Posted by "AlexanderSaydakov (via GitHub)" <gi...@apache.org>.
AlexanderSaydakov commented on code in PR #172:
URL: https://github.com/apache/datasketches-memory/pull/172#discussion_r1091132592


##########
datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/AllocateDirectWritableMap.java:
##########
@@ -60,4 +161,165 @@ public void force() {
       throw new RuntimeException(String.format("Encountered %s exception in force. " + e.getClass()));
     }
   }
+
+  public StepBoolean getValid() {
+    return deallocator.getValid();
+  }
+
+  public static boolean isFileReadOnly(final File file) {
+    return (!file.canWrite());
+  }
+
+  public boolean isLoaded() {
+    checkValidAndThread();
+    try {
+      final int pageCount = NioBits.pageCount(capacityBytes);
+      return (boolean) MAPPED_BYTE_BUFFER_ISLOADED0_METHOD
+          //isLoaded0 is effectively static, so ZERO_READ_ONLY_DIRECT_BYTE_BUFFER is not modified
+          .invoke(AccessByteBuffer.ZERO_READ_ONLY_DIRECT_BYTE_BUFFER,
+              nativeBaseOffset,
+              capacityBytes,
+              pageCount);
+    } catch (final  IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
+      throw new RuntimeException(
+              String.format("Encountered %s exception while loading", e.getClass()));
+    }
+  }
+
+  public void load() {
+    checkValidAndThread();
+    madvise();
+    // Performance optimization. Read a byte from each page to bring it into memory.
+    final int ps = NioBits.pageSize();
+    final int count = NioBits.pageCount(capacityBytes);
+    long offset = nativeBaseOffset;
+    for (int i = 0; i < count; i++) {
+      unsafe.getByte(offset);
+      offset += ps;
+    }
+  }
+
+  // Private methods
+  /**
+   * called by load(). Calls the native method load0 in MappedByteBuffer.java, implemented
+   * in MappedByteBuffer.c. See reference at top of class. load0 allows setting a mapping length
+   * of greater than 2GB.
+   */
+  private void madvise() {
+    try {
+      MAPPED_BYTE_BUFFER_LOAD0_METHOD
+        //load0 is effectively static, so ZERO_READ_ONLY_DIRECT_BYTE_BUFFER is not modified
+        .invoke(AccessByteBuffer.ZERO_READ_ONLY_DIRECT_BYTE_BUFFER,
+            nativeBaseOffset,
+            capacityBytes);
+    } catch (final  IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
+      throw new RuntimeException(
+          String.format("Encountered %s exception while loading", e.getClass()));
+    }
+  }
+
+  //Does the actual mapping work, resourceReadOnly must already be set
+  private static RandomAccessFile mapper(final File file, final long fileOffset,
+      final long capacityBytes, final boolean resourceReadOnly)  {
+
+    final String mode = resourceReadOnly ? "r" : "rw";
+    final RandomAccessFile raf;
+    try {
+      raf = new RandomAccessFile(file, mode);
+      if (fileOffset + capacityBytes > raf.length()) {
+        raf.setLength(fileOffset + capacityBytes);
+      }
+    } catch (final IOException e) {
+      throw new RuntimeException(e);
+    }
+    return raf;
+  }
+
+  /**
+   * Creates a mapping of the FileChannel starting at position and of size length to pages
+   * in the OS. This may throw OutOfMemory error if you have exhausted memory.
+   * You can try to force garbage collection and re-attempt.
+   *
+   * <p>map0 is a native method of FileChannelImpl.java implemented in FileChannelImpl.c.
+   * See reference at top of class.</p>
+   *
+   * @param fileChannel the FileChannel
+   * @param position the offset in bytes into the FileChannel
+   * @param lengthBytes the length in bytes
+   * @return the native base offset address
+   * @throws RuntimeException Encountered an exception while mapping
+   */
+  private static long map(final FileChannel fileChannel, final boolean resourceReadOnly,
+      final long position, final long lengthBytes) {
+    final int pagePosition = (int) (position % unsafe.pageSize());
+    final long mapPosition = position - pagePosition;
+    final long mapSize = lengthBytes + pagePosition;
+    final int mapMode = resourceReadOnly ? MAP_RO : MAP_RW;
+    //final boolean isSync = true; //required as of JDK14, but it is more complex
+    try {
+      final long nativeBaseOffset = //JDK14 add isSync
+        (long) FILE_CHANNEL_IMPL_MAP0_METHOD.invoke(fileChannel, mapMode, mapPosition, mapSize);
+      return nativeBaseOffset;
+    } catch (final InvocationTargetException e) {
+      throw new RuntimeException("Exception while mapping", e.getTargetException());
+    } catch (final IllegalAccessException e) {
+      throw new RuntimeException("Exception while mapping", e);
+    }
+  }
+
+  private static final class Deallocator implements Runnable {
+    private final RandomAccessFile myRaf;
+    private final FileChannel myFc;
+    //This is the only place the actual native offset is kept for use by unsafe.freeMemory();
+    private final long actualNativeBaseOffset;
+    private final long myCapacity;
+    private final StepBoolean valid = new StepBoolean(true); //only place for this
+
+    Deallocator(final long nativeBaseOffset, final long capacityBytes,
+        final RandomAccessFile raf) {
+      myRaf = raf;
+      assert myRaf != null;
+      myFc = myRaf.getChannel();
+      actualNativeBaseOffset = nativeBaseOffset;
+      assert actualNativeBaseOffset != 0;
+      myCapacity = capacityBytes;
+      assert myCapacity != 0;
+    }
+
+    StepBoolean getValid() {
+      return valid;
+    }
+
+    @Override
+    public void run() throws MemoryCloseException {
+      deallocate(true);
+    }
+
+    boolean deallocate(final boolean calledFromCleaner) throws MemoryCloseException {
+      if (valid.change()) {
+        if (calledFromCleaner) {
+          // Warn about non-deterministic resource cleanup.
+          LOG.warning("A WritableMapHandleImpl was not closed manually");
+        }
+        unmap();
+        return true;
+      }
+      return false;
+    }
+
+    /**
+     * Removes existing mapping.  <i>unmap0</i> is a native method in FileChannelImpl.c. See
+     * reference at top of class.
+     */
+    private void unmap() throws MemoryCloseException {
+      try {
+        FILE_CHANNEL_IMPL_UNMAP0_METHOD.invoke(myFc, actualNativeBaseOffset, myCapacity);
+        myRaf.close();
+      } catch (final IllegalAccessException | IllegalArgumentException | InvocationTargetException | IOException e) {
+        throw new MemoryCloseException(
+            String.format("Encountered %s exception while freeing memory", e.getClass()));
+      }
+    }
+  } //End of class Deallocator

Review Comment:
   seems like a leftover from some previous incarnation



-- 
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: commits-unsubscribe@datasketches.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@datasketches.apache.org
For additional commands, e-mail: commits-help@datasketches.apache.org


[GitHub] [datasketches-memory] leerho commented on a diff in pull request #172: Remove handles

Posted by "leerho (via GitHub)" <gi...@apache.org>.
leerho commented on code in PR #172:
URL: https://github.com/apache/datasketches-memory/pull/172#discussion_r1091212121


##########
datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/AllocateDirectWritableMap.java:
##########
@@ -60,4 +161,165 @@ public void force() {
       throw new RuntimeException(String.format("Encountered %s exception in force. " + e.getClass()));
     }
   }
+
+  public StepBoolean getValid() {
+    return deallocator.getValid();
+  }
+
+  public static boolean isFileReadOnly(final File file) {
+    return (!file.canWrite());
+  }
+
+  public boolean isLoaded() {
+    checkValidAndThread();
+    try {
+      final int pageCount = NioBits.pageCount(capacityBytes);
+      return (boolean) MAPPED_BYTE_BUFFER_ISLOADED0_METHOD
+          //isLoaded0 is effectively static, so ZERO_READ_ONLY_DIRECT_BYTE_BUFFER is not modified
+          .invoke(AccessByteBuffer.ZERO_READ_ONLY_DIRECT_BYTE_BUFFER,
+              nativeBaseOffset,
+              capacityBytes,
+              pageCount);
+    } catch (final  IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
+      throw new RuntimeException(
+              String.format("Encountered %s exception while loading", e.getClass()));
+    }
+  }
+
+  public void load() {
+    checkValidAndThread();
+    madvise();
+    // Performance optimization. Read a byte from each page to bring it into memory.
+    final int ps = NioBits.pageSize();
+    final int count = NioBits.pageCount(capacityBytes);
+    long offset = nativeBaseOffset;
+    for (int i = 0; i < count; i++) {
+      unsafe.getByte(offset);
+      offset += ps;
+    }
+  }
+
+  // Private methods
+  /**
+   * called by load(). Calls the native method load0 in MappedByteBuffer.java, implemented
+   * in MappedByteBuffer.c. See reference at top of class. load0 allows setting a mapping length
+   * of greater than 2GB.
+   */
+  private void madvise() {
+    try {
+      MAPPED_BYTE_BUFFER_LOAD0_METHOD
+        //load0 is effectively static, so ZERO_READ_ONLY_DIRECT_BYTE_BUFFER is not modified
+        .invoke(AccessByteBuffer.ZERO_READ_ONLY_DIRECT_BYTE_BUFFER,
+            nativeBaseOffset,
+            capacityBytes);
+    } catch (final  IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
+      throw new RuntimeException(
+          String.format("Encountered %s exception while loading", e.getClass()));
+    }
+  }
+
+  //Does the actual mapping work, resourceReadOnly must already be set
+  private static RandomAccessFile mapper(final File file, final long fileOffset,
+      final long capacityBytes, final boolean resourceReadOnly)  {
+
+    final String mode = resourceReadOnly ? "r" : "rw";
+    final RandomAccessFile raf;
+    try {
+      raf = new RandomAccessFile(file, mode);
+      if (fileOffset + capacityBytes > raf.length()) {
+        raf.setLength(fileOffset + capacityBytes);
+      }
+    } catch (final IOException e) {
+      throw new RuntimeException(e);
+    }
+    return raf;
+  }
+
+  /**
+   * Creates a mapping of the FileChannel starting at position and of size length to pages
+   * in the OS. This may throw OutOfMemory error if you have exhausted memory.
+   * You can try to force garbage collection and re-attempt.
+   *
+   * <p>map0 is a native method of FileChannelImpl.java implemented in FileChannelImpl.c.
+   * See reference at top of class.</p>
+   *
+   * @param fileChannel the FileChannel
+   * @param position the offset in bytes into the FileChannel
+   * @param lengthBytes the length in bytes
+   * @return the native base offset address
+   * @throws RuntimeException Encountered an exception while mapping
+   */
+  private static long map(final FileChannel fileChannel, final boolean resourceReadOnly,
+      final long position, final long lengthBytes) {
+    final int pagePosition = (int) (position % unsafe.pageSize());
+    final long mapPosition = position - pagePosition;
+    final long mapSize = lengthBytes + pagePosition;
+    final int mapMode = resourceReadOnly ? MAP_RO : MAP_RW;
+    //final boolean isSync = true; //required as of JDK14, but it is more complex
+    try {
+      final long nativeBaseOffset = //JDK14 add isSync
+        (long) FILE_CHANNEL_IMPL_MAP0_METHOD.invoke(fileChannel, mapMode, mapPosition, mapSize);
+      return nativeBaseOffset;
+    } catch (final InvocationTargetException e) {
+      throw new RuntimeException("Exception while mapping", e.getTargetException());
+    } catch (final IllegalAccessException e) {
+      throw new RuntimeException("Exception while mapping", e);
+    }
+  }
+
+  private static final class Deallocator implements Runnable {
+    private final RandomAccessFile myRaf;
+    private final FileChannel myFc;
+    //This is the only place the actual native offset is kept for use by unsafe.freeMemory();
+    private final long actualNativeBaseOffset;
+    private final long myCapacity;
+    private final StepBoolean valid = new StepBoolean(true); //only place for this
+
+    Deallocator(final long nativeBaseOffset, final long capacityBytes,
+        final RandomAccessFile raf) {
+      myRaf = raf;
+      assert myRaf != null;
+      myFc = myRaf.getChannel();
+      actualNativeBaseOffset = nativeBaseOffset;
+      assert actualNativeBaseOffset != 0;
+      myCapacity = capacityBytes;
+      assert myCapacity != 0;
+    }
+
+    StepBoolean getValid() {
+      return valid;
+    }
+
+    @Override
+    public void run() throws MemoryCloseException {
+      deallocate(true);
+    }
+
+    boolean deallocate(final boolean calledFromCleaner) throws MemoryCloseException {
+      if (valid.change()) {
+        if (calledFromCleaner) {
+          // Warn about non-deterministic resource cleanup.
+          LOG.warning("A WritableMapHandleImpl was not closed manually");
+        }
+        unmap();
+        return true;
+      }
+      return false;
+    }
+
+    /**
+     * Removes existing mapping.  <i>unmap0</i> is a native method in FileChannelImpl.c. See
+     * reference at top of class.
+     */
+    private void unmap() throws MemoryCloseException {
+      try {
+        FILE_CHANNEL_IMPL_UNMAP0_METHOD.invoke(myFc, actualNativeBaseOffset, myCapacity);
+        myRaf.close();
+      } catch (final IllegalAccessException | IllegalArgumentException | InvocationTargetException | IOException e) {
+        throw new MemoryCloseException(
+            String.format("Encountered %s exception while freeing memory", e.getClass()));
+      }
+    }
+  } //End of class Deallocator

Review Comment:
   It IS the end of the Deallocator class. What is wrong with that?



-- 
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: commits-unsubscribe@datasketches.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@datasketches.apache.org
For additional commands, e-mail: commits-help@datasketches.apache.org


[GitHub] [datasketches-memory] AlexanderSaydakov commented on a diff in pull request #172: Remove handles

Posted by "AlexanderSaydakov (via GitHub)" <gi...@apache.org>.
AlexanderSaydakov commented on code in PR #172:
URL: https://github.com/apache/datasketches-memory/pull/172#discussion_r1091155820


##########
datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/AllocateDirectMapMemoryTest.java:
##########
@@ -74,107 +71,72 @@ public void simpleMap2() throws Exception {
       println("");
       Memory mem2 = mem.region(43 + 76, 20);
       println("Mem Cap:       " + mem2.getCapacity());
-      println("Native Off:    " + ((BaseStateImpl)mem).getNativeBaseOffset());
+      println("Native Off:    " + ((ResourceImpl)mem).getNativeBaseOffset());
       println("Offset:        " + mem.getTotalOffset());
-      println("Cum Offset:    " + ((BaseStateImpl)mem2).getCumulativeOffset(0));
+      println("Cum Offset:    " + ((ResourceImpl)mem2).getCumulativeOffset(0));
       println("Total Offset: " + mem2.getTotalOffset());
       StringBuilder sb2 = new StringBuilder();
       mem2.getCharsFromUtf8(0, 12, sb2);
       println(sb2.toString());
-
-      rh.close();
     }
   }
 
   @Test
-  public void testIllegalArguments() throws Exception {
+  public void testIllegalArguments() {
     File file = getResourceFile("GettysburgAddress.txt");
-    try (MapHandle rh = Memory.map(file, -1, Integer.MAX_VALUE, ByteOrder.nativeOrder())) {
-      fail("Failed: testIllegalArgumentException: Position was negative.");
+    try (Memory mem = Memory.map(file, -1, Integer.MAX_VALUE, ByteOrder.nativeOrder())) {
+      fail("Failed: Position was negative.");
     } catch (IllegalArgumentException e) {
       //ok
     }
 
-    try (MapHandle rh = Memory.map(file, 0, -1, ByteOrder.nativeOrder())) {
-      fail("Failed: testIllegalArgumentException: Size was negative.");
+    try (Memory mem = Memory.map(file, 0, -1, ByteOrder.nativeOrder())) {
+      fail("Failed: Size was negative.");
     } catch (IllegalArgumentException e) {
       //ok
     }
   }
 
   @Test
-  public void testMapAndMultipleClose() throws Exception {
+  public void testMapAndMultipleClose() {
     File file = getResourceFile("GettysburgAddress.txt");
     long memCapacity = file.length();
-    try (MapHandle rh = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder())) {
-      Memory map = rh.get();
-      assertEquals(memCapacity, map.getCapacity());
-      rh.close();
-      rh.close();
-      map.getCapacity(); //throws assertion error
-    } catch (AssertionError e) {
-      //OK
+    try (Memory mem = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder())) {
+      assertEquals(memCapacity, mem.getCapacity());
     }
-  }
-
-  @Test
-  public void testReadFailAfterClose() throws Exception {
-    File file = getResourceFile("GettysburgAddress.txt");
-    long memCapacity = file.length();
-    try (MapHandle rh = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder())) {
-      Memory mmf = rh.get();
-      rh.close();
-      mmf.getByte(0);
-    } catch (AssertionError e) {
-      //OK
+    try {
+      Memory mem = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder());
+      mem.close();
+      mem.close(); //redundant close
+    } catch (MemoryCloseException e) { /* OK */ }
+    try {
+      Memory mem = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder());
+      mem.close();
+      mem.getCapacity(); //already closed
     }
+    catch (MemoryInvalidException e) { /* OK */ }
   }
 
   @Test
-  public void testLoad() throws Exception {
+  public void testReadFailAfterClose()  {
     File file = getResourceFile("GettysburgAddress.txt");
     long memCapacity = file.length();
-    try (MapHandle rh = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder())) {
-      rh.load();
-      assertTrue(rh.isLoaded());
-      rh.close();
+    try {
+      Memory mem = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder());
+      mem.close();
+      mem.getByte(0);

Review Comment:
   should this throw? if so, I would expect to see fail() right after



-- 
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: commits-unsubscribe@datasketches.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@datasketches.apache.org
For additional commands, e-mail: commits-help@datasketches.apache.org


[GitHub] [datasketches-memory] AlexanderSaydakov commented on a diff in pull request #172: Remove handles

Posted by "AlexanderSaydakov (via GitHub)" <gi...@apache.org>.
AlexanderSaydakov commented on code in PR #172:
URL: https://github.com/apache/datasketches-memory/pull/172#discussion_r1091171921


##########
datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/AllocateDirectWritableMap.java:
##########
@@ -60,4 +161,165 @@ public void force() {
       throw new RuntimeException(String.format("Encountered %s exception in force. " + e.getClass()));
     }
   }
+
+  public StepBoolean getValid() {
+    return deallocator.getValid();
+  }
+
+  public static boolean isFileReadOnly(final File file) {
+    return (!file.canWrite());
+  }
+
+  public boolean isLoaded() {
+    checkValidAndThread();
+    try {
+      final int pageCount = NioBits.pageCount(capacityBytes);
+      return (boolean) MAPPED_BYTE_BUFFER_ISLOADED0_METHOD
+          //isLoaded0 is effectively static, so ZERO_READ_ONLY_DIRECT_BYTE_BUFFER is not modified
+          .invoke(AccessByteBuffer.ZERO_READ_ONLY_DIRECT_BYTE_BUFFER,
+              nativeBaseOffset,
+              capacityBytes,
+              pageCount);
+    } catch (final  IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
+      throw new RuntimeException(
+              String.format("Encountered %s exception while loading", e.getClass()));
+    }
+  }
+
+  public void load() {
+    checkValidAndThread();
+    madvise();
+    // Performance optimization. Read a byte from each page to bring it into memory.
+    final int ps = NioBits.pageSize();
+    final int count = NioBits.pageCount(capacityBytes);
+    long offset = nativeBaseOffset;
+    for (int i = 0; i < count; i++) {
+      unsafe.getByte(offset);
+      offset += ps;
+    }
+  }
+
+  // Private methods
+  /**
+   * called by load(). Calls the native method load0 in MappedByteBuffer.java, implemented
+   * in MappedByteBuffer.c. See reference at top of class. load0 allows setting a mapping length
+   * of greater than 2GB.
+   */
+  private void madvise() {
+    try {
+      MAPPED_BYTE_BUFFER_LOAD0_METHOD
+        //load0 is effectively static, so ZERO_READ_ONLY_DIRECT_BYTE_BUFFER is not modified
+        .invoke(AccessByteBuffer.ZERO_READ_ONLY_DIRECT_BYTE_BUFFER,
+            nativeBaseOffset,
+            capacityBytes);
+    } catch (final  IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
+      throw new RuntimeException(
+          String.format("Encountered %s exception while loading", e.getClass()));
+    }
+  }
+
+  //Does the actual mapping work, resourceReadOnly must already be set
+  private static RandomAccessFile mapper(final File file, final long fileOffset,
+      final long capacityBytes, final boolean resourceReadOnly)  {
+
+    final String mode = resourceReadOnly ? "r" : "rw";
+    final RandomAccessFile raf;
+    try {
+      raf = new RandomAccessFile(file, mode);
+      if (fileOffset + capacityBytes > raf.length()) {
+        raf.setLength(fileOffset + capacityBytes);
+      }
+    } catch (final IOException e) {
+      throw new RuntimeException(e);
+    }
+    return raf;
+  }
+
+  /**
+   * Creates a mapping of the FileChannel starting at position and of size length to pages
+   * in the OS. This may throw OutOfMemory error if you have exhausted memory.
+   * You can try to force garbage collection and re-attempt.
+   *
+   * <p>map0 is a native method of FileChannelImpl.java implemented in FileChannelImpl.c.
+   * See reference at top of class.</p>
+   *
+   * @param fileChannel the FileChannel
+   * @param position the offset in bytes into the FileChannel
+   * @param lengthBytes the length in bytes
+   * @return the native base offset address
+   * @throws RuntimeException Encountered an exception while mapping
+   */
+  private static long map(final FileChannel fileChannel, final boolean resourceReadOnly,
+      final long position, final long lengthBytes) {
+    final int pagePosition = (int) (position % unsafe.pageSize());
+    final long mapPosition = position - pagePosition;
+    final long mapSize = lengthBytes + pagePosition;
+    final int mapMode = resourceReadOnly ? MAP_RO : MAP_RW;
+    //final boolean isSync = true; //required as of JDK14, but it is more complex
+    try {
+      final long nativeBaseOffset = //JDK14 add isSync
+        (long) FILE_CHANNEL_IMPL_MAP0_METHOD.invoke(fileChannel, mapMode, mapPosition, mapSize);
+      return nativeBaseOffset;
+    } catch (final InvocationTargetException e) {
+      throw new RuntimeException("Exception while mapping", e.getTargetException());
+    } catch (final IllegalAccessException e) {
+      throw new RuntimeException("Exception while mapping", e);
+    }
+  }
+
+  private static final class Deallocator implements Runnable {
+    private final RandomAccessFile myRaf;
+    private final FileChannel myFc;
+    //This is the only place the actual native offset is kept for use by unsafe.freeMemory();
+    private final long actualNativeBaseOffset;
+    private final long myCapacity;
+    private final StepBoolean valid = new StepBoolean(true); //only place for this
+
+    Deallocator(final long nativeBaseOffset, final long capacityBytes,
+        final RandomAccessFile raf) {
+      myRaf = raf;
+      assert myRaf != null;
+      myFc = myRaf.getChannel();
+      actualNativeBaseOffset = nativeBaseOffset;
+      assert actualNativeBaseOffset != 0;
+      myCapacity = capacityBytes;
+      assert myCapacity != 0;
+    }
+
+    StepBoolean getValid() {
+      return valid;
+    }
+
+    @Override
+    public void run() throws MemoryCloseException {
+      deallocate(true);
+    }
+
+    boolean deallocate(final boolean calledFromCleaner) throws MemoryCloseException {
+      if (valid.change()) {
+        if (calledFromCleaner) {
+          // Warn about non-deterministic resource cleanup.
+          LOG.warning("A WritableMapHandleImpl was not closed manually");
+        }
+        unmap();
+        return true;
+      }
+      return false;
+    }
+
+    /**
+     * Removes existing mapping.  <i>unmap0</i> is a native method in FileChannelImpl.c. See
+     * reference at top of class.
+     */
+    private void unmap() throws MemoryCloseException {
+      try {
+        FILE_CHANNEL_IMPL_UNMAP0_METHOD.invoke(myFc, actualNativeBaseOffset, myCapacity);
+        myRaf.close();
+      } catch (final IllegalAccessException | IllegalArgumentException | InvocationTargetException | IOException e) {
+        throw new MemoryCloseException(
+            String.format("Encountered %s exception while freeing memory", e.getClass()));
+      }
+    }
+  } //End of class Deallocator

Review Comment:
   I am referring to the comment //End of class Deallocator



-- 
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: commits-unsubscribe@datasketches.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@datasketches.apache.org
For additional commands, e-mail: commits-help@datasketches.apache.org


[GitHub] [datasketches-memory] AlexanderSaydakov commented on a diff in pull request #172: Remove handles

Posted by "AlexanderSaydakov (via GitHub)" <gi...@apache.org>.
AlexanderSaydakov commented on code in PR #172:
URL: https://github.com/apache/datasketches-memory/pull/172#discussion_r1091244821


##########
datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/AllocateDirectWritableMap.java:
##########
@@ -60,4 +161,165 @@ public void force() {
       throw new RuntimeException(String.format("Encountered %s exception in force. " + e.getClass()));
     }
   }
+
+  public StepBoolean getValid() {
+    return deallocator.getValid();
+  }
+
+  public static boolean isFileReadOnly(final File file) {
+    return (!file.canWrite());
+  }
+
+  public boolean isLoaded() {
+    checkValidAndThread();
+    try {
+      final int pageCount = NioBits.pageCount(capacityBytes);
+      return (boolean) MAPPED_BYTE_BUFFER_ISLOADED0_METHOD
+          //isLoaded0 is effectively static, so ZERO_READ_ONLY_DIRECT_BYTE_BUFFER is not modified
+          .invoke(AccessByteBuffer.ZERO_READ_ONLY_DIRECT_BYTE_BUFFER,
+              nativeBaseOffset,
+              capacityBytes,
+              pageCount);
+    } catch (final  IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
+      throw new RuntimeException(
+              String.format("Encountered %s exception while loading", e.getClass()));
+    }
+  }
+
+  public void load() {
+    checkValidAndThread();
+    madvise();
+    // Performance optimization. Read a byte from each page to bring it into memory.
+    final int ps = NioBits.pageSize();
+    final int count = NioBits.pageCount(capacityBytes);
+    long offset = nativeBaseOffset;
+    for (int i = 0; i < count; i++) {
+      unsafe.getByte(offset);
+      offset += ps;
+    }
+  }
+
+  // Private methods
+  /**
+   * called by load(). Calls the native method load0 in MappedByteBuffer.java, implemented
+   * in MappedByteBuffer.c. See reference at top of class. load0 allows setting a mapping length
+   * of greater than 2GB.
+   */
+  private void madvise() {
+    try {
+      MAPPED_BYTE_BUFFER_LOAD0_METHOD
+        //load0 is effectively static, so ZERO_READ_ONLY_DIRECT_BYTE_BUFFER is not modified
+        .invoke(AccessByteBuffer.ZERO_READ_ONLY_DIRECT_BYTE_BUFFER,
+            nativeBaseOffset,
+            capacityBytes);
+    } catch (final  IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
+      throw new RuntimeException(
+          String.format("Encountered %s exception while loading", e.getClass()));
+    }
+  }
+
+  //Does the actual mapping work, resourceReadOnly must already be set
+  private static RandomAccessFile mapper(final File file, final long fileOffset,
+      final long capacityBytes, final boolean resourceReadOnly)  {
+
+    final String mode = resourceReadOnly ? "r" : "rw";
+    final RandomAccessFile raf;
+    try {
+      raf = new RandomAccessFile(file, mode);
+      if (fileOffset + capacityBytes > raf.length()) {
+        raf.setLength(fileOffset + capacityBytes);
+      }
+    } catch (final IOException e) {
+      throw new RuntimeException(e);
+    }
+    return raf;
+  }
+
+  /**
+   * Creates a mapping of the FileChannel starting at position and of size length to pages
+   * in the OS. This may throw OutOfMemory error if you have exhausted memory.
+   * You can try to force garbage collection and re-attempt.
+   *
+   * <p>map0 is a native method of FileChannelImpl.java implemented in FileChannelImpl.c.
+   * See reference at top of class.</p>
+   *
+   * @param fileChannel the FileChannel
+   * @param position the offset in bytes into the FileChannel
+   * @param lengthBytes the length in bytes
+   * @return the native base offset address
+   * @throws RuntimeException Encountered an exception while mapping
+   */
+  private static long map(final FileChannel fileChannel, final boolean resourceReadOnly,
+      final long position, final long lengthBytes) {
+    final int pagePosition = (int) (position % unsafe.pageSize());
+    final long mapPosition = position - pagePosition;
+    final long mapSize = lengthBytes + pagePosition;
+    final int mapMode = resourceReadOnly ? MAP_RO : MAP_RW;
+    //final boolean isSync = true; //required as of JDK14, but it is more complex
+    try {
+      final long nativeBaseOffset = //JDK14 add isSync
+        (long) FILE_CHANNEL_IMPL_MAP0_METHOD.invoke(fileChannel, mapMode, mapPosition, mapSize);
+      return nativeBaseOffset;
+    } catch (final InvocationTargetException e) {
+      throw new RuntimeException("Exception while mapping", e.getTargetException());
+    } catch (final IllegalAccessException e) {
+      throw new RuntimeException("Exception while mapping", e);
+    }
+  }
+
+  private static final class Deallocator implements Runnable {
+    private final RandomAccessFile myRaf;
+    private final FileChannel myFc;
+    //This is the only place the actual native offset is kept for use by unsafe.freeMemory();
+    private final long actualNativeBaseOffset;
+    private final long myCapacity;
+    private final StepBoolean valid = new StepBoolean(true); //only place for this
+
+    Deallocator(final long nativeBaseOffset, final long capacityBytes,
+        final RandomAccessFile raf) {
+      myRaf = raf;
+      assert myRaf != null;
+      myFc = myRaf.getChannel();
+      actualNativeBaseOffset = nativeBaseOffset;
+      assert actualNativeBaseOffset != 0;
+      myCapacity = capacityBytes;
+      assert myCapacity != 0;
+    }
+
+    StepBoolean getValid() {
+      return valid;
+    }
+
+    @Override
+    public void run() throws MemoryCloseException {
+      deallocate(true);
+    }
+
+    boolean deallocate(final boolean calledFromCleaner) throws MemoryCloseException {
+      if (valid.change()) {
+        if (calledFromCleaner) {
+          // Warn about non-deterministic resource cleanup.
+          LOG.warning("A WritableMapHandleImpl was not closed manually");
+        }
+        unmap();
+        return true;
+      }
+      return false;
+    }
+
+    /**
+     * Removes existing mapping.  <i>unmap0</i> is a native method in FileChannelImpl.c. See
+     * reference at top of class.
+     */
+    private void unmap() throws MemoryCloseException {
+      try {
+        FILE_CHANNEL_IMPL_UNMAP0_METHOD.invoke(myFc, actualNativeBaseOffset, myCapacity);
+        myRaf.close();
+      } catch (final IllegalAccessException | IllegalArgumentException | InvocationTargetException | IOException e) {
+        throw new MemoryCloseException(
+            String.format("Encountered %s exception while freeing memory", e.getClass()));
+      }
+    }
+  } //End of class Deallocator

Review Comment:
   oh, right, an inner class, sorry



-- 
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: commits-unsubscribe@datasketches.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@datasketches.apache.org
For additional commands, e-mail: commits-help@datasketches.apache.org


[GitHub] [datasketches-memory] leerho commented on a diff in pull request #172: Remove handles

Posted by "leerho (via GitHub)" <gi...@apache.org>.
leerho commented on code in PR #172:
URL: https://github.com/apache/datasketches-memory/pull/172#discussion_r1091177853


##########
datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/AllocateDirectMapMemoryTest.java:
##########
@@ -74,107 +71,72 @@ public void simpleMap2() throws Exception {
       println("");
       Memory mem2 = mem.region(43 + 76, 20);
       println("Mem Cap:       " + mem2.getCapacity());
-      println("Native Off:    " + ((BaseStateImpl)mem).getNativeBaseOffset());
+      println("Native Off:    " + ((ResourceImpl)mem).getNativeBaseOffset());
       println("Offset:        " + mem.getTotalOffset());
-      println("Cum Offset:    " + ((BaseStateImpl)mem2).getCumulativeOffset(0));
+      println("Cum Offset:    " + ((ResourceImpl)mem2).getCumulativeOffset(0));
       println("Total Offset: " + mem2.getTotalOffset());
       StringBuilder sb2 = new StringBuilder();
       mem2.getCharsFromUtf8(0, 12, sb2);
       println(sb2.toString());
-
-      rh.close();
     }
   }
 
   @Test
-  public void testIllegalArguments() throws Exception {
+  public void testIllegalArguments() {
     File file = getResourceFile("GettysburgAddress.txt");
-    try (MapHandle rh = Memory.map(file, -1, Integer.MAX_VALUE, ByteOrder.nativeOrder())) {
-      fail("Failed: testIllegalArgumentException: Position was negative.");
+    try (Memory mem = Memory.map(file, -1, Integer.MAX_VALUE, ByteOrder.nativeOrder())) {
+      fail("Failed: Position was negative.");
     } catch (IllegalArgumentException e) {
       //ok
     }
 
-    try (MapHandle rh = Memory.map(file, 0, -1, ByteOrder.nativeOrder())) {
-      fail("Failed: testIllegalArgumentException: Size was negative.");
+    try (Memory mem = Memory.map(file, 0, -1, ByteOrder.nativeOrder())) {
+      fail("Failed: Size was negative.");
     } catch (IllegalArgumentException e) {
       //ok
     }
   }
 
   @Test
-  public void testMapAndMultipleClose() throws Exception {
+  public void testMapAndMultipleClose() {
     File file = getResourceFile("GettysburgAddress.txt");
     long memCapacity = file.length();
-    try (MapHandle rh = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder())) {
-      Memory map = rh.get();
-      assertEquals(memCapacity, map.getCapacity());
-      rh.close();
-      rh.close();
-      map.getCapacity(); //throws assertion error
-    } catch (AssertionError e) {
-      //OK
+    try (Memory mem = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder())) {
+      assertEquals(memCapacity, mem.getCapacity());
     }
-  }
-
-  @Test
-  public void testReadFailAfterClose() throws Exception {
-    File file = getResourceFile("GettysburgAddress.txt");
-    long memCapacity = file.length();
-    try (MapHandle rh = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder())) {
-      Memory mmf = rh.get();
-      rh.close();
-      mmf.getByte(0);
-    } catch (AssertionError e) {
-      //OK
+    try {
+      Memory mem = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder());
+      mem.close();
+      mem.close(); //redundant close
+    } catch (MemoryCloseException e) { /* OK */ }
+    try {
+      Memory mem = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder());
+      mem.close();
+      mem.getCapacity(); //already closed
     }
+    catch (MemoryInvalidException e) { /* OK */ }
   }
 
   @Test
-  public void testLoad() throws Exception {
+  public void testReadFailAfterClose()  {
     File file = getResourceFile("GettysburgAddress.txt");
     long memCapacity = file.length();
-    try (MapHandle rh = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder())) {
-      rh.load();
-      assertTrue(rh.isLoaded());
-      rh.close();
+    try {
+      Memory mem = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder());
+      mem.close();
+      mem.getByte(0);

Review Comment:
   Fixed.  I will search for all similar cases to these.



-- 
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: commits-unsubscribe@datasketches.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@datasketches.apache.org
For additional commands, e-mail: commits-help@datasketches.apache.org


[GitHub] [datasketches-memory] leerho commented on a diff in pull request #172: Remove handles

Posted by "leerho (via GitHub)" <gi...@apache.org>.
leerho commented on code in PR #172:
URL: https://github.com/apache/datasketches-memory/pull/172#discussion_r1091164237


##########
datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/AllocateDirectWritableMap.java:
##########
@@ -60,4 +161,165 @@ public void force() {
       throw new RuntimeException(String.format("Encountered %s exception in force. " + e.getClass()));
     }
   }
+
+  public StepBoolean getValid() {
+    return deallocator.getValid();
+  }
+
+  public static boolean isFileReadOnly(final File file) {
+    return (!file.canWrite());
+  }
+
+  public boolean isLoaded() {
+    checkValidAndThread();
+    try {
+      final int pageCount = NioBits.pageCount(capacityBytes);
+      return (boolean) MAPPED_BYTE_BUFFER_ISLOADED0_METHOD
+          //isLoaded0 is effectively static, so ZERO_READ_ONLY_DIRECT_BYTE_BUFFER is not modified
+          .invoke(AccessByteBuffer.ZERO_READ_ONLY_DIRECT_BYTE_BUFFER,
+              nativeBaseOffset,
+              capacityBytes,
+              pageCount);
+    } catch (final  IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
+      throw new RuntimeException(
+              String.format("Encountered %s exception while loading", e.getClass()));
+    }
+  }
+
+  public void load() {
+    checkValidAndThread();
+    madvise();
+    // Performance optimization. Read a byte from each page to bring it into memory.
+    final int ps = NioBits.pageSize();
+    final int count = NioBits.pageCount(capacityBytes);
+    long offset = nativeBaseOffset;
+    for (int i = 0; i < count; i++) {
+      unsafe.getByte(offset);
+      offset += ps;
+    }
+  }
+
+  // Private methods
+  /**
+   * called by load(). Calls the native method load0 in MappedByteBuffer.java, implemented
+   * in MappedByteBuffer.c. See reference at top of class. load0 allows setting a mapping length
+   * of greater than 2GB.
+   */
+  private void madvise() {
+    try {
+      MAPPED_BYTE_BUFFER_LOAD0_METHOD
+        //load0 is effectively static, so ZERO_READ_ONLY_DIRECT_BYTE_BUFFER is not modified
+        .invoke(AccessByteBuffer.ZERO_READ_ONLY_DIRECT_BYTE_BUFFER,
+            nativeBaseOffset,
+            capacityBytes);
+    } catch (final  IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
+      throw new RuntimeException(
+          String.format("Encountered %s exception while loading", e.getClass()));
+    }
+  }
+
+  //Does the actual mapping work, resourceReadOnly must already be set
+  private static RandomAccessFile mapper(final File file, final long fileOffset,
+      final long capacityBytes, final boolean resourceReadOnly)  {
+
+    final String mode = resourceReadOnly ? "r" : "rw";
+    final RandomAccessFile raf;
+    try {
+      raf = new RandomAccessFile(file, mode);
+      if (fileOffset + capacityBytes > raf.length()) {
+        raf.setLength(fileOffset + capacityBytes);
+      }
+    } catch (final IOException e) {
+      throw new RuntimeException(e);
+    }
+    return raf;
+  }
+
+  /**
+   * Creates a mapping of the FileChannel starting at position and of size length to pages
+   * in the OS. This may throw OutOfMemory error if you have exhausted memory.
+   * You can try to force garbage collection and re-attempt.
+   *
+   * <p>map0 is a native method of FileChannelImpl.java implemented in FileChannelImpl.c.
+   * See reference at top of class.</p>
+   *
+   * @param fileChannel the FileChannel
+   * @param position the offset in bytes into the FileChannel
+   * @param lengthBytes the length in bytes
+   * @return the native base offset address
+   * @throws RuntimeException Encountered an exception while mapping
+   */
+  private static long map(final FileChannel fileChannel, final boolean resourceReadOnly,
+      final long position, final long lengthBytes) {
+    final int pagePosition = (int) (position % unsafe.pageSize());
+    final long mapPosition = position - pagePosition;
+    final long mapSize = lengthBytes + pagePosition;
+    final int mapMode = resourceReadOnly ? MAP_RO : MAP_RW;
+    //final boolean isSync = true; //required as of JDK14, but it is more complex
+    try {
+      final long nativeBaseOffset = //JDK14 add isSync
+        (long) FILE_CHANNEL_IMPL_MAP0_METHOD.invoke(fileChannel, mapMode, mapPosition, mapSize);
+      return nativeBaseOffset;
+    } catch (final InvocationTargetException e) {
+      throw new RuntimeException("Exception while mapping", e.getTargetException());
+    } catch (final IllegalAccessException e) {
+      throw new RuntimeException("Exception while mapping", e);
+    }
+  }
+
+  private static final class Deallocator implements Runnable {
+    private final RandomAccessFile myRaf;
+    private final FileChannel myFc;
+    //This is the only place the actual native offset is kept for use by unsafe.freeMemory();
+    private final long actualNativeBaseOffset;
+    private final long myCapacity;
+    private final StepBoolean valid = new StepBoolean(true); //only place for this
+
+    Deallocator(final long nativeBaseOffset, final long capacityBytes,
+        final RandomAccessFile raf) {
+      myRaf = raf;
+      assert myRaf != null;
+      myFc = myRaf.getChannel();
+      actualNativeBaseOffset = nativeBaseOffset;
+      assert actualNativeBaseOffset != 0;
+      myCapacity = capacityBytes;
+      assert myCapacity != 0;
+    }
+
+    StepBoolean getValid() {
+      return valid;
+    }
+
+    @Override
+    public void run() throws MemoryCloseException {
+      deallocate(true);
+    }
+
+    boolean deallocate(final boolean calledFromCleaner) throws MemoryCloseException {
+      if (valid.change()) {
+        if (calledFromCleaner) {
+          // Warn about non-deterministic resource cleanup.
+          LOG.warning("A WritableMapHandleImpl was not closed manually");
+        }
+        unmap();
+        return true;
+      }
+      return false;
+    }
+
+    /**
+     * Removes existing mapping.  <i>unmap0</i> is a native method in FileChannelImpl.c. See
+     * reference at top of class.
+     */
+    private void unmap() throws MemoryCloseException {
+      try {
+        FILE_CHANNEL_IMPL_UNMAP0_METHOD.invoke(myFc, actualNativeBaseOffset, myCapacity);
+        myRaf.close();
+      } catch (final IllegalAccessException | IllegalArgumentException | InvocationTargetException | IOException e) {
+        throw new MemoryCloseException(
+            String.format("Encountered %s exception while freeing memory", e.getClass()));
+      }
+    }
+  } //End of class Deallocator

Review Comment:
   There is nothing "leftover" here.  This is all part of the actual closing process.



-- 
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: commits-unsubscribe@datasketches.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@datasketches.apache.org
For additional commands, e-mail: commits-help@datasketches.apache.org


[GitHub] [datasketches-memory] leerho commented on a diff in pull request #172: Remove handles

Posted by "leerho (via GitHub)" <gi...@apache.org>.
leerho commented on code in PR #172:
URL: https://github.com/apache/datasketches-memory/pull/172#discussion_r1091175771


##########
datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/AllocateDirectMapMemoryTest.java:
##########
@@ -74,107 +71,72 @@ public void simpleMap2() throws Exception {
       println("");
       Memory mem2 = mem.region(43 + 76, 20);
       println("Mem Cap:       " + mem2.getCapacity());
-      println("Native Off:    " + ((BaseStateImpl)mem).getNativeBaseOffset());
+      println("Native Off:    " + ((ResourceImpl)mem).getNativeBaseOffset());
       println("Offset:        " + mem.getTotalOffset());
-      println("Cum Offset:    " + ((BaseStateImpl)mem2).getCumulativeOffset(0));
+      println("Cum Offset:    " + ((ResourceImpl)mem2).getCumulativeOffset(0));
       println("Total Offset: " + mem2.getTotalOffset());
       StringBuilder sb2 = new StringBuilder();
       mem2.getCharsFromUtf8(0, 12, sb2);
       println(sb2.toString());
-
-      rh.close();
     }
   }
 
   @Test
-  public void testIllegalArguments() throws Exception {
+  public void testIllegalArguments() {
     File file = getResourceFile("GettysburgAddress.txt");
-    try (MapHandle rh = Memory.map(file, -1, Integer.MAX_VALUE, ByteOrder.nativeOrder())) {
-      fail("Failed: testIllegalArgumentException: Position was negative.");
+    try (Memory mem = Memory.map(file, -1, Integer.MAX_VALUE, ByteOrder.nativeOrder())) {
+      fail("Failed: Position was negative.");
     } catch (IllegalArgumentException e) {
       //ok
     }
 
-    try (MapHandle rh = Memory.map(file, 0, -1, ByteOrder.nativeOrder())) {
-      fail("Failed: testIllegalArgumentException: Size was negative.");
+    try (Memory mem = Memory.map(file, 0, -1, ByteOrder.nativeOrder())) {
+      fail("Failed: Size was negative.");
     } catch (IllegalArgumentException e) {
       //ok
     }
   }
 
   @Test
-  public void testMapAndMultipleClose() throws Exception {
+  public void testMapAndMultipleClose() {
     File file = getResourceFile("GettysburgAddress.txt");
     long memCapacity = file.length();
-    try (MapHandle rh = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder())) {
-      Memory map = rh.get();
-      assertEquals(memCapacity, map.getCapacity());
-      rh.close();
-      rh.close();
-      map.getCapacity(); //throws assertion error
-    } catch (AssertionError e) {
-      //OK
+    try (Memory mem = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder())) {
+      assertEquals(memCapacity, mem.getCapacity());
     }
-  }
-
-  @Test
-  public void testReadFailAfterClose() throws Exception {
-    File file = getResourceFile("GettysburgAddress.txt");
-    long memCapacity = file.length();
-    try (MapHandle rh = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder())) {
-      Memory mmf = rh.get();
-      rh.close();
-      mmf.getByte(0);
-    } catch (AssertionError e) {
-      //OK
+    try {
+      Memory mem = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder());
+      mem.close();
+      mem.close(); //redundant close
+    } catch (MemoryCloseException e) { /* OK */ }
+    try {
+      Memory mem = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder());
+      mem.close();
+      mem.getCapacity(); //already closed

Review Comment:
   Fixed.



-- 
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: commits-unsubscribe@datasketches.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@datasketches.apache.org
For additional commands, e-mail: commits-help@datasketches.apache.org


[GitHub] [datasketches-memory] AlexanderSaydakov commented on a diff in pull request #172: Remove handles

Posted by "AlexanderSaydakov (via GitHub)" <gi...@apache.org>.
AlexanderSaydakov commented on code in PR #172:
URL: https://github.com/apache/datasketches-memory/pull/172#discussion_r1091149369


##########
datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/AllocateDirectMapMemoryTest.java:
##########
@@ -25,47 +25,44 @@
 
 import static org.apache.datasketches.memory.internal.Util.getResourceFile;
 import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertFalse;
 import static org.testng.Assert.assertTrue;
 import static org.testng.Assert.fail;
 
 import java.io.File;
 import java.nio.ByteOrder;
 
-import org.apache.datasketches.memory.MapHandle;
 import org.apache.datasketches.memory.Memory;
-import org.testng.annotations.AfterClass;
+import org.apache.datasketches.memory.MemoryCloseException;
+import org.apache.datasketches.memory.MemoryInvalidException;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 public class AllocateDirectMapMemoryTest {
   private static final String LS = System.getProperty("line.separator");
-  MapHandle hand = null;
 
   @BeforeClass
   public void setReadOnly() {
     UtilTest.setGettysburgAddressFileToReadOnly();
   }
 
   @Test
-  public void simpleMap() throws Exception {
+  public void simpleMap() {
     File file = getResourceFile("GettysburgAddress.txt");
-    assertTrue(AllocateDirectMap.isFileReadOnly(file));
-    try (MapHandle rh = Memory.map(file)) {
-      rh.close();
-    }
+    assertTrue(AllocateDirectWritableMap.isFileReadOnly(file));
+    try (Memory mem = Memory.map(file)) {
+      mem.close(); //redundant close

Review Comment:
   is this is to test if a duplicate close throws? if so, why there is no action if it does not?



-- 
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: commits-unsubscribe@datasketches.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@datasketches.apache.org
For additional commands, e-mail: commits-help@datasketches.apache.org


[GitHub] [datasketches-memory] AlexanderSaydakov commented on a diff in pull request #172: Remove handles

Posted by "AlexanderSaydakov (via GitHub)" <gi...@apache.org>.
AlexanderSaydakov commented on code in PR #172:
URL: https://github.com/apache/datasketches-memory/pull/172#discussion_r1091145452


##########
datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/AllocateDirectMapMemoryTest.java:
##########
@@ -25,47 +25,44 @@
 
 import static org.apache.datasketches.memory.internal.Util.getResourceFile;
 import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertFalse;
 import static org.testng.Assert.assertTrue;
 import static org.testng.Assert.fail;
 
 import java.io.File;
 import java.nio.ByteOrder;
 
-import org.apache.datasketches.memory.MapHandle;
 import org.apache.datasketches.memory.Memory;
-import org.testng.annotations.AfterClass;
+import org.apache.datasketches.memory.MemoryCloseException;
+import org.apache.datasketches.memory.MemoryInvalidException;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 public class AllocateDirectMapMemoryTest {
   private static final String LS = System.getProperty("line.separator");
-  MapHandle hand = null;
 
   @BeforeClass
   public void setReadOnly() {
     UtilTest.setGettysburgAddressFileToReadOnly();
   }
 
   @Test
-  public void simpleMap() throws Exception {
+  public void simpleMap() {
     File file = getResourceFile("GettysburgAddress.txt");
-    assertTrue(AllocateDirectMap.isFileReadOnly(file));
-    try (MapHandle rh = Memory.map(file)) {
-      rh.close();
-    }
+    assertTrue(AllocateDirectWritableMap.isFileReadOnly(file));
+    try (Memory mem = Memory.map(file)) {
+      mem.close(); //redundant close

Review Comment:
   I would prefer the comment to explain why



-- 
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: commits-unsubscribe@datasketches.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@datasketches.apache.org
For additional commands, e-mail: commits-help@datasketches.apache.org


[GitHub] [datasketches-memory] AlexanderSaydakov commented on a diff in pull request #172: Remove handles

Posted by "AlexanderSaydakov (via GitHub)" <gi...@apache.org>.
AlexanderSaydakov commented on code in PR #172:
URL: https://github.com/apache/datasketches-memory/pull/172#discussion_r1091153106


##########
datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/AllocateDirectMapMemoryTest.java:
##########
@@ -74,107 +71,72 @@ public void simpleMap2() throws Exception {
       println("");
       Memory mem2 = mem.region(43 + 76, 20);
       println("Mem Cap:       " + mem2.getCapacity());
-      println("Native Off:    " + ((BaseStateImpl)mem).getNativeBaseOffset());
+      println("Native Off:    " + ((ResourceImpl)mem).getNativeBaseOffset());
       println("Offset:        " + mem.getTotalOffset());
-      println("Cum Offset:    " + ((BaseStateImpl)mem2).getCumulativeOffset(0));
+      println("Cum Offset:    " + ((ResourceImpl)mem2).getCumulativeOffset(0));
       println("Total Offset: " + mem2.getTotalOffset());
       StringBuilder sb2 = new StringBuilder();
       mem2.getCharsFromUtf8(0, 12, sb2);
       println(sb2.toString());
-
-      rh.close();
     }
   }
 
   @Test
-  public void testIllegalArguments() throws Exception {
+  public void testIllegalArguments() {
     File file = getResourceFile("GettysburgAddress.txt");
-    try (MapHandle rh = Memory.map(file, -1, Integer.MAX_VALUE, ByteOrder.nativeOrder())) {
-      fail("Failed: testIllegalArgumentException: Position was negative.");
+    try (Memory mem = Memory.map(file, -1, Integer.MAX_VALUE, ByteOrder.nativeOrder())) {
+      fail("Failed: Position was negative.");
     } catch (IllegalArgumentException e) {
       //ok
     }
 
-    try (MapHandle rh = Memory.map(file, 0, -1, ByteOrder.nativeOrder())) {
-      fail("Failed: testIllegalArgumentException: Size was negative.");
+    try (Memory mem = Memory.map(file, 0, -1, ByteOrder.nativeOrder())) {
+      fail("Failed: Size was negative.");
     } catch (IllegalArgumentException e) {
       //ok
     }
   }
 
   @Test
-  public void testMapAndMultipleClose() throws Exception {
+  public void testMapAndMultipleClose() {
     File file = getResourceFile("GettysburgAddress.txt");
     long memCapacity = file.length();
-    try (MapHandle rh = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder())) {
-      Memory map = rh.get();
-      assertEquals(memCapacity, map.getCapacity());
-      rh.close();
-      rh.close();
-      map.getCapacity(); //throws assertion error
-    } catch (AssertionError e) {
-      //OK
+    try (Memory mem = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder())) {
+      assertEquals(memCapacity, mem.getCapacity());
     }
-  }
-
-  @Test
-  public void testReadFailAfterClose() throws Exception {
-    File file = getResourceFile("GettysburgAddress.txt");
-    long memCapacity = file.length();
-    try (MapHandle rh = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder())) {
-      Memory mmf = rh.get();
-      rh.close();
-      mmf.getByte(0);
-    } catch (AssertionError e) {
-      //OK
+    try {
+      Memory mem = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder());
+      mem.close();
+      mem.close(); //redundant close
+    } catch (MemoryCloseException e) { /* OK */ }
+    try {
+      Memory mem = Memory.map(file, 0, memCapacity, ByteOrder.nativeOrder());
+      mem.close();
+      mem.getCapacity(); //already closed

Review Comment:
   so? should this throw or something?



-- 
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: commits-unsubscribe@datasketches.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@datasketches.apache.org
For additional commands, e-mail: commits-help@datasketches.apache.org


[GitHub] [datasketches-memory] leerho merged pull request #172: Remove handles

Posted by "leerho (via GitHub)" <gi...@apache.org>.
leerho merged PR #172:
URL: https://github.com/apache/datasketches-memory/pull/172


-- 
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: commits-unsubscribe@datasketches.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@datasketches.apache.org
For additional commands, e-mail: commits-help@datasketches.apache.org


[GitHub] [datasketches-memory] leerho commented on a diff in pull request #172: Remove handles

Posted by "leerho (via GitHub)" <gi...@apache.org>.
leerho commented on code in PR #172:
URL: https://github.com/apache/datasketches-memory/pull/172#discussion_r1091173372


##########
datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/AllocateDirectMapMemoryTest.java:
##########
@@ -25,47 +25,44 @@
 
 import static org.apache.datasketches.memory.internal.Util.getResourceFile;
 import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertFalse;
 import static org.testng.Assert.assertTrue;
 import static org.testng.Assert.fail;
 
 import java.io.File;
 import java.nio.ByteOrder;
 
-import org.apache.datasketches.memory.MapHandle;
 import org.apache.datasketches.memory.Memory;
-import org.testng.annotations.AfterClass;
+import org.apache.datasketches.memory.MemoryCloseException;
+import org.apache.datasketches.memory.MemoryInvalidException;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 public class AllocateDirectMapMemoryTest {
   private static final String LS = System.getProperty("line.separator");
-  MapHandle hand = null;
 
   @BeforeClass
   public void setReadOnly() {
     UtilTest.setGettysburgAddressFileToReadOnly();
   }
 
   @Test
-  public void simpleMap() throws Exception {
+  public void simpleMap() {
     File file = getResourceFile("GettysburgAddress.txt");
-    assertTrue(AllocateDirectMap.isFileReadOnly(file));
-    try (MapHandle rh = Memory.map(file)) {
-      rh.close();
-    }
+    assertTrue(AllocateDirectWritableMap.isFileReadOnly(file));
+    try (Memory mem = Memory.map(file)) {
+      mem.close(); //redundant close

Review Comment:
   Fixed.



-- 
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: commits-unsubscribe@datasketches.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@datasketches.apache.org
For additional commands, e-mail: commits-help@datasketches.apache.org