From f427ca2a0f30d70f42b4755d0d72318cfa0e4c77 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 13 Sep 2017 11:16:19 +0100 Subject: [PATCH 01/65] introduce ByteArrayMemoryBlock, IntArrayMemoryBlock, LongArrayMemoryBlock, and OffheaMemoryBlock --- .../apache/spark/unsafe/array/LongArray.java | 1 + .../unsafe/memory/ByteArrayMemoryBlock.java | 74 ++++++++++++++++++ .../unsafe/memory/HeapMemoryAllocator.java | 20 ++--- .../unsafe/memory/IntArrayMemoryBlock.java | 74 ++++++++++++++++++ .../unsafe/memory/LongArrayMemoryBlock.java | 75 +++++++++++++++++++ .../spark/unsafe/memory/MemoryAllocator.java | 4 +- .../spark/unsafe/memory/MemoryBlock.java | 41 ++++------ .../unsafe/memory/OffHeapMemoryBlock.java | 72 ++++++++++++++++++ .../unsafe/memory/UnsafeMemoryAllocator.java | 63 ++++++++++++---- .../spark/unsafe/PlatformUtilSuite.java | 4 +- .../spark/unsafe/array/LongArraySuite.java | 4 +- .../spark/memory/TaskMemoryManager.java | 22 +++--- .../shuffle/sort/ShuffleInMemorySorter.java | 4 +- .../unsafe/sort/UnsafeExternalSorter.java | 2 +- .../unsafe/sort/UnsafeInMemorySorter.java | 4 +- .../util/collection/ExternalSorterSuite.scala | 6 +- .../unsafe/sort/RadixSortSuite.scala | 11 +-- .../execution/benchmark/SortBenchmark.scala | 17 +++-- .../sql/execution/python/RowQueueSuite.scala | 4 +- 19 files changed, 413 insertions(+), 89 deletions(-) create mode 100644 common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java create mode 100644 common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java create mode 100644 common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java create mode 100644 common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java index 2cd39bd60c2ac..33ec69a3b3a41 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java @@ -18,6 +18,7 @@ package org.apache.spark.unsafe.array; import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.memory.LongArrayMemoryBlock; import org.apache.spark.unsafe.memory.MemoryBlock; /** diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java new file mode 100644 index 0000000000000..7d7666e88342a --- /dev/null +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -0,0 +1,74 @@ +/* + * 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.spark.unsafe.memory; + +import org.apache.spark.unsafe.Platform; + +/** + * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. + */ +public class ByteArrayMemoryBlock extends MemoryLocation implements MemoryBlock { + + private final long length; + + /** + * Optional page number; used when this MemoryBlock represents a page allocated by a + * TaskMemoryManager. This field is public so that it can be modified by the TaskMemoryManager, + * which lives in a different package. + */ + private int pageNumber = NO_PAGE_NUMBER; + + public ByteArrayMemoryBlock(byte[] obj, long offset, long length) { + super(obj, offset); + this.length = length; + } + + /** + * Returns the size of the memory block. + */ + public long size() { + return length; + } + + public void fill(byte value) { + Platform.setMemory(obj, offset, length, value); + } + + public MemoryBlock allocate(long offset, long size) { + return new ByteArrayMemoryBlock((byte[]) obj, offset, size); + } + + @Override + public void setPageNumber(int pageNum) { + this.pageNumber = pageNum; + } + + @Override + public int getPageNumber() { + return this.pageNumber; + } + + public byte[] getByteArray() { return (byte[])this.obj; } + + /** + * Creates a memory block pointing to the memory used by the byte array. + */ + public static ByteArrayMemoryBlock fromByteArray(final byte[] array) { + return new ByteArrayMemoryBlock(array, Platform.BYTE_ARRAY_OFFSET, array.length); + } +} diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java index 2733760dd19ef..da2c9ceee742d 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java @@ -58,7 +58,8 @@ public MemoryBlock allocate(long size) throws OutOfMemoryError { final long[] array = arrayReference.get(); if (array != null) { assert (array.length * 8L >= size); - MemoryBlock memory = new MemoryBlock(array, Platform.LONG_ARRAY_OFFSET, size); + MemoryBlock memory = + new LongArrayMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, size); if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE); } @@ -70,7 +71,7 @@ public MemoryBlock allocate(long size) throws OutOfMemoryError { } } long[] array = new long[numWords]; - MemoryBlock memory = new MemoryBlock(array, Platform.LONG_ARRAY_OFFSET, size); + MemoryBlock memory = new LongArrayMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, size); if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE); } @@ -79,12 +80,13 @@ public MemoryBlock allocate(long size) throws OutOfMemoryError { @Override public void free(MemoryBlock memory) { - assert (memory.obj != null) : + assert(memory instanceof LongArrayMemoryBlock); + assert (memory.getBaseObject() != null) : "baseObject was null; are you trying to use the on-heap allocator to free off-heap memory?"; - assert (memory.pageNumber != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : + assert (memory.getPageNumber() != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : "page has already been freed"; - assert ((memory.pageNumber == MemoryBlock.NO_PAGE_NUMBER) - || (memory.pageNumber == MemoryBlock.FREED_IN_TMM_PAGE_NUMBER)) : + assert ((memory.getPageNumber() == MemoryBlock.NO_PAGE_NUMBER) + || (memory.getPageNumber() == MemoryBlock.FREED_IN_TMM_PAGE_NUMBER)) : "TMM-allocated pages must first be freed via TMM.freePage(), not directly in allocator " + "free()"; @@ -94,12 +96,12 @@ public void free(MemoryBlock memory) { } // Mark the page as freed (so we can detect double-frees). - memory.pageNumber = MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER; + memory.setPageNumber(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER); // As an additional layer of defense against use-after-free bugs, we mutate the // MemoryBlock to null out its reference to the long[] array. - long[] array = (long[]) memory.obj; - memory.setObjAndOffset(null, 0); + long[] array = ((LongArrayMemoryBlock)memory).getLongArray(); + ((LongArrayMemoryBlock)memory).setObjAndOffset(null, 0); long alignedSize = ((size + 7) / 8) * 8; if (shouldPool(alignedSize)) { diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java new file mode 100644 index 0000000000000..514083dd7a3a0 --- /dev/null +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java @@ -0,0 +1,74 @@ +/* + * 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.spark.unsafe.memory; + +import org.apache.spark.unsafe.Platform; + +/** + * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. + */ +public class IntArrayMemoryBlock extends MemoryLocation implements MemoryBlock { + + private final long size; + + /** + * Optional page number; used when this MemoryBlock represents a page allocated by a + * TaskMemoryManager. This field is public so that it can be modified by the TaskMemoryManager, + * which lives in a different package. + */ + private int pageNumber = -1; + + public IntArrayMemoryBlock(int[] obj, long offset, long size) { + super(obj, offset); + this.size = size; + } + + /** + * Returns the size of the memory block. + */ + public long size() { + return size; + } + + public void fill(byte value) { + Platform.setMemory(obj, offset, size, value); + } + + public MemoryBlock allocate(long offset, long size) { + return new IntArrayMemoryBlock((int[]) obj, offset, size); + } + + @Override + public void setPageNumber(int pageNum) { + this.pageNumber = pageNum; + } + + @Override + public int getPageNumber() { + return this.pageNumber; + } + + public int[] getIntArray() { return (int[])this.obj; } + + /** + * Creates a memory block pointing to the memory used by the int array. + */ + public static IntArrayMemoryBlock fromIntArray(final int[] array) { + return new IntArrayMemoryBlock(array, Platform.INT_ARRAY_OFFSET, array.length*4); + } +} diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java new file mode 100644 index 0000000000000..31a157d12ee2b --- /dev/null +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java @@ -0,0 +1,75 @@ +/* + * 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.spark.unsafe.memory; + +import org.apache.spark.unsafe.Platform; + +/** + * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. + */ +public class LongArrayMemoryBlock extends MemoryLocation implements MemoryBlock { + + private final long size; + + /** + * Optional page number; used when this MemoryBlock represents a page allocated by a + * TaskMemoryManager. This field is public so that it can be modified by the TaskMemoryManager, + * which lives in a different package. + */ + private int pageNumber = -1; + + public LongArrayMemoryBlock(long[] obj, long offset, long size) { + super(obj, offset); + this.size = size; + } + + /** + * Returns the size of the memory block. + */ + public long size() { + return size; + } + + public void fill(byte value) { + Platform.setMemory(obj, offset, size, value); + } + + @Override + public MemoryBlock allocate(long offset, long size) { + return new LongArrayMemoryBlock((long[]) obj, offset, size); + } + + @Override + public void setPageNumber(int pageNum) { + this.pageNumber = pageNum; + } + + @Override + public int getPageNumber() { + return this.pageNumber; + } + + public long[] getLongArray() { return (long[])this.obj; } + + /** + * Creates a memory block pointing to the memory used by the long array. + */ + public static LongArrayMemoryBlock fromLongArray(final long[] array) { + return new LongArrayMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, array.length*8); + } +} diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java index 7b588681d9790..38315fb97b46a 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java @@ -38,7 +38,7 @@ public interface MemoryAllocator { void free(MemoryBlock memory); - MemoryAllocator UNSAFE = new UnsafeMemoryAllocator(); + UnsafeMemoryAllocator UNSAFE = new UnsafeMemoryAllocator(); - MemoryAllocator HEAP = new HeapMemoryAllocator(); + HeapMemoryAllocator HEAP = new HeapMemoryAllocator(); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index c333857358d30..6632669708d9a 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -22,10 +22,9 @@ import org.apache.spark.unsafe.Platform; /** - * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. + * A declaration of interfaces of MemoryBloock classes . */ -public class MemoryBlock extends MemoryLocation { - +public interface MemoryBlock { /** Special `pageNumber` value for pages which were not allocated by TaskMemoryManagers */ public static final int NO_PAGE_NUMBER = -1; @@ -45,38 +44,26 @@ public class MemoryBlock extends MemoryLocation { */ public static final int FREED_IN_ALLOCATOR_PAGE_NUMBER = -3; - private final long length; - /** - * Optional page number; used when this MemoryBlock represents a page allocated by a - * TaskMemoryManager. This field is public so that it can be modified by the TaskMemoryManager, - * which lives in a different package. + * Returns the size of the memory block. */ - public int pageNumber = NO_PAGE_NUMBER; + long size(); - public MemoryBlock(@Nullable Object obj, long offset, long length) { - super(obj, offset); - this.length = length; - } + Object getBaseObject(); - /** - * Returns the size of the memory block. - */ - public long size() { - return length; - } + long getBaseOffset(); + + void setPageNumber(int pageNum); + + int getPageNumber(); /** - * Creates a memory block pointing to the memory used by the long array. + * Fills the memory block with the specified byte value. */ - public static MemoryBlock fromLongArray(final long[] array) { - return new MemoryBlock(array, Platform.LONG_ARRAY_OFFSET, array.length * 8L); - } + void fill(byte value); /** - * Fills the memory block with the specified byte value. + * Instantiate the same type of MemoryBlock with new offset and size */ - public void fill(byte value) { - Platform.setMemory(obj, offset, length, value); - } + MemoryBlock allocate(long offset, long size); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java new file mode 100644 index 0000000000000..c4ec7c5e7160b --- /dev/null +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java @@ -0,0 +1,72 @@ +/* + * 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.spark.unsafe.memory; + +import org.apache.spark.unsafe.Platform; + +public class OffHeapMemoryBlock implements MemoryBlock { + private Object directBuffer; + private long address; + private final long length; + private int pageNumber = -1; + + public OffHeapMemoryBlock(Object directBuffer, long address, long size) { + this.address = address; + this.length = size; + this.directBuffer = directBuffer; + } + + @Override + public Object getBaseObject() { + return null; + } + + @Override + public long getBaseOffset() { + return this.address; + } + + public void setBaseOffset(long address) { + this.address = address; + } + + @Override + public long size() { + return this.length; + } + + @Override + public void fill(byte value) { + Platform.setMemory(null, address, length, value); + } + + @Override + public MemoryBlock allocate(long offset, long size) { + return new OffHeapMemoryBlock(address, offset, size); + } + + @Override + public void setPageNumber(int pageNum) { + this.pageNumber = pageNum; + } + + @Override + public int getPageNumber() { + return this.pageNumber; + } +} diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java index 4368fb615ba1e..b7cbeecf2d067 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -19,39 +19,76 @@ import org.apache.spark.unsafe.Platform; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.nio.ByteBuffer; + /** * A simple {@link MemoryAllocator} that uses {@code Unsafe} to allocate off-heap memory. */ public class UnsafeMemoryAllocator implements MemoryAllocator { + private static Method bufAddrMethod; + static { + try { + Class cb = UnsafeMemoryAllocator.class.getClassLoader().loadClass("java.nio.DirectByteBuffer"); + bufAddrMethod = cb.getMethod("address"); + bufAddrMethod.setAccessible(true); + } + catch(Exception ex) { + throw new RuntimeException(ex.getMessage(), ex); + } + } + @Override - public MemoryBlock allocate(long size) throws OutOfMemoryError { - long address = Platform.allocateMemory(size); - MemoryBlock memory = new MemoryBlock(null, address, size); - if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { - memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE); + public OffHeapMemoryBlock allocate(long size) throws OutOfMemoryError { + try { + Object b = ByteBuffer.allocateDirect((int)size); + long addr = (long)bufAddrMethod.invoke(b); + return new OffHeapMemoryBlock(b, addr, size); + } catch (IllegalAccessException e) { + throw new RuntimeException(e.getMessage(), e); + } catch (InvocationTargetException e) { + Throwable tex = e.getTargetException(); + if( tex instanceof OutOfMemoryError) { + throw (OutOfMemoryError) tex; + } + else { + throw new RuntimeException(e.getMessage(), e); + } } - return memory; } @Override public void free(MemoryBlock memory) { - assert (memory.obj == null) : + assert(memory instanceof OffHeapMemoryBlock); + assert (memory.getBaseObject() == null) : "baseObject not null; are you trying to use the off-heap allocator to free on-heap memory?"; - assert (memory.pageNumber != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : + assert (memory.getPageNumber() != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : "page has already been freed"; - assert ((memory.pageNumber == MemoryBlock.NO_PAGE_NUMBER) - || (memory.pageNumber == MemoryBlock.FREED_IN_TMM_PAGE_NUMBER)) : + assert ((memory.getPageNumber() == MemoryBlock.NO_PAGE_NUMBER) + || (memory.getPageNumber() == MemoryBlock.FREED_IN_TMM_PAGE_NUMBER)) : "TMM-allocated pages must be freed via TMM.freePage(), not directly in allocator free()"; if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_FREED_VALUE); } - Platform.freeMemory(memory.offset); + // As an additional layer of defense against use-after-free bugs, we mutate the // MemoryBlock to reset its pointer. - memory.offset = 0; + ((OffHeapMemoryBlock)memory).setBaseOffset(0); // Mark the page as freed (so we can detect double-frees). - memory.pageNumber = MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER; + memory.setPageNumber(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER); + + // DirectByteBuffers are deallocated automatically by JVM when they become + // unreachable much like normal Objects in heap + } + + public OffHeapMemoryBlock reallocate(OffHeapMemoryBlock block, long oldSize, long newSize) { + OffHeapMemoryBlock nb = this.allocate(newSize); + if( block.getBaseOffset() != 0 ) + Platform.copyMemory(block, block.getBaseOffset(), nb, nb.getBaseOffset(), oldSize); + + return nb; } } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java index 3ad9ac7b4de9c..583a148b3845d 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java @@ -81,7 +81,7 @@ public void freeingOnHeapMemoryBlockResetsBaseObjectAndOffset() { MemoryAllocator.HEAP.free(block); Assert.assertNull(block.getBaseObject()); Assert.assertEquals(0, block.getBaseOffset()); - Assert.assertEquals(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER, block.pageNumber); + Assert.assertEquals(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER, block.getPageNumber()); } @Test @@ -92,7 +92,7 @@ public void freeingOffHeapMemoryBlockResetsOffset() { MemoryAllocator.UNSAFE.free(block); Assert.assertNull(block.getBaseObject()); Assert.assertEquals(0, block.getBaseOffset()); - Assert.assertEquals(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER, block.pageNumber); + Assert.assertEquals(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER, block.getPageNumber()); } @Test(expected = AssertionError.class) diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java index fb8e53b3348f3..c12b4d1ea5c0e 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java @@ -20,14 +20,14 @@ import org.junit.Assert; import org.junit.Test; -import org.apache.spark.unsafe.memory.MemoryBlock; +import org.apache.spark.unsafe.memory.LongArrayMemoryBlock; public class LongArraySuite { @Test public void basicTest() { long[] bytes = new long[2]; - LongArray arr = new LongArray(MemoryBlock.fromLongArray(bytes)); + LongArray arr = new LongArray(LongArrayMemoryBlock.fromLongArray(bytes)); arr.set(0, 1L); arr.set(1, 2L); arr.set(1, 3L); diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index d07faf1da1248..8651a639c07f7 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -311,7 +311,7 @@ public MemoryBlock allocatePage(long size, MemoryConsumer consumer) { // this could trigger spilling to free some pages. return allocatePage(size, consumer); } - page.pageNumber = pageNumber; + page.setPageNumber(pageNumber); pageTable[pageNumber] = page; if (logger.isTraceEnabled()) { logger.trace("Allocate page number {} ({} bytes)", pageNumber, acquired); @@ -323,25 +323,25 @@ public MemoryBlock allocatePage(long size, MemoryConsumer consumer) { * Free a block of memory allocated via {@link TaskMemoryManager#allocatePage}. */ public void freePage(MemoryBlock page, MemoryConsumer consumer) { - assert (page.pageNumber != MemoryBlock.NO_PAGE_NUMBER) : + assert (page.getPageNumber() != MemoryBlock.NO_PAGE_NUMBER) : "Called freePage() on memory that wasn't allocated with allocatePage()"; - assert (page.pageNumber != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : + assert (page.getPageNumber() != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : "Called freePage() on a memory block that has already been freed"; - assert (page.pageNumber != MemoryBlock.FREED_IN_TMM_PAGE_NUMBER) : + assert (page.getPageNumber() != MemoryBlock.FREED_IN_TMM_PAGE_NUMBER) : "Called freePage() on a memory block that has already been freed"; - assert(allocatedPages.get(page.pageNumber)); - pageTable[page.pageNumber] = null; + assert(allocatedPages.get(page.getPageNumber())); + pageTable[page.getPageNumber()] = null; synchronized (this) { - allocatedPages.clear(page.pageNumber); + allocatedPages.clear(page.getPageNumber()); } if (logger.isTraceEnabled()) { - logger.trace("Freed page number {} ({} bytes)", page.pageNumber, page.size()); + logger.trace("Freed page number {} ({} bytes)", page.getPageNumber(), page.size()); } long pageSize = page.size(); // Clear the page number before passing the block to the MemoryAllocator's free(). // Doing this allows the MemoryAllocator to detect when a TaskMemoryManager-managed // page has been inappropriately directly freed without calling TMM.freePage(). - page.pageNumber = MemoryBlock.FREED_IN_TMM_PAGE_NUMBER; + page.setPageNumber(MemoryBlock.FREED_IN_TMM_PAGE_NUMBER); memoryManager.tungstenMemoryAllocator().free(page); releaseExecutionMemory(pageSize, consumer); } @@ -363,7 +363,7 @@ public long encodePageNumberAndOffset(MemoryBlock page, long offsetInPage) { // relative to the page's base offset; this relative offset will fit in 51 bits. offsetInPage -= page.getBaseOffset(); } - return encodePageNumberAndOffset(page.pageNumber, offsetInPage); + return encodePageNumberAndOffset(page.getPageNumber(), offsetInPage); } @VisibleForTesting @@ -434,7 +434,7 @@ public long cleanUpAllAllocatedMemory() { for (MemoryBlock page : pageTable) { if (page != null) { logger.debug("unreleased page: " + page + " in task " + taskAttemptId); - page.pageNumber = MemoryBlock.FREED_IN_TMM_PAGE_NUMBER; + page.setPageNumber(MemoryBlock.FREED_IN_TMM_PAGE_NUMBER); memoryManager.tungstenMemoryAllocator().free(page); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java index dc36809d8911f..2bfbcbc4c4d7a 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java @@ -22,6 +22,7 @@ import org.apache.spark.memory.MemoryConsumer; import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.LongArray; +import org.apache.spark.unsafe.memory.LongArrayMemoryBlock; import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.util.collection.Sorter; import org.apache.spark.util.collection.unsafe.sort.RadixSort; @@ -180,8 +181,7 @@ public ShuffleSorterIterator getSortedIterator() { PackedRecordPointer.PARTITION_ID_START_BYTE_INDEX, PackedRecordPointer.PARTITION_ID_END_BYTE_INDEX, false, false); } else { - MemoryBlock unused = new MemoryBlock( - array.getBaseObject(), + MemoryBlock unused = array.memoryBlock().allocate( array.getBaseOffset() + pos * 8L, (array.size() - pos) * 8L); LongArray buffer = new LongArray(unused); diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 66118f454159b..4fc19b1721518 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -544,7 +544,7 @@ public long spill() throws IOException { // is accessing the current record. We free this page in that caller's next loadNext() // call. for (MemoryBlock page : allocatedPages) { - if (!loaded || page.pageNumber != + if (!loaded || page.getPageNumber() != ((UnsafeInMemorySorter.SortedIterator)upstream).getCurrentPageNumber()) { released += page.size(); freePage(page); diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index b3c27d83da172..95866db51e36d 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -29,6 +29,7 @@ import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.UnsafeAlignedOffset; import org.apache.spark.unsafe.array.LongArray; +import org.apache.spark.unsafe.memory.LongArrayMemoryBlock; import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.util.collection.Sorter; @@ -348,8 +349,7 @@ public UnsafeSorterIterator getSortedIterator() { array, nullBoundaryPos, (pos - nullBoundaryPos) / 2L, 0, 7, radixSortSupport.sortDescending(), radixSortSupport.sortSigned()); } else { - MemoryBlock unused = new MemoryBlock( - array.getBaseObject(), + MemoryBlock unused = array.memoryBlock().allocate( array.getBaseOffset() + pos * 8L, (array.size() - pos) * 8L); LongArray buffer = new LongArray(unused); diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala index 47173b89e91e2..cbbb15c92e654 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark._ import org.apache.spark.memory.MemoryTestingUtils import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.unsafe.array.LongArray -import org.apache.spark.unsafe.memory.MemoryBlock +import org.apache.spark.unsafe.memory.LongArrayMemoryBlock import org.apache.spark.util.collection.unsafe.sort.{PrefixComparators, RecordPointerAndKeyPrefix, UnsafeSortDataFormat} class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { @@ -105,9 +105,9 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { // the form [150000000, 150000001, 150000002, ...., 300000000, 0, 1, 2, ..., 149999999] // that can trigger copyRange() in TimSort.mergeLo() or TimSort.mergeHi() val ref = Array.tabulate[Long](size) { i => if (i < size / 2) size / 2 + i else i } - val buf = new LongArray(MemoryBlock.fromLongArray(ref)) + val buf = new LongArray(LongArrayMemoryBlock.fromLongArray(ref)) val tmp = new Array[Long](size/2) - val tmpBuf = new LongArray(MemoryBlock.fromLongArray(tmp)) + val tmpBuf = new LongArray(LongArrayMemoryBlock.fromLongArray(tmp)) new Sorter(new UnsafeSortDataFormat(tmpBuf)).sort( buf, 0, size, new Comparator[RecordPointerAndKeyPrefix] { diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala index d5956ea32096a..a5b29676da47b 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala @@ -27,7 +27,7 @@ import com.google.common.primitives.Ints import org.apache.spark.SparkFunSuite import org.apache.spark.internal.Logging import org.apache.spark.unsafe.array.LongArray -import org.apache.spark.unsafe.memory.MemoryBlock +import org.apache.spark.unsafe.memory.LongArrayMemoryBlock import org.apache.spark.util.collection.Sorter import org.apache.spark.util.random.XORShiftRandom @@ -78,14 +78,14 @@ class RadixSortSuite extends SparkFunSuite with Logging { private def generateTestData(size: Long, rand: => Long): (Array[JLong], LongArray) = { val ref = Array.tabulate[Long](Ints.checkedCast(size)) { i => rand } val extended = ref ++ Array.fill[Long](Ints.checkedCast(size))(0) - (ref.map(i => new JLong(i)), new LongArray(MemoryBlock.fromLongArray(extended))) + (ref.map(i => new JLong(i)), new LongArray(LongArrayMemoryBlock.fromLongArray(extended))) } private def generateKeyPrefixTestData(size: Long, rand: => Long): (LongArray, LongArray) = { val ref = Array.tabulate[Long](Ints.checkedCast(size * 2)) { i => rand } val extended = ref ++ Array.fill[Long](Ints.checkedCast(size * 2))(0) - (new LongArray(MemoryBlock.fromLongArray(ref)), - new LongArray(MemoryBlock.fromLongArray(extended))) + (new LongArray(LongArrayMemoryBlock.fromLongArray(ref)), + new LongArray(LongArrayMemoryBlock.fromLongArray(extended))) } private def collectToArray(array: LongArray, offset: Int, length: Long): Array[Long] = { @@ -110,7 +110,8 @@ class RadixSortSuite extends SparkFunSuite with Logging { } private def referenceKeyPrefixSort(buf: LongArray, lo: Long, hi: Long, refCmp: PrefixComparator) { - val sortBuffer = new LongArray(MemoryBlock.fromLongArray(new Array[Long](buf.size().toInt))) + val sortBuffer = + new LongArray(LongArrayMemoryBlock.fromLongArray(new Array[Long](buf.size().toInt))) new Sorter(new UnsafeSortDataFormat(sortBuffer)).sort( buf, Ints.checkedCast(lo), Ints.checkedCast(hi), new Comparator[RecordPointerAndKeyPrefix] { override def compare( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala index 50ae26a3ff9d9..4845cac6b93dd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.benchmark import java.util.{Arrays, Comparator} import org.apache.spark.unsafe.array.LongArray -import org.apache.spark.unsafe.memory.MemoryBlock +import org.apache.spark.unsafe.memory.LongArrayMemoryBlock import org.apache.spark.util.Benchmark import org.apache.spark.util.collection.Sorter import org.apache.spark.util.collection.unsafe.sort._ @@ -36,7 +36,8 @@ import org.apache.spark.util.random.XORShiftRandom class SortBenchmark extends BenchmarkBase { private def referenceKeyPrefixSort(buf: LongArray, lo: Int, hi: Int, refCmp: PrefixComparator) { - val sortBuffer = new LongArray(MemoryBlock.fromLongArray(new Array[Long](buf.size().toInt))) + val sortBuffer = + new LongArray(LongArrayMemoryBlock.fromLongArray(new Array[Long](buf.size().toInt))) new Sorter(new UnsafeSortDataFormat(sortBuffer)).sort( buf, lo, hi, new Comparator[RecordPointerAndKeyPrefix] { override def compare( @@ -50,8 +51,8 @@ class SortBenchmark extends BenchmarkBase { private def generateKeyPrefixTestData(size: Int, rand: => Long): (LongArray, LongArray) = { val ref = Array.tabulate[Long](size * 2) { i => rand } val extended = ref ++ Array.fill[Long](size * 2)(0) - (new LongArray(MemoryBlock.fromLongArray(ref)), - new LongArray(MemoryBlock.fromLongArray(extended))) + (new LongArray(LongArrayMemoryBlock.fromLongArray(ref)), + new LongArray(LongArrayMemoryBlock.fromLongArray(extended))) } ignore("sort") { @@ -60,7 +61,7 @@ class SortBenchmark extends BenchmarkBase { val benchmark = new Benchmark("radix sort " + size, size) benchmark.addTimerCase("reference TimSort key prefix array") { timer => val array = Array.tabulate[Long](size * 2) { i => rand.nextLong } - val buf = new LongArray(MemoryBlock.fromLongArray(array)) + val buf = new LongArray(LongArrayMemoryBlock.fromLongArray(array)) timer.startTiming() referenceKeyPrefixSort(buf, 0, size, PrefixComparators.BINARY) timer.stopTiming() @@ -78,7 +79,7 @@ class SortBenchmark extends BenchmarkBase { array(i) = rand.nextLong & 0xff i += 1 } - val buf = new LongArray(MemoryBlock.fromLongArray(array)) + val buf = new LongArray(LongArrayMemoryBlock.fromLongArray(array)) timer.startTiming() RadixSort.sort(buf, size, 0, 7, false, false) timer.stopTiming() @@ -90,7 +91,7 @@ class SortBenchmark extends BenchmarkBase { array(i) = rand.nextLong & 0xffff i += 1 } - val buf = new LongArray(MemoryBlock.fromLongArray(array)) + val buf = new LongArray(LongArrayMemoryBlock.fromLongArray(array)) timer.startTiming() RadixSort.sort(buf, size, 0, 7, false, false) timer.stopTiming() @@ -102,7 +103,7 @@ class SortBenchmark extends BenchmarkBase { array(i) = rand.nextLong i += 1 } - val buf = new LongArray(MemoryBlock.fromLongArray(array)) + val buf = new LongArray(LongArrayMemoryBlock.fromLongArray(array)) timer.startTiming() RadixSort.sort(buf, size, 0, 7, false, false) timer.stopTiming() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala index ffda33cf906c5..5690a43ba28ec 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala @@ -22,13 +22,13 @@ import java.io.File import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.memory.{MemoryManager, TaskMemoryManager, TestMemoryManager} import org.apache.spark.sql.catalyst.expressions.UnsafeRow -import org.apache.spark.unsafe.memory.MemoryBlock +import org.apache.spark.unsafe.memory.LongArrayMemoryBlock import org.apache.spark.util.Utils class RowQueueSuite extends SparkFunSuite { test("in-memory queue") { - val page = MemoryBlock.fromLongArray(new Array[Long](1<<10)) + val page = LongArrayMemoryBlock.fromLongArray(new Array[Long](1<<10)) val queue = new InMemoryRowQueue(page, 1) { override def close() {} } From 5d7ccdb0e845afcaa430bac3d21b519d35d1e6f4 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 13 Sep 2017 18:15:25 +0100 Subject: [PATCH 02/65] OffHeapColumnVector uses UnsafeMemoryAllocator --- .../org/apache/spark/unsafe/Platform.java | 143 ++++++++++++- .../apache/spark/unsafe/array/LongArray.java | 10 +- .../unsafe/memory/ByteArrayMemoryBlock.java | 2 +- .../unsafe/memory/IntArrayMemoryBlock.java | 2 +- .../unsafe/memory/LongArrayMemoryBlock.java | 2 +- .../unsafe/memory/OffHeapMemoryBlock.java | 2 + .../unsafe/memory/UnsafeMemoryAllocator.java | 17 +- .../apache/spark/unsafe/types/ByteArray.java | 1 + .../spark/unsafe/array/LongArraySuite.java | 2 +- .../shuffle/sort/ShuffleInMemorySorter.java | 4 +- .../shuffle/sort/ShuffleSortDataFormat.java | 4 +- .../util/collection/ExternalSorterSuite.scala | 4 +- .../unsafe/sort/RadixSortSuite.scala | 8 +- .../vectorized/OffHeapColumnVector.java | 192 +++++++++--------- .../execution/benchmark/SortBenchmark.scala | 14 +- .../sql/execution/python/RowQueueSuite.scala | 2 +- 16 files changed, 267 insertions(+), 142 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java index aca6fca00c48b..7245b35d443f0 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java @@ -22,6 +22,7 @@ import java.lang.reflect.Method; import java.nio.ByteBuffer; +import org.apache.spark.unsafe.memory.MemoryBlock; import sun.misc.Cleaner; import sun.misc.Unsafe; @@ -75,67 +76,131 @@ public static boolean unaligned() { return unaligned; } + public static int getInt(MemoryBlock object, long offset) { + return _UNSAFE.getInt(object.getBaseObject(), offset); + } + public static int getInt(Object object, long offset) { return _UNSAFE.getInt(object, offset); } + public static void putInt(MemoryBlock object, long offset, int value) { + _UNSAFE.putInt(object.getBaseObject(), offset, value); + } + public static void putInt(Object object, long offset, int value) { _UNSAFE.putInt(object, offset, value); } + public static boolean getBoolean(MemoryBlock object, long offset) { + return _UNSAFE.getBoolean(object.getBaseObject(), offset); + } + public static boolean getBoolean(Object object, long offset) { return _UNSAFE.getBoolean(object, offset); } + public static void putBoolean(MemoryBlock object, long offset, boolean value) { + _UNSAFE.putBoolean(object.getBaseObject(), offset, value); + } + public static void putBoolean(Object object, long offset, boolean value) { _UNSAFE.putBoolean(object, offset, value); } + public static byte getByte(MemoryBlock object, long offset) { + return _UNSAFE.getByte(object.getBaseObject(), offset); + } + public static byte getByte(Object object, long offset) { return _UNSAFE.getByte(object, offset); } + public static void putByte(MemoryBlock object, long offset, byte value) { + _UNSAFE.putByte(object.getBaseObject(), offset, value); + } + public static void putByte(Object object, long offset, byte value) { _UNSAFE.putByte(object, offset, value); } + public static short getShort(MemoryBlock object, long offset) { + return _UNSAFE.getShort(object.getBaseObject(), offset); + } + public static short getShort(Object object, long offset) { return _UNSAFE.getShort(object, offset); } + public static void putShort(MemoryBlock object, long offset, short value) { + _UNSAFE.putShort(object.getBaseObject(), offset, value); + } + public static void putShort(Object object, long offset, short value) { _UNSAFE.putShort(object, offset, value); } + public static long getLong(MemoryBlock object, long offset) { + return _UNSAFE.getLong(object.getBaseObject(), offset); + } + public static long getLong(Object object, long offset) { return _UNSAFE.getLong(object, offset); } + public static void putLong(MemoryBlock object, long offset, long value) { + _UNSAFE.putLong(object.getBaseObject(), offset, value); + } + public static void putLong(Object object, long offset, long value) { _UNSAFE.putLong(object, offset, value); } + public static float getFloat(MemoryBlock object, long offset) { + return _UNSAFE.getFloat(object.getBaseObject(), offset); + } + public static float getFloat(Object object, long offset) { return _UNSAFE.getFloat(object, offset); } + public static void putFloat(MemoryBlock object, long offset, float value) { + _UNSAFE.putFloat(object.getBaseObject(), offset, value); + } + public static void putFloat(Object object, long offset, float value) { _UNSAFE.putFloat(object, offset, value); } + public static double getDouble(MemoryBlock object, long offset) { + return _UNSAFE.getDouble(object.getBaseObject(), offset); + } + public static double getDouble(Object object, long offset) { return _UNSAFE.getDouble(object, offset); } + public static void putDouble(MemoryBlock object, long offset, double value) { + _UNSAFE.putDouble(object.getBaseObject(), offset, value); + } + public static void putDouble(Object object, long offset, double value) { _UNSAFE.putDouble(object, offset, value); } - public static Object getObjectVolatile(Object object, long offset) { + public static Object getObjectVolatile(MemoryBlock object, long offset) { + return _UNSAFE.getObjectVolatile(object.getBaseObject(), offset); + } + + public static Object getObjectVolatile(byte[] object, long offset) { return _UNSAFE.getObjectVolatile(object, offset); } - public static void putObjectVolatile(Object object, long offset, Object value) { + public static void putObjectVolatile(MemoryBlock object, long offset, Object value) { + _UNSAFE.putObjectVolatile(object.getBaseObject(), offset, value); + } + + public static void putObjectVolatile(byte[] object, long offset, Object value) { _UNSAFE.putObjectVolatile(object, offset, value); } @@ -147,13 +212,6 @@ public static void freeMemory(long address) { _UNSAFE.freeMemory(address); } - public static long reallocateMemory(long address, long oldSize, long newSize) { - long newMemory = _UNSAFE.allocateMemory(newSize); - copyMemory(null, address, null, newMemory, oldSize); - freeMemory(address); - return newMemory; - } - /** * Uses internal JDK APIs to allocate a DirectByteBuffer while ignoring the JVM's * MaxDirectMemorySize limit (the default limit is too low and we do not want to require users @@ -187,7 +245,7 @@ public static void setMemory(long address, byte value, long size) { } public static void copyMemory( - Object src, long srcOffset, Object dst, long dstOffset, long length) { + Object src, long srcOffset, Object dst, long dstOffset, long length) { // Check if dstOffset is before or after srcOffset to determine if we should copy // forward or backwards. This is necessary in case src and dst overlap. if (dstOffset < srcOffset) { @@ -212,6 +270,71 @@ public static void copyMemory( } } + public static void copyMemory( + MemoryBlock src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { + Platform.copyMemory(src.getBaseObject(), srcOffset, dst.getBaseObject(), dstOffset, length); + } + + public static void copyMemory( + byte[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { + Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); + } + + public static void copyMemory( + short[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { + Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); + } + + public static void copyMemory( + int[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { + Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); + } + + public static void copyMemory( + long[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { + Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); + } + + public static void copyMemory( + float[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { + Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); + } + + public static void copyMemory( + double[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { + Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); + } + + public static void copyMemory( + MemoryBlock src, long srcOffset, byte[] dst, long dstOffset, long length) { + Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); + } + + public static void copyMemory( + MemoryBlock src, long srcOffset, short[] dst, long dstOffset, long length) { + Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); + } + + public static void copyMemory( + MemoryBlock src, long srcOffset, int[] dst, long dstOffset, long length) { + Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); + } + + public static void copyMemory( + MemoryBlock src, long srcOffset, long[] dst, long dstOffset, long length) { + Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); + } + + public static void copyMemory( + MemoryBlock src, long srcOffset, float[] dst, long dstOffset, long length) { + Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); + } + + public static void copyMemory( + MemoryBlock src, long srcOffset, double[] dst, long dstOffset, long length) { + Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); + } + /** * Raises an exception bypassing compiler checks for checked exceptions. */ diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java index 33ec69a3b3a41..97baa2c6cdfcb 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java @@ -34,7 +34,6 @@ public final class LongArray { private static final long WIDTH = 8; private final MemoryBlock memory; - private final Object baseObj; private final long baseOffset; private final long length; @@ -42,7 +41,6 @@ public final class LongArray { public LongArray(MemoryBlock memory) { assert memory.size() < (long) Integer.MAX_VALUE * 8: "Array size >= Integer.MAX_VALUE elements"; this.memory = memory; - this.baseObj = memory.getBaseObject(); this.baseOffset = memory.getBaseOffset(); this.length = memory.size() / WIDTH; } @@ -52,7 +50,7 @@ public MemoryBlock memoryBlock() { } public Object getBaseObject() { - return baseObj; + return memory.getBaseObject(); } public long getBaseOffset() { @@ -71,7 +69,7 @@ public long size() { */ public void zeroOut() { for (long off = baseOffset; off < baseOffset + length * WIDTH; off += WIDTH) { - Platform.putLong(baseObj, off, 0); + Platform.putLong(memory, off, 0); } } @@ -81,7 +79,7 @@ public void zeroOut() { public void set(int index, long value) { assert index >= 0 : "index (" + index + ") should >= 0"; assert index < length : "index (" + index + ") should < length (" + length + ")"; - Platform.putLong(baseObj, baseOffset + index * WIDTH, value); + Platform.putLong(memory, baseOffset + index * WIDTH, value); } /** @@ -90,6 +88,6 @@ public void set(int index, long value) { public long get(int index) { assert index >= 0 : "index (" + index + ") should >= 0"; assert index < length : "index (" + index + ") should < length (" + length + ")"; - return Platform.getLong(baseObj, baseOffset + index * WIDTH); + return Platform.getLong(memory, baseOffset + index * WIDTH); } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java index 7d7666e88342a..0ecf2cd26f94c 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -68,7 +68,7 @@ public int getPageNumber() { /** * Creates a memory block pointing to the memory used by the byte array. */ - public static ByteArrayMemoryBlock fromByteArray(final byte[] array) { + public static ByteArrayMemoryBlock fromArray(final byte[] array) { return new ByteArrayMemoryBlock(array, Platform.BYTE_ARRAY_OFFSET, array.length); } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java index 514083dd7a3a0..2069d850ece9b 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java @@ -68,7 +68,7 @@ public int getPageNumber() { /** * Creates a memory block pointing to the memory used by the int array. */ - public static IntArrayMemoryBlock fromIntArray(final int[] array) { + public static IntArrayMemoryBlock fromArray(final int[] array) { return new IntArrayMemoryBlock(array, Platform.INT_ARRAY_OFFSET, array.length*4); } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java index 31a157d12ee2b..f7fd21044646e 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java @@ -69,7 +69,7 @@ public int getPageNumber() { /** * Creates a memory block pointing to the memory used by the long array. */ - public static LongArrayMemoryBlock fromLongArray(final long[] array) { + public static LongArrayMemoryBlock fromArray(final long[] array) { return new LongArrayMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, array.length*8); } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java index c4ec7c5e7160b..69f6a3a068846 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java @@ -25,6 +25,8 @@ public class OffHeapMemoryBlock implements MemoryBlock { private final long length; private int pageNumber = -1; + static public final OffHeapMemoryBlock NULL = new OffHeapMemoryBlock(null, 0, 0); + public OffHeapMemoryBlock(Object directBuffer, long address, long size) { this.address = address; this.length = size; diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java index b7cbeecf2d067..7b9574a8b99dc 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -49,11 +49,10 @@ public OffHeapMemoryBlock allocate(long size) throws OutOfMemoryError { } catch (IllegalAccessException e) { throw new RuntimeException(e.getMessage(), e); } catch (InvocationTargetException e) { - Throwable tex = e.getTargetException(); - if( tex instanceof OutOfMemoryError) { - throw (OutOfMemoryError) tex; - } - else { + Throwable ex = e.getTargetException(); + if (ex instanceof OutOfMemoryError) { + throw (OutOfMemoryError) ex; + } else { throw new RuntimeException(e.getMessage(), e); } } @@ -85,10 +84,10 @@ public void free(MemoryBlock memory) { } public OffHeapMemoryBlock reallocate(OffHeapMemoryBlock block, long oldSize, long newSize) { - OffHeapMemoryBlock nb = this.allocate(newSize); - if( block.getBaseOffset() != 0 ) - Platform.copyMemory(block, block.getBaseOffset(), nb, nb.getBaseOffset(), oldSize); + OffHeapMemoryBlock mb = this.allocate(newSize); + if (block.getBaseOffset() != 0) + Platform.copyMemory(block, block.getBaseOffset(), mb, mb.getBaseOffset(), oldSize); - return nb; + return mb; } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java index c03caf0076f61..92d9b8059e682 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java @@ -18,6 +18,7 @@ package org.apache.spark.unsafe.types; import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.memory.MemoryBlock; import java.util.Arrays; diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java index c12b4d1ea5c0e..d4080a5dc9e93 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java @@ -27,7 +27,7 @@ public class LongArraySuite { @Test public void basicTest() { long[] bytes = new long[2]; - LongArray arr = new LongArray(LongArrayMemoryBlock.fromLongArray(bytes)); + LongArray arr = new LongArray(LongArrayMemoryBlock.fromArray(bytes)); arr.set(0, 1L); arr.set(1, 2L); arr.set(1, 3L); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java index 2bfbcbc4c4d7a..9bb2853f55a6a 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java @@ -107,9 +107,9 @@ public void reset() { public void expandPointerArray(LongArray newArray) { assert(newArray.size() > array.size()); Platform.copyMemory( - array.getBaseObject(), + array.memoryBlock(), array.getBaseOffset(), - newArray.getBaseObject(), + newArray.memoryBlock(), newArray.getBaseOffset(), pos * 8L ); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java index 717bdd79d47ef..8bf2d5d1fa94d 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java @@ -61,9 +61,9 @@ public void copyElement(LongArray src, int srcPos, LongArray dst, int dstPos) { @Override public void copyRange(LongArray src, int srcPos, LongArray dst, int dstPos, int length) { Platform.copyMemory( - src.getBaseObject(), + src.memoryBlock(), src.getBaseOffset() + srcPos * 8L, - dst.getBaseObject(), + dst.memoryBlock(), dst.getBaseOffset() + dstPos * 8L, length * 8L ); diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala index cbbb15c92e654..f9358478f3a42 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -105,9 +105,9 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { // the form [150000000, 150000001, 150000002, ...., 300000000, 0, 1, 2, ..., 149999999] // that can trigger copyRange() in TimSort.mergeLo() or TimSort.mergeHi() val ref = Array.tabulate[Long](size) { i => if (i < size / 2) size / 2 + i else i } - val buf = new LongArray(LongArrayMemoryBlock.fromLongArray(ref)) + val buf = new LongArray(LongArrayMemoryBlock.fromArray(ref)) val tmp = new Array[Long](size/2) - val tmpBuf = new LongArray(LongArrayMemoryBlock.fromLongArray(tmp)) + val tmpBuf = new LongArray(LongArrayMemoryBlock.fromArray(tmp)) new Sorter(new UnsafeSortDataFormat(tmpBuf)).sort( buf, 0, size, new Comparator[RecordPointerAndKeyPrefix] { diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala index a5b29676da47b..c1642a4d305a5 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala @@ -78,14 +78,14 @@ class RadixSortSuite extends SparkFunSuite with Logging { private def generateTestData(size: Long, rand: => Long): (Array[JLong], LongArray) = { val ref = Array.tabulate[Long](Ints.checkedCast(size)) { i => rand } val extended = ref ++ Array.fill[Long](Ints.checkedCast(size))(0) - (ref.map(i => new JLong(i)), new LongArray(LongArrayMemoryBlock.fromLongArray(extended))) + (ref.map(i => new JLong(i)), new LongArray(LongArrayMemoryBlock.fromArray(extended))) } private def generateKeyPrefixTestData(size: Long, rand: => Long): (LongArray, LongArray) = { val ref = Array.tabulate[Long](Ints.checkedCast(size * 2)) { i => rand } val extended = ref ++ Array.fill[Long](Ints.checkedCast(size * 2))(0) - (new LongArray(LongArrayMemoryBlock.fromLongArray(ref)), - new LongArray(LongArrayMemoryBlock.fromLongArray(extended))) + (new LongArray(LongArrayMemoryBlock.fromArray(ref)), + new LongArray(LongArrayMemoryBlock.fromArray(extended))) } private def collectToArray(array: LongArray, offset: Int, length: Long): Array[Long] = { @@ -111,7 +111,7 @@ class RadixSortSuite extends SparkFunSuite with Logging { private def referenceKeyPrefixSort(buf: LongArray, lo: Long, hi: Long, refCmp: PrefixComparator) { val sortBuffer = - new LongArray(LongArrayMemoryBlock.fromLongArray(new Array[Long](buf.size().toInt))) + new LongArray(LongArrayMemoryBlock.fromArray(new Array[Long](buf.size().toInt))) new Sorter(new UnsafeSortDataFormat(sortBuffer)).sort( buf, Ints.checkedCast(lo), Ints.checkedCast(hi), new Comparator[RecordPointerAndKeyPrefix] { override def compare( diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java index 754c26579ff08..cded0b4b9dcd7 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -24,6 +24,8 @@ import org.apache.spark.sql.types.*; import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.types.UTF8String; +import org.apache.spark.unsafe.memory.MemoryAllocator; +import org.apache.spark.unsafe.memory.OffHeapMemoryBlock; /** * Column data backed using offheap memory. @@ -57,20 +59,20 @@ public static OffHeapColumnVector[] allocateColumns(int capacity, StructField[] // The data stored in these two allocations need to maintain binary compatible. We can // directly pass this buffer to external components. - private long nulls; - private long data; + private OffHeapMemoryBlock nulls; + private OffHeapMemoryBlock data; // Only set if type is Array or Map. - private long lengthData; - private long offsetData; + private OffHeapMemoryBlock lengthData; + private OffHeapMemoryBlock offsetData; public OffHeapColumnVector(int capacity, DataType type) { super(capacity, type); - nulls = 0; - data = 0; - lengthData = 0; - offsetData = 0; + nulls = OffHeapMemoryBlock.NULL; + data = OffHeapMemoryBlock.NULL; + lengthData = OffHeapMemoryBlock.NULL; + offsetData = OffHeapMemoryBlock.NULL; reserveInternal(capacity); reset(); @@ -81,20 +83,20 @@ public OffHeapColumnVector(int capacity, DataType type) { */ @VisibleForTesting public long valuesNativeAddress() { - return data; + return data.getBaseOffset(); } @Override public void close() { super.close(); - Platform.freeMemory(nulls); - Platform.freeMemory(data); - Platform.freeMemory(lengthData); - Platform.freeMemory(offsetData); - nulls = 0; - data = 0; - lengthData = 0; - offsetData = 0; + MemoryAllocator.UNSAFE.free(nulls); + MemoryAllocator.UNSAFE.free(data); + MemoryAllocator.UNSAFE.free(lengthData); + MemoryAllocator.UNSAFE.free(offsetData); + nulls = OffHeapMemoryBlock.NULL; + data = OffHeapMemoryBlock.NULL; + lengthData = OffHeapMemoryBlock.NULL; + offsetData = OffHeapMemoryBlock.NULL; } // @@ -103,20 +105,20 @@ public void close() { @Override public void putNotNull(int rowId) { - Platform.putByte(null, nulls + rowId, (byte) 0); + Platform.putByte(nulls, nulls.getBaseOffset() + rowId, (byte) 0); } @Override public void putNull(int rowId) { - Platform.putByte(null, nulls + rowId, (byte) 1); + Platform.putByte(nulls, nulls.getBaseOffset() + rowId, (byte) 1); ++numNulls; } @Override public void putNulls(int rowId, int count) { - long offset = nulls + rowId; + long offset = nulls.getBaseOffset() + rowId; for (int i = 0; i < count; ++i, ++offset) { - Platform.putByte(null, offset, (byte) 1); + Platform.putByte(nulls, offset, (byte) 1); } numNulls += count; } @@ -124,15 +126,15 @@ public void putNulls(int rowId, int count) { @Override public void putNotNulls(int rowId, int count) { if (!hasNull()) return; - long offset = nulls + rowId; + long offset = nulls.getBaseOffset() + rowId; for (int i = 0; i < count; ++i, ++offset) { - Platform.putByte(null, offset, (byte) 0); + Platform.putByte(nulls, offset, (byte) 0); } } @Override public boolean isNullAt(int rowId) { - return Platform.getByte(null, nulls + rowId) == 1; + return Platform.getByte(nulls, nulls.getBaseOffset() + rowId) == 1; } // @@ -141,26 +143,26 @@ public boolean isNullAt(int rowId) { @Override public void putBoolean(int rowId, boolean value) { - Platform.putByte(null, data + rowId, (byte)((value) ? 1 : 0)); + Platform.putByte(data, data.getBaseOffset() + rowId, (byte)((value) ? 1 : 0)); } @Override public void putBooleans(int rowId, int count, boolean value) { byte v = (byte)((value) ? 1 : 0); for (int i = 0; i < count; ++i) { - Platform.putByte(null, data + rowId + i, v); + Platform.putByte(data, data.getBaseOffset() + rowId + i, v); } } @Override - public boolean getBoolean(int rowId) { return Platform.getByte(null, data + rowId) == 1; } + public boolean getBoolean(int rowId) { return Platform.getByte(data, data.getBaseOffset() + rowId) == 1; } @Override public boolean[] getBooleans(int rowId, int count) { assert(dictionary == null); boolean[] array = new boolean[count]; for (int i = 0; i < count; ++i) { - array[i] = (Platform.getByte(null, data + rowId + i) == 1); + array[i] = (Platform.getByte(data, data.getBaseOffset() + rowId + i) == 1); } return array; } @@ -171,26 +173,26 @@ public boolean[] getBooleans(int rowId, int count) { @Override public void putByte(int rowId, byte value) { - Platform.putByte(null, data + rowId, value); + Platform.putByte(data, data.getBaseOffset() + rowId, value); } @Override public void putBytes(int rowId, int count, byte value) { for (int i = 0; i < count; ++i) { - Platform.putByte(null, data + rowId + i, value); + Platform.putByte(data, data.getBaseOffset() + rowId + i, value); } } @Override public void putBytes(int rowId, int count, byte[] src, int srcIndex) { - Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, null, data + rowId, count); + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, data, data.getBaseOffset() + rowId, count); } @Override public byte getByte(int rowId) { if (dictionary == null) { - return Platform.getByte(null, data + rowId); + return Platform.getByte(data, data.getBaseOffset() + rowId); } else { return (byte) dictionary.decodeToInt(dictionaryIds.getDictId(rowId)); } @@ -200,13 +202,13 @@ public byte getByte(int rowId) { public byte[] getBytes(int rowId, int count) { assert(dictionary == null); byte[] array = new byte[count]; - Platform.copyMemory(null, data + rowId, array, Platform.BYTE_ARRAY_OFFSET, count); + Platform.copyMemory(data, data.getBaseOffset() + rowId, array, Platform.BYTE_ARRAY_OFFSET, count); return array; } @Override protected UTF8String getBytesAsUTF8String(int rowId, int count) { - return UTF8String.fromAddress(null, data + rowId, count); + return UTF8String.fromAddress(null, data.getBaseOffset() + rowId, count); } // @@ -215,33 +217,33 @@ protected UTF8String getBytesAsUTF8String(int rowId, int count) { @Override public void putShort(int rowId, short value) { - Platform.putShort(null, data + 2 * rowId, value); + Platform.putShort(data, data.getBaseOffset() + 2 * rowId, value); } @Override public void putShorts(int rowId, int count, short value) { - long offset = data + 2 * rowId; + long offset = data.getBaseOffset() + 2 * rowId; for (int i = 0; i < count; ++i, offset += 2) { - Platform.putShort(null, offset, value); + Platform.putShort(data, offset, value); } } @Override public void putShorts(int rowId, int count, short[] src, int srcIndex) { Platform.copyMemory(src, Platform.SHORT_ARRAY_OFFSET + srcIndex * 2, - null, data + 2 * rowId, count * 2); + data, data.getBaseOffset() + 2 * rowId, count * 2); } @Override public void putShorts(int rowId, int count, byte[] src, int srcIndex) { Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - null, data + rowId * 2, count * 2); + null, data.getBaseOffset() + rowId * 2, count * 2); } @Override public short getShort(int rowId) { if (dictionary == null) { - return Platform.getShort(null, data + 2 * rowId); + return Platform.getShort(data, data.getBaseOffset() + 2 * rowId); } else { return (short) dictionary.decodeToInt(dictionaryIds.getDictId(rowId)); } @@ -251,7 +253,7 @@ public short getShort(int rowId) { public short[] getShorts(int rowId, int count) { assert(dictionary == null); short[] array = new short[count]; - Platform.copyMemory(null, data + rowId * 2, array, Platform.SHORT_ARRAY_OFFSET, count * 2); + Platform.copyMemory(data, data.getBaseOffset() + rowId * 2, array, Platform.SHORT_ARRAY_OFFSET, count * 2); return array; } @@ -261,39 +263,39 @@ public short[] getShorts(int rowId, int count) { @Override public void putInt(int rowId, int value) { - Platform.putInt(null, data + 4 * rowId, value); + Platform.putInt(data, data.getBaseOffset() + 4 * rowId, value); } @Override public void putInts(int rowId, int count, int value) { - long offset = data + 4 * rowId; + long offset = data.getBaseOffset() + 4 * rowId; for (int i = 0; i < count; ++i, offset += 4) { - Platform.putInt(null, offset, value); + Platform.putInt(data, offset, value); } } @Override public void putInts(int rowId, int count, int[] src, int srcIndex) { Platform.copyMemory(src, Platform.INT_ARRAY_OFFSET + srcIndex * 4, - null, data + 4 * rowId, count * 4); + data, data.getBaseOffset() + 4 * rowId, count * 4); } @Override public void putInts(int rowId, int count, byte[] src, int srcIndex) { Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - null, data + rowId * 4, count * 4); + null, data.getBaseOffset() + rowId * 4, count * 4); } @Override public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { Platform.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, - null, data + 4 * rowId, count * 4); + data, data.getBaseOffset() + 4 * rowId, count * 4); } else { int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; - long offset = data + 4 * rowId; + long offset = data.getBaseOffset() + 4 * rowId; for (int i = 0; i < count; ++i, offset += 4, srcOffset += 4) { - Platform.putInt(null, offset, + Platform.putInt(data, offset, java.lang.Integer.reverseBytes(Platform.getInt(src, srcOffset))); } } @@ -302,7 +304,7 @@ public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) @Override public int getInt(int rowId) { if (dictionary == null) { - return Platform.getInt(null, data + 4 * rowId); + return Platform.getInt(data, data.getBaseOffset() + 4 * rowId); } else { return dictionary.decodeToInt(dictionaryIds.getDictId(rowId)); } @@ -312,7 +314,7 @@ public int getInt(int rowId) { public int[] getInts(int rowId, int count) { assert(dictionary == null); int[] array = new int[count]; - Platform.copyMemory(null, data + rowId * 4, array, Platform.INT_ARRAY_OFFSET, count * 4); + Platform.copyMemory(data, data.getBaseOffset() + rowId * 4, array, Platform.INT_ARRAY_OFFSET, count * 4); return array; } @@ -324,7 +326,7 @@ public int[] getInts(int rowId, int count) { public int getDictId(int rowId) { assert(dictionary == null) : "A ColumnVector dictionary should not have a dictionary for itself."; - return Platform.getInt(null, data + 4 * rowId); + return Platform.getInt(data, data.getBaseOffset() + 4 * rowId); } // @@ -333,39 +335,39 @@ public int getDictId(int rowId) { @Override public void putLong(int rowId, long value) { - Platform.putLong(null, data + 8 * rowId, value); + Platform.putLong(data, data.getBaseOffset() + 8 * rowId, value); } @Override public void putLongs(int rowId, int count, long value) { - long offset = data + 8 * rowId; + long offset = data.getBaseOffset() + 8 * rowId; for (int i = 0; i < count; ++i, offset += 8) { - Platform.putLong(null, offset, value); + Platform.putLong(data, offset, value); } } @Override public void putLongs(int rowId, int count, long[] src, int srcIndex) { Platform.copyMemory(src, Platform.LONG_ARRAY_OFFSET + srcIndex * 8, - null, data + 8 * rowId, count * 8); + data, data.getBaseOffset() + 8 * rowId, count * 8); } @Override public void putLongs(int rowId, int count, byte[] src, int srcIndex) { Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - null, data + rowId * 8, count * 8); + null, data.getBaseOffset() + rowId * 8, count * 8); } @Override public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { Platform.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, - null, data + 8 * rowId, count * 8); + data, data.getBaseOffset() + 8 * rowId, count * 8); } else { int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; - long offset = data + 8 * rowId; + long offset = data.getBaseOffset() + 8 * rowId; for (int i = 0; i < count; ++i, offset += 8, srcOffset += 8) { - Platform.putLong(null, offset, + Platform.putLong(data, offset, java.lang.Long.reverseBytes(Platform.getLong(src, srcOffset))); } } @@ -374,7 +376,7 @@ public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) @Override public long getLong(int rowId) { if (dictionary == null) { - return Platform.getLong(null, data + 8 * rowId); + return Platform.getLong(data, data.getBaseOffset() + 8 * rowId); } else { return dictionary.decodeToLong(dictionaryIds.getDictId(rowId)); } @@ -384,7 +386,7 @@ public long getLong(int rowId) { public long[] getLongs(int rowId, int count) { assert(dictionary == null); long[] array = new long[count]; - Platform.copyMemory(null, data + rowId * 8, array, Platform.LONG_ARRAY_OFFSET, count * 8); + Platform.copyMemory(data, data.getBaseOffset() + rowId * 8, array, Platform.LONG_ARRAY_OFFSET, count * 8); return array; } @@ -394,33 +396,33 @@ public long[] getLongs(int rowId, int count) { @Override public void putFloat(int rowId, float value) { - Platform.putFloat(null, data + rowId * 4, value); + Platform.putFloat(data, data.getBaseOffset() + rowId * 4, value); } @Override public void putFloats(int rowId, int count, float value) { - long offset = data + 4 * rowId; + long offset = data.getBaseOffset() + 4 * rowId; for (int i = 0; i < count; ++i, offset += 4) { - Platform.putFloat(null, offset, value); + Platform.putFloat(data, offset, value); } } @Override public void putFloats(int rowId, int count, float[] src, int srcIndex) { Platform.copyMemory(src, Platform.FLOAT_ARRAY_OFFSET + srcIndex * 4, - null, data + 4 * rowId, count * 4); + data, data.getBaseOffset() + 4 * rowId, count * 4); } @Override public void putFloats(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - null, data + rowId * 4, count * 4); + data, data.getBaseOffset() + rowId * 4, count * 4); } else { ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); - long offset = data + 4 * rowId; + long offset = data.getBaseOffset() + 4 * rowId; for (int i = 0; i < count; ++i, offset += 4) { - Platform.putFloat(null, offset, bb.getFloat(srcIndex + (4 * i))); + Platform.putFloat(data, offset, bb.getFloat(srcIndex + (4 * i))); } } } @@ -428,7 +430,7 @@ public void putFloats(int rowId, int count, byte[] src, int srcIndex) { @Override public float getFloat(int rowId) { if (dictionary == null) { - return Platform.getFloat(null, data + rowId * 4); + return Platform.getFloat(data, data.getBaseOffset() + rowId * 4); } else { return dictionary.decodeToFloat(dictionaryIds.getDictId(rowId)); } @@ -438,7 +440,7 @@ public float getFloat(int rowId) { public float[] getFloats(int rowId, int count) { assert(dictionary == null); float[] array = new float[count]; - Platform.copyMemory(null, data + rowId * 4, array, Platform.FLOAT_ARRAY_OFFSET, count * 4); + Platform.copyMemory(data, data.getBaseOffset() + rowId * 4, array, Platform.FLOAT_ARRAY_OFFSET, count * 4); return array; } @@ -449,33 +451,33 @@ public float[] getFloats(int rowId, int count) { @Override public void putDouble(int rowId, double value) { - Platform.putDouble(null, data + rowId * 8, value); + Platform.putDouble(data, data.getBaseOffset() + rowId * 8, value); } @Override public void putDoubles(int rowId, int count, double value) { - long offset = data + 8 * rowId; + long offset = data.getBaseOffset() + 8 * rowId; for (int i = 0; i < count; ++i, offset += 8) { - Platform.putDouble(null, offset, value); + Platform.putDouble(data, offset, value); } } @Override public void putDoubles(int rowId, int count, double[] src, int srcIndex) { Platform.copyMemory(src, Platform.DOUBLE_ARRAY_OFFSET + srcIndex * 8, - null, data + 8 * rowId, count * 8); + data, data.getBaseOffset() + 8 * rowId, count * 8); } @Override public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - null, data + rowId * 8, count * 8); + data, data.getBaseOffset() + rowId * 8, count * 8); } else { ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); - long offset = data + 8 * rowId; + long offset = data.getBaseOffset() + 8 * rowId; for (int i = 0; i < count; ++i, offset += 8) { - Platform.putDouble(null, offset, bb.getDouble(srcIndex + (8 * i))); + Platform.putDouble(data, offset, bb.getDouble(srcIndex + (8 * i))); } } } @@ -483,7 +485,7 @@ public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { @Override public double getDouble(int rowId) { if (dictionary == null) { - return Platform.getDouble(null, data + rowId * 8); + return Platform.getDouble(data, data.getBaseOffset() + rowId * 8); } else { return dictionary.decodeToDouble(dictionaryIds.getDictId(rowId)); } @@ -493,7 +495,7 @@ public double getDouble(int rowId) { public double[] getDoubles(int rowId, int count) { assert(dictionary == null); double[] array = new double[count]; - Platform.copyMemory(null, data + rowId * 8, array, Platform.DOUBLE_ARRAY_OFFSET, count * 8); + Platform.copyMemory(data, data.getBaseOffset() + rowId * 8, array, Platform.DOUBLE_ARRAY_OFFSET, count * 8); return array; } @@ -503,55 +505,55 @@ public double[] getDoubles(int rowId, int count) { @Override public void putArray(int rowId, int offset, int length) { assert(offset >= 0 && offset + length <= childColumns[0].capacity); - Platform.putInt(null, lengthData + 4 * rowId, length); - Platform.putInt(null, offsetData + 4 * rowId, offset); + Platform.putInt(lengthData, lengthData.getBaseOffset() + 4 * rowId, length); + Platform.putInt(offsetData, offsetData.getBaseOffset() + 4 * rowId, offset); } @Override public int getArrayLength(int rowId) { - return Platform.getInt(null, lengthData + 4 * rowId); + return Platform.getInt(lengthData, lengthData.getBaseOffset() + 4 * rowId); } @Override public int getArrayOffset(int rowId) { - return Platform.getInt(null, offsetData + 4 * rowId); + return Platform.getInt(offsetData, offsetData.getBaseOffset() + 4 * rowId); } // APIs dealing with ByteArrays @Override public int putByteArray(int rowId, byte[] value, int offset, int length) { int result = arrayData().appendBytes(length, value, offset); - Platform.putInt(null, lengthData + 4 * rowId, length); - Platform.putInt(null, offsetData + 4 * rowId, result); + Platform.putInt(lengthData, lengthData.getBaseOffset() + 4 * rowId, length); + Platform.putInt(offsetData, offsetData.getBaseOffset() + 4 * rowId, result); return result; } // Split out the slow path. @Override protected void reserveInternal(int newCapacity) { - int oldCapacity = (nulls == 0L) ? 0 : capacity; + int oldCapacity = (nulls == OffHeapMemoryBlock.NULL) ? 0 : capacity; if (isArray() || type instanceof MapType) { this.lengthData = - Platform.reallocateMemory(lengthData, oldCapacity * 4, newCapacity * 4); + MemoryAllocator.UNSAFE.reallocate(lengthData, oldCapacity * 4, newCapacity * 4); this.offsetData = - Platform.reallocateMemory(offsetData, oldCapacity * 4, newCapacity * 4); + MemoryAllocator.UNSAFE.reallocate(offsetData, oldCapacity * 4, newCapacity * 4); } else if (type instanceof ByteType || type instanceof BooleanType) { - this.data = Platform.reallocateMemory(data, oldCapacity, newCapacity); + this.data = MemoryAllocator.UNSAFE.reallocate(data, oldCapacity, newCapacity); } else if (type instanceof ShortType) { - this.data = Platform.reallocateMemory(data, oldCapacity * 2, newCapacity * 2); + this.data = MemoryAllocator.UNSAFE.reallocate(data, oldCapacity * 2, newCapacity * 2); } else if (type instanceof IntegerType || type instanceof FloatType || type instanceof DateType || DecimalType.is32BitDecimalType(type)) { - this.data = Platform.reallocateMemory(data, oldCapacity * 4, newCapacity * 4); + this.data = MemoryAllocator.UNSAFE.reallocate(data, oldCapacity * 4, newCapacity * 4); } else if (type instanceof LongType || type instanceof DoubleType || DecimalType.is64BitDecimalType(type) || type instanceof TimestampType) { - this.data = Platform.reallocateMemory(data, oldCapacity * 8, newCapacity * 8); + this.data = MemoryAllocator.UNSAFE.reallocate(data, oldCapacity * 8, newCapacity * 8); } else if (childColumns != null) { // Nothing to store. } else { throw new RuntimeException("Unhandled " + type); } - this.nulls = Platform.reallocateMemory(nulls, oldCapacity, newCapacity); - Platform.setMemory(nulls + oldCapacity, (byte)0, newCapacity - oldCapacity); + this.nulls = MemoryAllocator.UNSAFE.reallocate(nulls, oldCapacity, newCapacity); + Platform.setMemory(nulls.getBaseOffset() + oldCapacity, (byte)0, newCapacity - oldCapacity); capacity = newCapacity; } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala index 4845cac6b93dd..2b93a98f28477 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala @@ -37,7 +37,7 @@ class SortBenchmark extends BenchmarkBase { private def referenceKeyPrefixSort(buf: LongArray, lo: Int, hi: Int, refCmp: PrefixComparator) { val sortBuffer = - new LongArray(LongArrayMemoryBlock.fromLongArray(new Array[Long](buf.size().toInt))) + new LongArray(LongArrayMemoryBlock.fromArray(new Array[Long](buf.size().toInt))) new Sorter(new UnsafeSortDataFormat(sortBuffer)).sort( buf, lo, hi, new Comparator[RecordPointerAndKeyPrefix] { override def compare( @@ -51,8 +51,8 @@ class SortBenchmark extends BenchmarkBase { private def generateKeyPrefixTestData(size: Int, rand: => Long): (LongArray, LongArray) = { val ref = Array.tabulate[Long](size * 2) { i => rand } val extended = ref ++ Array.fill[Long](size * 2)(0) - (new LongArray(LongArrayMemoryBlock.fromLongArray(ref)), - new LongArray(LongArrayMemoryBlock.fromLongArray(extended))) + (new LongArray(LongArrayMemoryBlock.fromArray(ref)), + new LongArray(LongArrayMemoryBlock.fromArray(extended))) } ignore("sort") { @@ -61,7 +61,7 @@ class SortBenchmark extends BenchmarkBase { val benchmark = new Benchmark("radix sort " + size, size) benchmark.addTimerCase("reference TimSort key prefix array") { timer => val array = Array.tabulate[Long](size * 2) { i => rand.nextLong } - val buf = new LongArray(LongArrayMemoryBlock.fromLongArray(array)) + val buf = new LongArray(LongArrayMemoryBlock.fromArray(array)) timer.startTiming() referenceKeyPrefixSort(buf, 0, size, PrefixComparators.BINARY) timer.stopTiming() @@ -79,7 +79,7 @@ class SortBenchmark extends BenchmarkBase { array(i) = rand.nextLong & 0xff i += 1 } - val buf = new LongArray(LongArrayMemoryBlock.fromLongArray(array)) + val buf = new LongArray(LongArrayMemoryBlock.fromArray(array)) timer.startTiming() RadixSort.sort(buf, size, 0, 7, false, false) timer.stopTiming() @@ -91,7 +91,7 @@ class SortBenchmark extends BenchmarkBase { array(i) = rand.nextLong & 0xffff i += 1 } - val buf = new LongArray(LongArrayMemoryBlock.fromLongArray(array)) + val buf = new LongArray(LongArrayMemoryBlock.fromArray(array)) timer.startTiming() RadixSort.sort(buf, size, 0, 7, false, false) timer.stopTiming() @@ -103,7 +103,7 @@ class SortBenchmark extends BenchmarkBase { array(i) = rand.nextLong i += 1 } - val buf = new LongArray(LongArrayMemoryBlock.fromLongArray(array)) + val buf = new LongArray(LongArrayMemoryBlock.fromArray(array)) timer.startTiming() RadixSort.sort(buf, size, 0, 7, false, false) timer.stopTiming() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala index 5690a43ba28ec..baf7281269089 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.util.Utils class RowQueueSuite extends SparkFunSuite { test("in-memory queue") { - val page = LongArrayMemoryBlock.fromLongArray(new Array[Long](1<<10)) + val page = LongArrayMemoryBlock.fromArray(new Array[Long](1<<10)) val queue = new InMemoryRowQueue(page, 1) { override def close() {} } From 251fa09d4421085a6c97f56bb38108980a79b5c8 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 13 Sep 2017 18:27:09 +0100 Subject: [PATCH 03/65] UTF8String uses UnsafeMemoryAllocator --- .../spark/unsafe/array/ByteArrayMethods.java | 37 +++++++++++++++++ .../spark/unsafe/hash/Murmur3_x86_32.java | 26 +++++++++++- .../apache/spark/unsafe/types/UTF8String.java | 40 ++++++++++++------- .../spark/unsafe/types/UTF8StringSuite.java | 13 ++++-- .../catalyst/expressions/UnsafeArrayData.java | 5 ++- .../sql/catalyst/expressions/UnsafeRow.java | 5 ++- 6 files changed, 104 insertions(+), 22 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java index a6b1f7a16d605..fca2a4ad71967 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java @@ -18,6 +18,7 @@ package org.apache.spark.unsafe.array; import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.memory.MemoryBlock; public class ByteArrayMethods { @@ -52,6 +53,42 @@ public static int roundNumberOfBytesToNearestWord(int numBytes) { * Optimized byte array equality check for byte arrays. * @return true if the arrays are equal, false otherwise */ + public static boolean arrayEquals( + MemoryBlock leftBase, long leftOffset, MemoryBlock rightBase, long rightOffset, final long length) { + int i = 0; + + // check if stars align and we can get both offsets to be aligned + if ((leftOffset % 8) == (rightOffset % 8)) { + while ((leftOffset + i) % 8 != 0 && i < length) { + if (Platform.getByte(leftBase, leftOffset + i) != + Platform.getByte(rightBase, rightOffset + i)) { + return false; + } + i += 1; + } + } + // for architectures that suport unaligned accesses, chew it up 8 bytes at a time + if (unaligned || (((leftOffset + i) % 8 == 0) && ((rightOffset + i) % 8 == 0))) { + while (i <= length - 8) { + if (Platform.getLong(leftBase, leftOffset + i) != + Platform.getLong(rightBase, rightOffset + i)) { + return false; + } + i += 8; + } + } + // this will finish off the unaligned comparisons, or do the entire aligned + // comparison whichever is needed. + while (i < length) { + if (Platform.getByte(leftBase, leftOffset + i) != + Platform.getByte(rightBase, rightOffset + i)) { + return false; + } + i += 1; + } + return true; + } + public static boolean arrayEquals( Object leftBase, long leftOffset, Object rightBase, long rightOffset, final long length) { int i = 0; diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index d239de6083ad0..50090036a3b40 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -18,6 +18,7 @@ package org.apache.spark.unsafe.hash; import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.memory.MemoryBlock; /** * 32-bit Murmur3 hasher. This is based on Guava's Murmur3_32HashFunction. @@ -59,6 +60,18 @@ public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, i return fmix(h1, lengthInBytes); } + public static int hashUnsafeBytes(MemoryBlock base, long offset, int lengthInBytes, int seed) { + assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; + int lengthAligned = lengthInBytes - lengthInBytes % 4; + int h1 = hashBytesByInt(base, offset, lengthAligned, seed); + for (int i = lengthAligned; i < lengthInBytes; i++) { + int halfWord = Platform.getByte(base, offset + i); + int k1 = mixK1(halfWord); + h1 = mixH1(h1, k1); + } + return fmix(h1, lengthInBytes); + } + public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes, int seed) { // This is not compatible with original and another implementations. // But remain it for backward compatibility for the components existing before 2.3. @@ -76,7 +89,7 @@ public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes, i public static int hashUnsafeBytes2(Object base, long offset, int lengthInBytes, int seed) { // This is compatible with original and another implementations. // Use this method for new components after Spark 2.3. - assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; + assert (lengthInBytes >= 0) : "lengthInBytes cannot be negative"; int lengthAligned = lengthInBytes - lengthInBytes % 4; int h1 = hashBytesByInt(base, offset, lengthAligned, seed); int k1 = 0; @@ -87,6 +100,17 @@ public static int hashUnsafeBytes2(Object base, long offset, int lengthInBytes, return fmix(h1, lengthInBytes); } + private static int hashBytesByInt(MemoryBlock base, long offset, int lengthInBytes, int seed) { + assert (lengthInBytes % 4 == 0); + int h1 = seed; + for (int i = 0; i < lengthInBytes; i += 4) { + int halfWord = Platform.getInt(base, offset + i); + int k1 = mixK1(halfWord); + h1 = mixH1(h1, k1); + } + return h1; + } + private static int hashBytesByInt(Object base, long offset, int lengthInBytes, int seed) { assert (lengthInBytes % 4 == 0); int h1 = seed; diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index b0d0c44823e68..2be4d9a5a71c1 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -33,6 +33,8 @@ import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; +import org.apache.spark.unsafe.memory.ByteArrayMemoryBlock; +import org.apache.spark.unsafe.memory.MemoryBlock; import static org.apache.spark.unsafe.Platform.*; @@ -50,11 +52,11 @@ public final class UTF8String implements Comparable, Externalizable, // These are only updated by readExternal() or read() @Nonnull - private Object base; + private MemoryBlock base; private long offset; private int numBytes; - public Object getBaseObject() { return base; } + public MemoryBlock getBaseObject() { return base; } public long getBaseOffset() { return offset; } private static int[] bytesOfCodePointInUTF8 = {2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, @@ -99,7 +101,7 @@ public static UTF8String fromBytes(byte[] bytes, int offset, int numBytes) { /** * Creates an UTF8String from given address (base and offset) and length. */ - public static UTF8String fromAddress(Object base, long offset, int numBytes) { + public static UTF8String fromAddress(MemoryBlock base, long offset, int numBytes) { return new UTF8String(base, offset, numBytes); } @@ -119,7 +121,11 @@ public static UTF8String blankString(int length) { return fromBytes(spaces); } - protected UTF8String(Object base, long offset, int numBytes) { + protected UTF8String(byte[] bytes, long offset, int numBytes) { + this(ByteArrayMemoryBlock.fromArray(bytes), offset, numBytes); + } + + protected UTF8String(MemoryBlock base, long offset, int numBytes) { this.base = base; this.offset = offset; this.numBytes = numBytes; @@ -127,7 +133,7 @@ protected UTF8String(Object base, long offset, int numBytes) { // for serialization public UTF8String() { - this(null, 0, 0); + this((MemoryBlock)null, 0, 0); } /** @@ -135,7 +141,11 @@ public UTF8String() { * The target memory address must already been allocated, and have enough space to hold all the * bytes in this string. */ - public void writeToMemory(Object target, long targetOffset) { + public void writeToMemory(byte[] target, long targetOffset) { + Platform.copyMemory(base, offset, target, targetOffset, numBytes); + } + + public void writeToMemory(MemoryBlock target, long targetOffset) { Platform.copyMemory(base, offset, target, targetOffset, numBytes); } @@ -156,8 +166,8 @@ public void writeTo(ByteBuffer buffer) { */ @Nonnull public ByteBuffer getByteBuffer() { - if (base instanceof byte[] && offset >= BYTE_ARRAY_OFFSET) { - final byte[] bytes = (byte[]) base; + if (base instanceof ByteArrayMemoryBlock && offset >= BYTE_ARRAY_OFFSET) { + final byte[] bytes = ((ByteArrayMemoryBlock) base).getByteArray(); // the offset includes an object header... this is only needed for unsafe copies final long arrayOffset = offset - BYTE_ARRAY_OFFSET; @@ -257,9 +267,9 @@ public long getPrefix() { */ public byte[] getBytes() { // avoid copy if `base` is `byte[]` - if (offset == BYTE_ARRAY_OFFSET && base instanceof byte[] - && ((byte[]) base).length == numBytes) { - return (byte[]) base; + if (offset == BYTE_ARRAY_OFFSET && base instanceof ByteArrayMemoryBlock + && (((ByteArrayMemoryBlock) base).getByteArray()).length == numBytes) { + return ((ByteArrayMemoryBlock) base).getByteArray(); } else { byte[] bytes = new byte[numBytes]; copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, numBytes); @@ -1367,8 +1377,8 @@ public void writeExternal(ObjectOutput out) throws IOException { public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { offset = BYTE_ARRAY_OFFSET; numBytes = in.readInt(); - base = new byte[numBytes]; - in.readFully((byte[]) base); + base = ByteArrayMemoryBlock.fromArray(new byte[numBytes]); + in.readFully( ((ByteArrayMemoryBlock)base).getByteArray() ); } @Override @@ -1382,8 +1392,8 @@ public void write(Kryo kryo, Output out) { public void read(Kryo kryo, Input in) { this.offset = BYTE_ARRAY_OFFSET; this.numBytes = in.readInt(); - this.base = new byte[numBytes]; - in.read((byte[]) base); + base = ByteArrayMemoryBlock.fromArray(new byte[numBytes]); + in.read( ((ByteArrayMemoryBlock)base).getByteArray() ); } } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java index 9b303fa5bc6c5..86d50cdb3beff 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java @@ -26,6 +26,8 @@ import com.google.common.collect.ImmutableMap; import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.memory.ByteArrayMemoryBlock; +import org.apache.spark.unsafe.memory.IntArrayMemoryBlock; import org.junit.Test; import static org.junit.Assert.*; @@ -515,7 +517,8 @@ public void writeToOutputStreamUnderflow() throws IOException { final byte[] test = "01234567".getBytes(StandardCharsets.UTF_8); for (int i = 1; i <= Platform.BYTE_ARRAY_OFFSET; ++i) { - UTF8String.fromAddress(test, Platform.BYTE_ARRAY_OFFSET - i, test.length + i) + UTF8String.fromAddress(ByteArrayMemoryBlock.fromArray(test), + Platform.BYTE_ARRAY_OFFSET - i, test.length + i) .writeTo(outputStream); final ByteBuffer buffer = ByteBuffer.wrap(outputStream.toByteArray(), i, test.length); assertEquals("01234567", StandardCharsets.UTF_8.decode(buffer).toString()); @@ -530,7 +533,8 @@ public void writeToOutputStreamSlice() throws IOException { for (int i = 0; i < test.length; ++i) { for (int j = 0; j < test.length - i; ++j) { - UTF8String.fromAddress(test, Platform.BYTE_ARRAY_OFFSET + i, j) + UTF8String.fromAddress(ByteArrayMemoryBlock.fromArray(test), + Platform.BYTE_ARRAY_OFFSET + i, j) .writeTo(outputStream); assertArrayEquals(Arrays.copyOfRange(test, i, i + j), outputStream.toByteArray()); @@ -561,7 +565,8 @@ public void writeToOutputStreamOverflow() throws IOException { for (final long offset : offsets) { try { - fromAddress(test, BYTE_ARRAY_OFFSET + offset, test.length) + fromAddress(ByteArrayMemoryBlock.fromArray(test), + BYTE_ARRAY_OFFSET + offset, test.length) .writeTo(outputStream); throw new IllegalStateException(Long.toString(offset)); @@ -605,7 +610,7 @@ public void writeToOutputStreamIntArray() throws IOException { } final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - fromAddress(array, Platform.INT_ARRAY_OFFSET, length) + fromAddress(IntArrayMemoryBlock.fromArray(array), Platform.INT_ARRAY_OFFSET, length) .writeTo(outputStream); assertEquals("大千世界", outputStream.toString("UTF-8")); } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java index d18542b188f71..2e49431bce86d 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java @@ -27,6 +27,8 @@ import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.bitset.BitSetMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; +import org.apache.spark.unsafe.memory.ByteArrayMemoryBlock; +import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; @@ -230,7 +232,8 @@ public UTF8String getUTF8String(int ordinal) { final long offsetAndSize = getLong(ordinal); final int offset = (int) (offsetAndSize >> 32); final int size = (int) offsetAndSize; - return UTF8String.fromAddress(baseObject, baseOffset + offset, size); + MemoryBlock mb = new ByteArrayMemoryBlock((byte[])baseObject, baseOffset, size); + return UTF8String.fromAddress(mb, mb.getBaseOffset(), (int)mb.size()); } @Override diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index 71c086029cc5b..4c0e83bb99ec8 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -37,6 +37,8 @@ import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.bitset.BitSetMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; +import org.apache.spark.unsafe.memory.ByteArrayMemoryBlock; +import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; @@ -414,7 +416,8 @@ public UTF8String getUTF8String(int ordinal) { final long offsetAndSize = getLong(ordinal); final int offset = (int) (offsetAndSize >> 32); final int size = (int) offsetAndSize; - return UTF8String.fromAddress(baseObject, baseOffset + offset, size); + MemoryBlock mb = new ByteArrayMemoryBlock((byte[])baseObject, baseOffset + offset, size); + return UTF8String.fromAddress(mb, mb.getBaseOffset(), (int)mb.size()); } @Override From 790bbe7f3ac52b2b1e4375684998737d6127a552 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 13 Sep 2017 18:36:57 +0100 Subject: [PATCH 04/65] Platform.copymemory() in UsafeInMemorySorter uses new MemoryBlock --- .../util/collection/unsafe/sort/UnsafeInMemorySorter.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index 95866db51e36d..964aa72a3ad9d 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -29,7 +29,6 @@ import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.UnsafeAlignedOffset; import org.apache.spark.unsafe.array.LongArray; -import org.apache.spark.unsafe.memory.LongArrayMemoryBlock; import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.util.collection.Sorter; @@ -218,9 +217,9 @@ public void expandPointerArray(LongArray newArray) { throw new SparkOutOfMemoryError("Not enough memory to grow pointer array"); } Platform.copyMemory( - array.getBaseObject(), + array.memoryBlock(), array.getBaseOffset(), - newArray.getBaseObject(), + newArray.memoryBlock(), newArray.getBaseOffset(), pos * 8L); consumer.freeArray(array); From 93a792e7729f213fcef66f0ed9b33f45259d1ec3 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 14 Sep 2017 16:34:48 +0100 Subject: [PATCH 05/65] address review comments --- .../org/apache/spark/unsafe/Platform.java | 134 +------------ .../spark/unsafe/array/ByteArrayMethods.java | 18 +- .../apache/spark/unsafe/array/LongArray.java | 8 +- .../spark/unsafe/hash/Murmur3_x86_32.java | 10 +- .../unsafe/memory/ByteArrayMemoryBlock.java | 6 +- .../unsafe/memory/HeapMemoryAllocator.java | 2 +- .../unsafe/memory/IntArrayMemoryBlock.java | 8 +- .../unsafe/memory/LongArrayMemoryBlock.java | 8 +- .../spark/unsafe/memory/MemoryBlock.java | 180 ++++++++++++++++-- .../unsafe/memory/OffHeapMemoryBlock.java | 19 +- .../unsafe/memory/UnsafeMemoryAllocator.java | 22 +-- .../apache/spark/unsafe/types/ByteArray.java | 1 - .../apache/spark/unsafe/types/UTF8String.java | 54 +++--- .../shuffle/sort/ShuffleInMemorySorter.java | 4 +- .../shuffle/sort/ShuffleSortDataFormat.java | 4 +- .../unsafe/sort/UnsafeInMemorySorter.java | 3 +- .../vectorized/OffHeapColumnVector.java | 109 +++++------ 17 files changed, 300 insertions(+), 290 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java index 7245b35d443f0..e39959594ccc4 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java @@ -22,7 +22,6 @@ import java.lang.reflect.Method; import java.nio.ByteBuffer; -import org.apache.spark.unsafe.memory.MemoryBlock; import sun.misc.Cleaner; import sun.misc.Unsafe; @@ -76,131 +75,67 @@ public static boolean unaligned() { return unaligned; } - public static int getInt(MemoryBlock object, long offset) { - return _UNSAFE.getInt(object.getBaseObject(), offset); - } - public static int getInt(Object object, long offset) { return _UNSAFE.getInt(object, offset); } - public static void putInt(MemoryBlock object, long offset, int value) { - _UNSAFE.putInt(object.getBaseObject(), offset, value); - } - public static void putInt(Object object, long offset, int value) { _UNSAFE.putInt(object, offset, value); } - public static boolean getBoolean(MemoryBlock object, long offset) { - return _UNSAFE.getBoolean(object.getBaseObject(), offset); - } - public static boolean getBoolean(Object object, long offset) { return _UNSAFE.getBoolean(object, offset); } - public static void putBoolean(MemoryBlock object, long offset, boolean value) { - _UNSAFE.putBoolean(object.getBaseObject(), offset, value); - } - public static void putBoolean(Object object, long offset, boolean value) { _UNSAFE.putBoolean(object, offset, value); } - public static byte getByte(MemoryBlock object, long offset) { - return _UNSAFE.getByte(object.getBaseObject(), offset); - } - public static byte getByte(Object object, long offset) { return _UNSAFE.getByte(object, offset); } - public static void putByte(MemoryBlock object, long offset, byte value) { - _UNSAFE.putByte(object.getBaseObject(), offset, value); - } - public static void putByte(Object object, long offset, byte value) { _UNSAFE.putByte(object, offset, value); } - public static short getShort(MemoryBlock object, long offset) { - return _UNSAFE.getShort(object.getBaseObject(), offset); - } - public static short getShort(Object object, long offset) { return _UNSAFE.getShort(object, offset); } - public static void putShort(MemoryBlock object, long offset, short value) { - _UNSAFE.putShort(object.getBaseObject(), offset, value); - } - public static void putShort(Object object, long offset, short value) { _UNSAFE.putShort(object, offset, value); } - public static long getLong(MemoryBlock object, long offset) { - return _UNSAFE.getLong(object.getBaseObject(), offset); - } - public static long getLong(Object object, long offset) { return _UNSAFE.getLong(object, offset); } - public static void putLong(MemoryBlock object, long offset, long value) { - _UNSAFE.putLong(object.getBaseObject(), offset, value); - } - public static void putLong(Object object, long offset, long value) { _UNSAFE.putLong(object, offset, value); } - public static float getFloat(MemoryBlock object, long offset) { - return _UNSAFE.getFloat(object.getBaseObject(), offset); - } - public static float getFloat(Object object, long offset) { return _UNSAFE.getFloat(object, offset); } - public static void putFloat(MemoryBlock object, long offset, float value) { - _UNSAFE.putFloat(object.getBaseObject(), offset, value); - } - public static void putFloat(Object object, long offset, float value) { _UNSAFE.putFloat(object, offset, value); } - public static double getDouble(MemoryBlock object, long offset) { - return _UNSAFE.getDouble(object.getBaseObject(), offset); - } - public static double getDouble(Object object, long offset) { return _UNSAFE.getDouble(object, offset); } - public static void putDouble(MemoryBlock object, long offset, double value) { - _UNSAFE.putDouble(object.getBaseObject(), offset, value); - } - public static void putDouble(Object object, long offset, double value) { _UNSAFE.putDouble(object, offset, value); } - public static Object getObjectVolatile(MemoryBlock object, long offset) { - return _UNSAFE.getObjectVolatile(object.getBaseObject(), offset); - } - - public static Object getObjectVolatile(byte[] object, long offset) { + public static Object getObjectVolatile(Object object, long offset) { return _UNSAFE.getObjectVolatile(object, offset); } - public static void putObjectVolatile(MemoryBlock object, long offset, Object value) { - _UNSAFE.putObjectVolatile(object.getBaseObject(), offset, value); - } - - public static void putObjectVolatile(byte[] object, long offset, Object value) { + public static void putObjectVolatile(Object object, long offset, Object value) { _UNSAFE.putObjectVolatile(object, offset, value); } @@ -270,71 +205,6 @@ public static void copyMemory( } } - public static void copyMemory( - MemoryBlock src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { - Platform.copyMemory(src.getBaseObject(), srcOffset, dst.getBaseObject(), dstOffset, length); - } - - public static void copyMemory( - byte[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { - Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); - } - - public static void copyMemory( - short[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { - Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); - } - - public static void copyMemory( - int[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { - Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); - } - - public static void copyMemory( - long[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { - Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); - } - - public static void copyMemory( - float[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { - Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); - } - - public static void copyMemory( - double[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { - Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); - } - - public static void copyMemory( - MemoryBlock src, long srcOffset, byte[] dst, long dstOffset, long length) { - Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); - } - - public static void copyMemory( - MemoryBlock src, long srcOffset, short[] dst, long dstOffset, long length) { - Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); - } - - public static void copyMemory( - MemoryBlock src, long srcOffset, int[] dst, long dstOffset, long length) { - Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); - } - - public static void copyMemory( - MemoryBlock src, long srcOffset, long[] dst, long dstOffset, long length) { - Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); - } - - public static void copyMemory( - MemoryBlock src, long srcOffset, float[] dst, long dstOffset, long length) { - Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); - } - - public static void copyMemory( - MemoryBlock src, long srcOffset, double[] dst, long dstOffset, long length) { - Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); - } - /** * Raises an exception bypassing compiler checks for checked exceptions. */ diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java index fca2a4ad71967..fad303b9f09ec 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java @@ -57,21 +57,21 @@ public static boolean arrayEquals( MemoryBlock leftBase, long leftOffset, MemoryBlock rightBase, long rightOffset, final long length) { int i = 0; - // check if stars align and we can get both offsets to be aligned + // check if starts align and we can get both offsets to be aligned if ((leftOffset % 8) == (rightOffset % 8)) { while ((leftOffset + i) % 8 != 0 && i < length) { - if (Platform.getByte(leftBase, leftOffset + i) != - Platform.getByte(rightBase, rightOffset + i)) { + if (MemoryBlock.getByte(leftBase, leftOffset + i) != + MemoryBlock.getByte(rightBase, rightOffset + i)) { return false; } i += 1; } } - // for architectures that suport unaligned accesses, chew it up 8 bytes at a time + // for architectures that support unaligned accesses, chew it up 8 bytes at a time if (unaligned || (((leftOffset + i) % 8 == 0) && ((rightOffset + i) % 8 == 0))) { while (i <= length - 8) { - if (Platform.getLong(leftBase, leftOffset + i) != - Platform.getLong(rightBase, rightOffset + i)) { + if (MemoryBlock.getLong(leftBase, leftOffset + i) != + MemoryBlock.getLong(rightBase, rightOffset + i)) { return false; } i += 8; @@ -80,8 +80,8 @@ public static boolean arrayEquals( // this will finish off the unaligned comparisons, or do the entire aligned // comparison whichever is needed. while (i < length) { - if (Platform.getByte(leftBase, leftOffset + i) != - Platform.getByte(rightBase, rightOffset + i)) { + if (MemoryBlock.getByte(leftBase, leftOffset + i) != + MemoryBlock.getByte(rightBase, rightOffset + i)) { return false; } i += 1; @@ -93,7 +93,7 @@ public static boolean arrayEquals( Object leftBase, long leftOffset, Object rightBase, long rightOffset, final long length) { int i = 0; - // check if stars align and we can get both offsets to be aligned + // check if starts align and we can get both offsets to be aligned if ((leftOffset % 8) == (rightOffset % 8)) { while ((leftOffset + i) % 8 != 0 && i < length) { if (Platform.getByte(leftBase, leftOffset + i) != diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java index 97baa2c6cdfcb..eb9b2172c3ce3 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java @@ -17,8 +17,6 @@ package org.apache.spark.unsafe.array; -import org.apache.spark.unsafe.Platform; -import org.apache.spark.unsafe.memory.LongArrayMemoryBlock; import org.apache.spark.unsafe.memory.MemoryBlock; /** @@ -69,7 +67,7 @@ public long size() { */ public void zeroOut() { for (long off = baseOffset; off < baseOffset + length * WIDTH; off += WIDTH) { - Platform.putLong(memory, off, 0); + MemoryBlock.putLong(memory, off, 0); } } @@ -79,7 +77,7 @@ public void zeroOut() { public void set(int index, long value) { assert index >= 0 : "index (" + index + ") should >= 0"; assert index < length : "index (" + index + ") should < length (" + length + ")"; - Platform.putLong(memory, baseOffset + index * WIDTH, value); + MemoryBlock.putLong(memory, baseOffset + index * WIDTH, value); } /** @@ -88,6 +86,6 @@ public void set(int index, long value) { public long get(int index) { assert index >= 0 : "index (" + index + ") should >= 0"; assert index < length : "index (" + index + ") should < length (" + length + ")"; - return Platform.getLong(memory, baseOffset + index * WIDTH); + return MemoryBlock.getLong(memory, baseOffset + index * WIDTH); } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index 50090036a3b40..8e428fde3f69c 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -60,12 +60,12 @@ public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, i return fmix(h1, lengthInBytes); } - public static int hashUnsafeBytes(MemoryBlock base, long offset, int lengthInBytes, int seed) { + public static int hashUnsafeBytesMB(MemoryBlock base, long offset, int lengthInBytes, int seed) { assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; int lengthAligned = lengthInBytes - lengthInBytes % 4; - int h1 = hashBytesByInt(base, offset, lengthAligned, seed); + int h1 = hashBytesByIntMB(base, offset, lengthAligned, seed); for (int i = lengthAligned; i < lengthInBytes; i++) { - int halfWord = Platform.getByte(base, offset + i); + int halfWord = MemoryBlock.getByte(base, offset + i); int k1 = mixK1(halfWord); h1 = mixH1(h1, k1); } @@ -100,11 +100,11 @@ public static int hashUnsafeBytes2(Object base, long offset, int lengthInBytes, return fmix(h1, lengthInBytes); } - private static int hashBytesByInt(MemoryBlock base, long offset, int lengthInBytes, int seed) { + private static int hashBytesByIntMB(MemoryBlock base, long offset, int lengthInBytes, int seed) { assert (lengthInBytes % 4 == 0); int h1 = seed; for (int i = 0; i < lengthInBytes; i += 4) { - int halfWord = Platform.getInt(base, offset + i); + int halfWord = MemoryBlock.getInt(base, offset + i); int k1 = mixK1(halfWord); h1 = mixH1(h1, k1); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java index 0ecf2cd26f94c..7c31f003bb6f7 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -22,14 +22,14 @@ /** * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. */ -public class ByteArrayMemoryBlock extends MemoryLocation implements MemoryBlock { +public class ByteArrayMemoryBlock extends MemoryBlock { private final long length; /** * Optional page number; used when this MemoryBlock represents a page allocated by a - * TaskMemoryManager. This field is public so that it can be modified by the TaskMemoryManager, - * which lives in a different package. + * TaskMemoryManager. This field can be updated using setPageNumber method so that + * this can be modified by the TaskMemoryManage, which lives in a different package. */ private int pageNumber = NO_PAGE_NUMBER; diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java index da2c9ceee742d..aaa251f7d285e 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java @@ -101,7 +101,7 @@ public void free(MemoryBlock memory) { // As an additional layer of defense against use-after-free bugs, we mutate the // MemoryBlock to null out its reference to the long[] array. long[] array = ((LongArrayMemoryBlock)memory).getLongArray(); - ((LongArrayMemoryBlock)memory).setObjAndOffset(null, 0); + memory.resetObjAndOffset(); long alignedSize = ((size + 7) / 8) * 8; if (shouldPool(alignedSize)) { diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java index 2069d850ece9b..01d98cd08fa89 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java @@ -22,16 +22,16 @@ /** * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. */ -public class IntArrayMemoryBlock extends MemoryLocation implements MemoryBlock { +public class IntArrayMemoryBlock extends MemoryBlock { private final long size; /** * Optional page number; used when this MemoryBlock represents a page allocated by a - * TaskMemoryManager. This field is public so that it can be modified by the TaskMemoryManager, - * which lives in a different package. + * TaskMemoryManager. This field can be updated using setPageNumber method so that + * this can be modified by the TaskMemoryManage, which lives in a different package. */ - private int pageNumber = -1; + private int pageNumber = NO_PAGE_NUMBER; public IntArrayMemoryBlock(int[] obj, long offset, long size) { super(obj, offset); diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java index f7fd21044646e..a24c79b54f795 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java @@ -22,16 +22,16 @@ /** * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. */ -public class LongArrayMemoryBlock extends MemoryLocation implements MemoryBlock { +public class LongArrayMemoryBlock extends MemoryBlock { private final long size; /** * Optional page number; used when this MemoryBlock represents a page allocated by a - * TaskMemoryManager. This field is public so that it can be modified by the TaskMemoryManager, - * which lives in a different package. + * TaskMemoryManager. This field can be updated using setPageNumber method so that + * this can be modified by the TaskMemoryManage, which lives in a different package. */ - private int pageNumber = -1; + private int pageNumber = NO_PAGE_NUMBER; public LongArrayMemoryBlock(long[] obj, long offset, long size) { super(obj, offset); diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index 6632669708d9a..e190319301aa6 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -17,14 +17,14 @@ package org.apache.spark.unsafe.memory; -import javax.annotation.Nullable; - import org.apache.spark.unsafe.Platform; +import javax.annotation.Nullable; + /** - * A declaration of interfaces of MemoryBloock classes . + * A declaration of interfaces of MemoryBlock classes . */ -public interface MemoryBlock { +public abstract class MemoryBlock { /** Special `pageNumber` value for pages which were not allocated by TaskMemoryManagers */ public static final int NO_PAGE_NUMBER = -1; @@ -44,26 +44,180 @@ public interface MemoryBlock { */ public static final int FREED_IN_ALLOCATOR_PAGE_NUMBER = -3; + @Nullable + protected Object obj; + + protected long offset; + + public MemoryBlock(@Nullable Object obj, long offset) { + this.obj = obj; + this.offset = offset; + } + + public MemoryBlock() { + this(null, 0); + } + + public final Object getBaseObject() { + return obj; + } + + public final long getBaseOffset() { + return offset; + } + + public final void resetObjAndOffset() { + this.obj = null; + this.offset = 0; + } + /** * Returns the size of the memory block. */ - long size(); + public abstract long size(); - Object getBaseObject(); + public abstract void setPageNumber(int pageNum); - long getBaseOffset(); - - void setPageNumber(int pageNum); - - int getPageNumber(); + public abstract int getPageNumber(); /** * Fills the memory block with the specified byte value. */ - void fill(byte value); + public abstract void fill(byte value); /** * Instantiate the same type of MemoryBlock with new offset and size */ - MemoryBlock allocate(long offset, long size); + public abstract MemoryBlock allocate(long offset, long size); + + + public static final int getInt(MemoryBlock object, long offset) { + return Platform.getInt(object.getBaseObject(), offset); + } + + public static final void putInt(MemoryBlock object, long offset, int value) { + Platform.putInt(object.getBaseObject(), offset, value); + } + + + public static final boolean getBoolean(MemoryBlock object, long offset) { + return Platform.getBoolean(object.getBaseObject(), offset); + } + + public static final void putBoolean(MemoryBlock object, long offset, boolean value) { + Platform.putBoolean(object.getBaseObject(), offset, value); + } + + public static final byte getByte(MemoryBlock object, long offset) { + return Platform.getByte(object.getBaseObject(), offset); + } + + public static final void putByte(MemoryBlock object, long offset, byte value) { + Platform.putByte(object.getBaseObject(), offset, value); + } + + public static final short getShort(MemoryBlock object, long offset) { + return Platform.getShort(object.getBaseObject(), offset); + } + + public static final void putShort(MemoryBlock object, long offset, short value) { + Platform.putShort(object.getBaseObject(), offset, value); + } + + public static final long getLong(MemoryBlock object, long offset) { + return Platform.getLong(object.getBaseObject(), offset); + } + + public static final void putLong(MemoryBlock object, long offset, long value) { + Platform.putLong(object.getBaseObject(), offset, value); + } + + public static final float getFloat(MemoryBlock object, long offset) { + return Platform.getFloat(object.getBaseObject(), offset); + } + + public static final void putFloat(MemoryBlock object, long offset, float value) { + Platform.putFloat(object.getBaseObject(), offset, value); + } + + public static final double getDouble(MemoryBlock object, long offset) { + return Platform.getDouble(object.getBaseObject(), offset); + } + + public static final void putDouble(MemoryBlock object, long offset, double value) { + Platform.putDouble(object.getBaseObject(), offset, value); + } + + public static final Object getObjectVolatile(MemoryBlock object, long offset) { + return Platform.getObjectVolatile(object.getBaseObject(), offset); + } + + public static final void putObjectVolatile(MemoryBlock object, long offset, Object value) { + Platform.putObjectVolatile(object.getBaseObject(), offset, value); + } + + public static final void copyMemory( + MemoryBlock src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { + Platform.copyMemory(src.getBaseObject(), srcOffset, dst.getBaseObject(), dstOffset, length); + } + + public static final void copyMemory( + byte[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { + Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); + } + + public static final void copyMemory( + short[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { + Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); + } + + public static final void copyMemory( + int[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { + Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); + } + + public static final void copyMemory( + long[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { + Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); + } + + public static void copyMemory( + float[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { + Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); + } + + public static void copyMemory( + double[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { + Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); + } + + public static void copyMemory( + MemoryBlock src, long srcOffset, byte[] dst, long dstOffset, long length) { + Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); + } + + public static void copyMemory( + MemoryBlock src, long srcOffset, short[] dst, long dstOffset, long length) { + Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); + } + + public static void copyMemory( + MemoryBlock src, long srcOffset, int[] dst, long dstOffset, long length) { + Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); + } + + public static void copyMemory( + MemoryBlock src, long srcOffset, long[] dst, long dstOffset, long length) { + Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); + } + + public static void copyMemory( + MemoryBlock src, long srcOffset, float[] dst, long dstOffset, long length) { + Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); + } + + public static void copyMemory( + MemoryBlock src, long srcOffset, double[] dst, long dstOffset, long length) { + Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); + } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java index 69f6a3a068846..086dd37195479 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java @@ -19,30 +19,27 @@ import org.apache.spark.unsafe.Platform; -public class OffHeapMemoryBlock implements MemoryBlock { +public class OffHeapMemoryBlock extends MemoryBlock { private Object directBuffer; private long address; private final long length; + + /** + * Optional page number; used when this MemoryBlock represents a page allocated by a + * TaskMemoryManager. This field can be updated using setPageNumber method so that + * this can be modified by the TaskMemoryManage, which lives in a different package. + */ private int pageNumber = -1; static public final OffHeapMemoryBlock NULL = new OffHeapMemoryBlock(null, 0, 0); public OffHeapMemoryBlock(Object directBuffer, long address, long size) { + super(null, address); this.address = address; this.length = size; this.directBuffer = directBuffer; } - @Override - public Object getBaseObject() { - return null; - } - - @Override - public long getBaseOffset() { - return this.address; - } - public void setBaseOffset(long address) { this.address = address; } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java index 7b9574a8b99dc..cd69305b24555 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -23,6 +23,8 @@ import java.lang.reflect.Method; import java.nio.ByteBuffer; +import sun.nio.ch.DirectBuffer; + /** * A simple {@link MemoryAllocator} that uses {@code Unsafe} to allocate off-heap memory. */ @@ -42,20 +44,12 @@ public class UnsafeMemoryAllocator implements MemoryAllocator { @Override public OffHeapMemoryBlock allocate(long size) throws OutOfMemoryError { - try { - Object b = ByteBuffer.allocateDirect((int)size); - long addr = (long)bufAddrMethod.invoke(b); - return new OffHeapMemoryBlock(b, addr, size); - } catch (IllegalAccessException e) { - throw new RuntimeException(e.getMessage(), e); - } catch (InvocationTargetException e) { - Throwable ex = e.getTargetException(); - if (ex instanceof OutOfMemoryError) { - throw (OutOfMemoryError) ex; - } else { - throw new RuntimeException(e.getMessage(), e); - } + Object buffer = ByteBuffer.allocateDirect((int)size); + if (buffer instanceof DirectBuffer) { + long addr = ((DirectBuffer) buffer).address(); + return new OffHeapMemoryBlock(buffer, addr, size); } + throw new UnsupportedOperationException("A ByteBuffer does not have an address in off-heap"); } @Override @@ -86,7 +80,7 @@ public void free(MemoryBlock memory) { public OffHeapMemoryBlock reallocate(OffHeapMemoryBlock block, long oldSize, long newSize) { OffHeapMemoryBlock mb = this.allocate(newSize); if (block.getBaseOffset() != 0) - Platform.copyMemory(block, block.getBaseOffset(), mb, mb.getBaseOffset(), oldSize); + MemoryBlock.copyMemory(block, block.getBaseOffset(), mb, mb.getBaseOffset(), oldSize); return mb; } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java index 92d9b8059e682..c03caf0076f61 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java @@ -18,7 +18,6 @@ package org.apache.spark.unsafe.types; import org.apache.spark.unsafe.Platform; -import org.apache.spark.unsafe.memory.MemoryBlock; import java.util.Arrays; diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 2be4d9a5a71c1..c80d674994bd9 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -142,11 +142,11 @@ public UTF8String() { * bytes in this string. */ public void writeToMemory(byte[] target, long targetOffset) { - Platform.copyMemory(base, offset, target, targetOffset, numBytes); + MemoryBlock.copyMemory(base, offset, target, targetOffset, numBytes); } public void writeToMemory(MemoryBlock target, long targetOffset) { - Platform.copyMemory(base, offset, target, targetOffset, numBytes); + MemoryBlock.copyMemory(base, offset, target, targetOffset, numBytes); } public void writeTo(ByteBuffer buffer) { @@ -233,12 +233,12 @@ public long getPrefix() { long mask = 0; if (IS_LITTLE_ENDIAN) { if (numBytes >= 8) { - p = Platform.getLong(base, offset); + p = MemoryBlock.getLong(base, offset); } else if (numBytes > 4) { - p = Platform.getLong(base, offset); + p = MemoryBlock.getLong(base, offset); mask = (1L << (8 - numBytes) * 8) - 1; } else if (numBytes > 0) { - p = (long) Platform.getInt(base, offset); + p = (long) MemoryBlock.getInt(base, offset); mask = (1L << (8 - numBytes) * 8) - 1; } else { p = 0; @@ -247,12 +247,12 @@ public long getPrefix() { } else { // byteOrder == ByteOrder.BIG_ENDIAN if (numBytes >= 8) { - p = Platform.getLong(base, offset); + p = MemoryBlock.getLong(base, offset); } else if (numBytes > 4) { - p = Platform.getLong(base, offset); + p = MemoryBlock.getLong(base, offset); mask = (1L << (8 - numBytes) * 8) - 1; } else if (numBytes > 0) { - p = ((long) Platform.getInt(base, offset)) << 32; + p = ((long) MemoryBlock.getInt(base, offset)) << 32; mask = (1L << (8 - numBytes) * 8) - 1; } else { p = 0; @@ -272,7 +272,7 @@ public byte[] getBytes() { return ((ByteArrayMemoryBlock) base).getByteArray(); } else { byte[] bytes = new byte[numBytes]; - copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, numBytes); + MemoryBlock.copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, numBytes); return bytes; } } @@ -302,7 +302,7 @@ public UTF8String substring(final int start, final int until) { if (i > j) { byte[] bytes = new byte[i - j]; - copyMemory(base, offset + j, bytes, BYTE_ARRAY_OFFSET, i - j); + MemoryBlock.copyMemory(base, offset + j, bytes, BYTE_ARRAY_OFFSET, i - j); return fromBytes(bytes); } else { return EMPTY_UTF8; @@ -343,7 +343,7 @@ public boolean contains(final UTF8String substring) { * Returns the byte at position `i`. */ private byte getByte(int i) { - return Platform.getByte(base, offset + i); + return MemoryBlock.getByte(base, offset + i); } private boolean matchAt(final UTF8String s, int pos) { @@ -502,7 +502,7 @@ public int findInSet(UTF8String match) { private UTF8String copyUTF8String(int start, int end) { int len = end - start + 1; byte[] newBytes = new byte[len]; - copyMemory(base, offset + start, newBytes, BYTE_ARRAY_OFFSET, len); + MemoryBlock.copyMemory(base, offset + start, newBytes, BYTE_ARRAY_OFFSET, len); return UTF8String.fromBytes(newBytes); } @@ -649,7 +649,7 @@ public UTF8String reverse() { int i = 0; // position in byte while (i < numBytes) { int len = numBytesForFirstByte(getByte(i)); - copyMemory(this.base, this.offset + i, result, + MemoryBlock.copyMemory(this.base, this.offset + i, result, BYTE_ARRAY_OFFSET + result.length - i - len, len); i += len; @@ -664,7 +664,7 @@ public UTF8String repeat(int times) { } byte[] newBytes = new byte[numBytes * times]; - copyMemory(this.base, this.offset, newBytes, BYTE_ARRAY_OFFSET, numBytes); + MemoryBlock.copyMemory(this.base, this.offset, newBytes, BYTE_ARRAY_OFFSET, numBytes); int copied = 1; while (copied < times) { @@ -764,7 +764,7 @@ public UTF8String subStringIndex(UTF8String delim, int count) { return EMPTY_UTF8; } byte[] bytes = new byte[idx]; - copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, idx); + MemoryBlock.copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, idx); return fromBytes(bytes); } else { @@ -784,7 +784,7 @@ public UTF8String subStringIndex(UTF8String delim, int count) { } int size = numBytes - delim.numBytes - idx; byte[] bytes = new byte[size]; - copyMemory(base, offset + idx + delim.numBytes, bytes, BYTE_ARRAY_OFFSET, size); + MemoryBlock.copyMemory(base, offset + idx + delim.numBytes, bytes, BYTE_ARRAY_OFFSET, size); return fromBytes(bytes); } } @@ -807,15 +807,15 @@ public UTF8String rpad(int len, UTF8String pad) { UTF8String remain = pad.substring(0, spaces - padChars * count); byte[] data = new byte[this.numBytes + pad.numBytes * count + remain.numBytes]; - copyMemory(this.base, this.offset, data, BYTE_ARRAY_OFFSET, this.numBytes); + MemoryBlock.copyMemory(this.base, this.offset, data, BYTE_ARRAY_OFFSET, this.numBytes); int offset = this.numBytes; int idx = 0; while (idx < count) { - copyMemory(pad.base, pad.offset, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); + MemoryBlock.copyMemory(pad.base, pad.offset, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); ++ idx; offset += pad.numBytes; } - copyMemory(remain.base, remain.offset, data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); + MemoryBlock.copyMemory(remain.base, remain.offset, data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); return UTF8String.fromBytes(data); } @@ -843,13 +843,13 @@ public UTF8String lpad(int len, UTF8String pad) { int offset = 0; int idx = 0; while (idx < count) { - copyMemory(pad.base, pad.offset, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); + MemoryBlock.copyMemory(pad.base, pad.offset, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); ++ idx; offset += pad.numBytes; } - copyMemory(remain.base, remain.offset, data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); + MemoryBlock.copyMemory(remain.base, remain.offset, data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); offset += remain.numBytes; - copyMemory(this.base, this.offset, data, BYTE_ARRAY_OFFSET + offset, numBytes()); + MemoryBlock.copyMemory(this.base, this.offset, data, BYTE_ARRAY_OFFSET + offset, numBytes()); return UTF8String.fromBytes(data); } @@ -874,7 +874,7 @@ public static UTF8String concat(UTF8String... inputs) { int offset = 0; for (int i = 0; i < inputs.length; i++) { int len = inputs[i].numBytes; - copyMemory( + MemoryBlock.copyMemory( inputs[i].base, inputs[i].offset, result, BYTE_ARRAY_OFFSET + offset, len); @@ -914,7 +914,7 @@ public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) { for (int i = 0, j = 0; i < inputs.length; i++) { if (inputs[i] != null) { int len = inputs[i].numBytes; - copyMemory( + MemoryBlock.copyMemory( inputs[i].base, inputs[i].offset, result, BYTE_ARRAY_OFFSET + offset, len); @@ -923,7 +923,7 @@ public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) { j++; // Add separator if this is not the last input. if (j < numInputs) { - copyMemory( + MemoryBlock.copyMemory( separator.base, separator.offset, result, BYTE_ARRAY_OFFSET + offset, separator.numBytes); @@ -1198,7 +1198,7 @@ public UTF8String clone() { public UTF8String copy() { byte[] bytes = new byte[numBytes]; - copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, numBytes); + MemoryBlock.copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, numBytes); return fromBytes(bytes); } @@ -1312,7 +1312,7 @@ public int levenshteinDistance(UTF8String other) { @Override public int hashCode() { - return Murmur3_x86_32.hashUnsafeBytes(base, offset, numBytes, 42); + return Murmur3_x86_32.hashUnsafeBytesMB(base, offset, numBytes, 42); } /** diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java index 9bb2853f55a6a..cc6699932bf3c 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java @@ -20,9 +20,7 @@ import java.util.Comparator; import org.apache.spark.memory.MemoryConsumer; -import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.LongArray; -import org.apache.spark.unsafe.memory.LongArrayMemoryBlock; import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.util.collection.Sorter; import org.apache.spark.util.collection.unsafe.sort.RadixSort; @@ -106,7 +104,7 @@ public void reset() { public void expandPointerArray(LongArray newArray) { assert(newArray.size() > array.size()); - Platform.copyMemory( + MemoryBlock.copyMemory( array.memoryBlock(), array.getBaseOffset(), newArray.memoryBlock(), diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java index 8bf2d5d1fa94d..8dbe06905ef57 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java @@ -17,8 +17,8 @@ package org.apache.spark.shuffle.sort; -import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.LongArray; +import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.util.collection.SortDataFormat; final class ShuffleSortDataFormat extends SortDataFormat { @@ -60,7 +60,7 @@ public void copyElement(LongArray src, int srcPos, LongArray dst, int dstPos) { @Override public void copyRange(LongArray src, int srcPos, LongArray dst, int dstPos, int length) { - Platform.copyMemory( + MemoryBlock.copyMemory( src.memoryBlock(), src.getBaseOffset() + srcPos * 8L, dst.memoryBlock(), diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index 964aa72a3ad9d..a4a24c8c62c0c 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -26,7 +26,6 @@ import org.apache.spark.memory.MemoryConsumer; import org.apache.spark.memory.SparkOutOfMemoryError; import org.apache.spark.memory.TaskMemoryManager; -import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.UnsafeAlignedOffset; import org.apache.spark.unsafe.array.LongArray; import org.apache.spark.unsafe.memory.MemoryBlock; @@ -216,7 +215,7 @@ public void expandPointerArray(LongArray newArray) { if (newArray.size() < array.size()) { throw new SparkOutOfMemoryError("Not enough memory to grow pointer array"); } - Platform.copyMemory( + MemoryBlock.copyMemory( array.memoryBlock(), array.getBaseOffset(), newArray.memoryBlock(), diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java index cded0b4b9dcd7..024217bced02a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -25,6 +25,7 @@ import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.types.UTF8String; import org.apache.spark.unsafe.memory.MemoryAllocator; +import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.unsafe.memory.OffHeapMemoryBlock; /** @@ -105,12 +106,12 @@ public void close() { @Override public void putNotNull(int rowId) { - Platform.putByte(nulls, nulls.getBaseOffset() + rowId, (byte) 0); + MemoryBlock.putByte(nulls, nulls.getBaseOffset() + rowId, (byte) 0); } @Override public void putNull(int rowId) { - Platform.putByte(nulls, nulls.getBaseOffset() + rowId, (byte) 1); + MemoryBlock.putByte(nulls, nulls.getBaseOffset() + rowId, (byte) 1); ++numNulls; } @@ -118,7 +119,7 @@ public void putNull(int rowId) { public void putNulls(int rowId, int count) { long offset = nulls.getBaseOffset() + rowId; for (int i = 0; i < count; ++i, ++offset) { - Platform.putByte(nulls, offset, (byte) 1); + MemoryBlock.putByte(nulls, offset, (byte) 1); } numNulls += count; } @@ -128,13 +129,13 @@ public void putNotNulls(int rowId, int count) { if (!hasNull()) return; long offset = nulls.getBaseOffset() + rowId; for (int i = 0; i < count; ++i, ++offset) { - Platform.putByte(nulls, offset, (byte) 0); + MemoryBlock.putByte(nulls, offset, (byte) 0); } } @Override public boolean isNullAt(int rowId) { - return Platform.getByte(nulls, nulls.getBaseOffset() + rowId) == 1; + return MemoryBlock.getByte(nulls, nulls.getBaseOffset() + rowId) == 1; } // @@ -143,26 +144,26 @@ public boolean isNullAt(int rowId) { @Override public void putBoolean(int rowId, boolean value) { - Platform.putByte(data, data.getBaseOffset() + rowId, (byte)((value) ? 1 : 0)); + MemoryBlock.putByte(data, data.getBaseOffset() + rowId, (byte)((value) ? 1 : 0)); } @Override public void putBooleans(int rowId, int count, boolean value) { byte v = (byte)((value) ? 1 : 0); for (int i = 0; i < count; ++i) { - Platform.putByte(data, data.getBaseOffset() + rowId + i, v); + MemoryBlock.putByte(data, data.getBaseOffset() + rowId + i, v); } } @Override - public boolean getBoolean(int rowId) { return Platform.getByte(data, data.getBaseOffset() + rowId) == 1; } + public boolean getBoolean(int rowId) { return MemoryBlock.getByte(data, data.getBaseOffset() + rowId) == 1; } @Override public boolean[] getBooleans(int rowId, int count) { assert(dictionary == null); boolean[] array = new boolean[count]; for (int i = 0; i < count; ++i) { - array[i] = (Platform.getByte(data, data.getBaseOffset() + rowId + i) == 1); + array[i] = (MemoryBlock.getByte(data, data.getBaseOffset() + rowId + i) == 1); } return array; } @@ -173,26 +174,26 @@ public boolean[] getBooleans(int rowId, int count) { @Override public void putByte(int rowId, byte value) { - Platform.putByte(data, data.getBaseOffset() + rowId, value); + MemoryBlock.putByte(data, data.getBaseOffset() + rowId, value); } @Override public void putBytes(int rowId, int count, byte value) { for (int i = 0; i < count; ++i) { - Platform.putByte(data, data.getBaseOffset() + rowId + i, value); + MemoryBlock.putByte(data, data.getBaseOffset() + rowId + i, value); } } @Override public void putBytes(int rowId, int count, byte[] src, int srcIndex) { - Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, data, data.getBaseOffset() + rowId, count); + MemoryBlock.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, data, data.getBaseOffset() + rowId, count); } @Override public byte getByte(int rowId) { if (dictionary == null) { - return Platform.getByte(data, data.getBaseOffset() + rowId); + return MemoryBlock.getByte(data, data.getBaseOffset() + rowId); } else { return (byte) dictionary.decodeToInt(dictionaryIds.getDictId(rowId)); } @@ -202,7 +203,7 @@ public byte getByte(int rowId) { public byte[] getBytes(int rowId, int count) { assert(dictionary == null); byte[] array = new byte[count]; - Platform.copyMemory(data, data.getBaseOffset() + rowId, array, Platform.BYTE_ARRAY_OFFSET, count); + MemoryBlock.copyMemory(data, data.getBaseOffset() + rowId, array, Platform.BYTE_ARRAY_OFFSET, count); return array; } @@ -217,20 +218,20 @@ protected UTF8String getBytesAsUTF8String(int rowId, int count) { @Override public void putShort(int rowId, short value) { - Platform.putShort(data, data.getBaseOffset() + 2 * rowId, value); + MemoryBlock.putShort(data, data.getBaseOffset() + 2 * rowId, value); } @Override public void putShorts(int rowId, int count, short value) { long offset = data.getBaseOffset() + 2 * rowId; for (int i = 0; i < count; ++i, offset += 2) { - Platform.putShort(data, offset, value); + MemoryBlock.putShort(data, offset, value); } } @Override public void putShorts(int rowId, int count, short[] src, int srcIndex) { - Platform.copyMemory(src, Platform.SHORT_ARRAY_OFFSET + srcIndex * 2, + MemoryBlock.copyMemory(src, Platform.SHORT_ARRAY_OFFSET + srcIndex * 2, data, data.getBaseOffset() + 2 * rowId, count * 2); } @@ -243,7 +244,7 @@ public void putShorts(int rowId, int count, byte[] src, int srcIndex) { @Override public short getShort(int rowId) { if (dictionary == null) { - return Platform.getShort(data, data.getBaseOffset() + 2 * rowId); + return MemoryBlock.getShort(data, data.getBaseOffset() + 2 * rowId); } else { return (short) dictionary.decodeToInt(dictionaryIds.getDictId(rowId)); } @@ -253,7 +254,7 @@ public short getShort(int rowId) { public short[] getShorts(int rowId, int count) { assert(dictionary == null); short[] array = new short[count]; - Platform.copyMemory(data, data.getBaseOffset() + rowId * 2, array, Platform.SHORT_ARRAY_OFFSET, count * 2); + MemoryBlock.copyMemory(data, data.getBaseOffset() + rowId * 2, array, Platform.SHORT_ARRAY_OFFSET, count * 2); return array; } @@ -263,20 +264,20 @@ public short[] getShorts(int rowId, int count) { @Override public void putInt(int rowId, int value) { - Platform.putInt(data, data.getBaseOffset() + 4 * rowId, value); + MemoryBlock.putInt(data, data.getBaseOffset() + 4 * rowId, value); } @Override public void putInts(int rowId, int count, int value) { long offset = data.getBaseOffset() + 4 * rowId; for (int i = 0; i < count; ++i, offset += 4) { - Platform.putInt(data, offset, value); + MemoryBlock.putInt(data, offset, value); } } @Override public void putInts(int rowId, int count, int[] src, int srcIndex) { - Platform.copyMemory(src, Platform.INT_ARRAY_OFFSET + srcIndex * 4, + MemoryBlock.copyMemory(src, Platform.INT_ARRAY_OFFSET + srcIndex * 4, data, data.getBaseOffset() + 4 * rowId, count * 4); } @@ -289,13 +290,13 @@ public void putInts(int rowId, int count, byte[] src, int srcIndex) { @Override public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { - Platform.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, + MemoryBlock.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, data, data.getBaseOffset() + 4 * rowId, count * 4); } else { int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; long offset = data.getBaseOffset() + 4 * rowId; for (int i = 0; i < count; ++i, offset += 4, srcOffset += 4) { - Platform.putInt(data, offset, + MemoryBlock.putInt(data, offset, java.lang.Integer.reverseBytes(Platform.getInt(src, srcOffset))); } } @@ -304,7 +305,7 @@ public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) @Override public int getInt(int rowId) { if (dictionary == null) { - return Platform.getInt(data, data.getBaseOffset() + 4 * rowId); + return MemoryBlock.getInt(data, data.getBaseOffset() + 4 * rowId); } else { return dictionary.decodeToInt(dictionaryIds.getDictId(rowId)); } @@ -314,7 +315,7 @@ public int getInt(int rowId) { public int[] getInts(int rowId, int count) { assert(dictionary == null); int[] array = new int[count]; - Platform.copyMemory(data, data.getBaseOffset() + rowId * 4, array, Platform.INT_ARRAY_OFFSET, count * 4); + MemoryBlock.copyMemory(data, data.getBaseOffset() + rowId * 4, array, Platform.INT_ARRAY_OFFSET, count * 4); return array; } @@ -326,7 +327,7 @@ public int[] getInts(int rowId, int count) { public int getDictId(int rowId) { assert(dictionary == null) : "A ColumnVector dictionary should not have a dictionary for itself."; - return Platform.getInt(data, data.getBaseOffset() + 4 * rowId); + return MemoryBlock.getInt(data, data.getBaseOffset() + 4 * rowId); } // @@ -335,20 +336,20 @@ public int getDictId(int rowId) { @Override public void putLong(int rowId, long value) { - Platform.putLong(data, data.getBaseOffset() + 8 * rowId, value); + MemoryBlock.putLong(data, data.getBaseOffset() + 8 * rowId, value); } @Override public void putLongs(int rowId, int count, long value) { long offset = data.getBaseOffset() + 8 * rowId; for (int i = 0; i < count; ++i, offset += 8) { - Platform.putLong(data, offset, value); + MemoryBlock.putLong(data, offset, value); } } @Override public void putLongs(int rowId, int count, long[] src, int srcIndex) { - Platform.copyMemory(src, Platform.LONG_ARRAY_OFFSET + srcIndex * 8, + MemoryBlock.copyMemory(src, Platform.LONG_ARRAY_OFFSET + srcIndex * 8, data, data.getBaseOffset() + 8 * rowId, count * 8); } @@ -361,13 +362,13 @@ public void putLongs(int rowId, int count, byte[] src, int srcIndex) { @Override public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { - Platform.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, + MemoryBlock.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, data, data.getBaseOffset() + 8 * rowId, count * 8); } else { int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; long offset = data.getBaseOffset() + 8 * rowId; for (int i = 0; i < count; ++i, offset += 8, srcOffset += 8) { - Platform.putLong(data, offset, + MemoryBlock.putLong(data, offset, java.lang.Long.reverseBytes(Platform.getLong(src, srcOffset))); } } @@ -376,7 +377,7 @@ public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) @Override public long getLong(int rowId) { if (dictionary == null) { - return Platform.getLong(data, data.getBaseOffset() + 8 * rowId); + return MemoryBlock.getLong(data, data.getBaseOffset() + 8 * rowId); } else { return dictionary.decodeToLong(dictionaryIds.getDictId(rowId)); } @@ -386,7 +387,7 @@ public long getLong(int rowId) { public long[] getLongs(int rowId, int count) { assert(dictionary == null); long[] array = new long[count]; - Platform.copyMemory(data, data.getBaseOffset() + rowId * 8, array, Platform.LONG_ARRAY_OFFSET, count * 8); + MemoryBlock.copyMemory(data, data.getBaseOffset() + rowId * 8, array, Platform.LONG_ARRAY_OFFSET, count * 8); return array; } @@ -396,33 +397,33 @@ public long[] getLongs(int rowId, int count) { @Override public void putFloat(int rowId, float value) { - Platform.putFloat(data, data.getBaseOffset() + rowId * 4, value); + MemoryBlock.putFloat(data, data.getBaseOffset() + rowId * 4, value); } @Override public void putFloats(int rowId, int count, float value) { long offset = data.getBaseOffset() + 4 * rowId; for (int i = 0; i < count; ++i, offset += 4) { - Platform.putFloat(data, offset, value); + MemoryBlock.putFloat(data, offset, value); } } @Override public void putFloats(int rowId, int count, float[] src, int srcIndex) { - Platform.copyMemory(src, Platform.FLOAT_ARRAY_OFFSET + srcIndex * 4, + MemoryBlock.copyMemory(src, Platform.FLOAT_ARRAY_OFFSET + srcIndex * 4, data, data.getBaseOffset() + 4 * rowId, count * 4); } @Override public void putFloats(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { - Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + MemoryBlock.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, data, data.getBaseOffset() + rowId * 4, count * 4); } else { ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); long offset = data.getBaseOffset() + 4 * rowId; for (int i = 0; i < count; ++i, offset += 4) { - Platform.putFloat(data, offset, bb.getFloat(srcIndex + (4 * i))); + MemoryBlock.putFloat(data, offset, bb.getFloat(srcIndex + (4 * i))); } } } @@ -430,7 +431,7 @@ public void putFloats(int rowId, int count, byte[] src, int srcIndex) { @Override public float getFloat(int rowId) { if (dictionary == null) { - return Platform.getFloat(data, data.getBaseOffset() + rowId * 4); + return MemoryBlock.getFloat(data, data.getBaseOffset() + rowId * 4); } else { return dictionary.decodeToFloat(dictionaryIds.getDictId(rowId)); } @@ -440,7 +441,7 @@ public float getFloat(int rowId) { public float[] getFloats(int rowId, int count) { assert(dictionary == null); float[] array = new float[count]; - Platform.copyMemory(data, data.getBaseOffset() + rowId * 4, array, Platform.FLOAT_ARRAY_OFFSET, count * 4); + MemoryBlock.copyMemory(data, data.getBaseOffset() + rowId * 4, array, Platform.FLOAT_ARRAY_OFFSET, count * 4); return array; } @@ -451,33 +452,33 @@ public float[] getFloats(int rowId, int count) { @Override public void putDouble(int rowId, double value) { - Platform.putDouble(data, data.getBaseOffset() + rowId * 8, value); + MemoryBlock.putDouble(data, data.getBaseOffset() + rowId * 8, value); } @Override public void putDoubles(int rowId, int count, double value) { long offset = data.getBaseOffset() + 8 * rowId; for (int i = 0; i < count; ++i, offset += 8) { - Platform.putDouble(data, offset, value); + MemoryBlock.putDouble(data, offset, value); } } @Override public void putDoubles(int rowId, int count, double[] src, int srcIndex) { - Platform.copyMemory(src, Platform.DOUBLE_ARRAY_OFFSET + srcIndex * 8, + MemoryBlock.copyMemory(src, Platform.DOUBLE_ARRAY_OFFSET + srcIndex * 8, data, data.getBaseOffset() + 8 * rowId, count * 8); } @Override public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { - Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + MemoryBlock.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, data, data.getBaseOffset() + rowId * 8, count * 8); } else { ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); long offset = data.getBaseOffset() + 8 * rowId; for (int i = 0; i < count; ++i, offset += 8) { - Platform.putDouble(data, offset, bb.getDouble(srcIndex + (8 * i))); + MemoryBlock.putDouble(data, offset, bb.getDouble(srcIndex + (8 * i))); } } } @@ -485,7 +486,7 @@ public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { @Override public double getDouble(int rowId) { if (dictionary == null) { - return Platform.getDouble(data, data.getBaseOffset() + rowId * 8); + return MemoryBlock.getDouble(data, data.getBaseOffset() + rowId * 8); } else { return dictionary.decodeToDouble(dictionaryIds.getDictId(rowId)); } @@ -495,7 +496,7 @@ public double getDouble(int rowId) { public double[] getDoubles(int rowId, int count) { assert(dictionary == null); double[] array = new double[count]; - Platform.copyMemory(data, data.getBaseOffset() + rowId * 8, array, Platform.DOUBLE_ARRAY_OFFSET, count * 8); + MemoryBlock.copyMemory(data, data.getBaseOffset() + rowId * 8, array, Platform.DOUBLE_ARRAY_OFFSET, count * 8); return array; } @@ -505,26 +506,26 @@ public double[] getDoubles(int rowId, int count) { @Override public void putArray(int rowId, int offset, int length) { assert(offset >= 0 && offset + length <= childColumns[0].capacity); - Platform.putInt(lengthData, lengthData.getBaseOffset() + 4 * rowId, length); - Platform.putInt(offsetData, offsetData.getBaseOffset() + 4 * rowId, offset); + MemoryBlock.putInt(lengthData, lengthData.getBaseOffset() + 4 * rowId, length); + MemoryBlock.putInt(offsetData, offsetData.getBaseOffset() + 4 * rowId, offset); } @Override public int getArrayLength(int rowId) { - return Platform.getInt(lengthData, lengthData.getBaseOffset() + 4 * rowId); + return MemoryBlock.getInt(lengthData, lengthData.getBaseOffset() + 4 * rowId); } @Override public int getArrayOffset(int rowId) { - return Platform.getInt(offsetData, offsetData.getBaseOffset() + 4 * rowId); + return MemoryBlock.getInt(offsetData, offsetData.getBaseOffset() + 4 * rowId); } // APIs dealing with ByteArrays @Override public int putByteArray(int rowId, byte[] value, int offset, int length) { int result = arrayData().appendBytes(length, value, offset); - Platform.putInt(lengthData, lengthData.getBaseOffset() + 4 * rowId, length); - Platform.putInt(offsetData, offsetData.getBaseOffset() + 4 * rowId, result); + MemoryBlock.putInt(lengthData, lengthData.getBaseOffset() + 4 * rowId, length); + MemoryBlock.putInt(offsetData, offsetData.getBaseOffset() + 4 * rowId, result); return result; } From 0beab0308cdac57e85daf7794170a0ca899ab568 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 14 Sep 2017 18:53:33 +0100 Subject: [PATCH 06/65] fix test failures (e.g. String in UnsafeArrayData) --- .../org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java | 2 +- .../main/java/org/apache/spark/unsafe/types/UTF8String.java | 4 ++-- .../spark/sql/catalyst/expressions/UnsafeArrayData.java | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java index 086dd37195479..e4a1541e12db1 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java @@ -56,7 +56,7 @@ public void fill(byte value) { @Override public MemoryBlock allocate(long offset, long size) { - return new OffHeapMemoryBlock(address, offset, size); + return new OffHeapMemoryBlock(directBuffer, address + offset, size); } @Override diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index c80d674994bd9..6e517a54940f6 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -1378,7 +1378,7 @@ public void readExternal(ObjectInput in) throws IOException, ClassNotFoundExcept offset = BYTE_ARRAY_OFFSET; numBytes = in.readInt(); base = ByteArrayMemoryBlock.fromArray(new byte[numBytes]); - in.readFully( ((ByteArrayMemoryBlock)base).getByteArray() ); + in.readFully(((ByteArrayMemoryBlock)base).getByteArray()); } @Override @@ -1393,7 +1393,7 @@ public void read(Kryo kryo, Input in) { this.offset = BYTE_ARRAY_OFFSET; this.numBytes = in.readInt(); base = ByteArrayMemoryBlock.fromArray(new byte[numBytes]); - in.read( ((ByteArrayMemoryBlock)base).getByteArray() ); + in.read(((ByteArrayMemoryBlock)base).getByteArray()); } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java index 2e49431bce86d..58ab26bbed819 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java @@ -232,7 +232,7 @@ public UTF8String getUTF8String(int ordinal) { final long offsetAndSize = getLong(ordinal); final int offset = (int) (offsetAndSize >> 32); final int size = (int) offsetAndSize; - MemoryBlock mb = new ByteArrayMemoryBlock((byte[])baseObject, baseOffset, size); + MemoryBlock mb = new ByteArrayMemoryBlock((byte[])baseObject, baseOffset + offset, size); return UTF8String.fromAddress(mb, mb.getBaseOffset(), (int)mb.size()); } From fcf764c1aebdc847675f710c17ec8477d6022a40 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Mon, 18 Sep 2017 17:13:13 +0100 Subject: [PATCH 07/65] fix failures --- .../sql/catalyst/expressions/HiveHasher.java | 5 +++ .../spark/unsafe/hash/Murmur3_x86_32.java | 19 ++---------- .../sql/catalyst/expressions/UnsafeRow.java | 7 +++-- .../spark/sql/catalyst/expressions/XXH64.java | 5 +++ .../spark/sql/catalyst/expressions/hash.scala | 31 ++++++++++++++++--- 5 files changed, 42 insertions(+), 25 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java index 73577437ac506..a20ee19874427 100644 --- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions; import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.memory.MemoryBlock; /** * Simulates Hive's hashing function from Hive v1.2.1 @@ -38,6 +39,10 @@ public static int hashLong(long input) { return (int) ((input >>> 32) ^ input); } + public static int hashUnsafeBytesMB(MemoryBlock base, long offset, int lengthInBytes) { + return hashUnsafeBytes(base.getBaseObject(), offset, lengthInBytes); + } + public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes) { assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; int result = 0; diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index 8e428fde3f69c..d9811484d5b9f 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -61,15 +61,7 @@ public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, i } public static int hashUnsafeBytesMB(MemoryBlock base, long offset, int lengthInBytes, int seed) { - assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; - int lengthAligned = lengthInBytes - lengthInBytes % 4; - int h1 = hashBytesByIntMB(base, offset, lengthAligned, seed); - for (int i = lengthAligned; i < lengthInBytes; i++) { - int halfWord = MemoryBlock.getByte(base, offset + i); - int k1 = mixK1(halfWord); - h1 = mixH1(h1, k1); - } - return fmix(h1, lengthInBytes); + return hashUnsafeBytes(base.getBaseObject(), offset, lengthInBytes, seed); } public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes, int seed) { @@ -101,14 +93,7 @@ public static int hashUnsafeBytes2(Object base, long offset, int lengthInBytes, } private static int hashBytesByIntMB(MemoryBlock base, long offset, int lengthInBytes, int seed) { - assert (lengthInBytes % 4 == 0); - int h1 = seed; - for (int i = 0; i < lengthInBytes; i += 4) { - int halfWord = MemoryBlock.getInt(base, offset + i); - int k1 = mixK1(halfWord); - h1 = mixH1(h1, k1); - } - return h1; + return hashBytesByInt(base.getBaseObject(), offset, lengthInBytes, seed); } private static int hashBytesByInt(Object base, long offset, int lengthInBytes, int seed) { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index 4c0e83bb99ec8..8a0228e5094b2 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -37,8 +37,7 @@ import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.bitset.BitSetMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; -import org.apache.spark.unsafe.memory.ByteArrayMemoryBlock; -import org.apache.spark.unsafe.memory.MemoryBlock; +import org.apache.spark.unsafe.memory.*; import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; @@ -416,7 +415,9 @@ public UTF8String getUTF8String(int ordinal) { final long offsetAndSize = getLong(ordinal); final int offset = (int) (offsetAndSize >> 32); final int size = (int) offsetAndSize; - MemoryBlock mb = new ByteArrayMemoryBlock((byte[])baseObject, baseOffset + offset, size); + MemoryBlock mb = (baseObject instanceof byte[]) ? + new ByteArrayMemoryBlock((byte[])baseObject, baseOffset + offset, size) : + new LongArrayMemoryBlock((long[])baseObject, baseOffset + offset, size); return UTF8String.fromAddress(mb, mb.getBaseOffset(), (int)mb.size()); } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java index f37ef83ad92b4..445348b5a4ff2 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions; import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.memory.MemoryBlock; // scalastyle: off /** @@ -85,6 +86,10 @@ public long hashUnsafeBytes(Object base, long offset, int length) { return hashUnsafeBytes(base, offset, length, seed); } + public static long hashUnsafeBytesMB(MemoryBlock base, long offset, int length, long seed) { + return hashUnsafeBytes(base.getBaseObject(), offset, length, seed); + } + public static long hashUnsafeBytes(Object base, long offset, int length, long seed) { assert (length >= 0) : "lengthInBytes cannot be negative"; long hash = hashBytesByWords(base, offset, length, seed); diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala index 055ebf6c0da54..e961a48b00882 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.hash.Murmur3_x86_32 +import org.apache.spark.unsafe.memory.MemoryBlock import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} //////////////////////////////////////////////////////////////////////////////////////////////////// @@ -362,7 +363,7 @@ abstract class HashExpression[E] extends Expression { val baseObject = s"$input.getBaseObject()" val baseOffset = s"$input.getBaseOffset()" val numBytes = s"$input.numBytes()" - s"$result = $hasherClassName.hashUnsafeBytes($baseObject, $baseOffset, $numBytes, $result);" + s"$result = $hasherClassName.hashUnsafeBytesMB($baseObject, $baseOffset, $numBytes, $result);" } protected def genHashForMap( @@ -464,6 +465,8 @@ abstract class InterpretedHashFunction { protected def hashUnsafeBytes(base: AnyRef, offset: Long, length: Int, seed: Long): Long + protected def hashUnsafeBytesMB(base: MemoryBlock, offset: Long, length: Int, seed: Long): Long + /** * Computes hash of a given `value` of type `dataType`. The caller needs to check the validity * of input `value`. @@ -490,7 +493,7 @@ abstract class InterpretedHashFunction { case a: Array[Byte] => hashUnsafeBytes(a, Platform.BYTE_ARRAY_OFFSET, a.length, seed) case s: UTF8String => - hashUnsafeBytes(s.getBaseObject, s.getBaseOffset, s.numBytes(), seed) + hashUnsafeBytesMB(s.getBaseObject, s.getBaseOffset, s.numBytes(), seed) case array: ArrayData => val elementType = dataType match { @@ -577,9 +580,15 @@ object Murmur3HashFunction extends InterpretedHashFunction { Murmur3_x86_32.hashLong(l, seed.toInt) } - override protected def hashUnsafeBytes(base: AnyRef, offset: Long, len: Int, seed: Long): Long = { + override protected def hashUnsafeBytes( + base: AnyRef, offset: Long, len: Int, seed: Long): Long = { Murmur3_x86_32.hashUnsafeBytes(base, offset, len, seed.toInt) } + + override protected def hashUnsafeBytesMB( + base: MemoryBlock, offset: Long, len: Int, seed: Long): Long = { + Murmur3_x86_32.hashUnsafeBytesMB(base, offset, len, seed.toInt) + } } /** @@ -604,9 +613,15 @@ object XxHash64Function extends InterpretedHashFunction { override protected def hashLong(l: Long, seed: Long): Long = XXH64.hashLong(l, seed) - override protected def hashUnsafeBytes(base: AnyRef, offset: Long, len: Int, seed: Long): Long = { + override protected def hashUnsafeBytes( + base: AnyRef, offset: Long, len: Int, seed: Long): Long = { XXH64.hashUnsafeBytes(base, offset, len, seed) } + + override protected def hashUnsafeBytesMB( + base: MemoryBlock, offset: Long, len: Int, seed: Long): Long = { + XXH64.hashUnsafeBytesMB(base, offset, len, seed) + } } /** @@ -804,10 +819,16 @@ object HiveHashFunction extends InterpretedHashFunction { HiveHasher.hashLong(l) } - override protected def hashUnsafeBytes(base: AnyRef, offset: Long, len: Int, seed: Long): Long = { + override protected def hashUnsafeBytes( + base: AnyRef, offset: Long, len: Int, seed: Long): Long = { HiveHasher.hashUnsafeBytes(base, offset, len) } + override protected def hashUnsafeBytesMB( + base: MemoryBlock, offset: Long, len: Int, seed: Long): Long = { + HiveHasher.hashUnsafeBytesMB(base, offset, len) + } + private val HIVE_DECIMAL_MAX_PRECISION = 38 private val HIVE_DECIMAL_MAX_SCALE = 38 From d2d2e50f8a2baf41d5b85127bf888da6f8bca343 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 21 Sep 2017 19:45:55 +0100 Subject: [PATCH 08/65] minor update of UTF8String constructor --- .../src/main/java/org/apache/spark/unsafe/types/UTF8String.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 6e517a54940f6..9b89ab47608fb 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -122,7 +122,7 @@ public static UTF8String blankString(int length) { } protected UTF8String(byte[] bytes, long offset, int numBytes) { - this(ByteArrayMemoryBlock.fromArray(bytes), offset, numBytes); + this(new ByteArrayMemoryBlock(bytes, offset, numBytes), offset, numBytes); } protected UTF8String(MemoryBlock base, long offset, int numBytes) { From f5e10bb52c33856ddd3e1b1f8483b170e0167c53 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 22 Sep 2017 12:00:12 +0100 Subject: [PATCH 09/65] rename method name --- .../sql/catalyst/expressions/HiveHasher.java | 2 +- .../spark/unsafe/hash/Murmur3_x86_32.java | 2 +- .../spark/sql/catalyst/expressions/XXH64.java | 2 +- .../spark/sql/catalyst/expressions/hash.scala | 18 +++++++++--------- 4 files changed, 12 insertions(+), 12 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java index a20ee19874427..77f96e11492b0 100644 --- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java @@ -39,7 +39,7 @@ public static int hashLong(long input) { return (int) ((input >>> 32) ^ input); } - public static int hashUnsafeBytesMB(MemoryBlock base, long offset, int lengthInBytes) { + public static int hashUnsafeBlock(MemoryBlock base, long offset, int lengthInBytes) { return hashUnsafeBytes(base.getBaseObject(), offset, lengthInBytes); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index d9811484d5b9f..518b93c7e374a 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -60,7 +60,7 @@ public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, i return fmix(h1, lengthInBytes); } - public static int hashUnsafeBytesMB(MemoryBlock base, long offset, int lengthInBytes, int seed) { + public static int hashUnsafeBlock(MemoryBlock base, long offset, int lengthInBytes, int seed) { return hashUnsafeBytes(base.getBaseObject(), offset, lengthInBytes, seed); } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java index 445348b5a4ff2..baf55516711bc 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java @@ -86,7 +86,7 @@ public long hashUnsafeBytes(Object base, long offset, int length) { return hashUnsafeBytes(base, offset, length, seed); } - public static long hashUnsafeBytesMB(MemoryBlock base, long offset, int length, long seed) { + public static long hashUnsafeBlock(MemoryBlock base, long offset, int length, long seed) { return hashUnsafeBytes(base.getBaseObject(), offset, length, seed); } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala index e961a48b00882..294d3d6918f41 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala @@ -363,7 +363,7 @@ abstract class HashExpression[E] extends Expression { val baseObject = s"$input.getBaseObject()" val baseOffset = s"$input.getBaseOffset()" val numBytes = s"$input.numBytes()" - s"$result = $hasherClassName.hashUnsafeBytesMB($baseObject, $baseOffset, $numBytes, $result);" + s"$result = $hasherClassName.hashUnsafeBlock($baseObject, $baseOffset, $numBytes, $result);" } protected def genHashForMap( @@ -465,7 +465,7 @@ abstract class InterpretedHashFunction { protected def hashUnsafeBytes(base: AnyRef, offset: Long, length: Int, seed: Long): Long - protected def hashUnsafeBytesMB(base: MemoryBlock, offset: Long, length: Int, seed: Long): Long + protected def hashUnsafeBlock(base: MemoryBlock, offset: Long, length: Int, seed: Long): Long /** * Computes hash of a given `value` of type `dataType`. The caller needs to check the validity @@ -493,7 +493,7 @@ abstract class InterpretedHashFunction { case a: Array[Byte] => hashUnsafeBytes(a, Platform.BYTE_ARRAY_OFFSET, a.length, seed) case s: UTF8String => - hashUnsafeBytesMB(s.getBaseObject, s.getBaseOffset, s.numBytes(), seed) + hashUnsafeBlock(s.getBaseObject, s.getBaseOffset, s.numBytes(), seed) case array: ArrayData => val elementType = dataType match { @@ -585,9 +585,9 @@ object Murmur3HashFunction extends InterpretedHashFunction { Murmur3_x86_32.hashUnsafeBytes(base, offset, len, seed.toInt) } - override protected def hashUnsafeBytesMB( + override protected def hashUnsafeBlock( base: MemoryBlock, offset: Long, len: Int, seed: Long): Long = { - Murmur3_x86_32.hashUnsafeBytesMB(base, offset, len, seed.toInt) + Murmur3_x86_32.hashUnsafeBlock(base, offset, len, seed.toInt) } } @@ -618,9 +618,9 @@ object XxHash64Function extends InterpretedHashFunction { XXH64.hashUnsafeBytes(base, offset, len, seed) } - override protected def hashUnsafeBytesMB( + override protected def hashUnsafeBlock( base: MemoryBlock, offset: Long, len: Int, seed: Long): Long = { - XXH64.hashUnsafeBytesMB(base, offset, len, seed) + XXH64.hashUnsafeBlock(base, offset, len, seed) } } @@ -824,9 +824,9 @@ object HiveHashFunction extends InterpretedHashFunction { HiveHasher.hashUnsafeBytes(base, offset, len) } - override protected def hashUnsafeBytesMB( + override protected def hashUnsafeBlock( base: MemoryBlock, offset: Long, len: Int, seed: Long): Long = { - HiveHasher.hashUnsafeBytesMB(base, offset, len) + HiveHasher.hashUnsafeBlock(base, offset, len) } private val HIVE_DECIMAL_MAX_PRECISION = 38 From 1905e8ca4b3b8200fa56f5fc91899bb420a07628 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 22 Sep 2017 12:01:13 +0100 Subject: [PATCH 10/65] remove unused code --- .../spark/unsafe/memory/UnsafeMemoryAllocator.java | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java index cd69305b24555..694b3e5bd966f 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -30,18 +30,6 @@ */ public class UnsafeMemoryAllocator implements MemoryAllocator { - private static Method bufAddrMethod; - static { - try { - Class cb = UnsafeMemoryAllocator.class.getClassLoader().loadClass("java.nio.DirectByteBuffer"); - bufAddrMethod = cb.getMethod("address"); - bufAddrMethod.setAccessible(true); - } - catch(Exception ex) { - throw new RuntimeException(ex.getMessage(), ex); - } - } - @Override public OffHeapMemoryBlock allocate(long size) throws OutOfMemoryError { Object buffer = ByteBuffer.allocateDirect((int)size); From 7778e586e94130749cec3f54a60b6fb24514647a Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 22 Sep 2017 12:02:30 +0100 Subject: [PATCH 11/65] update arrayEquals --- .../spark/unsafe/array/ByteArrayMethods.java | 35 ++----------------- .../apache/spark/unsafe/types/UTF8String.java | 18 +++++----- 2 files changed, 11 insertions(+), 42 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java index fad303b9f09ec..fc71953463552 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java @@ -53,40 +53,9 @@ public static int roundNumberOfBytesToNearestWord(int numBytes) { * Optimized byte array equality check for byte arrays. * @return true if the arrays are equal, false otherwise */ - public static boolean arrayEquals( + public static boolean arrayEqualsBlock( MemoryBlock leftBase, long leftOffset, MemoryBlock rightBase, long rightOffset, final long length) { - int i = 0; - - // check if starts align and we can get both offsets to be aligned - if ((leftOffset % 8) == (rightOffset % 8)) { - while ((leftOffset + i) % 8 != 0 && i < length) { - if (MemoryBlock.getByte(leftBase, leftOffset + i) != - MemoryBlock.getByte(rightBase, rightOffset + i)) { - return false; - } - i += 1; - } - } - // for architectures that support unaligned accesses, chew it up 8 bytes at a time - if (unaligned || (((leftOffset + i) % 8 == 0) && ((rightOffset + i) % 8 == 0))) { - while (i <= length - 8) { - if (MemoryBlock.getLong(leftBase, leftOffset + i) != - MemoryBlock.getLong(rightBase, rightOffset + i)) { - return false; - } - i += 8; - } - } - // this will finish off the unaligned comparisons, or do the entire aligned - // comparison whichever is needed. - while (i < length) { - if (MemoryBlock.getByte(leftBase, leftOffset + i) != - MemoryBlock.getByte(rightBase, rightOffset + i)) { - return false; - } - i += 1; - } - return true; + return arrayEquals(leftBase.getBaseObject(), leftOffset, rightBase.getBaseObject(), rightOffset, length); } public static boolean arrayEquals( diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 9b89ab47608fb..598e575237983 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -350,7 +350,7 @@ private boolean matchAt(final UTF8String s, int pos) { if (s.numBytes + pos > numBytes || pos < 0) { return false; } - return ByteArrayMethods.arrayEquals(base, offset + pos, s.base, s.offset, s.numBytes); + return ByteArrayMethods.arrayEqualsBlock(base, offset + pos, s.base, s.offset, s.numBytes); } public boolean startsWith(final UTF8String prefix) { @@ -477,7 +477,7 @@ public int findInSet(UTF8String match) { for (int i = 0; i < numBytes; i++) { if (getByte(i) == (byte) ',') { if (i - (lastComma + 1) == match.numBytes && - ByteArrayMethods.arrayEquals(base, offset + (lastComma + 1), match.base, match.offset, + ByteArrayMethods.arrayEqualsBlock(base, offset + (lastComma + 1), match.base, match.offset, match.numBytes)) { return n; } @@ -486,7 +486,7 @@ public int findInSet(UTF8String match) { } } if (numBytes - (lastComma + 1) == match.numBytes && - ByteArrayMethods.arrayEquals(base, offset + (lastComma + 1), match.base, match.offset, + ByteArrayMethods.arrayEqualsBlock(base, offset + (lastComma + 1), match.base, match.offset, match.numBytes)) { return n; } @@ -701,7 +701,7 @@ public int indexOf(UTF8String v, int start) { if (i + v.numBytes > numBytes) { return -1; } - if (ByteArrayMethods.arrayEquals(base, offset + i, v.base, v.offset, v.numBytes)) { + if (ByteArrayMethods.arrayEqualsBlock(base, offset + i, v.base, v.offset, v.numBytes)) { return c; } i += numBytesForFirstByte(getByte(i)); @@ -717,7 +717,7 @@ public int indexOf(UTF8String v, int start) { private int find(UTF8String str, int start) { assert (str.numBytes > 0); while (start <= numBytes - str.numBytes) { - if (ByteArrayMethods.arrayEquals(base, offset + start, str.base, str.offset, str.numBytes)) { + if (ByteArrayMethods.arrayEqualsBlock(base, offset + start, str.base, str.offset, str.numBytes)) { return start; } start += 1; @@ -731,7 +731,7 @@ private int find(UTF8String str, int start) { private int rfind(UTF8String str, int start) { assert (str.numBytes > 0); while (start >= 0) { - if (ByteArrayMethods.arrayEquals(base, offset + start, str.base, str.offset, str.numBytes)) { + if (ByteArrayMethods.arrayEqualsBlock(base, offset + start, str.base, str.offset, str.numBytes)) { return start; } start -= 1; @@ -1240,7 +1240,7 @@ public boolean equals(final Object other) { if (numBytes != o.numBytes) { return false; } - return ByteArrayMethods.arrayEquals(base, offset, o.base, o.offset, numBytes); + return ByteArrayMethods.arrayEqualsBlock(base, offset, o.base, o.offset, numBytes); } else { return false; } @@ -1296,7 +1296,7 @@ public int levenshteinDistance(UTF8String other) { num_bytes_j != numBytesForFirstByte(s.getByte(i_bytes))) { cost = 1; } else { - cost = (ByteArrayMethods.arrayEquals(t.base, t.offset + j_bytes, s.base, + cost = (ByteArrayMethods.arrayEqualsBlock(t.base, t.offset + j_bytes, s.base, s.offset + i_bytes, num_bytes_j)) ? 0 : 1; } d[i + 1] = Math.min(Math.min(d[i] + 1, p[i + 1] + 1), p[i] + cost); @@ -1312,7 +1312,7 @@ public int levenshteinDistance(UTF8String other) { @Override public int hashCode() { - return Murmur3_x86_32.hashUnsafeBytesMB(base, offset, numBytes, 42); + return Murmur3_x86_32.hashUnsafeBlock(base, offset, numBytes, 42); } /** From 4f96c82b151b78641bcfc92a65913048b055cfee Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 22 Sep 2017 14:18:01 +0100 Subject: [PATCH 12/65] rebase master --- .../java/org/apache/spark/unsafe/types/UTF8String.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 598e575237983..7af947dcc9382 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -1207,10 +1207,10 @@ public int compareTo(@Nonnull final UTF8String other) { int len = Math.min(numBytes, other.numBytes); int wordMax = (len / 8) * 8; long roffset = other.offset; - Object rbase = other.base; + MemoryBlock rbase = other.base; for (int i = 0; i < wordMax; i += 8) { - long left = getLong(base, offset + i); - long right = getLong(rbase, roffset + i); + long left = MemoryBlock.getLong(base, offset + i); + long right = MemoryBlock.getLong(rbase, roffset + i); if (left != right) { if (IS_LITTLE_ENDIAN) { return Long.compareUnsigned(Long.reverseBytes(left), Long.reverseBytes(right)); @@ -1221,7 +1221,7 @@ public int compareTo(@Nonnull final UTF8String other) { } for (int i = wordMax; i < len; i++) { // In UTF-8, the byte should be unsigned, so we should compare them as unsigned int. - int res = (getByte(i) & 0xFF) - (Platform.getByte(rbase, roffset + i) & 0xFF); + int res = (getByte(i) & 0xFF) - (MemoryBlock.getByte(rbase, roffset + i) & 0xFF); if (res != 0) { return res; } From d1d6ae90589c0fae6c64af0cb95696e270446228 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 22 Sep 2017 15:51:40 +0100 Subject: [PATCH 13/65] make more methods final --- .../unsafe/memory/ByteArrayMemoryBlock.java | 33 +++-------------- .../unsafe/memory/IntArrayMemoryBlock.java | 35 +++---------------- .../unsafe/memory/LongArrayMemoryBlock.java | 34 +++--------------- .../spark/unsafe/memory/MemoryBlock.java | 28 +++++++++++---- .../unsafe/memory/OffHeapMemoryBlock.java | 31 +++------------- .../unsafe/memory/UnsafeMemoryAllocator.java | 2 +- 6 files changed, 41 insertions(+), 122 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java index 7c31f003bb6f7..132b2b54a8c27 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -22,47 +22,22 @@ /** * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. */ -public class ByteArrayMemoryBlock extends MemoryBlock { - - private final long length; - - /** - * Optional page number; used when this MemoryBlock represents a page allocated by a - * TaskMemoryManager. This field can be updated using setPageNumber method so that - * this can be modified by the TaskMemoryManage, which lives in a different package. - */ - private int pageNumber = NO_PAGE_NUMBER; +public final class ByteArrayMemoryBlock extends MemoryBlock { public ByteArrayMemoryBlock(byte[] obj, long offset, long length) { - super(obj, offset); - this.length = length; - } - - /** - * Returns the size of the memory block. - */ - public long size() { - return length; + super(obj, offset, length); } + @Override public void fill(byte value) { Platform.setMemory(obj, offset, length, value); } + @Override public MemoryBlock allocate(long offset, long size) { return new ByteArrayMemoryBlock((byte[]) obj, offset, size); } - @Override - public void setPageNumber(int pageNum) { - this.pageNumber = pageNum; - } - - @Override - public int getPageNumber() { - return this.pageNumber; - } - public byte[] getByteArray() { return (byte[])this.obj; } /** diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java index 01d98cd08fa89..b8ccd73fe6147 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java @@ -22,47 +22,22 @@ /** * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. */ -public class IntArrayMemoryBlock extends MemoryBlock { - - private final long size; - - /** - * Optional page number; used when this MemoryBlock represents a page allocated by a - * TaskMemoryManager. This field can be updated using setPageNumber method so that - * this can be modified by the TaskMemoryManage, which lives in a different package. - */ - private int pageNumber = NO_PAGE_NUMBER; +public final class IntArrayMemoryBlock extends MemoryBlock { public IntArrayMemoryBlock(int[] obj, long offset, long size) { - super(obj, offset); - this.size = size; - } - - /** - * Returns the size of the memory block. - */ - public long size() { - return size; + super(obj, offset, size); } + @Override public void fill(byte value) { - Platform.setMemory(obj, offset, size, value); + Platform.setMemory(obj, offset, length, value); } + @Override public MemoryBlock allocate(long offset, long size) { return new IntArrayMemoryBlock((int[]) obj, offset, size); } - @Override - public void setPageNumber(int pageNum) { - this.pageNumber = pageNum; - } - - @Override - public int getPageNumber() { - return this.pageNumber; - } - public int[] getIntArray() { return (int[])this.obj; } /** diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java index a24c79b54f795..0f8fd8fc3937b 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java @@ -22,31 +22,15 @@ /** * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. */ -public class LongArrayMemoryBlock extends MemoryBlock { - - private final long size; - - /** - * Optional page number; used when this MemoryBlock represents a page allocated by a - * TaskMemoryManager. This field can be updated using setPageNumber method so that - * this can be modified by the TaskMemoryManage, which lives in a different package. - */ - private int pageNumber = NO_PAGE_NUMBER; +public final class LongArrayMemoryBlock extends MemoryBlock { public LongArrayMemoryBlock(long[] obj, long offset, long size) { - super(obj, offset); - this.size = size; - } - - /** - * Returns the size of the memory block. - */ - public long size() { - return size; + super(obj, offset, size); } + @Override public void fill(byte value) { - Platform.setMemory(obj, offset, size, value); + Platform.setMemory(obj, offset, length, value); } @Override @@ -54,16 +38,6 @@ public MemoryBlock allocate(long offset, long size) { return new LongArrayMemoryBlock((long[]) obj, offset, size); } - @Override - public void setPageNumber(int pageNum) { - this.pageNumber = pageNum; - } - - @Override - public int getPageNumber() { - return this.pageNumber; - } - public long[] getLongArray() { return (long[])this.obj; } /** diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index e190319301aa6..36cf57f063192 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -49,13 +49,23 @@ public abstract class MemoryBlock { protected long offset; - public MemoryBlock(@Nullable Object obj, long offset) { + protected final long length; + + /** + * Optional page number; used when this MemoryBlock represents a page allocated by a + * TaskMemoryManager. This field can be updated using setPageNumber method so that + * this can be modified by the TaskMemoryManage, which lives in a different package. + */ + private int pageNumber = NO_PAGE_NUMBER; + + public MemoryBlock(@Nullable Object obj, long offset, long length) { this.obj = obj; this.offset = offset; + this.length = length; } public MemoryBlock() { - this(null, 0); + this(null, 0, 0); } public final Object getBaseObject() { @@ -66,7 +76,7 @@ public final long getBaseOffset() { return offset; } - public final void resetObjAndOffset() { + public void resetObjAndOffset() { this.obj = null; this.offset = 0; } @@ -74,11 +84,17 @@ public final void resetObjAndOffset() { /** * Returns the size of the memory block. */ - public abstract long size(); + public final long size() { + return this.length; + } - public abstract void setPageNumber(int pageNum); + public final void setPageNumber(int pageNum) { + this.pageNumber = pageNum; + } - public abstract int getPageNumber(); + public final int getPageNumber() { + return this.pageNumber; + } /** * Fills the memory block with the specified byte value. diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java index e4a1541e12db1..717715f50e8c4 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java @@ -20,33 +20,22 @@ import org.apache.spark.unsafe.Platform; public class OffHeapMemoryBlock extends MemoryBlock { + private Object directBuffer; private long address; - private final long length; - - /** - * Optional page number; used when this MemoryBlock represents a page allocated by a - * TaskMemoryManager. This field can be updated using setPageNumber method so that - * this can be modified by the TaskMemoryManage, which lives in a different package. - */ - private int pageNumber = -1; static public final OffHeapMemoryBlock NULL = new OffHeapMemoryBlock(null, 0, 0); public OffHeapMemoryBlock(Object directBuffer, long address, long size) { - super(null, address); + super(null, address, size); this.address = address; - this.length = size; this.directBuffer = directBuffer; } - public void setBaseOffset(long address) { - this.address = address; - } - @Override - public long size() { - return this.length; + public void resetObjAndOffset() { + this.directBuffer = null; + this.address = 0; } @Override @@ -58,14 +47,4 @@ public void fill(byte value) { public MemoryBlock allocate(long offset, long size) { return new OffHeapMemoryBlock(directBuffer, address + offset, size); } - - @Override - public void setPageNumber(int pageNum) { - this.pageNumber = pageNum; - } - - @Override - public int getPageNumber() { - return this.pageNumber; - } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java index 694b3e5bd966f..0a0c5aa6aa85f 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -57,7 +57,7 @@ public void free(MemoryBlock memory) { // As an additional layer of defense against use-after-free bugs, we mutate the // MemoryBlock to reset its pointer. - ((OffHeapMemoryBlock)memory).setBaseOffset(0); + memory.resetObjAndOffset(); // Mark the page as freed (so we can detect double-frees). memory.setPageNumber(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER); From 914dcd11d0d5ef014284868f2794cb4e5baa0958 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 22 Sep 2017 16:39:57 +0100 Subject: [PATCH 14/65] make fill method final in MemoryBlock --- .../spark/unsafe/memory/ByteArrayMemoryBlock.java | 5 ----- .../spark/unsafe/memory/IntArrayMemoryBlock.java | 5 ----- .../spark/unsafe/memory/LongArrayMemoryBlock.java | 5 ----- .../org/apache/spark/unsafe/memory/MemoryBlock.java | 10 ++++++---- .../apache/spark/unsafe/memory/OffHeapMemoryBlock.java | 5 ----- 5 files changed, 6 insertions(+), 24 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java index 132b2b54a8c27..8d69938d3570d 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -28,11 +28,6 @@ public ByteArrayMemoryBlock(byte[] obj, long offset, long length) { super(obj, offset, length); } - @Override - public void fill(byte value) { - Platform.setMemory(obj, offset, length, value); - } - @Override public MemoryBlock allocate(long offset, long size) { return new ByteArrayMemoryBlock((byte[]) obj, offset, size); diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java index b8ccd73fe6147..9199fd66dc878 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java @@ -28,11 +28,6 @@ public IntArrayMemoryBlock(int[] obj, long offset, long size) { super(obj, offset, size); } - @Override - public void fill(byte value) { - Platform.setMemory(obj, offset, length, value); - } - @Override public MemoryBlock allocate(long offset, long size) { return new IntArrayMemoryBlock((int[]) obj, offset, size); diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java index 0f8fd8fc3937b..e341f4c22e509 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java @@ -28,11 +28,6 @@ public LongArrayMemoryBlock(long[] obj, long offset, long size) { super(obj, offset, size); } - @Override - public void fill(byte value) { - Platform.setMemory(obj, offset, length, value); - } - @Override public MemoryBlock allocate(long offset, long size) { return new LongArrayMemoryBlock((long[]) obj, offset, size); diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index 36cf57f063192..a720b56bf6194 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -85,21 +85,23 @@ public void resetObjAndOffset() { * Returns the size of the memory block. */ public final long size() { - return this.length; + return length; } public final void setPageNumber(int pageNum) { - this.pageNumber = pageNum; + pageNumber = pageNum; } public final int getPageNumber() { - return this.pageNumber; + return pageNumber; } /** * Fills the memory block with the specified byte value. */ - public abstract void fill(byte value); + public final void fill(byte value) { + Platform.setMemory(obj, offset, length, value); + } /** * Instantiate the same type of MemoryBlock with new offset and size diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java index 717715f50e8c4..b63f0adbf37ed 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java @@ -38,11 +38,6 @@ public void resetObjAndOffset() { this.address = 0; } - @Override - public void fill(byte value) { - Platform.setMemory(null, address, length, value); - } - @Override public MemoryBlock allocate(long offset, long size) { return new OffHeapMemoryBlock(directBuffer, address + offset, size); From 336e4b7bfd7edcb861edeac3ca115dead785b68a Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 23 Sep 2017 12:35:15 +0100 Subject: [PATCH 15/65] fix test failures --- .../apache/spark/unsafe/memory/UnsafeMemoryAllocator.java | 6 +++++- .../scala/org/apache/spark/mllib/feature/HashingTF.scala | 2 +- .../spark/sql/catalyst/expressions/UnsafeArrayData.java | 5 ++++- .../apache/spark/sql/catalyst/expressions/UnsafeRow.java | 4 ++-- .../org/apache/spark/sql/catalyst/expressions/hash.scala | 2 +- .../spark/sql/catalyst/expressions/HiveHasherSuite.java | 2 +- 6 files changed, 14 insertions(+), 7 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java index 0a0c5aa6aa85f..8561439869c15 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -35,7 +35,11 @@ public OffHeapMemoryBlock allocate(long size) throws OutOfMemoryError { Object buffer = ByteBuffer.allocateDirect((int)size); if (buffer instanceof DirectBuffer) { long addr = ((DirectBuffer) buffer).address(); - return new OffHeapMemoryBlock(buffer, addr, size); + OffHeapMemoryBlock memory = new OffHeapMemoryBlock(buffer, addr, size); + if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { + memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE); + } + return memory; } throw new UnsupportedOperationException("A ByteBuffer does not have an address in off-heap"); } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala index 8935c8496cdbb..f1a15192688b6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala @@ -160,7 +160,7 @@ object HashingTF { case d: Double => hashLong(java.lang.Double.doubleToLongBits(d), seed) case s: String => val utf8 = UTF8String.fromString(s) - hashUnsafeBytes(utf8.getBaseObject, utf8.getBaseOffset, utf8.numBytes(), seed) + hashUnsafeBlock(utf8.getBaseObject, utf8.getBaseOffset, utf8.numBytes(), seed) case _ => throw new SparkException("HashingTF with murmur3 algorithm does not " + s"support type ${term.getClass.getCanonicalName} of input data.") } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java index 58ab26bbed819..7094eecdd741b 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java @@ -28,6 +28,7 @@ import org.apache.spark.unsafe.bitset.BitSetMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; import org.apache.spark.unsafe.memory.ByteArrayMemoryBlock; +import org.apache.spark.unsafe.memory.LongArrayMemoryBlock; import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; @@ -232,7 +233,9 @@ public UTF8String getUTF8String(int ordinal) { final long offsetAndSize = getLong(ordinal); final int offset = (int) (offsetAndSize >> 32); final int size = (int) offsetAndSize; - MemoryBlock mb = new ByteArrayMemoryBlock((byte[])baseObject, baseOffset + offset, size); + MemoryBlock mb = (baseObject instanceof byte[]) ? + new ByteArrayMemoryBlock((byte[]) baseObject, baseOffset + offset, size) : + new LongArrayMemoryBlock((long[]) baseObject, baseOffset + offset, size); return UTF8String.fromAddress(mb, mb.getBaseOffset(), (int)mb.size()); } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index 8a0228e5094b2..68432996c8cf2 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -416,8 +416,8 @@ public UTF8String getUTF8String(int ordinal) { final int offset = (int) (offsetAndSize >> 32); final int size = (int) offsetAndSize; MemoryBlock mb = (baseObject instanceof byte[]) ? - new ByteArrayMemoryBlock((byte[])baseObject, baseOffset + offset, size) : - new LongArrayMemoryBlock((long[])baseObject, baseOffset + offset, size); + new ByteArrayMemoryBlock((byte[]) baseObject, baseOffset + offset, size) : + new LongArrayMemoryBlock((long[]) baseObject, baseOffset + offset, size); return UTF8String.fromAddress(mb, mb.getBaseOffset(), (int)mb.size()); } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala index 294d3d6918f41..118cd34c27176 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala @@ -731,7 +731,7 @@ case class HiveHash(children: Seq[Expression]) extends HashExpression[Int] { val baseObject = s"$input.getBaseObject()" val baseOffset = s"$input.getBaseOffset()" val numBytes = s"$input.numBytes()" - s"$result = $hasherClassName.hashUnsafeBytes($baseObject, $baseOffset, $numBytes);" + s"$result = $hasherClassName.hashUnsafeBlock($baseObject, $baseOffset, $numBytes);" } override protected def genHashForArray( diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java index b67c6f3e6e85e..298d6b86e6dc0 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java @@ -53,7 +53,7 @@ public void testKnownStringAndIntInputs() { for (int i = 0; i < inputs.length; i++) { UTF8String s = UTF8String.fromString("val_" + inputs[i]); - int hash = HiveHasher.hashUnsafeBytes(s.getBaseObject(), s.getBaseOffset(), s.numBytes()); + int hash = HiveHasher.hashUnsafeBlock(s.getBaseObject(), s.getBaseOffset(), s.numBytes()); Assert.assertEquals(expected[i], ((31 * inputs[i]) + hash)); } } From 5be9ccb163832e1895b045730a06e79eb3b171cf Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 24 Sep 2017 15:00:40 +0100 Subject: [PATCH 16/65] add testsuite --- .../spark/unsafe/memory/MemoryBlockSuite.java | 99 +++++++++++++++++++ 1 file changed, 99 insertions(+) create mode 100644 common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java new file mode 100644 index 0000000000000..666f486f7899c --- /dev/null +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java @@ -0,0 +1,99 @@ +/* + * 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.spark.unsafe.memory; + +import org.apache.spark.unsafe.Platform; +import org.junit.Assert; +import org.junit.Test; + +import org.apache.spark.unsafe.memory.LongArrayMemoryBlock; + +public class MemoryBlockSuite { + + private void check(MemoryBlock memory, Object obj, long offset, int length) { + memory.setPageNumber(1); + memory.fill((byte)-1); + MemoryBlock.putBoolean(memory, offset, true); + MemoryBlock.putByte(memory, offset + 1, (byte)127); + MemoryBlock.putShort(memory, offset + 2, (short)257); + MemoryBlock.putInt(memory, offset + 4, 0x20000002); + MemoryBlock.putLong(memory, offset + 8, -1L); + MemoryBlock.putFloat(memory, offset + 16, 1.0F); + MemoryBlock.putDouble(memory, offset + 20, 2.0); + MemoryBlock.copyMemory(memory, offset, memory, offset + 28, 4); + + Assert.assertEquals(obj, memory.getBaseObject()); + Assert.assertEquals(offset, memory.getBaseOffset()); + Assert.assertEquals(length, memory.size()); + Assert.assertEquals(1, memory.getPageNumber()); + Assert.assertEquals(true, MemoryBlock.getBoolean(memory, offset)); + Assert.assertEquals((byte)127, MemoryBlock.getByte(memory, offset + 1 )); + Assert.assertEquals((short)257, MemoryBlock.getShort(memory, offset + 2)); + Assert.assertEquals(0x20000002, MemoryBlock.getInt(memory, offset + 4)); + Assert.assertEquals(-1L, MemoryBlock.getLong(memory, offset + 8)); + Assert.assertEquals(1.0F, MemoryBlock.getFloat(memory, offset + 16), 0); + Assert.assertEquals(2.0, MemoryBlock.getDouble(memory, offset + 20), 0); + Assert.assertEquals(true, MemoryBlock.getBoolean(memory, offset + 28)); + Assert.assertEquals((byte)127, MemoryBlock.getByte(memory, offset + 29 )); + Assert.assertEquals((short)257, MemoryBlock.getShort(memory, offset + 30)); + for (int i = 32; i < memory.size(); i++) { + Assert.assertEquals((byte) -1, MemoryBlock.getByte(memory, offset + i)); + } + } + + @Test + public void ByteArrayMemoryBlockTest() { + byte[] obj = new byte[36]; + long offset = Platform.BYTE_ARRAY_OFFSET; + int length = obj.length; + MemoryBlock memory = new ByteArrayMemoryBlock(obj, offset, length); + + check(memory, obj, offset, length); + } + + @Test + public void IntArrayMemoryBlockTest() { + int[] obj = new int[9]; + long offset = Platform.INT_ARRAY_OFFSET; + int length = obj.length; + MemoryBlock memory = new IntArrayMemoryBlock(obj, offset, length); + + check(memory, obj, offset, length); + } + + @Test + public void LongArrayMemoryBlockTest() { + long[] obj = new long[5]; + long offset = Platform.LONG_ARRAY_OFFSET; + int length = obj.length; + MemoryBlock memory = new LongArrayMemoryBlock(obj, offset, length); + + check(memory, obj, offset, length); + } + + @Test + public void OffHeapArrayMemoryBlockTest() { + MemoryAllocator memoryAllocator = new UnsafeMemoryAllocator(); + MemoryBlock memory = memoryAllocator.allocate(36); + Object obj = memory.getBaseObject(); + long offset = memory.getBaseOffset(); + int length = 36; + + check(memory, obj, offset, length); + } +} From 43e6b572bd893bd42e58df930a34b7e31549a49a Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 24 Sep 2017 19:10:49 +0100 Subject: [PATCH 17/65] pass concrete type to the first argument of Platform.get*/put* to get better performance --- .../apache/spark/unsafe/array/LongArray.java | 6 +- .../unsafe/memory/ByteArrayMemoryBlock.java | 156 ++++++++++---- .../unsafe/memory/IntArrayMemoryBlock.java | 156 ++++++++++---- .../unsafe/memory/LongArrayMemoryBlock.java | 156 ++++++++++---- .../spark/unsafe/memory/MemoryBlock.java | 81 +++---- .../unsafe/memory/OffHeapMemoryBlock.java | 147 +++++++++---- .../apache/spark/unsafe/types/UTF8String.java | 20 +- .../spark/unsafe/memory/MemoryBlockSuite.java | 198 +++++++++--------- .../vectorized/OffHeapColumnVector.java | 78 ++++--- 9 files changed, 612 insertions(+), 386 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java index eb9b2172c3ce3..7e086e9a7867e 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java @@ -67,7 +67,7 @@ public long size() { */ public void zeroOut() { for (long off = baseOffset; off < baseOffset + length * WIDTH; off += WIDTH) { - MemoryBlock.putLong(memory, off, 0); + memory.putLong(off, 0); } } @@ -77,7 +77,7 @@ public void zeroOut() { public void set(int index, long value) { assert index >= 0 : "index (" + index + ") should >= 0"; assert index < length : "index (" + index + ") should < length (" + length + ")"; - MemoryBlock.putLong(memory, baseOffset + index * WIDTH, value); + memory.putLong(baseOffset + index * WIDTH, value); } /** @@ -86,6 +86,6 @@ public void set(int index, long value) { public long get(int index) { assert index >= 0 : "index (" + index + ") should >= 0"; assert index < length : "index (" + index + ") should < length (" + length + ")"; - return MemoryBlock.getLong(memory, baseOffset + index * WIDTH); + return memory.getLong(baseOffset + index * WIDTH); } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java index 8d69938d3570d..f470c8fd7c42e 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -1,44 +1,112 @@ -/* - * 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.spark.unsafe.memory; - -import org.apache.spark.unsafe.Platform; - -/** - * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. - */ -public final class ByteArrayMemoryBlock extends MemoryBlock { - - public ByteArrayMemoryBlock(byte[] obj, long offset, long length) { - super(obj, offset, length); - } - - @Override - public MemoryBlock allocate(long offset, long size) { - return new ByteArrayMemoryBlock((byte[]) obj, offset, size); - } - - public byte[] getByteArray() { return (byte[])this.obj; } - - /** - * Creates a memory block pointing to the memory used by the byte array. - */ - public static ByteArrayMemoryBlock fromArray(final byte[] array) { - return new ByteArrayMemoryBlock(array, Platform.BYTE_ARRAY_OFFSET, array.length); - } -} +/* + * 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.spark.unsafe.memory; + +import org.apache.spark.unsafe.Platform; + +/** + * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. + */ +public final class ByteArrayMemoryBlock extends MemoryBlock { + + private final byte[] array; + + public ByteArrayMemoryBlock(byte[] obj, long offset, long length) { + super(obj, offset, length); + this.array = obj; + } + + @Override + public MemoryBlock allocate(long offset, long size) { + return new ByteArrayMemoryBlock(array, offset, size); + } + + public byte[] getByteArray() { return array; } + + /** + * Creates a memory block pointing to the memory used by the byte array. + */ + public static ByteArrayMemoryBlock fromArray(final byte[] array) { + return new ByteArrayMemoryBlock(array, Platform.BYTE_ARRAY_OFFSET, array.length); + } + + + public final int getInt(long offset) { + return Platform.getInt(array, offset); + } + + public final void putInt(long offset, int value) { + Platform.putInt(array, offset, value); + } + + public final boolean getBoolean(long offset) { + return Platform.getBoolean(array, offset); + } + + public final void putBoolean(long offset, boolean value) { + Platform.putBoolean(array, offset, value); + } + + public final byte getByte(long offset) { + return Platform.getByte(array, offset); + } + + public final void putByte(long offset, byte value) { + Platform.putByte(array, offset, value); + } + + public final short getShort(long offset) { + return Platform.getShort(array, offset); + } + + public final void putShort(long offset, short value) { + Platform.putShort(array, offset, value); + } + + public final long getLong(long offset) { + return Platform.getLong(array, offset); + } + + public final void putLong(long offset, long value) { + Platform.putLong(array, offset, value); + } + + public final float getFloat(long offset) { + return Platform.getFloat(array, offset); + } + + public final void putFloat(long offset, float value) { + Platform.putFloat(array, offset, value); + } + + public final double getDouble(long offset) { + return Platform.getDouble(array, offset); + } + + public final void putDouble(long offset, double value) { + Platform.putDouble(array, offset, value); + } + + public final Object getObjectVolatile(long offset) { + return Platform.getObjectVolatile(array, offset); + } + + public final void putObjectVolatile(long offset, Object value) { + Platform.putObjectVolatile(array, offset, value); + } +} diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java index 9199fd66dc878..734b2083e8cde 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java @@ -1,44 +1,112 @@ -/* - * 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.spark.unsafe.memory; - -import org.apache.spark.unsafe.Platform; - -/** - * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. - */ -public final class IntArrayMemoryBlock extends MemoryBlock { - - public IntArrayMemoryBlock(int[] obj, long offset, long size) { - super(obj, offset, size); - } - - @Override - public MemoryBlock allocate(long offset, long size) { - return new IntArrayMemoryBlock((int[]) obj, offset, size); - } - - public int[] getIntArray() { return (int[])this.obj; } - - /** - * Creates a memory block pointing to the memory used by the int array. - */ - public static IntArrayMemoryBlock fromArray(final int[] array) { - return new IntArrayMemoryBlock(array, Platform.INT_ARRAY_OFFSET, array.length*4); - } -} +/* + * 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.spark.unsafe.memory; + +import org.apache.spark.unsafe.Platform; + +/** + * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. + */ +public final class IntArrayMemoryBlock extends MemoryBlock { + + private final int[] array; + + public IntArrayMemoryBlock(int[] obj, long offset, long size) { + super(obj, offset, size); + this.array = obj; + } + + @Override + public MemoryBlock allocate(long offset, long size) { + return new IntArrayMemoryBlock(array, offset, size); + } + + public int[] getIntArray() { return array; } + + /** + * Creates a memory block pointing to the memory used by the int array. + */ + public static IntArrayMemoryBlock fromArray(final int[] array) { + return new IntArrayMemoryBlock(array, Platform.INT_ARRAY_OFFSET, array.length * 4); + } + + + public final int getInt(long offset) { + return Platform.getInt(array, offset); + } + + public final void putInt(long offset, int value) { + Platform.putInt(array, offset, value); + } + + public final boolean getBoolean(long offset) { + return Platform.getBoolean(array, offset); + } + + public final void putBoolean(long offset, boolean value) { + Platform.putBoolean(array, offset, value); + } + + public final byte getByte(long offset) { + return Platform.getByte(array, offset); + } + + public final void putByte(long offset, byte value) { + Platform.putByte(array, offset, value); + } + + public final short getShort(long offset) { + return Platform.getShort(array, offset); + } + + public final void putShort(long offset, short value) { + Platform.putShort(array, offset, value); + } + + public final long getLong(long offset) { + return Platform.getLong(array, offset); + } + + public final void putLong(long offset, long value) { + Platform.putLong(array, offset, value); + } + + public final float getFloat(long offset) { + return Platform.getFloat(array, offset); + } + + public final void putFloat(long offset, float value) { + Platform.putFloat(array, offset, value); + } + + public final double getDouble(long offset) { + return Platform.getDouble(array, offset); + } + + public final void putDouble(long offset, double value) { + Platform.putDouble(array, offset, value); + } + + public final Object getObjectVolatile(long offset) { + return Platform.getObjectVolatile(array, offset); + } + + public final void putObjectVolatile(long offset, Object value) { + Platform.putObjectVolatile(array, offset, value); + } +} diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java index e341f4c22e509..2b1c6a33825aa 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java @@ -1,44 +1,112 @@ -/* - * 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.spark.unsafe.memory; - -import org.apache.spark.unsafe.Platform; - -/** - * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. - */ -public final class LongArrayMemoryBlock extends MemoryBlock { - - public LongArrayMemoryBlock(long[] obj, long offset, long size) { - super(obj, offset, size); - } - - @Override - public MemoryBlock allocate(long offset, long size) { - return new LongArrayMemoryBlock((long[]) obj, offset, size); - } - - public long[] getLongArray() { return (long[])this.obj; } - - /** - * Creates a memory block pointing to the memory used by the long array. - */ - public static LongArrayMemoryBlock fromArray(final long[] array) { - return new LongArrayMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, array.length*8); - } -} +/* + * 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.spark.unsafe.memory; + +import org.apache.spark.unsafe.Platform; + +/** + * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. + */ +public final class LongArrayMemoryBlock extends MemoryBlock { + + private final long[] array; + + public LongArrayMemoryBlock(long[] obj, long offset, long size) { + super(obj, offset, size); + this.array = obj; + } + + @Override + public MemoryBlock allocate(long offset, long size) { + return new LongArrayMemoryBlock(array, offset, size); + } + + public long[] getLongArray() { return array; } + + /** + * Creates a memory block pointing to the memory used by the long array. + */ + public static LongArrayMemoryBlock fromArray(final long[] array) { + return new LongArrayMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, array.length * 8); + } + + + public final int getInt(long offset) { + return Platform.getInt(array, offset); + } + + public final void putInt(long offset, int value) { + Platform.putInt(array, offset, value); + } + + public final boolean getBoolean(long offset) { + return Platform.getBoolean(array, offset); + } + + public final void putBoolean(long offset, boolean value) { + Platform.putBoolean(array, offset, value); + } + + public final byte getByte(long offset) { + return Platform.getByte(array, offset); + } + + public final void putByte(long offset, byte value) { + Platform.putByte(array, offset, value); + } + + public final short getShort(long offset) { + return Platform.getShort(array, offset); + } + + public final void putShort(long offset, short value) { + Platform.putShort(array, offset, value); + } + + public final long getLong(long offset) { + return Platform.getLong(array, offset); + } + + public final void putLong(long offset, long value) { + Platform.putLong(array, offset, value); + } + + public final float getFloat(long offset) { + return Platform.getFloat(array, offset); + } + + public final void putFloat(long offset, float value) { + Platform.putFloat(array, offset, value); + } + + public final double getDouble(long offset) { + return Platform.getDouble(array, offset); + } + + public final void putDouble(long offset, double value) { + Platform.putDouble(array, offset, value); + } + + public final Object getObjectVolatile(long offset) { + return Platform.getObjectVolatile(array, offset); + } + + public final void putObjectVolatile(long offset, Object value) { + Platform.putObjectVolatile(array, offset, value); + } +} diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index a720b56bf6194..c489d2da86b51 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -109,70 +109,37 @@ public final void fill(byte value) { public abstract MemoryBlock allocate(long offset, long size); - public static final int getInt(MemoryBlock object, long offset) { - return Platform.getInt(object.getBaseObject(), offset); - } + public abstract int getInt(long offset); - public static final void putInt(MemoryBlock object, long offset, int value) { - Platform.putInt(object.getBaseObject(), offset, value); - } + public abstract void putInt(long offset, int value); + public abstract boolean getBoolean(long offset); - public static final boolean getBoolean(MemoryBlock object, long offset) { - return Platform.getBoolean(object.getBaseObject(), offset); - } + public abstract void putBoolean(long offset, boolean value); - public static final void putBoolean(MemoryBlock object, long offset, boolean value) { - Platform.putBoolean(object.getBaseObject(), offset, value); - } + public abstract byte getByte(long offset); - public static final byte getByte(MemoryBlock object, long offset) { - return Platform.getByte(object.getBaseObject(), offset); - } + public abstract void putByte(long offset, byte value); - public static final void putByte(MemoryBlock object, long offset, byte value) { - Platform.putByte(object.getBaseObject(), offset, value); - } + public abstract short getShort(long offset); - public static final short getShort(MemoryBlock object, long offset) { - return Platform.getShort(object.getBaseObject(), offset); - } + public abstract void putShort(long offset, short value); - public static final void putShort(MemoryBlock object, long offset, short value) { - Platform.putShort(object.getBaseObject(), offset, value); - } + public abstract long getLong(long offset); - public static final long getLong(MemoryBlock object, long offset) { - return Platform.getLong(object.getBaseObject(), offset); - } + public abstract void putLong(long offset, long value); - public static final void putLong(MemoryBlock object, long offset, long value) { - Platform.putLong(object.getBaseObject(), offset, value); - } + public abstract float getFloat(long offset); - public static final float getFloat(MemoryBlock object, long offset) { - return Platform.getFloat(object.getBaseObject(), offset); - } + public abstract void putFloat(long offset, float value); - public static final void putFloat(MemoryBlock object, long offset, float value) { - Platform.putFloat(object.getBaseObject(), offset, value); - } + public abstract double getDouble(long offset); - public static final double getDouble(MemoryBlock object, long offset) { - return Platform.getDouble(object.getBaseObject(), offset); - } + public abstract void putDouble(long offset, double value); - public static final void putDouble(MemoryBlock object, long offset, double value) { - Platform.putDouble(object.getBaseObject(), offset, value); - } - - public static final Object getObjectVolatile(MemoryBlock object, long offset) { - return Platform.getObjectVolatile(object.getBaseObject(), offset); - } + public abstract Object getObjectVolatile(long offset); - public static final void putObjectVolatile(MemoryBlock object, long offset, Object value) { - Platform.putObjectVolatile(object.getBaseObject(), offset, value); - } + public abstract void putObjectVolatile(long offset, Object value); public static final void copyMemory( MemoryBlock src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { @@ -199,42 +166,42 @@ public static final void copyMemory( Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); } - public static void copyMemory( + public static final void copyMemory( float[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); } - public static void copyMemory( + public static final void copyMemory( double[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); } - public static void copyMemory( + public static final void copyMemory( MemoryBlock src, long srcOffset, byte[] dst, long dstOffset, long length) { Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); } - public static void copyMemory( + public static final void copyMemory( MemoryBlock src, long srcOffset, short[] dst, long dstOffset, long length) { Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); } - public static void copyMemory( + public static final void copyMemory( MemoryBlock src, long srcOffset, int[] dst, long dstOffset, long length) { Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); } - public static void copyMemory( + public static final void copyMemory( MemoryBlock src, long srcOffset, long[] dst, long dstOffset, long length) { Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); } - public static void copyMemory( + public static final void copyMemory( MemoryBlock src, long srcOffset, float[] dst, long dstOffset, long length) { Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); } - public static void copyMemory( + public static final void copyMemory( MemoryBlock src, long srcOffset, double[] dst, long dstOffset, long length) { Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java index b63f0adbf37ed..5f574db5ad04c 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java @@ -1,45 +1,102 @@ -/* - * 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.spark.unsafe.memory; - -import org.apache.spark.unsafe.Platform; - -public class OffHeapMemoryBlock extends MemoryBlock { - - private Object directBuffer; - private long address; - - static public final OffHeapMemoryBlock NULL = new OffHeapMemoryBlock(null, 0, 0); - - public OffHeapMemoryBlock(Object directBuffer, long address, long size) { - super(null, address, size); - this.address = address; - this.directBuffer = directBuffer; - } - - @Override - public void resetObjAndOffset() { - this.directBuffer = null; - this.address = 0; - } - - @Override - public MemoryBlock allocate(long offset, long size) { - return new OffHeapMemoryBlock(directBuffer, address + offset, size); - } -} +/* + * 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.spark.unsafe.memory; + +import org.apache.spark.unsafe.Platform; + +public class OffHeapMemoryBlock extends MemoryBlock { + private Object directBuffer; + private final long address; + + static public final OffHeapMemoryBlock NULL = new OffHeapMemoryBlock(null, 0, 0); + + public OffHeapMemoryBlock(Object directBuffer, long address, long size) { + super(null, address, size); + this.address = address; + this.directBuffer = directBuffer; + } + + @Override + public MemoryBlock allocate(long offset, long size) { + return new OffHeapMemoryBlock(directBuffer, offset, size); + } + + public final int getInt(long offset) { + return Platform.getInt(null, offset); + } + + public final void putInt(long offset, int value) { + Platform.putInt(null, offset, value); + } + + public final boolean getBoolean(long offset) { + return Platform.getBoolean(null, offset); + } + + public final void putBoolean(long offset, boolean value) { + Platform.putBoolean(null, offset, value); + } + + public final byte getByte(long offset) { + return Platform.getByte(null, offset); + } + + public final void putByte(long offset, byte value) { + Platform.putByte(null, offset, value); + } + + public final short getShort(long offset) { + return Platform.getShort(null, offset); + } + + public final void putShort(long offset, short value) { + Platform.putShort(null, offset, value); + } + + public final long getLong(long offset) { + return Platform.getLong(null, offset); + } + + public final void putLong(long offset, long value) { + Platform.putLong(null, offset, value); + } + + public final float getFloat(long offset) { + return Platform.getFloat(null, offset); + } + + public final void putFloat(long offset, float value) { + Platform.putFloat(null, offset, value); + } + + public final double getDouble(long offset) { + return Platform.getDouble(null, offset); + } + + public final void putDouble(long offset, double value) { + Platform.putDouble(null, offset, value); + } + + public final Object getObjectVolatile(long offset) { + return Platform.getObjectVolatile(null, offset); + } + + public final void putObjectVolatile(long offset, Object value) { + Platform.putObjectVolatile(null, offset, value); + } +} diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 7af947dcc9382..a13cfb3111424 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -233,12 +233,12 @@ public long getPrefix() { long mask = 0; if (IS_LITTLE_ENDIAN) { if (numBytes >= 8) { - p = MemoryBlock.getLong(base, offset); + p = base.getLong(offset); } else if (numBytes > 4) { - p = MemoryBlock.getLong(base, offset); + p = base.getLong(offset); mask = (1L << (8 - numBytes) * 8) - 1; } else if (numBytes > 0) { - p = (long) MemoryBlock.getInt(base, offset); + p = (long) base.getInt(offset); mask = (1L << (8 - numBytes) * 8) - 1; } else { p = 0; @@ -247,12 +247,12 @@ public long getPrefix() { } else { // byteOrder == ByteOrder.BIG_ENDIAN if (numBytes >= 8) { - p = MemoryBlock.getLong(base, offset); + p = base.getLong(offset); } else if (numBytes > 4) { - p = MemoryBlock.getLong(base, offset); + p = base.getLong(offset); mask = (1L << (8 - numBytes) * 8) - 1; } else if (numBytes > 0) { - p = ((long) MemoryBlock.getInt(base, offset)) << 32; + p = ((long) base.getInt(offset)) << 32; mask = (1L << (8 - numBytes) * 8) - 1; } else { p = 0; @@ -343,7 +343,7 @@ public boolean contains(final UTF8String substring) { * Returns the byte at position `i`. */ private byte getByte(int i) { - return MemoryBlock.getByte(base, offset + i); + return base.getByte(offset + i); } private boolean matchAt(final UTF8String s, int pos) { @@ -1209,8 +1209,8 @@ public int compareTo(@Nonnull final UTF8String other) { long roffset = other.offset; MemoryBlock rbase = other.base; for (int i = 0; i < wordMax; i += 8) { - long left = MemoryBlock.getLong(base, offset + i); - long right = MemoryBlock.getLong(rbase, roffset + i); + long left = base.getLong(offset + i); + long right = rbase.getLong(roffset + i); if (left != right) { if (IS_LITTLE_ENDIAN) { return Long.compareUnsigned(Long.reverseBytes(left), Long.reverseBytes(right)); @@ -1221,7 +1221,7 @@ public int compareTo(@Nonnull final UTF8String other) { } for (int i = wordMax; i < len; i++) { // In UTF-8, the byte should be unsigned, so we should compare them as unsigned int. - int res = (getByte(i) & 0xFF) - (MemoryBlock.getByte(rbase, roffset + i) & 0xFF); + int res = (getByte(i) & 0xFF) - (rbase.getByte(roffset + i) & 0xFF); if (res != 0) { return res; } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java index 666f486f7899c..59a5f2222fad2 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java @@ -1,99 +1,99 @@ -/* - * 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.spark.unsafe.memory; - -import org.apache.spark.unsafe.Platform; -import org.junit.Assert; -import org.junit.Test; - -import org.apache.spark.unsafe.memory.LongArrayMemoryBlock; - -public class MemoryBlockSuite { - - private void check(MemoryBlock memory, Object obj, long offset, int length) { - memory.setPageNumber(1); - memory.fill((byte)-1); - MemoryBlock.putBoolean(memory, offset, true); - MemoryBlock.putByte(memory, offset + 1, (byte)127); - MemoryBlock.putShort(memory, offset + 2, (short)257); - MemoryBlock.putInt(memory, offset + 4, 0x20000002); - MemoryBlock.putLong(memory, offset + 8, -1L); - MemoryBlock.putFloat(memory, offset + 16, 1.0F); - MemoryBlock.putDouble(memory, offset + 20, 2.0); - MemoryBlock.copyMemory(memory, offset, memory, offset + 28, 4); - - Assert.assertEquals(obj, memory.getBaseObject()); - Assert.assertEquals(offset, memory.getBaseOffset()); - Assert.assertEquals(length, memory.size()); - Assert.assertEquals(1, memory.getPageNumber()); - Assert.assertEquals(true, MemoryBlock.getBoolean(memory, offset)); - Assert.assertEquals((byte)127, MemoryBlock.getByte(memory, offset + 1 )); - Assert.assertEquals((short)257, MemoryBlock.getShort(memory, offset + 2)); - Assert.assertEquals(0x20000002, MemoryBlock.getInt(memory, offset + 4)); - Assert.assertEquals(-1L, MemoryBlock.getLong(memory, offset + 8)); - Assert.assertEquals(1.0F, MemoryBlock.getFloat(memory, offset + 16), 0); - Assert.assertEquals(2.0, MemoryBlock.getDouble(memory, offset + 20), 0); - Assert.assertEquals(true, MemoryBlock.getBoolean(memory, offset + 28)); - Assert.assertEquals((byte)127, MemoryBlock.getByte(memory, offset + 29 )); - Assert.assertEquals((short)257, MemoryBlock.getShort(memory, offset + 30)); - for (int i = 32; i < memory.size(); i++) { - Assert.assertEquals((byte) -1, MemoryBlock.getByte(memory, offset + i)); - } - } - - @Test - public void ByteArrayMemoryBlockTest() { - byte[] obj = new byte[36]; - long offset = Platform.BYTE_ARRAY_OFFSET; - int length = obj.length; - MemoryBlock memory = new ByteArrayMemoryBlock(obj, offset, length); - - check(memory, obj, offset, length); - } - - @Test - public void IntArrayMemoryBlockTest() { - int[] obj = new int[9]; - long offset = Platform.INT_ARRAY_OFFSET; - int length = obj.length; - MemoryBlock memory = new IntArrayMemoryBlock(obj, offset, length); - - check(memory, obj, offset, length); - } - - @Test - public void LongArrayMemoryBlockTest() { - long[] obj = new long[5]; - long offset = Platform.LONG_ARRAY_OFFSET; - int length = obj.length; - MemoryBlock memory = new LongArrayMemoryBlock(obj, offset, length); - - check(memory, obj, offset, length); - } - - @Test - public void OffHeapArrayMemoryBlockTest() { - MemoryAllocator memoryAllocator = new UnsafeMemoryAllocator(); - MemoryBlock memory = memoryAllocator.allocate(36); - Object obj = memory.getBaseObject(); - long offset = memory.getBaseOffset(); - int length = 36; - - check(memory, obj, offset, length); - } -} +/* + * 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.spark.unsafe.memory; + +import org.apache.spark.unsafe.Platform; +import org.junit.Assert; +import org.junit.Test; + +import org.apache.spark.unsafe.memory.LongArrayMemoryBlock; + +public class MemoryBlockSuite { + + private void check(MemoryBlock memory, Object obj, long offset, int length) { + memory.setPageNumber(1); + memory.fill((byte)-1); + memory.putBoolean(offset, true); + memory.putByte(offset + 1, (byte)127); + memory.putShort(offset + 2, (short)257); + memory.putInt(offset + 4, 0x20000002); + memory.putLong(offset + 8, -1L); + memory.putFloat(offset + 16, 1.0F); + memory.putDouble(offset + 20, 2.0); + MemoryBlock.copyMemory(memory, offset, memory, offset + 28, 4); + + Assert.assertEquals(obj, memory.getBaseObject()); + Assert.assertEquals(offset, memory.getBaseOffset()); + Assert.assertEquals(length, memory.size()); + Assert.assertEquals(1, memory.getPageNumber()); + Assert.assertEquals(true, memory.getBoolean(offset)); + Assert.assertEquals((byte)127, memory.getByte(offset + 1 )); + Assert.assertEquals((short)257, memory.getShort(offset + 2)); + Assert.assertEquals(0x20000002, memory.getInt(offset + 4)); + Assert.assertEquals(-1L, memory.getLong(offset + 8)); + Assert.assertEquals(1.0F, memory.getFloat(offset + 16), 0); + Assert.assertEquals(2.0, memory.getDouble(offset + 20), 0); + Assert.assertEquals(true, memory.getBoolean(offset + 28)); + Assert.assertEquals((byte)127, memory.getByte(offset + 29 )); + Assert.assertEquals((short)257, memory.getShort(offset + 30)); + for (int i = 32; i < memory.size(); i++) { + Assert.assertEquals((byte) -1, memory.getByte(offset + i)); + } + } + + @Test + public void ByteArrayMemoryBlockTest() { + byte[] obj = new byte[36]; + long offset = Platform.BYTE_ARRAY_OFFSET; + int length = obj.length; + MemoryBlock memory = new ByteArrayMemoryBlock(obj, offset, length); + + check(memory, obj, offset, length); + } + + @Test + public void IntArrayMemoryBlockTest() { + int[] obj = new int[9]; + long offset = Platform.INT_ARRAY_OFFSET; + int length = obj.length; + MemoryBlock memory = new IntArrayMemoryBlock(obj, offset, length); + + check(memory, obj, offset, length); + } + + @Test + public void LongArrayMemoryBlockTest() { + long[] obj = new long[5]; + long offset = Platform.LONG_ARRAY_OFFSET; + int length = obj.length; + MemoryBlock memory = new LongArrayMemoryBlock(obj, offset, length); + + check(memory, obj, offset, length); + } + + @Test + public void OffHeapArrayMemoryBlockTest() { + MemoryAllocator memoryAllocator = new UnsafeMemoryAllocator(); + MemoryBlock memory = memoryAllocator.allocate(36); + Object obj = memory.getBaseObject(); + long offset = memory.getBaseOffset(); + int length = 36; + + check(memory, obj, offset, length); + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java index 024217bced02a..01a092f786a45 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -106,12 +106,12 @@ public void close() { @Override public void putNotNull(int rowId) { - MemoryBlock.putByte(nulls, nulls.getBaseOffset() + rowId, (byte) 0); + nulls.putByte(nulls.getBaseOffset() + rowId, (byte) 0); } @Override public void putNull(int rowId) { - MemoryBlock.putByte(nulls, nulls.getBaseOffset() + rowId, (byte) 1); + nulls.putByte(nulls.getBaseOffset() + rowId, (byte) 1); ++numNulls; } @@ -119,7 +119,7 @@ public void putNull(int rowId) { public void putNulls(int rowId, int count) { long offset = nulls.getBaseOffset() + rowId; for (int i = 0; i < count; ++i, ++offset) { - MemoryBlock.putByte(nulls, offset, (byte) 1); + nulls.putByte(offset, (byte) 1); } numNulls += count; } @@ -129,13 +129,13 @@ public void putNotNulls(int rowId, int count) { if (!hasNull()) return; long offset = nulls.getBaseOffset() + rowId; for (int i = 0; i < count; ++i, ++offset) { - MemoryBlock.putByte(nulls, offset, (byte) 0); + nulls.putByte(offset, (byte) 0); } } @Override public boolean isNullAt(int rowId) { - return MemoryBlock.getByte(nulls, nulls.getBaseOffset() + rowId) == 1; + return nulls.getByte(nulls.getBaseOffset() + rowId) == 1; } // @@ -144,26 +144,26 @@ public boolean isNullAt(int rowId) { @Override public void putBoolean(int rowId, boolean value) { - MemoryBlock.putByte(data, data.getBaseOffset() + rowId, (byte)((value) ? 1 : 0)); + data.putByte(data.getBaseOffset() + rowId, (byte)((value) ? 1 : 0)); } @Override public void putBooleans(int rowId, int count, boolean value) { byte v = (byte)((value) ? 1 : 0); for (int i = 0; i < count; ++i) { - MemoryBlock.putByte(data, data.getBaseOffset() + rowId + i, v); + data.putByte(data.getBaseOffset() + rowId + i, v); } } @Override - public boolean getBoolean(int rowId) { return MemoryBlock.getByte(data, data.getBaseOffset() + rowId) == 1; } + public boolean getBoolean(int rowId) { return data.getByte(data.getBaseOffset() + rowId) == 1; } @Override public boolean[] getBooleans(int rowId, int count) { assert(dictionary == null); boolean[] array = new boolean[count]; for (int i = 0; i < count; ++i) { - array[i] = (MemoryBlock.getByte(data, data.getBaseOffset() + rowId + i) == 1); + array[i] = (data.getByte(data.getBaseOffset() + rowId + i) == 1); } return array; } @@ -174,14 +174,14 @@ public boolean[] getBooleans(int rowId, int count) { @Override public void putByte(int rowId, byte value) { - MemoryBlock.putByte(data, data.getBaseOffset() + rowId, value); + data.putByte(data.getBaseOffset() + rowId, value); } @Override public void putBytes(int rowId, int count, byte value) { for (int i = 0; i < count; ++i) { - MemoryBlock.putByte(data, data.getBaseOffset() + rowId + i, value); + data.putByte(data.getBaseOffset() + rowId + i, value); } } @@ -193,7 +193,7 @@ public void putBytes(int rowId, int count, byte[] src, int srcIndex) { @Override public byte getByte(int rowId) { if (dictionary == null) { - return MemoryBlock.getByte(data, data.getBaseOffset() + rowId); + return data.getByte(data.getBaseOffset() + rowId); } else { return (byte) dictionary.decodeToInt(dictionaryIds.getDictId(rowId)); } @@ -218,14 +218,14 @@ protected UTF8String getBytesAsUTF8String(int rowId, int count) { @Override public void putShort(int rowId, short value) { - MemoryBlock.putShort(data, data.getBaseOffset() + 2 * rowId, value); + data.putShort(data.getBaseOffset() + 2 * rowId, value); } @Override public void putShorts(int rowId, int count, short value) { long offset = data.getBaseOffset() + 2 * rowId; for (int i = 0; i < count; ++i, offset += 2) { - MemoryBlock.putShort(data, offset, value); + data.putShort(offset, value); } } @@ -244,7 +244,7 @@ public void putShorts(int rowId, int count, byte[] src, int srcIndex) { @Override public short getShort(int rowId) { if (dictionary == null) { - return MemoryBlock.getShort(data, data.getBaseOffset() + 2 * rowId); + return data.getShort(data.getBaseOffset() + 2 * rowId); } else { return (short) dictionary.decodeToInt(dictionaryIds.getDictId(rowId)); } @@ -264,14 +264,14 @@ public short[] getShorts(int rowId, int count) { @Override public void putInt(int rowId, int value) { - MemoryBlock.putInt(data, data.getBaseOffset() + 4 * rowId, value); + data.putInt(data.getBaseOffset() + 4 * rowId, value); } @Override public void putInts(int rowId, int count, int value) { long offset = data.getBaseOffset() + 4 * rowId; for (int i = 0; i < count; ++i, offset += 4) { - MemoryBlock.putInt(data, offset, value); + data.putInt(offset, value); } } @@ -296,8 +296,7 @@ public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; long offset = data.getBaseOffset() + 4 * rowId; for (int i = 0; i < count; ++i, offset += 4, srcOffset += 4) { - MemoryBlock.putInt(data, offset, - java.lang.Integer.reverseBytes(Platform.getInt(src, srcOffset))); + data.putInt(offset, java.lang.Integer.reverseBytes(Platform.getInt(src, srcOffset))); } } } @@ -305,7 +304,7 @@ public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) @Override public int getInt(int rowId) { if (dictionary == null) { - return MemoryBlock.getInt(data, data.getBaseOffset() + 4 * rowId); + return data.getInt(data.getBaseOffset() + 4 * rowId); } else { return dictionary.decodeToInt(dictionaryIds.getDictId(rowId)); } @@ -327,7 +326,7 @@ public int[] getInts(int rowId, int count) { public int getDictId(int rowId) { assert(dictionary == null) : "A ColumnVector dictionary should not have a dictionary for itself."; - return MemoryBlock.getInt(data, data.getBaseOffset() + 4 * rowId); + return data.getInt(data.getBaseOffset() + 4 * rowId); } // @@ -336,14 +335,14 @@ public int getDictId(int rowId) { @Override public void putLong(int rowId, long value) { - MemoryBlock.putLong(data, data.getBaseOffset() + 8 * rowId, value); + data.putLong(data.getBaseOffset() + 8 * rowId, value); } @Override public void putLongs(int rowId, int count, long value) { long offset = data.getBaseOffset() + 8 * rowId; for (int i = 0; i < count; ++i, offset += 8) { - MemoryBlock.putLong(data, offset, value); + data.putLong(offset, value); } } @@ -368,8 +367,7 @@ public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; long offset = data.getBaseOffset() + 8 * rowId; for (int i = 0; i < count; ++i, offset += 8, srcOffset += 8) { - MemoryBlock.putLong(data, offset, - java.lang.Long.reverseBytes(Platform.getLong(src, srcOffset))); + data.putLong(offset, java.lang.Long.reverseBytes(Platform.getLong(src, srcOffset))); } } } @@ -377,7 +375,7 @@ public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) @Override public long getLong(int rowId) { if (dictionary == null) { - return MemoryBlock.getLong(data, data.getBaseOffset() + 8 * rowId); + return data.getLong(data.getBaseOffset() + 8 * rowId); } else { return dictionary.decodeToLong(dictionaryIds.getDictId(rowId)); } @@ -397,14 +395,14 @@ public long[] getLongs(int rowId, int count) { @Override public void putFloat(int rowId, float value) { - MemoryBlock.putFloat(data, data.getBaseOffset() + rowId * 4, value); + data.putFloat(data.getBaseOffset() + rowId * 4, value); } @Override public void putFloats(int rowId, int count, float value) { long offset = data.getBaseOffset() + 4 * rowId; for (int i = 0; i < count; ++i, offset += 4) { - MemoryBlock.putFloat(data, offset, value); + data.putFloat(offset, value); } } @@ -423,7 +421,7 @@ public void putFloats(int rowId, int count, byte[] src, int srcIndex) { ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); long offset = data.getBaseOffset() + 4 * rowId; for (int i = 0; i < count; ++i, offset += 4) { - MemoryBlock.putFloat(data, offset, bb.getFloat(srcIndex + (4 * i))); + data.putFloat(offset, bb.getFloat(srcIndex + (4 * i))); } } } @@ -431,7 +429,7 @@ public void putFloats(int rowId, int count, byte[] src, int srcIndex) { @Override public float getFloat(int rowId) { if (dictionary == null) { - return MemoryBlock.getFloat(data, data.getBaseOffset() + rowId * 4); + return data.getFloat(data.getBaseOffset() + rowId * 4); } else { return dictionary.decodeToFloat(dictionaryIds.getDictId(rowId)); } @@ -452,14 +450,14 @@ public float[] getFloats(int rowId, int count) { @Override public void putDouble(int rowId, double value) { - MemoryBlock.putDouble(data, data.getBaseOffset() + rowId * 8, value); + data.putDouble(data.getBaseOffset() + rowId * 8, value); } @Override public void putDoubles(int rowId, int count, double value) { long offset = data.getBaseOffset() + 8 * rowId; for (int i = 0; i < count; ++i, offset += 8) { - MemoryBlock.putDouble(data, offset, value); + data.putDouble(offset, value); } } @@ -478,7 +476,7 @@ public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); long offset = data.getBaseOffset() + 8 * rowId; for (int i = 0; i < count; ++i, offset += 8) { - MemoryBlock.putDouble(data, offset, bb.getDouble(srcIndex + (8 * i))); + data.putDouble(offset, bb.getDouble(srcIndex + (8 * i))); } } } @@ -486,7 +484,7 @@ public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { @Override public double getDouble(int rowId) { if (dictionary == null) { - return MemoryBlock.getDouble(data, data.getBaseOffset() + rowId * 8); + return data.getDouble(data.getBaseOffset() + rowId * 8); } else { return dictionary.decodeToDouble(dictionaryIds.getDictId(rowId)); } @@ -506,26 +504,26 @@ public double[] getDoubles(int rowId, int count) { @Override public void putArray(int rowId, int offset, int length) { assert(offset >= 0 && offset + length <= childColumns[0].capacity); - MemoryBlock.putInt(lengthData, lengthData.getBaseOffset() + 4 * rowId, length); - MemoryBlock.putInt(offsetData, offsetData.getBaseOffset() + 4 * rowId, offset); + lengthData.putInt(lengthData.getBaseOffset() + 4 * rowId, length); + offsetData.putInt(offsetData.getBaseOffset() + 4 * rowId, offset); } @Override public int getArrayLength(int rowId) { - return MemoryBlock.getInt(lengthData, lengthData.getBaseOffset() + 4 * rowId); + return lengthData.getInt(lengthData.getBaseOffset() + 4 * rowId); } @Override public int getArrayOffset(int rowId) { - return MemoryBlock.getInt(offsetData, offsetData.getBaseOffset() + 4 * rowId); + return offsetData.getInt(offsetData.getBaseOffset() + 4 * rowId); } // APIs dealing with ByteArrays @Override public int putByteArray(int rowId, byte[] value, int offset, int length) { int result = arrayData().appendBytes(length, value, offset); - MemoryBlock.putInt(lengthData, lengthData.getBaseOffset() + 4 * rowId, length); - MemoryBlock.putInt(offsetData, offsetData.getBaseOffset() + 4 * rowId, result); + lengthData.putInt(lengthData.getBaseOffset() + 4 * rowId, length); + offsetData.putInt(offsetData.getBaseOffset() + 4 * rowId, result); return result; } From 05f024e566f828e9c3f836430c9c7b34da5e954b Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 28 Sep 2017 02:51:38 +0100 Subject: [PATCH 18/65] rename methods related to hash --- .../sql/catalyst/expressions/HiveHasher.java | 2 +- .../spark/unsafe/hash/Murmur3_x86_32.java | 10 ++++----- .../apache/spark/unsafe/types/UTF8String.java | 2 +- .../spark/mllib/feature/HashingTF.scala | 2 +- .../spark/sql/catalyst/expressions/XXH64.java | 2 +- .../spark/sql/catalyst/expressions/hash.scala | 21 ++++++++++--------- .../catalyst/expressions/HiveHasherSuite.java | 2 +- 7 files changed, 21 insertions(+), 20 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java index 77f96e11492b0..147c367b75898 100644 --- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java @@ -39,7 +39,7 @@ public static int hashLong(long input) { return (int) ((input >>> 32) ^ input); } - public static int hashUnsafeBlock(MemoryBlock base, long offset, int lengthInBytes) { + public static int hashUnsafeBytesBlock(MemoryBlock base, long offset, int lengthInBytes) { return hashUnsafeBytes(base.getBaseObject(), offset, lengthInBytes); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index 518b93c7e374a..6f105616f063e 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -53,6 +53,10 @@ public int hashUnsafeWords(Object base, long offset, int lengthInBytes) { return hashUnsafeWords(base, offset, lengthInBytes, seed); } + public static int hashUnsafeWordsBlock(MemoryBlock base, long offset, int lengthInBytes, int seed) { + return hashUnsafeWords(base.getBaseObject(), offset, lengthInBytes, seed); + } + public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, int seed) { // This is based on Guava's `Murmur32_Hasher.processRemaining(ByteBuffer)` method. assert (lengthInBytes % 8 == 0): "lengthInBytes must be a multiple of 8 (word-aligned)"; @@ -60,7 +64,7 @@ public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, i return fmix(h1, lengthInBytes); } - public static int hashUnsafeBlock(MemoryBlock base, long offset, int lengthInBytes, int seed) { + public static int hashUnsafeBytesBlock(MemoryBlock base, long offset, int lengthInBytes, int seed) { return hashUnsafeBytes(base.getBaseObject(), offset, lengthInBytes, seed); } @@ -92,10 +96,6 @@ public static int hashUnsafeBytes2(Object base, long offset, int lengthInBytes, return fmix(h1, lengthInBytes); } - private static int hashBytesByIntMB(MemoryBlock base, long offset, int lengthInBytes, int seed) { - return hashBytesByInt(base.getBaseObject(), offset, lengthInBytes, seed); - } - private static int hashBytesByInt(Object base, long offset, int lengthInBytes, int seed) { assert (lengthInBytes % 4 == 0); int h1 = seed; diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index a13cfb3111424..d2794c8e67cb0 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -1312,7 +1312,7 @@ public int levenshteinDistance(UTF8String other) { @Override public int hashCode() { - return Murmur3_x86_32.hashUnsafeBlock(base, offset, numBytes, 42); + return Murmur3_x86_32.hashUnsafeBytesBlock(base, offset, numBytes, 42); } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala index f1a15192688b6..b02fa48e3e1be 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala @@ -160,7 +160,7 @@ object HashingTF { case d: Double => hashLong(java.lang.Double.doubleToLongBits(d), seed) case s: String => val utf8 = UTF8String.fromString(s) - hashUnsafeBlock(utf8.getBaseObject, utf8.getBaseOffset, utf8.numBytes(), seed) + hashUnsafeBytesBlock(utf8.getBaseObject, utf8.getBaseOffset, utf8.numBytes(), seed) case _ => throw new SparkException("HashingTF with murmur3 algorithm does not " + s"support type ${term.getClass.getCanonicalName} of input data.") } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java index baf55516711bc..9db6b94b2b8e4 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java @@ -86,7 +86,7 @@ public long hashUnsafeBytes(Object base, long offset, int length) { return hashUnsafeBytes(base, offset, length, seed); } - public static long hashUnsafeBlock(MemoryBlock base, long offset, int length, long seed) { + public static long hashUnsafeBytesBlock(MemoryBlock base, long offset, int length, long seed) { return hashUnsafeBytes(base.getBaseObject(), offset, length, seed); } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala index 118cd34c27176..c9db59745d89b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala @@ -363,7 +363,7 @@ abstract class HashExpression[E] extends Expression { val baseObject = s"$input.getBaseObject()" val baseOffset = s"$input.getBaseOffset()" val numBytes = s"$input.numBytes()" - s"$result = $hasherClassName.hashUnsafeBlock($baseObject, $baseOffset, $numBytes, $result);" + s"$result = $hasherClassName.hashUnsafeBytesBlock($baseObject, $baseOffset, $numBytes, $result);" } protected def genHashForMap( @@ -465,7 +465,8 @@ abstract class InterpretedHashFunction { protected def hashUnsafeBytes(base: AnyRef, offset: Long, length: Int, seed: Long): Long - protected def hashUnsafeBlock(base: MemoryBlock, offset: Long, length: Int, seed: Long): Long + protected def hashUnsafeBytesBlock( + base: MemoryBlock, offset: Long, length: Int, seed: Long): Long /** * Computes hash of a given `value` of type `dataType`. The caller needs to check the validity @@ -493,7 +494,7 @@ abstract class InterpretedHashFunction { case a: Array[Byte] => hashUnsafeBytes(a, Platform.BYTE_ARRAY_OFFSET, a.length, seed) case s: UTF8String => - hashUnsafeBlock(s.getBaseObject, s.getBaseOffset, s.numBytes(), seed) + hashUnsafeBytesBlock(s.getBaseObject, s.getBaseOffset, s.numBytes(), seed) case array: ArrayData => val elementType = dataType match { @@ -585,9 +586,9 @@ object Murmur3HashFunction extends InterpretedHashFunction { Murmur3_x86_32.hashUnsafeBytes(base, offset, len, seed.toInt) } - override protected def hashUnsafeBlock( + override protected def hashUnsafeBytesBlock( base: MemoryBlock, offset: Long, len: Int, seed: Long): Long = { - Murmur3_x86_32.hashUnsafeBlock(base, offset, len, seed.toInt) + Murmur3_x86_32.hashUnsafeBytesBlock(base, offset, len, seed.toInt) } } @@ -618,9 +619,9 @@ object XxHash64Function extends InterpretedHashFunction { XXH64.hashUnsafeBytes(base, offset, len, seed) } - override protected def hashUnsafeBlock( + override protected def hashUnsafeBytesBlock( base: MemoryBlock, offset: Long, len: Int, seed: Long): Long = { - XXH64.hashUnsafeBlock(base, offset, len, seed) + XXH64.hashUnsafeBytesBlock(base, offset, len, seed) } } @@ -731,7 +732,7 @@ case class HiveHash(children: Seq[Expression]) extends HashExpression[Int] { val baseObject = s"$input.getBaseObject()" val baseOffset = s"$input.getBaseOffset()" val numBytes = s"$input.numBytes()" - s"$result = $hasherClassName.hashUnsafeBlock($baseObject, $baseOffset, $numBytes);" + s"$result = $hasherClassName.hashUnsafeBytesBlock($baseObject, $baseOffset, $numBytes);" } override protected def genHashForArray( @@ -824,9 +825,9 @@ object HiveHashFunction extends InterpretedHashFunction { HiveHasher.hashUnsafeBytes(base, offset, len) } - override protected def hashUnsafeBlock( + override protected def hashUnsafeBytesBlock( base: MemoryBlock, offset: Long, len: Int, seed: Long): Long = { - HiveHasher.hashUnsafeBlock(base, offset, len) + HiveHasher.hashUnsafeBytesBlock(base, offset, len) } private val HIVE_DECIMAL_MAX_PRECISION = 38 diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java index 298d6b86e6dc0..0f1b692ea43be 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java @@ -53,7 +53,7 @@ public void testKnownStringAndIntInputs() { for (int i = 0; i < inputs.length; i++) { UTF8String s = UTF8String.fromString("val_" + inputs[i]); - int hash = HiveHasher.hashUnsafeBlock(s.getBaseObject(), s.getBaseOffset(), s.numBytes()); + int hash = HiveHasher.hashUnsafeBytesBlock(s.getBaseObject(), s.getBaseOffset(), s.numBytes()); Assert.assertEquals(expected[i], ((31 * inputs[i]) + hash)); } } From 9071cf6449123400f3d774664e1709337b05c555 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 28 Sep 2017 02:52:48 +0100 Subject: [PATCH 19/65] added methods for MemoryBlock --- .../spark/unsafe/bitset/BitSetMethods.java | 35 +++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java index c8c57381f332f..b2665715785ce 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java @@ -18,6 +18,7 @@ package org.apache.spark.unsafe.bitset; import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.memory.MemoryBlock; /** * Methods for working with fixed-size uncompressed bitsets. @@ -45,6 +46,14 @@ public static void set(Object baseObject, long baseOffset, int index) { Platform.putLong(baseObject, wordOffset, word | mask); } + public static void setBlock(MemoryBlock base, long baseOffset, int index) { + assert index >= 0 : "index (" + index + ") should >= 0"; + final long mask = 1L << (index & 0x3f); // mod 64 and shift + final long wordOffset = baseOffset + (index >> 6) * WORD_SIZE; + final long word = base.getLong(wordOffset); + base.putLong(wordOffset, word | mask); + } + /** * Sets the bit at the specified index to {@code false}. */ @@ -56,6 +65,14 @@ public static void unset(Object baseObject, long baseOffset, int index) { Platform.putLong(baseObject, wordOffset, word & ~mask); } + public static void unsetBlock(MemoryBlock base, long baseOffset, int index) { + assert index >= 0 : "index (" + index + ") should >= 0"; + final long mask = 1L << (index & 0x3f); // mod 64 and shift + final long wordOffset = baseOffset + (index >> 6) * WORD_SIZE; + final long word = base.getLong(wordOffset); + base.putLong(wordOffset, word & ~mask); + } + /** * Returns {@code true} if the bit is set at the specified index. */ @@ -67,6 +84,14 @@ public static boolean isSet(Object baseObject, long baseOffset, int index) { return (word & mask) != 0; } + public static boolean isSetBlock(MemoryBlock base, long baseOffset, int index) { + assert index >= 0 : "index (" + index + ") should >= 0"; + final long mask = 1L << (index & 0x3f); // mod 64 and shift + final long wordOffset = baseOffset + (index >> 6) * WORD_SIZE; + final long word = base.getLong(wordOffset); + return (word & mask) != 0; + } + /** * Returns {@code true} if any bit is set. */ @@ -80,6 +105,16 @@ public static boolean anySet(Object baseObject, long baseOffset, long bitSetWidt return false; } + public static boolean anySetBlock(MemoryBlock base, long baseOffset, long bitSetWidthInWords) { + long addr = baseOffset; + for (int i = 0; i < bitSetWidthInWords; i++, addr += WORD_SIZE) { + if (base.getLong(addr) != 0) { + return true; + } + } + return false; + } + /** * Returns the index of the first bit that is set to true that occurs on or after the * specified starting index. If no such bit exists then {@code -1} is returned. From 37ee9fa07a8f8faaeb097164422bb15958fa4b1c Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 28 Sep 2017 04:44:30 +0100 Subject: [PATCH 20/65] rebase with master --- .../org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java index 8561439869c15..13a1e7ef749af 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -46,8 +46,9 @@ public OffHeapMemoryBlock allocate(long size) throws OutOfMemoryError { @Override public void free(MemoryBlock memory) { + if (memory == OffHeapMemoryBlock.NULL) return; assert(memory instanceof OffHeapMemoryBlock); - assert (memory.getBaseObject() == null) : + assert (memory.getBaseObject() != null) : "baseObject not null; are you trying to use the off-heap allocator to free on-heap memory?"; assert (memory.getPageNumber() != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : "page has already been freed"; From d0b5d59bb31fe2845477ee243008992686e2f2a2 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 28 Sep 2017 05:37:25 +0100 Subject: [PATCH 21/65] fix scala style error --- .../scala/org/apache/spark/sql/catalyst/expressions/hash.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala index c9db59745d89b..9dcda27f79e49 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala @@ -363,7 +363,8 @@ abstract class HashExpression[E] extends Expression { val baseObject = s"$input.getBaseObject()" val baseOffset = s"$input.getBaseOffset()" val numBytes = s"$input.numBytes()" - s"$result = $hasherClassName.hashUnsafeBytesBlock($baseObject, $baseOffset, $numBytes, $result);" + s"$result = $hasherClassName.hashUnsafeBytesBlock(" + + s"$baseObject, $baseOffset, $numBytes, $result);" } protected def genHashForMap( From 5cdad44717ccb510d4114d14cfff304bef9f5bb4 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 14 Oct 2017 08:29:14 +0100 Subject: [PATCH 22/65] use MemoryBlock in Murmur3 for performance reason --- .../spark/unsafe/hash/Murmur3_x86_32.java | 21 ++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index 6f105616f063e..601b992c72c25 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -65,7 +65,15 @@ public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, i } public static int hashUnsafeBytesBlock(MemoryBlock base, long offset, int lengthInBytes, int seed) { - return hashUnsafeBytes(base.getBaseObject(), offset, lengthInBytes, seed); + assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; + int lengthAligned = lengthInBytes - lengthInBytes % 4; + int h1 = hashBytesByIntBlock(base, offset, lengthAligned, seed); + for (int i = lengthAligned; i < lengthInBytes; i++) { + int halfWord = base.getByte(offset + i); + int k1 = mixK1(halfWord); + h1 = mixH1(h1, k1); + } + return fmix(h1, lengthInBytes); } public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes, int seed) { @@ -96,6 +104,17 @@ public static int hashUnsafeBytes2(Object base, long offset, int lengthInBytes, return fmix(h1, lengthInBytes); } + private static int hashBytesByIntBlock(MemoryBlock base, long offset, int lengthInBytes, int seed) { + assert (lengthInBytes % 4 == 0); + int h1 = seed; + for (int i = 0; i < lengthInBytes; i += 4) { + int halfWord = base.getInt(offset + i); + int k1 = mixK1(halfWord); + h1 = mixH1(h1, k1); + } + return h1; + } + private static int hashBytesByInt(Object base, long offset, int lengthInBytes, int seed) { assert (lengthInBytes % 4 == 0); int h1 = seed; From 91028fa2ae34bb3ae667692112b8455d4394cbbd Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 14 Oct 2017 08:29:30 +0100 Subject: [PATCH 23/65] fix typo in comment --- .../main/java/org/apache/spark/unsafe/memory/MemoryBlock.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index c489d2da86b51..254e76faa2364 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -54,7 +54,7 @@ public abstract class MemoryBlock { /** * Optional page number; used when this MemoryBlock represents a page allocated by a * TaskMemoryManager. This field can be updated using setPageNumber method so that - * this can be modified by the TaskMemoryManage, which lives in a different package. + * this can be modified by the TaskMemoryManager, which lives in a different package. */ private int pageNumber = NO_PAGE_NUMBER; From 0210bd1e5f46f81617a35493d2cd0b737b4cf85d Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 29 Oct 2017 13:32:38 +0100 Subject: [PATCH 24/65] address review comment --- .../apache/spark/unsafe/array/LongArray.java | 9 +- .../spark/unsafe/memory/MemoryBlock.java | 27 ++++- .../unsafe/memory/OffHeapMemoryBlock.java | 11 +- .../unsafe/memory/UnsafeMemoryAllocator.java | 24 ++--- .../apache/spark/unsafe/types/UTF8String.java | 100 +++++++++++------- .../catalyst/expressions/UnsafeArrayData.java | 6 +- .../sql/catalyst/expressions/UnsafeRow.java | 6 +- 7 files changed, 108 insertions(+), 75 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java index 7e086e9a7867e..206cc6b5c4440 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java @@ -32,14 +32,12 @@ public final class LongArray { private static final long WIDTH = 8; private final MemoryBlock memory; - private final long baseOffset; private final long length; public LongArray(MemoryBlock memory) { assert memory.size() < (long) Integer.MAX_VALUE * 8: "Array size >= Integer.MAX_VALUE elements"; this.memory = memory; - this.baseOffset = memory.getBaseOffset(); this.length = memory.size() / WIDTH; } @@ -52,7 +50,7 @@ public Object getBaseObject() { } public long getBaseOffset() { - return baseOffset; + return memory.getBaseOffset(); } /** @@ -66,6 +64,7 @@ public long size() { * Fill this all with 0L. */ public void zeroOut() { + long baseOffset = memory.getBaseOffset(); for (long off = baseOffset; off < baseOffset + length * WIDTH; off += WIDTH) { memory.putLong(off, 0); } @@ -77,7 +76,7 @@ public void zeroOut() { public void set(int index, long value) { assert index >= 0 : "index (" + index + ") should >= 0"; assert index < length : "index (" + index + ") should < length (" + length + ")"; - memory.putLong(baseOffset + index * WIDTH, value); + memory.putLong(memory.getBaseOffset() + index * WIDTH, value); } /** @@ -86,6 +85,6 @@ public void set(int index, long value) { public long get(int index) { assert index >= 0 : "index (" + index + ") should >= 0"; assert index < length : "index (" + index + ") should < length (" + length + ")"; - return memory.getLong(baseOffset + index * WIDTH); + return memory.getLong(memory.getBaseOffset() + index * WIDTH); } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index 254e76faa2364..e097618a1f832 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -17,10 +17,10 @@ package org.apache.spark.unsafe.memory; -import org.apache.spark.unsafe.Platform; - import javax.annotation.Nullable; +import org.apache.spark.unsafe.Platform; + /** * A declaration of interfaces of MemoryBlock classes . */ @@ -103,6 +103,29 @@ public final void fill(byte value) { Platform.setMemory(obj, offset, length, value); } + /** + * Instantiate MemoryBlock for given object type with new offset + */ + public final static MemoryBlock allocateFromObject(Object obj, long offset, long length) { + MemoryBlock mb = null; + if (obj instanceof byte[]) { + byte[] array = (byte[])obj; + mb = new ByteArrayMemoryBlock(array, offset, length); + } else if (obj instanceof int[]) { + int[] array = (int[])obj; + mb = new IntArrayMemoryBlock(array, offset, length); + } else if (obj instanceof long[]) { + long[] array = (long[])obj; + mb = new LongArrayMemoryBlock(array, offset, length); + } else if (obj == null) { + // we assume that to pass null pointer means off-heap + mb = new OffHeapMemoryBlock(offset, length); + } else { + throw new UnsupportedOperationException(obj.getClass() + " is not supported now"); + } + return mb; + } + /** * Instantiate the same type of MemoryBlock with new offset and size */ diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java index 5f574db5ad04c..4dcc40bc28f46 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java @@ -20,20 +20,15 @@ import org.apache.spark.unsafe.Platform; public class OffHeapMemoryBlock extends MemoryBlock { - private Object directBuffer; - private final long address; + static public final OffHeapMemoryBlock NULL = new OffHeapMemoryBlock(0, 0); - static public final OffHeapMemoryBlock NULL = new OffHeapMemoryBlock(null, 0, 0); - - public OffHeapMemoryBlock(Object directBuffer, long address, long size) { + public OffHeapMemoryBlock(long address, long size) { super(null, address, size); - this.address = address; - this.directBuffer = directBuffer; } @Override public MemoryBlock allocate(long offset, long size) { - return new OffHeapMemoryBlock(directBuffer, offset, size); + return new OffHeapMemoryBlock(offset, size); } public final int getInt(long offset) { diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java index 13a1e7ef749af..3e8626e33936b 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -32,16 +32,15 @@ public class UnsafeMemoryAllocator implements MemoryAllocator { @Override public OffHeapMemoryBlock allocate(long size) throws OutOfMemoryError { - Object buffer = ByteBuffer.allocateDirect((int)size); - if (buffer instanceof DirectBuffer) { - long addr = ((DirectBuffer) buffer).address(); - OffHeapMemoryBlock memory = new OffHeapMemoryBlock(buffer, addr, size); - if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { - memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE); - } - return memory; + // No usage of DirectByteBuffer.allocateDirect is current design + // Platform.allocateMemory is used here. + // http://downloads.typesafe.com/website/presentations/ScalaDaysSF2015/T4_Xin_Performance_Optimization.pdf#page=26 + long address = Platform.allocateMemory(size); + OffHeapMemoryBlock memory = new OffHeapMemoryBlock(address, size); + if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { + memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE); } - throw new UnsupportedOperationException("A ByteBuffer does not have an address in off-heap"); + return memory; } @Override @@ -60,21 +59,20 @@ public void free(MemoryBlock memory) { memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_FREED_VALUE); } + Platform.freeMemory(memory.offset); + // As an additional layer of defense against use-after-free bugs, we mutate the // MemoryBlock to reset its pointer. memory.resetObjAndOffset(); // Mark the page as freed (so we can detect double-frees). memory.setPageNumber(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER); - - // DirectByteBuffers are deallocated automatically by JVM when they become - // unreachable much like normal Objects in heap } public OffHeapMemoryBlock reallocate(OffHeapMemoryBlock block, long oldSize, long newSize) { OffHeapMemoryBlock mb = this.allocate(newSize); if (block.getBaseOffset() != 0) MemoryBlock.copyMemory(block, block.getBaseOffset(), mb, mb.getBaseOffset(), oldSize); - + free(block); return mb; } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index d2794c8e67cb0..bd2bede2cadd3 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -53,11 +53,10 @@ public final class UTF8String implements Comparable, Externalizable, // These are only updated by readExternal() or read() @Nonnull private MemoryBlock base; - private long offset; private int numBytes; public MemoryBlock getBaseObject() { return base; } - public long getBaseOffset() { return offset; } + public long getBaseOffset() { return base.getBaseOffset(); } private static int[] bytesOfCodePointInUTF8 = {2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, @@ -102,7 +101,14 @@ public static UTF8String fromBytes(byte[] bytes, int offset, int numBytes) { * Creates an UTF8String from given address (base and offset) and length. */ public static UTF8String fromAddress(MemoryBlock base, long offset, int numBytes) { - return new UTF8String(base, offset, numBytes); + return new UTF8String(base.allocate(offset, numBytes), numBytes); + } + + /** + * Creates an UTF8String from given memory block. + */ + public static UTF8String fromMemoryBlock(MemoryBlock base) { + return new UTF8String(base, (int)base.size()); } /** @@ -122,18 +128,17 @@ public static UTF8String blankString(int length) { } protected UTF8String(byte[] bytes, long offset, int numBytes) { - this(new ByteArrayMemoryBlock(bytes, offset, numBytes), offset, numBytes); + this(new ByteArrayMemoryBlock(bytes, offset, numBytes), numBytes); } - protected UTF8String(MemoryBlock base, long offset, int numBytes) { + protected UTF8String(MemoryBlock base, int numBytes) { this.base = base; - this.offset = offset; this.numBytes = numBytes; } // for serialization public UTF8String() { - this((MemoryBlock)null, 0, 0); + this((MemoryBlock)null,0); } /** @@ -142,11 +147,11 @@ public UTF8String() { * bytes in this string. */ public void writeToMemory(byte[] target, long targetOffset) { - MemoryBlock.copyMemory(base, offset, target, targetOffset, numBytes); + MemoryBlock.copyMemory(base, base.getBaseOffset(), target, targetOffset, numBytes); } public void writeToMemory(MemoryBlock target, long targetOffset) { - MemoryBlock.copyMemory(base, offset, target, targetOffset, numBytes); + MemoryBlock.copyMemory(base, base.getBaseOffset(), target, targetOffset, numBytes); } public void writeTo(ByteBuffer buffer) { @@ -166,6 +171,7 @@ public void writeTo(ByteBuffer buffer) { */ @Nonnull public ByteBuffer getByteBuffer() { + long offset = base.getBaseOffset(); if (base instanceof ByteArrayMemoryBlock && offset >= BYTE_ARRAY_OFFSET) { final byte[] bytes = ((ByteArrayMemoryBlock) base).getByteArray(); @@ -231,6 +237,7 @@ public long getPrefix() { // After getting the data, we use a mask to mask out data that is not part of the string. long p; long mask = 0; + long offset = base.getBaseOffset(); if (IS_LITTLE_ENDIAN) { if (numBytes >= 8) { p = base.getLong(offset); @@ -267,6 +274,7 @@ public long getPrefix() { */ public byte[] getBytes() { // avoid copy if `base` is `byte[]` + long offset = base.getBaseOffset(); if (offset == BYTE_ARRAY_OFFSET && base instanceof ByteArrayMemoryBlock && (((ByteArrayMemoryBlock) base).getByteArray()).length == numBytes) { return ((ByteArrayMemoryBlock) base).getByteArray(); @@ -302,7 +310,7 @@ public UTF8String substring(final int start, final int until) { if (i > j) { byte[] bytes = new byte[i - j]; - MemoryBlock.copyMemory(base, offset + j, bytes, BYTE_ARRAY_OFFSET, i - j); + MemoryBlock.copyMemory(base, base.getBaseOffset() + j, bytes, BYTE_ARRAY_OFFSET, i - j); return fromBytes(bytes); } else { return EMPTY_UTF8; @@ -343,14 +351,14 @@ public boolean contains(final UTF8String substring) { * Returns the byte at position `i`. */ private byte getByte(int i) { - return base.getByte(offset + i); + return base.getByte(getBaseOffset() + i); } private boolean matchAt(final UTF8String s, int pos) { if (s.numBytes + pos > numBytes || pos < 0) { return false; } - return ByteArrayMethods.arrayEqualsBlock(base, offset + pos, s.base, s.offset, s.numBytes); + return ByteArrayMethods.arrayEqualsBlock(base, getBaseOffset() + pos, s.base, s.getBaseOffset(), s.numBytes); } public boolean startsWith(final UTF8String prefix) { @@ -473,11 +481,13 @@ public int findInSet(UTF8String match) { return 0; } + long offset = getBaseOffset(); + long matchOffset = match.getBaseOffset(); int n = 1, lastComma = -1; for (int i = 0; i < numBytes; i++) { if (getByte(i) == (byte) ',') { if (i - (lastComma + 1) == match.numBytes && - ByteArrayMethods.arrayEqualsBlock(base, offset + (lastComma + 1), match.base, match.offset, + ByteArrayMethods.arrayEqualsBlock(base, offset + (lastComma + 1), match.base, matchOffset, match.numBytes)) { return n; } @@ -486,7 +496,7 @@ public int findInSet(UTF8String match) { } } if (numBytes - (lastComma + 1) == match.numBytes && - ByteArrayMethods.arrayEqualsBlock(base, offset + (lastComma + 1), match.base, match.offset, + ByteArrayMethods.arrayEqualsBlock(base, offset + (lastComma + 1), match.base, matchOffset, match.numBytes)) { return n; } @@ -502,7 +512,7 @@ public int findInSet(UTF8String match) { private UTF8String copyUTF8String(int start, int end) { int len = end - start + 1; byte[] newBytes = new byte[len]; - MemoryBlock.copyMemory(base, offset + start, newBytes, BYTE_ARRAY_OFFSET, len); + MemoryBlock.copyMemory(base, getBaseOffset() + start, newBytes, BYTE_ARRAY_OFFSET, len); return UTF8String.fromBytes(newBytes); } @@ -646,10 +656,11 @@ public UTF8String trimRight(UTF8String trimString) { public UTF8String reverse() { byte[] result = new byte[this.numBytes]; + long offset = getBaseOffset(); int i = 0; // position in byte while (i < numBytes) { int len = numBytesForFirstByte(getByte(i)); - MemoryBlock.copyMemory(this.base, this.offset + i, result, + MemoryBlock.copyMemory(this.base, offset + i, result, BYTE_ARRAY_OFFSET + result.length - i - len, len); i += len; @@ -664,7 +675,7 @@ public UTF8String repeat(int times) { } byte[] newBytes = new byte[numBytes * times]; - MemoryBlock.copyMemory(this.base, this.offset, newBytes, BYTE_ARRAY_OFFSET, numBytes); + MemoryBlock.copyMemory(this.base, this.getBaseOffset(), newBytes, BYTE_ARRAY_OFFSET, numBytes); int copied = 1; while (copied < times) { @@ -697,11 +708,13 @@ public int indexOf(UTF8String v, int start) { c += 1; } + long offset = getBaseOffset(); + long vOffset = v.getBaseOffset(); do { if (i + v.numBytes > numBytes) { return -1; } - if (ByteArrayMethods.arrayEqualsBlock(base, offset + i, v.base, v.offset, v.numBytes)) { + if (ByteArrayMethods.arrayEqualsBlock(base, offset + i, v.base, vOffset, v.numBytes)) { return c; } i += numBytesForFirstByte(getByte(i)); @@ -716,8 +729,10 @@ public int indexOf(UTF8String v, int start) { */ private int find(UTF8String str, int start) { assert (str.numBytes > 0); + long offset = getBaseOffset(); + long strOffset = str.getBaseOffset(); while (start <= numBytes - str.numBytes) { - if (ByteArrayMethods.arrayEqualsBlock(base, offset + start, str.base, str.offset, str.numBytes)) { + if (ByteArrayMethods.arrayEqualsBlock(base, offset + start, str.base, strOffset, str.numBytes)) { return start; } start += 1; @@ -730,8 +745,10 @@ private int find(UTF8String str, int start) { */ private int rfind(UTF8String str, int start) { assert (str.numBytes > 0); + long offset = getBaseOffset(); + long strOffset = str.getBaseOffset(); while (start >= 0) { - if (ByteArrayMethods.arrayEqualsBlock(base, offset + start, str.base, str.offset, str.numBytes)) { + if (ByteArrayMethods.arrayEqualsBlock(base, offset + start, str.base, strOffset, str.numBytes)) { return start; } start -= 1; @@ -764,7 +781,7 @@ public UTF8String subStringIndex(UTF8String delim, int count) { return EMPTY_UTF8; } byte[] bytes = new byte[idx]; - MemoryBlock.copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, idx); + MemoryBlock.copyMemory(base, getBaseOffset(), bytes, BYTE_ARRAY_OFFSET, idx); return fromBytes(bytes); } else { @@ -784,7 +801,7 @@ public UTF8String subStringIndex(UTF8String delim, int count) { } int size = numBytes - delim.numBytes - idx; byte[] bytes = new byte[size]; - MemoryBlock.copyMemory(base, offset + idx + delim.numBytes, bytes, BYTE_ARRAY_OFFSET, size); + MemoryBlock.copyMemory(base, getBaseOffset() + idx + delim.numBytes, bytes, BYTE_ARRAY_OFFSET, size); return fromBytes(bytes); } } @@ -807,15 +824,16 @@ public UTF8String rpad(int len, UTF8String pad) { UTF8String remain = pad.substring(0, spaces - padChars * count); byte[] data = new byte[this.numBytes + pad.numBytes * count + remain.numBytes]; - MemoryBlock.copyMemory(this.base, this.offset, data, BYTE_ARRAY_OFFSET, this.numBytes); + MemoryBlock.copyMemory(this.base, this.getBaseOffset(), data, BYTE_ARRAY_OFFSET, this.numBytes); int offset = this.numBytes; int idx = 0; + long padOffset = pad.getBaseOffset(); while (idx < count) { - MemoryBlock.copyMemory(pad.base, pad.offset, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); + MemoryBlock.copyMemory(pad.base, padOffset, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); ++ idx; offset += pad.numBytes; } - MemoryBlock.copyMemory(remain.base, remain.offset, data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); + MemoryBlock.copyMemory(remain.base, remain.getBaseOffset(), data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); return UTF8String.fromBytes(data); } @@ -842,14 +860,15 @@ public UTF8String lpad(int len, UTF8String pad) { int offset = 0; int idx = 0; + long padOffset = pad.getBaseOffset(); while (idx < count) { - MemoryBlock.copyMemory(pad.base, pad.offset, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); + MemoryBlock.copyMemory(pad.base, padOffset, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); ++ idx; offset += pad.numBytes; } - MemoryBlock.copyMemory(remain.base, remain.offset, data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); + MemoryBlock.copyMemory(remain.base, remain.getBaseOffset(), data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); offset += remain.numBytes; - MemoryBlock.copyMemory(this.base, this.offset, data, BYTE_ARRAY_OFFSET + offset, numBytes()); + MemoryBlock.copyMemory(this.base, this.getBaseOffset(), data, BYTE_ARRAY_OFFSET + offset, numBytes()); return UTF8String.fromBytes(data); } @@ -875,7 +894,7 @@ public static UTF8String concat(UTF8String... inputs) { for (int i = 0; i < inputs.length; i++) { int len = inputs[i].numBytes; MemoryBlock.copyMemory( - inputs[i].base, inputs[i].offset, + inputs[i].base, inputs[i].getBaseOffset(), result, BYTE_ARRAY_OFFSET + offset, len); offset += len; @@ -915,7 +934,7 @@ public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) { if (inputs[i] != null) { int len = inputs[i].numBytes; MemoryBlock.copyMemory( - inputs[i].base, inputs[i].offset, + inputs[i].base, inputs[i].getBaseOffset(), result, BYTE_ARRAY_OFFSET + offset, len); offset += len; @@ -924,7 +943,7 @@ public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) { // Add separator if this is not the last input. if (j < numInputs) { MemoryBlock.copyMemory( - separator.base, separator.offset, + separator.base, separator.getBaseOffset(), result, BYTE_ARRAY_OFFSET + offset, separator.numBytes); offset += separator.numBytes; @@ -1198,7 +1217,7 @@ public UTF8String clone() { public UTF8String copy() { byte[] bytes = new byte[numBytes]; - MemoryBlock.copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, numBytes); + MemoryBlock.copyMemory(base, getBaseOffset(), bytes, BYTE_ARRAY_OFFSET, numBytes); return fromBytes(bytes); } @@ -1206,7 +1225,8 @@ public UTF8String copy() { public int compareTo(@Nonnull final UTF8String other) { int len = Math.min(numBytes, other.numBytes); int wordMax = (len / 8) * 8; - long roffset = other.offset; + long offset = getBaseOffset(); + long roffset = other.getBaseOffset(); MemoryBlock rbase = other.base; for (int i = 0; i < wordMax; i += 8) { long left = base.getLong(offset + i); @@ -1240,7 +1260,7 @@ public boolean equals(final Object other) { if (numBytes != o.numBytes) { return false; } - return ByteArrayMethods.arrayEqualsBlock(base, offset, o.base, o.offset, numBytes); + return ByteArrayMethods.arrayEqualsBlock(base, getBaseOffset(), o.base, o.getBaseOffset(), numBytes); } else { return false; } @@ -1287,6 +1307,8 @@ public int levenshteinDistance(UTF8String other) { p[i] = i; } + long sOffset = s.getBaseOffset(); + long tOffset = t.getBaseOffset(); for (j = 0, j_bytes = 0; j < m; j_bytes += num_bytes_j, j++) { num_bytes_j = numBytesForFirstByte(t.getByte(j_bytes)); d[0] = j + 1; @@ -1296,8 +1318,8 @@ public int levenshteinDistance(UTF8String other) { num_bytes_j != numBytesForFirstByte(s.getByte(i_bytes))) { cost = 1; } else { - cost = (ByteArrayMethods.arrayEqualsBlock(t.base, t.offset + j_bytes, s.base, - s.offset + i_bytes, num_bytes_j)) ? 0 : 1; + cost = (ByteArrayMethods.arrayEqualsBlock(t.base, tOffset + j_bytes, s.base, + sOffset + i_bytes, num_bytes_j)) ? 0 : 1; } d[i + 1] = Math.min(Math.min(d[i] + 1, p[i + 1] + 1), p[i] + cost); } @@ -1312,7 +1334,7 @@ public int levenshteinDistance(UTF8String other) { @Override public int hashCode() { - return Murmur3_x86_32.hashUnsafeBytesBlock(base, offset, numBytes, 42); + return Murmur3_x86_32.hashUnsafeBytesBlock(base, getBaseOffset(), numBytes, 42); } /** @@ -1375,7 +1397,7 @@ public void writeExternal(ObjectOutput out) throws IOException { } public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - offset = BYTE_ARRAY_OFFSET; + // offset = BYTE_ARRAY_OFFSET; numBytes = in.readInt(); base = ByteArrayMemoryBlock.fromArray(new byte[numBytes]); in.readFully(((ByteArrayMemoryBlock)base).getByteArray()); @@ -1390,7 +1412,7 @@ public void write(Kryo kryo, Output out) { @Override public void read(Kryo kryo, Input in) { - this.offset = BYTE_ARRAY_OFFSET; + // this.offset = BYTE_ARRAY_OFFSET; this.numBytes = in.readInt(); base = ByteArrayMemoryBlock.fromArray(new byte[numBytes]); in.read(((ByteArrayMemoryBlock)base).getByteArray()); diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java index 7094eecdd741b..a936e884b45c4 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java @@ -233,10 +233,8 @@ public UTF8String getUTF8String(int ordinal) { final long offsetAndSize = getLong(ordinal); final int offset = (int) (offsetAndSize >> 32); final int size = (int) offsetAndSize; - MemoryBlock mb = (baseObject instanceof byte[]) ? - new ByteArrayMemoryBlock((byte[]) baseObject, baseOffset + offset, size) : - new LongArrayMemoryBlock((long[]) baseObject, baseOffset + offset, size); - return UTF8String.fromAddress(mb, mb.getBaseOffset(), (int)mb.size()); + MemoryBlock mb = MemoryBlock.allocateFromObject(baseObject, baseOffset + offset, size); + return UTF8String.fromMemoryBlock(mb); } @Override diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index 68432996c8cf2..e612af931c351 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -415,10 +415,8 @@ public UTF8String getUTF8String(int ordinal) { final long offsetAndSize = getLong(ordinal); final int offset = (int) (offsetAndSize >> 32); final int size = (int) offsetAndSize; - MemoryBlock mb = (baseObject instanceof byte[]) ? - new ByteArrayMemoryBlock((byte[]) baseObject, baseOffset + offset, size) : - new LongArrayMemoryBlock((long[]) baseObject, baseOffset + offset, size); - return UTF8String.fromAddress(mb, mb.getBaseOffset(), (int)mb.size()); + MemoryBlock mb = MemoryBlock.allocateFromObject(baseObject, baseOffset + offset, size); + return UTF8String.fromMemoryBlock(mb); } @Override From df6dad3762f4e918d503df75ae8fce052af8bf43 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 28 Nov 2017 06:08:52 +0000 Subject: [PATCH 25/65] rebase with master --- .../spark/sql/execution/vectorized/OffHeapColumnVector.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java index 01a092f786a45..e738307063e0e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -209,7 +209,7 @@ public byte[] getBytes(int rowId, int count) { @Override protected UTF8String getBytesAsUTF8String(int rowId, int count) { - return UTF8String.fromAddress(null, data.getBaseOffset() + rowId, count); + return UTF8String.fromAddress(data, data.getBaseOffset() + rowId, count); } // From 1fa47a8c291ab54a8c1e386737769f467a76a672 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 20 Feb 2018 12:17:26 +0000 Subject: [PATCH 26/65] fix failures --- .../org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java | 2 +- .../java/org/apache/spark/memory/TaskMemoryManagerSuite.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java index 3e8626e33936b..1d9fe8a200059 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -47,7 +47,7 @@ public OffHeapMemoryBlock allocate(long size) throws OutOfMemoryError { public void free(MemoryBlock memory) { if (memory == OffHeapMemoryBlock.NULL) return; assert(memory instanceof OffHeapMemoryBlock); - assert (memory.getBaseObject() != null) : + assert (memory.getBaseObject() == null) : "baseObject not null; are you trying to use the off-heap allocator to free on-heap memory?"; assert (memory.getPageNumber() != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : "page has already been freed"; diff --git a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java index a0664b30d6cc2..d7d2d0b012bd3 100644 --- a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java +++ b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java @@ -76,7 +76,7 @@ public void freeingPageSetsPageNumberToSpecialConstant() { final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.ON_HEAP); final MemoryBlock dataPage = manager.allocatePage(256, c); c.freePage(dataPage); - Assert.assertEquals(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER, dataPage.pageNumber); + Assert.assertEquals(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER, dataPage.getPageNumber()); } @Test(expected = AssertionError.class) From 01f9c8e8146ff8d21d18feba59c0c2ad83299e2a Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 20 Feb 2018 17:30:11 +0000 Subject: [PATCH 27/65] fix failures in ArrowColumnVectorSuite and FeatureHasherSuite --- .../apache/spark/unsafe/hash/Murmur3_x86_32.java | 15 +++++++++++++++ .../apache/spark/unsafe/memory/MemoryBlock.java | 2 +- .../spark/unsafe/memory/OffHeapMemoryBlock.java | 5 +++++ .../apache/spark/ml/feature/FeatureHasher.scala | 4 ++-- .../spark/sql/vectorized/ArrowColumnVector.java | 5 ++++- 5 files changed, 27 insertions(+), 4 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index 601b992c72c25..26c7980a5bbea 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -104,6 +104,21 @@ public static int hashUnsafeBytes2(Object base, long offset, int lengthInBytes, return fmix(h1, lengthInBytes); } + public static int hashUnsafeBytes2Block( + MemoryBlock base, long offset, int lengthInBytes, int seed) { + // This is compatible with original and another implementations. + // Use this method for new components after Spark 2.3. + assert (lengthInBytes >= 0) : "lengthInBytes cannot be negative"; + int lengthAligned = lengthInBytes - lengthInBytes % 4; + int h1 = hashBytesByIntBlock(base, offset, lengthAligned, seed); + int k1 = 0; + for (int i = lengthAligned, shift = 0; i < lengthInBytes; i++, shift += 8) { + k1 ^= (base.getByte(offset + i) & 0xFF) << shift; + } + h1 ^= mixK1(k1); + return fmix(h1, lengthInBytes); + } + private static int hashBytesByIntBlock(MemoryBlock base, long offset, int lengthInBytes, int seed) { assert (lengthInBytes % 4 == 0); int h1 = seed; diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index e097618a1f832..0411f58c2d1ee 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -49,7 +49,7 @@ public abstract class MemoryBlock { protected long offset; - protected final long length; + protected long length; /** * Optional page number; used when this MemoryBlock represents a page allocated by a diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java index 4dcc40bc28f46..8e690cb2a7661 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java @@ -26,6 +26,11 @@ public OffHeapMemoryBlock(long address, long size) { super(null, address, size); } + public void setAddressAndSize(long address, long size) { + this.offset = address; + this.length = size; + } + @Override public MemoryBlock allocate(long offset, long size) { return new OffHeapMemoryBlock(offset, size); diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala index c78f61ac3ef71..d5694cd416fe0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala @@ -29,7 +29,7 @@ import org.apache.spark.mllib.feature.{HashingTF => OldHashingTF} import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.hash.Murmur3_x86_32.{hashInt, hashLong, hashUnsafeBytes2} +import org.apache.spark.unsafe.hash.Murmur3_x86_32.{hashInt, hashLong, hashUnsafeBytes2Block} import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils import org.apache.spark.util.collection.OpenHashMap @@ -244,7 +244,7 @@ object FeatureHasher extends DefaultParamsReadable[FeatureHasher] { case d: Double => hashLong(java.lang.Double.doubleToLongBits(d), seed) case s: String => val utf8 = UTF8String.fromString(s) - hashUnsafeBytes2(utf8.getBaseObject, utf8.getBaseOffset, utf8.numBytes(), seed) + hashUnsafeBytes2Block(utf8.getBaseObject, utf8.getBaseOffset, utf8.numBytes(), seed) case _ => throw new SparkException("FeatureHasher with murmur3 algorithm does not " + s"support type ${term.getClass.getCanonicalName} of input data.") } diff --git a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java index f8e37e995a17f..80b189d820166 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java @@ -25,6 +25,7 @@ import org.apache.spark.annotation.InterfaceStability; import org.apache.spark.sql.execution.arrow.ArrowUtils; import org.apache.spark.sql.types.*; +import org.apache.spark.unsafe.memory.OffHeapMemoryBlock; import org.apache.spark.unsafe.types.UTF8String; /** @@ -365,6 +366,7 @@ private static class StringAccessor extends ArrowVectorAccessor { private final VarCharVector accessor; private final NullableVarCharHolder stringResult = new NullableVarCharHolder(); + private final OffHeapMemoryBlock mb = new OffHeapMemoryBlock(0L, 0L); StringAccessor(VarCharVector vector) { super(vector); @@ -377,7 +379,8 @@ final UTF8String getUTF8String(int rowId) { if (stringResult.isSet == 0) { return null; } else { - return UTF8String.fromAddress(null, + mb.setAddressAndSize(stringResult.buffer.memoryAddress(), stringResult.buffer.capacity()); + return UTF8String.fromAddress(mb, stringResult.buffer.memoryAddress() + stringResult.start, stringResult.end - stringResult.start); } From 2ed8f82f13288e00e6352bdccbac0dd890c91715 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 21 Feb 2018 17:48:41 +0000 Subject: [PATCH 28/65] address review comments rename LongArrayMemoryBlock to OnHeapMemoryBlock remove intArrayMemoryBlock separate copyMemory to copyFrom and writeTo and make them non-static add assertion for range check --- .../unsafe/memory/ByteArrayMemoryBlock.java | 92 +++++++- .../unsafe/memory/HeapMemoryAllocator.java | 8 +- .../unsafe/memory/IntArrayMemoryBlock.java | 112 ---------- .../unsafe/memory/LongArrayMemoryBlock.java | 112 ---------- .../spark/unsafe/memory/MemoryBlock.java | 67 ++---- .../spark/unsafe/memory/MemoryLocation.java | 54 ----- .../unsafe/memory/OffHeapMemoryBlock.java | 86 ++++++++ .../unsafe/memory/OnHeapMemoryBlock.java | 199 ++++++++++++++++++ .../apache/spark/unsafe/types/UTF8String.java | 43 ++-- .../spark/unsafe/array/LongArraySuite.java | 4 +- .../spark/unsafe/memory/MemoryBlockSuite.java | 51 +++-- .../spark/unsafe/types/UTF8StringSuite.java | 20 +- .../util/collection/ExternalSorterSuite.scala | 6 +- .../unsafe/sort/RadixSortSuite.scala | 10 +- .../catalyst/expressions/UnsafeArrayData.java | 2 +- .../vectorized/OffHeapColumnVector.java | 58 ++--- .../execution/benchmark/SortBenchmark.scala | 16 +- .../sql/execution/python/RowQueueSuite.scala | 4 +- 18 files changed, 506 insertions(+), 438 deletions(-) delete mode 100644 common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java delete mode 100644 common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java delete mode 100644 common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java create mode 100644 common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java index f470c8fd7c42e..3d10718d002a5 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -20,7 +20,7 @@ import org.apache.spark.unsafe.Platform; /** - * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. + * A consecutive block of memory with a byte array on Java heap. */ public final class ByteArrayMemoryBlock extends MemoryBlock { @@ -47,58 +47,72 @@ public static ByteArrayMemoryBlock fromArray(final byte[] array) { public final int getInt(long offset) { + // UTF8String.getPrefix() assumes data is 4-byte aligned + assert(offset + 4 - Platform.BYTE_ARRAY_OFFSET <= ((array.length + 3) / 4) * 4); return Platform.getInt(array, offset); } public final void putInt(long offset, int value) { + assert(offset + 4 - Platform.BYTE_ARRAY_OFFSET <= array.length); Platform.putInt(array, offset, value); } public final boolean getBoolean(long offset) { + assert(offset + 1 - Platform.BYTE_ARRAY_OFFSET <= array.length); return Platform.getBoolean(array, offset); } public final void putBoolean(long offset, boolean value) { + assert(offset + 1 - Platform.BYTE_ARRAY_OFFSET <= array.length); Platform.putBoolean(array, offset, value); } public final byte getByte(long offset) { - return Platform.getByte(array, offset); + return array[(int)(offset - Platform.BYTE_ARRAY_OFFSET)]; } public final void putByte(long offset, byte value) { - Platform.putByte(array, offset, value); + array[(int)(offset - Platform.BYTE_ARRAY_OFFSET)] = value; } public final short getShort(long offset) { + assert(offset + 2 - Platform.BYTE_ARRAY_OFFSET <= array.length); return Platform.getShort(array, offset); } public final void putShort(long offset, short value) { + assert(offset + 2 - Platform.BYTE_ARRAY_OFFSET <= array.length); Platform.putShort(array, offset, value); } public final long getLong(long offset) { + // UTF8String.getPrefix() assumes data is 8-byte aligned + assert(offset + 8 - Platform.BYTE_ARRAY_OFFSET <= ((array.length + 7) / 8) * 8); return Platform.getLong(array, offset); } public final void putLong(long offset, long value) { + assert(offset + 8 - Platform.BYTE_ARRAY_OFFSET <= array.length); Platform.putLong(array, offset, value); } public final float getFloat(long offset) { + assert(offset + 4 - Platform.BYTE_ARRAY_OFFSET <= array.length); return Platform.getFloat(array, offset); } public final void putFloat(long offset, float value) { + assert(offset + 4 - Platform.BYTE_ARRAY_OFFSET <= array.length); Platform.putFloat(array, offset, value); } public final double getDouble(long offset) { + assert(offset + 8 - Platform.BYTE_ARRAY_OFFSET <= array.length); return Platform.getDouble(array, offset); } public final void putDouble(long offset, double value) { + assert(offset + 8 - Platform.BYTE_ARRAY_OFFSET <= array.length); Platform.putDouble(array, offset, value); } @@ -109,4 +123,76 @@ public final Object getObjectVolatile(long offset) { public final void putObjectVolatile(long offset, Object value) { Platform.putObjectVolatile(array, offset, value); } + + public final void copyFrom(byte[] src, long srcOffset, long dstOffset, long length) { + assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= src.length); + assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); + Platform.copyMemory(src, srcOffset, array, dstOffset, length); + } + + public final void copyFrom(short[] src, long srcOffset, long dstOffset, long length) { + assert(srcOffset - Platform.SHORT_ARRAY_OFFSET + length <= src.length * 2); + assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); + Platform.copyMemory(src, srcOffset, array, dstOffset, length); + } + + public final void copyFrom(int[] src, long srcOffset, long dstOffset, long length) { + assert(srcOffset - Platform.INT_ARRAY_OFFSET + length <= src.length * 4); + assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); + Platform.copyMemory(src, srcOffset, array, dstOffset, length); + } + + public final void copyFrom(long[] src, long srcOffset, long dstOffset, long length) { + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= src.length * 8); + assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); + Platform.copyMemory(src, srcOffset, array, dstOffset, length); + } + + public final void copyFrom(float[] src, long srcOffset, long dstOffset, long length) { + assert(srcOffset - Platform.FLOAT_ARRAY_OFFSET + length <= src.length * 4); + assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); + Platform.copyMemory(src, srcOffset, array, dstOffset, length); + } + + public final void copyFrom(double[] src, long srcOffset, long dstOffset, long length) { + assert(srcOffset - Platform.DOUBLE_ARRAY_OFFSET + length <= src.length * 8); + assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + Platform.copyMemory(src, srcOffset, array, dstOffset, length); + } + + public final void writeTo(long srcOffset, byte[] dst, long dstOffset, long length) { + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= dst.length); + Platform.copyMemory(array, srcOffset, dst, dstOffset, length); + } + + public final void writeTo(long srcOffset, short[] dst, long dstOffset, long length) { + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(dstOffset - Platform.SHORT_ARRAY_OFFSET + length <= dst.length * 2); + Platform.copyMemory(array, srcOffset, dst, dstOffset, length); + } + + public final void writeTo(long srcOffset, int[] dst, long dstOffset, long length) { + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(dstOffset - Platform.INT_ARRAY_OFFSET + length <= dst.length * 4); + Platform.copyMemory(array, srcOffset, dst, dstOffset, length); + } + + public final void writeTo(long srcOffset, long[] dst, long dstOffset, long length) { + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= dst.length * 8); + Platform.copyMemory(array, srcOffset, dst, dstOffset, length); + } + + public final void writeTo(long srcOffset, float[] dst, long dstOffset, long length) { + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(dstOffset - Platform.FLOAT_ARRAY_OFFSET + length <= dst.length * 4); + Platform.copyMemory(array, srcOffset, dst, dstOffset, length); + } + + public final void writeTo(long srcOffset, double[] dst, long dstOffset, long length) { + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(dstOffset - Platform.DOUBLE_ARRAY_OFFSET + length <= dst.length * 8); + Platform.copyMemory(array, srcOffset, dst, dstOffset, length); + } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java index aaa251f7d285e..727e06ee4ccb9 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java @@ -59,7 +59,7 @@ public MemoryBlock allocate(long size) throws OutOfMemoryError { if (array != null) { assert (array.length * 8L >= size); MemoryBlock memory = - new LongArrayMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, size); + new OnHeapMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, size); if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE); } @@ -71,7 +71,7 @@ public MemoryBlock allocate(long size) throws OutOfMemoryError { } } long[] array = new long[numWords]; - MemoryBlock memory = new LongArrayMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, size); + MemoryBlock memory = new OnHeapMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, size); if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE); } @@ -80,7 +80,7 @@ public MemoryBlock allocate(long size) throws OutOfMemoryError { @Override public void free(MemoryBlock memory) { - assert(memory instanceof LongArrayMemoryBlock); + assert(memory instanceof OnHeapMemoryBlock); assert (memory.getBaseObject() != null) : "baseObject was null; are you trying to use the on-heap allocator to free off-heap memory?"; assert (memory.getPageNumber() != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : @@ -100,7 +100,7 @@ public void free(MemoryBlock memory) { // As an additional layer of defense against use-after-free bugs, we mutate the // MemoryBlock to null out its reference to the long[] array. - long[] array = ((LongArrayMemoryBlock)memory).getLongArray(); + long[] array = ((OnHeapMemoryBlock)memory).getLongArray(); memory.resetObjAndOffset(); long alignedSize = ((size + 7) / 8) * 8; diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java deleted file mode 100644 index 734b2083e8cde..0000000000000 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/IntArrayMemoryBlock.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * 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.spark.unsafe.memory; - -import org.apache.spark.unsafe.Platform; - -/** - * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. - */ -public final class IntArrayMemoryBlock extends MemoryBlock { - - private final int[] array; - - public IntArrayMemoryBlock(int[] obj, long offset, long size) { - super(obj, offset, size); - this.array = obj; - } - - @Override - public MemoryBlock allocate(long offset, long size) { - return new IntArrayMemoryBlock(array, offset, size); - } - - public int[] getIntArray() { return array; } - - /** - * Creates a memory block pointing to the memory used by the int array. - */ - public static IntArrayMemoryBlock fromArray(final int[] array) { - return new IntArrayMemoryBlock(array, Platform.INT_ARRAY_OFFSET, array.length * 4); - } - - - public final int getInt(long offset) { - return Platform.getInt(array, offset); - } - - public final void putInt(long offset, int value) { - Platform.putInt(array, offset, value); - } - - public final boolean getBoolean(long offset) { - return Platform.getBoolean(array, offset); - } - - public final void putBoolean(long offset, boolean value) { - Platform.putBoolean(array, offset, value); - } - - public final byte getByte(long offset) { - return Platform.getByte(array, offset); - } - - public final void putByte(long offset, byte value) { - Platform.putByte(array, offset, value); - } - - public final short getShort(long offset) { - return Platform.getShort(array, offset); - } - - public final void putShort(long offset, short value) { - Platform.putShort(array, offset, value); - } - - public final long getLong(long offset) { - return Platform.getLong(array, offset); - } - - public final void putLong(long offset, long value) { - Platform.putLong(array, offset, value); - } - - public final float getFloat(long offset) { - return Platform.getFloat(array, offset); - } - - public final void putFloat(long offset, float value) { - Platform.putFloat(array, offset, value); - } - - public final double getDouble(long offset) { - return Platform.getDouble(array, offset); - } - - public final void putDouble(long offset, double value) { - Platform.putDouble(array, offset, value); - } - - public final Object getObjectVolatile(long offset) { - return Platform.getObjectVolatile(array, offset); - } - - public final void putObjectVolatile(long offset, Object value) { - Platform.putObjectVolatile(array, offset, value); - } -} diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java deleted file mode 100644 index 2b1c6a33825aa..0000000000000 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/LongArrayMemoryBlock.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * 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.spark.unsafe.memory; - -import org.apache.spark.unsafe.Platform; - -/** - * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. - */ -public final class LongArrayMemoryBlock extends MemoryBlock { - - private final long[] array; - - public LongArrayMemoryBlock(long[] obj, long offset, long size) { - super(obj, offset, size); - this.array = obj; - } - - @Override - public MemoryBlock allocate(long offset, long size) { - return new LongArrayMemoryBlock(array, offset, size); - } - - public long[] getLongArray() { return array; } - - /** - * Creates a memory block pointing to the memory used by the long array. - */ - public static LongArrayMemoryBlock fromArray(final long[] array) { - return new LongArrayMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, array.length * 8); - } - - - public final int getInt(long offset) { - return Platform.getInt(array, offset); - } - - public final void putInt(long offset, int value) { - Platform.putInt(array, offset, value); - } - - public final boolean getBoolean(long offset) { - return Platform.getBoolean(array, offset); - } - - public final void putBoolean(long offset, boolean value) { - Platform.putBoolean(array, offset, value); - } - - public final byte getByte(long offset) { - return Platform.getByte(array, offset); - } - - public final void putByte(long offset, byte value) { - Platform.putByte(array, offset, value); - } - - public final short getShort(long offset) { - return Platform.getShort(array, offset); - } - - public final void putShort(long offset, short value) { - Platform.putShort(array, offset, value); - } - - public final long getLong(long offset) { - return Platform.getLong(array, offset); - } - - public final void putLong(long offset, long value) { - Platform.putLong(array, offset, value); - } - - public final float getFloat(long offset) { - return Platform.getFloat(array, offset); - } - - public final void putFloat(long offset, float value) { - Platform.putFloat(array, offset, value); - } - - public final double getDouble(long offset) { - return Platform.getDouble(array, offset); - } - - public final void putDouble(long offset, double value) { - Platform.putDouble(array, offset, value); - } - - public final Object getObjectVolatile(long offset) { - return Platform.getObjectVolatile(array, offset); - } - - public final void putObjectVolatile(long offset, Object value) { - Platform.putObjectVolatile(array, offset, value); - } -} diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index 0411f58c2d1ee..bdbefb8e7eea0 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -111,12 +111,9 @@ public final static MemoryBlock allocateFromObject(Object obj, long offset, long if (obj instanceof byte[]) { byte[] array = (byte[])obj; mb = new ByteArrayMemoryBlock(array, offset, length); - } else if (obj instanceof int[]) { - int[] array = (int[])obj; - mb = new IntArrayMemoryBlock(array, offset, length); } else if (obj instanceof long[]) { long[] array = (long[])obj; - mb = new LongArrayMemoryBlock(array, offset, length); + mb = new OnHeapMemoryBlock(array, offset, length); } else if (obj == null) { // we assume that to pass null pointer means off-heap mb = new OffHeapMemoryBlock(offset, length); @@ -164,68 +161,32 @@ public final static MemoryBlock allocateFromObject(Object obj, long offset, long public abstract void putObjectVolatile(long offset, Object value); - public static final void copyMemory( + public static void copyMemory( MemoryBlock src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { Platform.copyMemory(src.getBaseObject(), srcOffset, dst.getBaseObject(), dstOffset, length); } - public static final void copyMemory( - byte[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { - Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); - } + public abstract void copyFrom(byte[] src, long srcOffset, long dstOffset, long length); - public static final void copyMemory( - short[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { - Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); - } + public abstract void copyFrom(short[] src, long srcOffset, long dstOffset, long length); - public static final void copyMemory( - int[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { - Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); - } + public abstract void copyFrom(int[] src, long srcOffset, long dstOffset, long length); - public static final void copyMemory( - long[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { - Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); - } + public abstract void copyFrom(long[] src, long srcOffset, long dstOffset, long length); - public static final void copyMemory( - float[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { - Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); - } + public abstract void copyFrom(float[] src, long srcOffset, long dstOffset, long length); - public static final void copyMemory( - double[] src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { - Platform.copyMemory(src, srcOffset, dst.getBaseObject(), dstOffset, length); - } + public abstract void copyFrom(double[] src, long srcOffset, long dstOffset, long length); - public static final void copyMemory( - MemoryBlock src, long srcOffset, byte[] dst, long dstOffset, long length) { - Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); - } + public abstract void writeTo(long srcOffset, byte[] dst, long dstOffset, long length); - public static final void copyMemory( - MemoryBlock src, long srcOffset, short[] dst, long dstOffset, long length) { - Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); - } + public abstract void writeTo(long srcOffset, short[] dst, long dstOffset, long length); - public static final void copyMemory( - MemoryBlock src, long srcOffset, int[] dst, long dstOffset, long length) { - Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); - } + public abstract void writeTo(long srcOffset, int[] dst, long dstOffset, long length); - public static final void copyMemory( - MemoryBlock src, long srcOffset, long[] dst, long dstOffset, long length) { - Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); - } + public abstract void writeTo(long srcOffset, long[] dst, long dstOffset, long length); - public static final void copyMemory( - MemoryBlock src, long srcOffset, float[] dst, long dstOffset, long length) { - Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); - } + public abstract void writeTo(long srcOffset, float[] dst, long dstOffset, long length); - public static final void copyMemory( - MemoryBlock src, long srcOffset, double[] dst, long dstOffset, long length) { - Platform.copyMemory(src.getBaseObject(), srcOffset, dst, dstOffset, length); - } + public abstract void writeTo(long srcOffset, double[] dst, long dstOffset, long length); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java deleted file mode 100644 index 74ebc87dc978c..0000000000000 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.spark.unsafe.memory; - -import javax.annotation.Nullable; - -/** - * A memory location. Tracked either by a memory address (with off-heap allocation), - * or by an offset from a JVM object (in-heap allocation). - */ -public class MemoryLocation { - - @Nullable - Object obj; - - long offset; - - public MemoryLocation(@Nullable Object obj, long offset) { - this.obj = obj; - this.offset = offset; - } - - public MemoryLocation() { - this(null, 0); - } - - public void setObjAndOffset(Object newObj, long newOffset) { - this.obj = newObj; - this.offset = newOffset; - } - - public final Object getBaseObject() { - return obj; - } - - public final long getBaseOffset() { - return offset; - } -} diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java index 8e690cb2a7661..6996c104986ad 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java @@ -37,58 +37,72 @@ public MemoryBlock allocate(long offset, long size) { } public final int getInt(long offset) { + assert(offset + 4 <= this.offset + this.length); return Platform.getInt(null, offset); } public final void putInt(long offset, int value) { + assert(offset + 4 <= this.offset + this.length); Platform.putInt(null, offset, value); } public final boolean getBoolean(long offset) { + assert(offset + 1 <= this.offset + this.length); return Platform.getBoolean(null, offset); } public final void putBoolean(long offset, boolean value) { + assert(offset + 1 <= this.offset + this.length); Platform.putBoolean(null, offset, value); } public final byte getByte(long offset) { + assert(offset + 1 <= this.offset + this.length); return Platform.getByte(null, offset); } public final void putByte(long offset, byte value) { + assert(offset + 1 <= this.offset + this.length); Platform.putByte(null, offset, value); } public final short getShort(long offset) { + assert(offset + 2 <= this.offset + this.length); return Platform.getShort(null, offset); } public final void putShort(long offset, short value) { + assert(offset + 2 <= this.offset + this.length); Platform.putShort(null, offset, value); } public final long getLong(long offset) { + assert(offset + 8 <= this.offset + this.length); return Platform.getLong(null, offset); } public final void putLong(long offset, long value) { + assert(offset + 8 <= this.offset + this.length); Platform.putLong(null, offset, value); } public final float getFloat(long offset) { + assert(offset + 4 <= this.offset + this.length); return Platform.getFloat(null, offset); } public final void putFloat(long offset, float value) { + assert(offset + 4 <= this.offset + this.length); Platform.putFloat(null, offset, value); } public final double getDouble(long offset) { + assert(offset + 8 <= this.offset + this.length); return Platform.getDouble(null, offset); } public final void putDouble(long offset, double value) { + assert(offset + 8 <= this.offset + this.length); Platform.putDouble(null, offset, value); } @@ -99,4 +113,76 @@ public final Object getObjectVolatile(long offset) { public final void putObjectVolatile(long offset, Object value) { Platform.putObjectVolatile(null, offset, value); } + + public final void copyFrom(byte[] src, long srcOffset, long dstOffset, long length) { + assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= src.length); + assert(dstOffset + length <= this.offset + this.length); + Platform.copyMemory(src, srcOffset, null, dstOffset, length); + } + + public final void copyFrom(short[] src, long srcOffset, long dstOffset, long length) { + assert(srcOffset - Platform.SHORT_ARRAY_OFFSET + length <= src.length * 2); + assert(dstOffset + length <= this.offset + this.length); + Platform.copyMemory(src, srcOffset, null, dstOffset, length); + } + + public final void copyFrom(int[] src, long srcOffset, long dstOffset, long length) { + assert(srcOffset - Platform.INT_ARRAY_OFFSET + length <= src.length * 4); + assert(dstOffset + length <= this.offset + this.length); + Platform.copyMemory(src, srcOffset, null, dstOffset, length); + } + + public final void copyFrom(long[] src, long srcOffset, long dstOffset, long length) { + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= src.length * 8); + assert(dstOffset + length <= this.offset + this.length); + Platform.copyMemory(src, srcOffset, null, dstOffset, length); + } + + public final void copyFrom(float[] src, long srcOffset, long dstOffset, long length) { + assert(srcOffset - Platform.FLOAT_ARRAY_OFFSET + length <= src.length * 4); + assert(dstOffset + length <= this.offset + this.length); + Platform.copyMemory(src, srcOffset, null, dstOffset, length); + } + + public final void copyFrom(double[] src, long srcOffset, long dstOffset, long length) { + assert(srcOffset - Platform.DOUBLE_ARRAY_OFFSET + length <= src.length * 8); + assert(dstOffset + length <= this.offset + this.length); + Platform.copyMemory(src, srcOffset, null, dstOffset, length); + } + + public final void writeTo(long srcOffset, byte[] dst, long dstOffset, long length) { + assert(srcOffset + length <= this.offset + this.length); + assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= dst.length); + Platform.copyMemory(null, srcOffset, dst, dstOffset, length); + } + + public final void writeTo(long srcOffset, short[] dst, long dstOffset, long length) { + assert(srcOffset + length <= this.offset + this.length); + assert(dstOffset - Platform.SHORT_ARRAY_OFFSET + length <= dst.length * 2); + Platform.copyMemory(null, srcOffset, dst, dstOffset, length); + } + + public final void writeTo(long srcOffset, int[] dst, long dstOffset, long length) { + assert(srcOffset + length <= this.offset + this.length); + assert(dstOffset - Platform.INT_ARRAY_OFFSET + length <= dst.length * 4); + Platform.copyMemory(null, srcOffset, dst, dstOffset, length); + } + + public final void writeTo(long srcOffset, long[] dst, long dstOffset, long length) { + assert(srcOffset + length <= this.offset + this.length); + assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= dst.length * 8); + Platform.copyMemory(null, srcOffset, dst, dstOffset, length); + } + + public final void writeTo(long srcOffset, float[] dst, long dstOffset, long length) { + assert(srcOffset + length <= this.offset + this.length); + assert(dstOffset - Platform.FLOAT_ARRAY_OFFSET + length <= dst.length * 4); + Platform.copyMemory(null, srcOffset, dst, dstOffset, length); + } + + public final void writeTo(long srcOffset, double[] dst, long dstOffset, long length) { + assert(srcOffset + length <= this.offset + this.length); + assert(dstOffset - Platform.DOUBLE_ARRAY_OFFSET + length <= dst.length * 8); + Platform.copyMemory(null, srcOffset, dst, dstOffset, length); + } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java new file mode 100644 index 0000000000000..f1e12338bb7ab --- /dev/null +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java @@ -0,0 +1,199 @@ +/* + * 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.spark.unsafe.memory; + +import org.apache.spark.unsafe.Platform; + +/** + * A consecutive block of memory with a long array on Java heap. + */ +public final class OnHeapMemoryBlock extends MemoryBlock { + + private final long[] array; + + public OnHeapMemoryBlock(long[] obj, long offset, long size) { + super(obj, offset, size); + this.array = obj; + assert(offset - Platform.LONG_ARRAY_OFFSET + size <= obj.length * 8L); + } + + @Override + public MemoryBlock allocate(long offset, long size) { + return new OnHeapMemoryBlock(array, offset, size); + } + + public long[] getLongArray() { return array; } + + /** + * Creates a memory block pointing to the memory used by the long array. + */ + public static OnHeapMemoryBlock fromArray(final long[] array) { + return new OnHeapMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, array.length * 8); + } + + + public final int getInt(long offset) { + assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + return Platform.getInt(array, offset); + } + + public final void putInt(long offset, int value) { + assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + Platform.putInt(array, offset, value); + } + + public final boolean getBoolean(long offset) { + assert(offset + 1 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + return Platform.getBoolean(array, offset); + } + + public final void putBoolean(long offset, boolean value) { + assert(offset + 1 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + Platform.putBoolean(array, offset, value); + } + + public final byte getByte(long offset) { + assert(offset + 2 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + return Platform.getByte(array, offset); + } + + public final void putByte(long offset, byte value) { + assert(offset + 2 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + Platform.putByte(array, offset, value); + } + + public final short getShort(long offset) { + assert(offset + 2 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + return Platform.getShort(array, offset); + } + + public final void putShort(long offset, short value) { + assert(offset + 2 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + Platform.putShort(array, offset, value); + } + + public final long getLong(long offset) { + assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + return Platform.getLong(array, offset); + } + + public final void putLong(long offset, long value) { + assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + Platform.putLong(array, offset, value); + } + + public final float getFloat(long offset) { + assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + return Platform.getFloat(array, offset); + } + + public final void putFloat(long offset, float value) { + assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + Platform.putFloat(array, offset, value); + } + + public final double getDouble(long offset) { + assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + return Platform.getDouble(array, offset); + } + + public final void putDouble(long offset, double value) { + assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + Platform.putDouble(array, offset, value); + } + + public final Object getObjectVolatile(long offset) { + return Platform.getObjectVolatile(array, offset); + } + + public final void putObjectVolatile(long offset, Object value) { + Platform.putObjectVolatile(array, offset, value); + } + + public final void copyFrom(byte[] src, long srcOffset, long dstOffset, long length) { + assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= src.length); + assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); + Platform.copyMemory(src, srcOffset, array, dstOffset, length); + } + + public final void copyFrom(short[] src, long srcOffset, long dstOffset, long length) { + assert(srcOffset - Platform.SHORT_ARRAY_OFFSET + length <= src.length * 2); + assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); + Platform.copyMemory(src, srcOffset, array, dstOffset, length); + } + + public final void copyFrom(int[] src, long srcOffset, long dstOffset, long length) { + assert(srcOffset - Platform.INT_ARRAY_OFFSET + length <= src.length * 4); + assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); + Platform.copyMemory(src, srcOffset, array, dstOffset, length); + } + + public final void copyFrom(long[] src, long srcOffset, long dstOffset, long length) { + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= src.length * 8); + assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); + Platform.copyMemory(src, srcOffset, array, dstOffset, length); + } + + public final void copyFrom(float[] src, long srcOffset, long dstOffset, long length) { + assert(srcOffset - Platform.FLOAT_ARRAY_OFFSET + length <= src.length * 4); + assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); + Platform.copyMemory(src, srcOffset, array, dstOffset, length); + } + + public final void copyFrom(double[] src, long srcOffset, long dstOffset, long length) { + assert(srcOffset - Platform.DOUBLE_ARRAY_OFFSET + length <= src.length * 8); + assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + Platform.copyMemory(src, srcOffset, array, dstOffset, length); + } + + public final void writeTo(long srcOffset, byte[] dst, long dstOffset, long length) { + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= dst.length); + Platform.copyMemory(array, srcOffset, dst, dstOffset, length); + } + + public final void writeTo(long srcOffset, short[] dst, long dstOffset, long length) { + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(dstOffset - Platform.SHORT_ARRAY_OFFSET + length <= dst.length * 2); + Platform.copyMemory(array, srcOffset, dst, dstOffset, length); + } + + public final void writeTo(long srcOffset, int[] dst, long dstOffset, long length) { + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(dstOffset - Platform.INT_ARRAY_OFFSET + length <= dst.length * 4); + Platform.copyMemory(array, srcOffset, dst, dstOffset, length); + } + + public final void writeTo(long srcOffset, long[] dst, long dstOffset, long length) { + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= dst.length * 8); + Platform.copyMemory(array, srcOffset, dst, dstOffset, length); + } + + public final void writeTo(long srcOffset, float[] dst, long dstOffset, long length) { + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(dstOffset - Platform.FLOAT_ARRAY_OFFSET + length <= dst.length * 4); + Platform.copyMemory(array, srcOffset, dst, dstOffset, length); + } + + public final void writeTo(long srcOffset, double[] dst, long dstOffset, long length) { + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(dstOffset - Platform.DOUBLE_ARRAY_OFFSET + length <= dst.length * 8); + Platform.copyMemory(array, srcOffset, dst, dstOffset, length); + } +} diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index bd2bede2cadd3..14098a6a72772 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -147,7 +147,7 @@ public UTF8String() { * bytes in this string. */ public void writeToMemory(byte[] target, long targetOffset) { - MemoryBlock.copyMemory(base, base.getBaseOffset(), target, targetOffset, numBytes); + base.writeTo(base.getBaseOffset(), target, targetOffset, numBytes); } public void writeToMemory(MemoryBlock target, long targetOffset) { @@ -280,7 +280,7 @@ public byte[] getBytes() { return ((ByteArrayMemoryBlock) base).getByteArray(); } else { byte[] bytes = new byte[numBytes]; - MemoryBlock.copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, numBytes); + base.writeTo(offset, bytes, BYTE_ARRAY_OFFSET, numBytes); return bytes; } } @@ -310,7 +310,7 @@ public UTF8String substring(final int start, final int until) { if (i > j) { byte[] bytes = new byte[i - j]; - MemoryBlock.copyMemory(base, base.getBaseOffset() + j, bytes, BYTE_ARRAY_OFFSET, i - j); + base.writeTo(base.getBaseOffset() + j, bytes, BYTE_ARRAY_OFFSET, i - j); return fromBytes(bytes); } else { return EMPTY_UTF8; @@ -512,7 +512,7 @@ public int findInSet(UTF8String match) { private UTF8String copyUTF8String(int start, int end) { int len = end - start + 1; byte[] newBytes = new byte[len]; - MemoryBlock.copyMemory(base, getBaseOffset() + start, newBytes, BYTE_ARRAY_OFFSET, len); + base.writeTo(getBaseOffset() + start, newBytes, BYTE_ARRAY_OFFSET, len); return UTF8String.fromBytes(newBytes); } @@ -660,8 +660,7 @@ public UTF8String reverse() { int i = 0; // position in byte while (i < numBytes) { int len = numBytesForFirstByte(getByte(i)); - MemoryBlock.copyMemory(this.base, offset + i, result, - BYTE_ARRAY_OFFSET + result.length - i - len, len); + base.writeTo(offset + i, result, BYTE_ARRAY_OFFSET + result.length - i - len, len); i += len; } @@ -675,7 +674,7 @@ public UTF8String repeat(int times) { } byte[] newBytes = new byte[numBytes * times]; - MemoryBlock.copyMemory(this.base, this.getBaseOffset(), newBytes, BYTE_ARRAY_OFFSET, numBytes); + base.writeTo(this.getBaseOffset(), newBytes, BYTE_ARRAY_OFFSET, numBytes); int copied = 1; while (copied < times) { @@ -781,7 +780,7 @@ public UTF8String subStringIndex(UTF8String delim, int count) { return EMPTY_UTF8; } byte[] bytes = new byte[idx]; - MemoryBlock.copyMemory(base, getBaseOffset(), bytes, BYTE_ARRAY_OFFSET, idx); + base.writeTo(getBaseOffset(), bytes, BYTE_ARRAY_OFFSET, idx); return fromBytes(bytes); } else { @@ -801,7 +800,7 @@ public UTF8String subStringIndex(UTF8String delim, int count) { } int size = numBytes - delim.numBytes - idx; byte[] bytes = new byte[size]; - MemoryBlock.copyMemory(base, getBaseOffset() + idx + delim.numBytes, bytes, BYTE_ARRAY_OFFSET, size); + base.writeTo(getBaseOffset() + idx + delim.numBytes, bytes, BYTE_ARRAY_OFFSET, size); return fromBytes(bytes); } } @@ -824,16 +823,16 @@ public UTF8String rpad(int len, UTF8String pad) { UTF8String remain = pad.substring(0, spaces - padChars * count); byte[] data = new byte[this.numBytes + pad.numBytes * count + remain.numBytes]; - MemoryBlock.copyMemory(this.base, this.getBaseOffset(), data, BYTE_ARRAY_OFFSET, this.numBytes); + base.writeTo(this.getBaseOffset(), data, BYTE_ARRAY_OFFSET, this.numBytes); int offset = this.numBytes; int idx = 0; long padOffset = pad.getBaseOffset(); while (idx < count) { - MemoryBlock.copyMemory(pad.base, padOffset, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); + pad.base.writeTo(padOffset, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); ++ idx; offset += pad.numBytes; } - MemoryBlock.copyMemory(remain.base, remain.getBaseOffset(), data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); + remain.base.writeTo(remain.getBaseOffset(), data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); return UTF8String.fromBytes(data); } @@ -862,13 +861,13 @@ public UTF8String lpad(int len, UTF8String pad) { int idx = 0; long padOffset = pad.getBaseOffset(); while (idx < count) { - MemoryBlock.copyMemory(pad.base, padOffset, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); + pad.base.writeTo(padOffset, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); ++ idx; offset += pad.numBytes; } - MemoryBlock.copyMemory(remain.base, remain.getBaseOffset(), data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); + remain.base.writeTo(remain.getBaseOffset(), data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); offset += remain.numBytes; - MemoryBlock.copyMemory(this.base, this.getBaseOffset(), data, BYTE_ARRAY_OFFSET + offset, numBytes()); + base.writeTo(this.getBaseOffset(), data, BYTE_ARRAY_OFFSET + offset, numBytes()); return UTF8String.fromBytes(data); } @@ -893,8 +892,8 @@ public static UTF8String concat(UTF8String... inputs) { int offset = 0; for (int i = 0; i < inputs.length; i++) { int len = inputs[i].numBytes; - MemoryBlock.copyMemory( - inputs[i].base, inputs[i].getBaseOffset(), + inputs[i].base.writeTo( + inputs[i].getBaseOffset(), result, BYTE_ARRAY_OFFSET + offset, len); offset += len; @@ -933,8 +932,8 @@ public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) { for (int i = 0, j = 0; i < inputs.length; i++) { if (inputs[i] != null) { int len = inputs[i].numBytes; - MemoryBlock.copyMemory( - inputs[i].base, inputs[i].getBaseOffset(), + inputs[i].base.writeTo( + inputs[i].getBaseOffset(), result, BYTE_ARRAY_OFFSET + offset, len); offset += len; @@ -942,8 +941,8 @@ public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) { j++; // Add separator if this is not the last input. if (j < numInputs) { - MemoryBlock.copyMemory( - separator.base, separator.getBaseOffset(), + separator.base.writeTo( + separator.getBaseOffset(), result, BYTE_ARRAY_OFFSET + offset, separator.numBytes); offset += separator.numBytes; @@ -1217,7 +1216,7 @@ public UTF8String clone() { public UTF8String copy() { byte[] bytes = new byte[numBytes]; - MemoryBlock.copyMemory(base, getBaseOffset(), bytes, BYTE_ARRAY_OFFSET, numBytes); + base.writeTo(getBaseOffset(), bytes, BYTE_ARRAY_OFFSET, numBytes); return fromBytes(bytes); } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java index d4080a5dc9e93..623fddaede9fe 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java @@ -20,14 +20,14 @@ import org.junit.Assert; import org.junit.Test; -import org.apache.spark.unsafe.memory.LongArrayMemoryBlock; +import org.apache.spark.unsafe.memory.OnHeapMemoryBlock; public class LongArraySuite { @Test public void basicTest() { long[] bytes = new long[2]; - LongArray arr = new LongArray(LongArrayMemoryBlock.fromArray(bytes)); + LongArray arr = new LongArray(OnHeapMemoryBlock.fromArray(bytes)); arr.set(0, 1L); arr.set(1, 2L); arr.set(1, 3L); diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java index 59a5f2222fad2..d44f50544c069 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java @@ -21,9 +21,11 @@ import org.junit.Assert; import org.junit.Test; -import org.apache.spark.unsafe.memory.LongArrayMemoryBlock; +import java.nio.ByteOrder; public class MemoryBlockSuite { + private static final boolean bigEndianPlatform = + ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); private void check(MemoryBlock memory, Object obj, long offset, int length) { memory.setPageNumber(1); @@ -36,6 +38,12 @@ private void check(MemoryBlock memory, Object obj, long offset, int length) { memory.putFloat(offset + 16, 1.0F); memory.putDouble(offset + 20, 2.0); MemoryBlock.copyMemory(memory, offset, memory, offset + 28, 4); + int[] a = new int[2]; + a[0] = 0x12345678; + a[1] = 0x13579BDF; + memory.copyFrom(a, Platform.INT_ARRAY_OFFSET, offset + 32, 8); + byte[] b = new byte[8]; + memory.writeTo(offset + 32, b, Platform.BYTE_ARRAY_OFFSET, 8); Assert.assertEquals(obj, memory.getBaseObject()); Assert.assertEquals(offset, memory.getBaseOffset()); @@ -51,14 +59,31 @@ private void check(MemoryBlock memory, Object obj, long offset, int length) { Assert.assertEquals(true, memory.getBoolean(offset + 28)); Assert.assertEquals((byte)127, memory.getByte(offset + 29 )); Assert.assertEquals((short)257, memory.getShort(offset + 30)); - for (int i = 32; i < memory.size(); i++) { + Assert.assertEquals(a[0], memory.getInt(offset + 32)); + Assert.assertEquals(a[1], memory.getInt(offset + 36)); + if (bigEndianPlatform) { + Assert.assertEquals(a[0], + ((int)b[0] & 0xff) << 24 | ((int)b[1] & 0xff) << 16 | + ((int)b[2] & 0xff) << 8 | ((int)b[3] & 0xff)); + Assert.assertEquals(a[1], + ((int)b[4] & 0xff) << 24 | ((int)b[5] & 0xff) << 16 | + ((int)b[6] & 0xff) << 8 | ((int)b[7] & 0xff)); + } else { + Assert.assertEquals(a[0], + ((int)b[3] & 0xff) << 24 | ((int)b[2] & 0xff) << 16 | + ((int)b[1] & 0xff) << 8 | ((int)b[0] & 0xff)); + Assert.assertEquals(a[1], + ((int)b[7] & 0xff) << 24 | ((int)b[6] & 0xff) << 16 | + ((int)b[5] & 0xff) << 8 | ((int)b[4] & 0xff)); + } + for (int i = 40; i < memory.size(); i++) { Assert.assertEquals((byte) -1, memory.getByte(offset + i)); } } @Test public void ByteArrayMemoryBlockTest() { - byte[] obj = new byte[36]; + byte[] obj = new byte[48]; long offset = Platform.BYTE_ARRAY_OFFSET; int length = obj.length; MemoryBlock memory = new ByteArrayMemoryBlock(obj, offset, length); @@ -67,21 +92,11 @@ public void ByteArrayMemoryBlockTest() { } @Test - public void IntArrayMemoryBlockTest() { - int[] obj = new int[9]; - long offset = Platform.INT_ARRAY_OFFSET; - int length = obj.length; - MemoryBlock memory = new IntArrayMemoryBlock(obj, offset, length); - - check(memory, obj, offset, length); - } - - @Test - public void LongArrayMemoryBlockTest() { - long[] obj = new long[5]; + public void OnHeapMemoryBlockTest() { + long[] obj = new long[6]; long offset = Platform.LONG_ARRAY_OFFSET; int length = obj.length; - MemoryBlock memory = new LongArrayMemoryBlock(obj, offset, length); + MemoryBlock memory = new OnHeapMemoryBlock(obj, offset, length); check(memory, obj, offset, length); } @@ -89,10 +104,10 @@ public void LongArrayMemoryBlockTest() { @Test public void OffHeapArrayMemoryBlockTest() { MemoryAllocator memoryAllocator = new UnsafeMemoryAllocator(); - MemoryBlock memory = memoryAllocator.allocate(36); + MemoryBlock memory = memoryAllocator.allocate(48); Object obj = memory.getBaseObject(); long offset = memory.getBaseOffset(); - int length = 36; + int length = 48; check(memory, obj, offset, length); } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java index 86d50cdb3beff..3b174a45d7b0a 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java @@ -27,7 +27,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.memory.ByteArrayMemoryBlock; -import org.apache.spark.unsafe.memory.IntArrayMemoryBlock; +import org.apache.spark.unsafe.memory.OnHeapMemoryBlock; import org.junit.Test; import static org.junit.Assert.*; @@ -593,26 +593,26 @@ public void writeToOutputStream() throws IOException { } @Test - public void writeToOutputStreamIntArray() throws IOException { + public void writeToOutputStreamLongArray() throws IOException { // verify that writes work on objects that are not byte arrays - final ByteBuffer buffer = StandardCharsets.UTF_8.encode("大千世界"); + final ByteBuffer buffer = StandardCharsets.UTF_8.encode("3千大千世界"); buffer.position(0); buffer.order(ByteOrder.nativeOrder()); final int length = buffer.limit(); - assertEquals(12, length); + assertEquals(16, length); - final int ints = length / 4; - final int[] array = new int[ints]; + final int longs = length / 8; + final long[] array = new long[longs]; - for (int i = 0; i < ints; ++i) { - array[i] = buffer.getInt(); + for (int i = 0; i < longs; ++i) { + array[i] = buffer.getLong(); } final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - fromAddress(IntArrayMemoryBlock.fromArray(array), Platform.INT_ARRAY_OFFSET, length) + fromAddress(OnHeapMemoryBlock.fromArray(array), Platform.LONG_ARRAY_OFFSET, length) .writeTo(outputStream); - assertEquals("大千世界", outputStream.toString("UTF-8")); + assertEquals("3千大千世界", outputStream.toString("UTF-8")); } @Test diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala index f9358478f3a42..507ec191f307b 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark._ import org.apache.spark.memory.MemoryTestingUtils import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.unsafe.array.LongArray -import org.apache.spark.unsafe.memory.LongArrayMemoryBlock +import org.apache.spark.unsafe.memory.OnHeapMemoryBlock import org.apache.spark.util.collection.unsafe.sort.{PrefixComparators, RecordPointerAndKeyPrefix, UnsafeSortDataFormat} class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { @@ -105,9 +105,9 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { // the form [150000000, 150000001, 150000002, ...., 300000000, 0, 1, 2, ..., 149999999] // that can trigger copyRange() in TimSort.mergeLo() or TimSort.mergeHi() val ref = Array.tabulate[Long](size) { i => if (i < size / 2) size / 2 + i else i } - val buf = new LongArray(LongArrayMemoryBlock.fromArray(ref)) + val buf = new LongArray(OnHeapMemoryBlock.fromArray(ref)) val tmp = new Array[Long](size/2) - val tmpBuf = new LongArray(LongArrayMemoryBlock.fromArray(tmp)) + val tmpBuf = new LongArray(OnHeapMemoryBlock.fromArray(tmp)) new Sorter(new UnsafeSortDataFormat(tmpBuf)).sort( buf, 0, size, new Comparator[RecordPointerAndKeyPrefix] { diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala index c1642a4d305a5..639fbc0179103 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala @@ -27,7 +27,7 @@ import com.google.common.primitives.Ints import org.apache.spark.SparkFunSuite import org.apache.spark.internal.Logging import org.apache.spark.unsafe.array.LongArray -import org.apache.spark.unsafe.memory.LongArrayMemoryBlock +import org.apache.spark.unsafe.memory.OnHeapMemoryBlock import org.apache.spark.util.collection.Sorter import org.apache.spark.util.random.XORShiftRandom @@ -78,14 +78,14 @@ class RadixSortSuite extends SparkFunSuite with Logging { private def generateTestData(size: Long, rand: => Long): (Array[JLong], LongArray) = { val ref = Array.tabulate[Long](Ints.checkedCast(size)) { i => rand } val extended = ref ++ Array.fill[Long](Ints.checkedCast(size))(0) - (ref.map(i => new JLong(i)), new LongArray(LongArrayMemoryBlock.fromArray(extended))) + (ref.map(i => new JLong(i)), new LongArray(OnHeapMemoryBlock.fromArray(extended))) } private def generateKeyPrefixTestData(size: Long, rand: => Long): (LongArray, LongArray) = { val ref = Array.tabulate[Long](Ints.checkedCast(size * 2)) { i => rand } val extended = ref ++ Array.fill[Long](Ints.checkedCast(size * 2))(0) - (new LongArray(LongArrayMemoryBlock.fromArray(ref)), - new LongArray(LongArrayMemoryBlock.fromArray(extended))) + (new LongArray(OnHeapMemoryBlock.fromArray(ref)), + new LongArray(OnHeapMemoryBlock.fromArray(extended))) } private def collectToArray(array: LongArray, offset: Int, length: Long): Array[Long] = { @@ -111,7 +111,7 @@ class RadixSortSuite extends SparkFunSuite with Logging { private def referenceKeyPrefixSort(buf: LongArray, lo: Long, hi: Long, refCmp: PrefixComparator) { val sortBuffer = - new LongArray(LongArrayMemoryBlock.fromArray(new Array[Long](buf.size().toInt))) + new LongArray(OnHeapMemoryBlock.fromArray(new Array[Long](buf.size().toInt))) new Sorter(new UnsafeSortDataFormat(sortBuffer)).sort( buf, Ints.checkedCast(lo), Ints.checkedCast(hi), new Comparator[RecordPointerAndKeyPrefix] { override def compare( diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java index a936e884b45c4..7fd3d18cb5046 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java @@ -28,7 +28,7 @@ import org.apache.spark.unsafe.bitset.BitSetMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; import org.apache.spark.unsafe.memory.ByteArrayMemoryBlock; -import org.apache.spark.unsafe.memory.LongArrayMemoryBlock; +import org.apache.spark.unsafe.memory.OnHeapMemoryBlock; import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java index e738307063e0e..fe6f310bad90c 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -187,7 +187,7 @@ public void putBytes(int rowId, int count, byte value) { @Override public void putBytes(int rowId, int count, byte[] src, int srcIndex) { - MemoryBlock.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, data, data.getBaseOffset() + rowId, count); + data.copyFrom(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, data.getBaseOffset() + rowId, count); } @Override @@ -203,7 +203,7 @@ public byte getByte(int rowId) { public byte[] getBytes(int rowId, int count) { assert(dictionary == null); byte[] array = new byte[count]; - MemoryBlock.copyMemory(data, data.getBaseOffset() + rowId, array, Platform.BYTE_ARRAY_OFFSET, count); + data.writeTo(data.getBaseOffset() + rowId, array, Platform.BYTE_ARRAY_OFFSET, count); return array; } @@ -231,8 +231,8 @@ public void putShorts(int rowId, int count, short value) { @Override public void putShorts(int rowId, int count, short[] src, int srcIndex) { - MemoryBlock.copyMemory(src, Platform.SHORT_ARRAY_OFFSET + srcIndex * 2, - data, data.getBaseOffset() + 2 * rowId, count * 2); + data.copyFrom(src, Platform.SHORT_ARRAY_OFFSET + srcIndex * 2, + data.getBaseOffset() + 2 * rowId, count * 2); } @Override @@ -254,7 +254,7 @@ public short getShort(int rowId) { public short[] getShorts(int rowId, int count) { assert(dictionary == null); short[] array = new short[count]; - MemoryBlock.copyMemory(data, data.getBaseOffset() + rowId * 2, array, Platform.SHORT_ARRAY_OFFSET, count * 2); + data.writeTo(data.getBaseOffset() + rowId * 2, array, Platform.SHORT_ARRAY_OFFSET, count * 2); return array; } @@ -277,21 +277,21 @@ public void putInts(int rowId, int count, int value) { @Override public void putInts(int rowId, int count, int[] src, int srcIndex) { - MemoryBlock.copyMemory(src, Platform.INT_ARRAY_OFFSET + srcIndex * 4, - data, data.getBaseOffset() + 4 * rowId, count * 4); + data.copyFrom(src, Platform.INT_ARRAY_OFFSET + srcIndex * 4, + data.getBaseOffset() + 4 * rowId, count * 4); } @Override public void putInts(int rowId, int count, byte[] src, int srcIndex) { - Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - null, data.getBaseOffset() + rowId * 4, count * 4); + data.copyFrom(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + data.getBaseOffset() + rowId * 4, count * 4); } @Override public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { - MemoryBlock.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, - data, data.getBaseOffset() + 4 * rowId, count * 4); + data.copyFrom(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, + data.getBaseOffset() + 4 * rowId, count * 4); } else { int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; long offset = data.getBaseOffset() + 4 * rowId; @@ -314,7 +314,7 @@ public int getInt(int rowId) { public int[] getInts(int rowId, int count) { assert(dictionary == null); int[] array = new int[count]; - MemoryBlock.copyMemory(data, data.getBaseOffset() + rowId * 4, array, Platform.INT_ARRAY_OFFSET, count * 4); + data.writeTo(data.getBaseOffset() + rowId * 4, array, Platform.INT_ARRAY_OFFSET, count * 4); return array; } @@ -348,21 +348,21 @@ public void putLongs(int rowId, int count, long value) { @Override public void putLongs(int rowId, int count, long[] src, int srcIndex) { - MemoryBlock.copyMemory(src, Platform.LONG_ARRAY_OFFSET + srcIndex * 8, - data, data.getBaseOffset() + 8 * rowId, count * 8); + data.copyFrom(src, Platform.LONG_ARRAY_OFFSET + srcIndex * 8, + data.getBaseOffset() + 8 * rowId, count * 8); } @Override public void putLongs(int rowId, int count, byte[] src, int srcIndex) { - Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - null, data.getBaseOffset() + rowId * 8, count * 8); + data.copyFrom(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + data.getBaseOffset() + rowId * 8, count * 8); } @Override public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { - MemoryBlock.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, - data, data.getBaseOffset() + 8 * rowId, count * 8); + data.copyFrom(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, + data.getBaseOffset() + 8 * rowId, count * 8); } else { int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; long offset = data.getBaseOffset() + 8 * rowId; @@ -385,7 +385,7 @@ public long getLong(int rowId) { public long[] getLongs(int rowId, int count) { assert(dictionary == null); long[] array = new long[count]; - MemoryBlock.copyMemory(data, data.getBaseOffset() + rowId * 8, array, Platform.LONG_ARRAY_OFFSET, count * 8); + data.writeTo(data.getBaseOffset() + rowId * 8, array, Platform.LONG_ARRAY_OFFSET, count * 8); return array; } @@ -408,15 +408,15 @@ public void putFloats(int rowId, int count, float value) { @Override public void putFloats(int rowId, int count, float[] src, int srcIndex) { - MemoryBlock.copyMemory(src, Platform.FLOAT_ARRAY_OFFSET + srcIndex * 4, - data, data.getBaseOffset() + 4 * rowId, count * 4); + data.copyFrom(src, Platform.FLOAT_ARRAY_OFFSET + srcIndex * 4, + data.getBaseOffset() + 4 * rowId, count * 4); } @Override public void putFloats(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { - MemoryBlock.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - data, data.getBaseOffset() + rowId * 4, count * 4); + data.copyFrom(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + data.getBaseOffset() + rowId * 4, count * 4); } else { ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); long offset = data.getBaseOffset() + 4 * rowId; @@ -439,7 +439,7 @@ public float getFloat(int rowId) { public float[] getFloats(int rowId, int count) { assert(dictionary == null); float[] array = new float[count]; - MemoryBlock.copyMemory(data, data.getBaseOffset() + rowId * 4, array, Platform.FLOAT_ARRAY_OFFSET, count * 4); + data.writeTo(data.getBaseOffset() + rowId * 4, array, Platform.FLOAT_ARRAY_OFFSET, count * 4); return array; } @@ -463,15 +463,15 @@ public void putDoubles(int rowId, int count, double value) { @Override public void putDoubles(int rowId, int count, double[] src, int srcIndex) { - MemoryBlock.copyMemory(src, Platform.DOUBLE_ARRAY_OFFSET + srcIndex * 8, - data, data.getBaseOffset() + 8 * rowId, count * 8); + data.copyFrom(src, Platform.DOUBLE_ARRAY_OFFSET + srcIndex * 8, + data.getBaseOffset() + 8 * rowId, count * 8); } @Override public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { - MemoryBlock.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - data, data.getBaseOffset() + rowId * 8, count * 8); + data.copyFrom(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + data.getBaseOffset() + rowId * 8, count * 8); } else { ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); long offset = data.getBaseOffset() + 8 * rowId; @@ -494,7 +494,7 @@ public double getDouble(int rowId) { public double[] getDoubles(int rowId, int count) { assert(dictionary == null); double[] array = new double[count]; - MemoryBlock.copyMemory(data, data.getBaseOffset() + rowId * 8, array, Platform.DOUBLE_ARRAY_OFFSET, count * 8); + data.writeTo(data.getBaseOffset() + rowId * 8, array, Platform.DOUBLE_ARRAY_OFFSET, count * 8); return array; } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala index 2b93a98f28477..e1cea8a88ea4b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.benchmark import java.util.{Arrays, Comparator} import org.apache.spark.unsafe.array.LongArray -import org.apache.spark.unsafe.memory.LongArrayMemoryBlock +import org.apache.spark.unsafe.memory.OnHeapMemoryBlock import org.apache.spark.util.Benchmark import org.apache.spark.util.collection.Sorter import org.apache.spark.util.collection.unsafe.sort._ @@ -37,7 +37,7 @@ class SortBenchmark extends BenchmarkBase { private def referenceKeyPrefixSort(buf: LongArray, lo: Int, hi: Int, refCmp: PrefixComparator) { val sortBuffer = - new LongArray(LongArrayMemoryBlock.fromArray(new Array[Long](buf.size().toInt))) + new LongArray(OnHeapMemoryBlock.fromArray(new Array[Long](buf.size().toInt))) new Sorter(new UnsafeSortDataFormat(sortBuffer)).sort( buf, lo, hi, new Comparator[RecordPointerAndKeyPrefix] { override def compare( @@ -51,8 +51,8 @@ class SortBenchmark extends BenchmarkBase { private def generateKeyPrefixTestData(size: Int, rand: => Long): (LongArray, LongArray) = { val ref = Array.tabulate[Long](size * 2) { i => rand } val extended = ref ++ Array.fill[Long](size * 2)(0) - (new LongArray(LongArrayMemoryBlock.fromArray(ref)), - new LongArray(LongArrayMemoryBlock.fromArray(extended))) + (new LongArray(OnHeapMemoryBlock.fromArray(ref)), + new LongArray(OnHeapMemoryBlock.fromArray(extended))) } ignore("sort") { @@ -61,7 +61,7 @@ class SortBenchmark extends BenchmarkBase { val benchmark = new Benchmark("radix sort " + size, size) benchmark.addTimerCase("reference TimSort key prefix array") { timer => val array = Array.tabulate[Long](size * 2) { i => rand.nextLong } - val buf = new LongArray(LongArrayMemoryBlock.fromArray(array)) + val buf = new LongArray(OnHeapMemoryBlock.fromArray(array)) timer.startTiming() referenceKeyPrefixSort(buf, 0, size, PrefixComparators.BINARY) timer.stopTiming() @@ -79,7 +79,7 @@ class SortBenchmark extends BenchmarkBase { array(i) = rand.nextLong & 0xff i += 1 } - val buf = new LongArray(LongArrayMemoryBlock.fromArray(array)) + val buf = new LongArray(OnHeapMemoryBlock.fromArray(array)) timer.startTiming() RadixSort.sort(buf, size, 0, 7, false, false) timer.stopTiming() @@ -91,7 +91,7 @@ class SortBenchmark extends BenchmarkBase { array(i) = rand.nextLong & 0xffff i += 1 } - val buf = new LongArray(LongArrayMemoryBlock.fromArray(array)) + val buf = new LongArray(OnHeapMemoryBlock.fromArray(array)) timer.startTiming() RadixSort.sort(buf, size, 0, 7, false, false) timer.stopTiming() @@ -103,7 +103,7 @@ class SortBenchmark extends BenchmarkBase { array(i) = rand.nextLong i += 1 } - val buf = new LongArray(LongArrayMemoryBlock.fromArray(array)) + val buf = new LongArray(OnHeapMemoryBlock.fromArray(array)) timer.startTiming() RadixSort.sort(buf, size, 0, 7, false, false) timer.stopTiming() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala index baf7281269089..52f9fba08d88b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala @@ -22,13 +22,13 @@ import java.io.File import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.memory.{MemoryManager, TaskMemoryManager, TestMemoryManager} import org.apache.spark.sql.catalyst.expressions.UnsafeRow -import org.apache.spark.unsafe.memory.LongArrayMemoryBlock +import org.apache.spark.unsafe.memory.OnHeapMemoryBlock import org.apache.spark.util.Utils class RowQueueSuite extends SparkFunSuite { test("in-memory queue") { - val page = LongArrayMemoryBlock.fromArray(new Array[Long](1<<10)) + val page = OnHeapMemoryBlock.fromArray(new Array[Long](1<<10)) val queue = new InMemoryRowQueue(page, 1) { override def close() {} } From 5e3afd11a2dc76d2cd23264b4052abbf3f5d7e9d Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 23 Feb 2018 16:20:21 +0000 Subject: [PATCH 29/65] address review comments --- .../unsafe/memory/ByteArrayMemoryBlock.java | 30 +++++++------------ .../unsafe/memory/HeapMemoryAllocator.java | 5 ++-- .../spark/unsafe/memory/MemoryBlock.java | 14 +++++---- .../unsafe/memory/OffHeapMemoryBlock.java | 8 ----- .../unsafe/memory/OnHeapMemoryBlock.java | 26 ++++++---------- .../unsafe/memory/UnsafeMemoryAllocator.java | 2 +- .../spark/unsafe/memory/MemoryBlockSuite.java | 8 ++++- .../shuffle/sort/ShuffleInMemorySorter.java | 8 +---- .../unsafe/sort/UnsafeInMemorySorter.java | 7 +---- 9 files changed, 40 insertions(+), 68 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java index 3d10718d002a5..074ea9742ce1b 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -116,41 +116,33 @@ public final void putDouble(long offset, double value) { Platform.putDouble(array, offset, value); } - public final Object getObjectVolatile(long offset) { - return Platform.getObjectVolatile(array, offset); - } - - public final void putObjectVolatile(long offset, Object value) { - Platform.putObjectVolatile(array, offset, value); - } - public final void copyFrom(byte[] src, long srcOffset, long dstOffset, long length) { assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= src.length); - assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); + assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); Platform.copyMemory(src, srcOffset, array, dstOffset, length); } public final void copyFrom(short[] src, long srcOffset, long dstOffset, long length) { assert(srcOffset - Platform.SHORT_ARRAY_OFFSET + length <= src.length * 2); - assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); + assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); Platform.copyMemory(src, srcOffset, array, dstOffset, length); } public final void copyFrom(int[] src, long srcOffset, long dstOffset, long length) { assert(srcOffset - Platform.INT_ARRAY_OFFSET + length <= src.length * 4); - assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); + assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); Platform.copyMemory(src, srcOffset, array, dstOffset, length); } public final void copyFrom(long[] src, long srcOffset, long dstOffset, long length) { assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= src.length * 8); - assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); + assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); Platform.copyMemory(src, srcOffset, array, dstOffset, length); } public final void copyFrom(float[] src, long srcOffset, long dstOffset, long length) { assert(srcOffset - Platform.FLOAT_ARRAY_OFFSET + length <= src.length * 4); - assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); + assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); Platform.copyMemory(src, srcOffset, array, dstOffset, length); } @@ -161,37 +153,37 @@ public final void copyFrom(double[] src, long srcOffset, long dstOffset, long le } public final void writeTo(long srcOffset, byte[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= dst.length); Platform.copyMemory(array, srcOffset, dst, dstOffset, length); } public final void writeTo(long srcOffset, short[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); assert(dstOffset - Platform.SHORT_ARRAY_OFFSET + length <= dst.length * 2); Platform.copyMemory(array, srcOffset, dst, dstOffset, length); } public final void writeTo(long srcOffset, int[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); assert(dstOffset - Platform.INT_ARRAY_OFFSET + length <= dst.length * 4); Platform.copyMemory(array, srcOffset, dst, dstOffset, length); } public final void writeTo(long srcOffset, long[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= dst.length * 8); Platform.copyMemory(array, srcOffset, dst, dstOffset, length); } public final void writeTo(long srcOffset, float[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); assert(dstOffset - Platform.FLOAT_ARRAY_OFFSET + length <= dst.length * 4); Platform.copyMemory(array, srcOffset, dst, dstOffset, length); } public final void writeTo(long srcOffset, double[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); assert(dstOffset - Platform.DOUBLE_ARRAY_OFFSET + length <= dst.length * 8); Platform.copyMemory(array, srcOffset, dst, dstOffset, length); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java index 727e06ee4ccb9..dc5f0ad1976d8 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java @@ -58,8 +58,7 @@ public MemoryBlock allocate(long size) throws OutOfMemoryError { final long[] array = arrayReference.get(); if (array != null) { assert (array.length * 8L >= size); - MemoryBlock memory = - new OnHeapMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, size); + MemoryBlock memory = OnHeapMemoryBlock.fromArray(array); if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE); } @@ -71,7 +70,7 @@ public MemoryBlock allocate(long size) throws OutOfMemoryError { } } long[] array = new long[numWords]; - MemoryBlock memory = new OnHeapMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, size); + MemoryBlock memory = OnHeapMemoryBlock.fromArray(array); if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index bdbefb8e7eea0..ccb615684fd5d 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -22,7 +22,8 @@ import org.apache.spark.unsafe.Platform; /** - * A declaration of interfaces of MemoryBlock classes . + * A representation of a consecutive memory block in Spark. It defines the common interfaces + * for memory accessing and mutating. */ public abstract class MemoryBlock { /** Special `pageNumber` value for pages which were not allocated by TaskMemoryManagers */ @@ -157,15 +158,16 @@ public final static MemoryBlock allocateFromObject(Object obj, long offset, long public abstract void putDouble(long offset, double value); - public abstract Object getObjectVolatile(long offset); - - public abstract void putObjectVolatile(long offset, Object value); - - public static void copyMemory( + public static final void copyMemory( MemoryBlock src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { Platform.copyMemory(src.getBaseObject(), srcOffset, dst.getBaseObject(), dstOffset, length); } + public static final void copyMemory(MemoryBlock src, MemoryBlock dst, long length) { + Platform.copyMemory(src.getBaseObject(), src.getBaseOffset(), + dst.getBaseObject(), dst.getBaseOffset(), length); + } + public abstract void copyFrom(byte[] src, long srcOffset, long dstOffset, long length); public abstract void copyFrom(short[] src, long srcOffset, long dstOffset, long length); diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java index 6996c104986ad..cee88a94d4cab 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java @@ -106,14 +106,6 @@ public final void putDouble(long offset, double value) { Platform.putDouble(null, offset, value); } - public final Object getObjectVolatile(long offset) { - return Platform.getObjectVolatile(null, offset); - } - - public final void putObjectVolatile(long offset, Object value) { - Platform.putObjectVolatile(null, offset, value); - } - public final void copyFrom(byte[] src, long srcOffset, long dstOffset, long length) { assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= src.length); assert(dstOffset + length <= this.offset + this.length); diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java index f1e12338bb7ab..e6453f9d70695 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java @@ -68,12 +68,12 @@ public final void putBoolean(long offset, boolean value) { } public final byte getByte(long offset) { - assert(offset + 2 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + assert(offset + 1 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); return Platform.getByte(array, offset); } public final void putByte(long offset, byte value) { - assert(offset + 2 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + assert(offset + 1 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); Platform.putByte(array, offset, value); } @@ -117,14 +117,6 @@ public final void putDouble(long offset, double value) { Platform.putDouble(array, offset, value); } - public final Object getObjectVolatile(long offset) { - return Platform.getObjectVolatile(array, offset); - } - - public final void putObjectVolatile(long offset, Object value) { - Platform.putObjectVolatile(array, offset, value); - } - public final void copyFrom(byte[] src, long srcOffset, long dstOffset, long length) { assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= src.length); assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); @@ -157,42 +149,42 @@ public final void copyFrom(float[] src, long srcOffset, long dstOffset, long len public final void copyFrom(double[] src, long srcOffset, long dstOffset, long length) { assert(srcOffset - Platform.DOUBLE_ARRAY_OFFSET + length <= src.length * 8); - assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); Platform.copyMemory(src, srcOffset, array, dstOffset, length); } public final void writeTo(long srcOffset, byte[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= dst.length); Platform.copyMemory(array, srcOffset, dst, dstOffset, length); } public final void writeTo(long srcOffset, short[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); assert(dstOffset - Platform.SHORT_ARRAY_OFFSET + length <= dst.length * 2); Platform.copyMemory(array, srcOffset, dst, dstOffset, length); } public final void writeTo(long srcOffset, int[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); assert(dstOffset - Platform.INT_ARRAY_OFFSET + length <= dst.length * 4); Platform.copyMemory(array, srcOffset, dst, dstOffset, length); } public final void writeTo(long srcOffset, long[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= dst.length * 8); Platform.copyMemory(array, srcOffset, dst, dstOffset, length); } public final void writeTo(long srcOffset, float[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); assert(dstOffset - Platform.FLOAT_ARRAY_OFFSET + length <= dst.length * 4); Platform.copyMemory(array, srcOffset, dst, dstOffset, length); } public final void writeTo(long srcOffset, double[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); assert(dstOffset - Platform.DOUBLE_ARRAY_OFFSET + length <= dst.length * 8); Platform.copyMemory(array, srcOffset, dst, dstOffset, length); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java index 1d9fe8a200059..45d84954706e2 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -71,7 +71,7 @@ public void free(MemoryBlock memory) { public OffHeapMemoryBlock reallocate(OffHeapMemoryBlock block, long oldSize, long newSize) { OffHeapMemoryBlock mb = this.allocate(newSize); if (block.getBaseOffset() != 0) - MemoryBlock.copyMemory(block, block.getBaseOffset(), mb, mb.getBaseOffset(), oldSize); + MemoryBlock.copyMemory(block, mb, oldSize); free(block); return mb; } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java index d44f50544c069..52398e3d49635 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java @@ -86,8 +86,11 @@ public void ByteArrayMemoryBlockTest() { byte[] obj = new byte[48]; long offset = Platform.BYTE_ARRAY_OFFSET; int length = obj.length; + MemoryBlock memory = new ByteArrayMemoryBlock(obj, offset, length); + check(memory, obj, offset, length); + memory = ByteArrayMemoryBlock.fromArray(obj); check(memory, obj, offset, length); } @@ -95,9 +98,12 @@ public void ByteArrayMemoryBlockTest() { public void OnHeapMemoryBlockTest() { long[] obj = new long[6]; long offset = Platform.LONG_ARRAY_OFFSET; - int length = obj.length; + int length = obj.length * 8; + MemoryBlock memory = new OnHeapMemoryBlock(obj, offset, length); + check(memory, obj, offset, length); + memory = OnHeapMemoryBlock.fromArray(obj); check(memory, obj, offset, length); } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java index cc6699932bf3c..5f253ee9a4fbb 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java @@ -104,13 +104,7 @@ public void reset() { public void expandPointerArray(LongArray newArray) { assert(newArray.size() > array.size()); - MemoryBlock.copyMemory( - array.memoryBlock(), - array.getBaseOffset(), - newArray.memoryBlock(), - newArray.getBaseOffset(), - pos * 8L - ); + MemoryBlock.copyMemory(array.memoryBlock(), newArray.memoryBlock(),pos * 8L); consumer.freeArray(array); array = newArray; usableCapacity = getUsableCapacity(); diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index a4a24c8c62c0c..afa210aaafc66 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -215,12 +215,7 @@ public void expandPointerArray(LongArray newArray) { if (newArray.size() < array.size()) { throw new SparkOutOfMemoryError("Not enough memory to grow pointer array"); } - MemoryBlock.copyMemory( - array.memoryBlock(), - array.getBaseOffset(), - newArray.memoryBlock(), - newArray.getBaseOffset(), - pos * 8L); + MemoryBlock.copyMemory(array.memoryBlock(), newArray.memoryBlock(), pos * 8L); consumer.freeArray(array); array = newArray; usableCapacity = getUsableCapacity(); From 95fbdee04e9137938cdc76f7f4573116720357f5 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 24 Feb 2018 00:22:44 +0000 Subject: [PATCH 30/65] fix test failures --- .../spark/unsafe/memory/ByteArrayMemoryBlock.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java index 074ea9742ce1b..282ab81bf8022 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -148,42 +148,42 @@ public final void copyFrom(float[] src, long srcOffset, long dstOffset, long len public final void copyFrom(double[] src, long srcOffset, long dstOffset, long length) { assert(srcOffset - Platform.DOUBLE_ARRAY_OFFSET + length <= src.length * 8); - assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= this.length * 8); + assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length); Platform.copyMemory(src, srcOffset, array, dstOffset, length); } public final void writeTo(long srcOffset, byte[] dst, long dstOffset, long length) { - assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); + assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= dst.length); Platform.copyMemory(array, srcOffset, dst, dstOffset, length); } public final void writeTo(long srcOffset, short[] dst, long dstOffset, long length) { - assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); + assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); assert(dstOffset - Platform.SHORT_ARRAY_OFFSET + length <= dst.length * 2); Platform.copyMemory(array, srcOffset, dst, dstOffset, length); } public final void writeTo(long srcOffset, int[] dst, long dstOffset, long length) { - assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); + assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); assert(dstOffset - Platform.INT_ARRAY_OFFSET + length <= dst.length * 4); Platform.copyMemory(array, srcOffset, dst, dstOffset, length); } public final void writeTo(long srcOffset, long[] dst, long dstOffset, long length) { - assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); + assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= dst.length * 8); Platform.copyMemory(array, srcOffset, dst, dstOffset, length); } public final void writeTo(long srcOffset, float[] dst, long dstOffset, long length) { - assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); + assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); assert(dstOffset - Platform.FLOAT_ARRAY_OFFSET + length <= dst.length * 4); Platform.copyMemory(array, srcOffset, dst, dstOffset, length); } public final void writeTo(long srcOffset, double[] dst, long dstOffset, long length) { - assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); + assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); assert(dstOffset - Platform.DOUBLE_ARRAY_OFFSET + length <= dst.length * 8); Platform.copyMemory(array, srcOffset, dst, dstOffset, length); } From 9b4975b8487dc0a06ae66943cf3fdda5750c4a74 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 25 Feb 2018 05:00:19 +0000 Subject: [PATCH 31/65] fix test failures --- .../org/apache/spark/unsafe/memory/HeapMemoryAllocator.java | 4 ++-- .../org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java | 3 +++ 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java index dc5f0ad1976d8..acf28fd7ee59b 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java @@ -58,7 +58,7 @@ public MemoryBlock allocate(long size) throws OutOfMemoryError { final long[] array = arrayReference.get(); if (array != null) { assert (array.length * 8L >= size); - MemoryBlock memory = OnHeapMemoryBlock.fromArray(array); + MemoryBlock memory = OnHeapMemoryBlock.fromArray(array, size); if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE); } @@ -70,7 +70,7 @@ public MemoryBlock allocate(long size) throws OutOfMemoryError { } } long[] array = new long[numWords]; - MemoryBlock memory = OnHeapMemoryBlock.fromArray(array); + MemoryBlock memory = OnHeapMemoryBlock.fromArray(array, size); if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java index e6453f9d70695..f51c473df3d82 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java @@ -46,6 +46,9 @@ public static OnHeapMemoryBlock fromArray(final long[] array) { return new OnHeapMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, array.length * 8); } + public static OnHeapMemoryBlock fromArray(final long[] array, long size) { + return new OnHeapMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, size); + } public final int getInt(long offset) { assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); From 9e2697cc70939d8221572697991ebf7b6b29cd65 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 25 Feb 2018 08:45:27 +0000 Subject: [PATCH 32/65] remove MemoryBlock offset at caller site for allocate() and copyMemory() --- .../spark/unsafe/memory/ByteArrayMemoryBlock.java | 2 +- .../org/apache/spark/unsafe/memory/MemoryBlock.java | 3 ++- .../apache/spark/unsafe/memory/OffHeapMemoryBlock.java | 2 +- .../apache/spark/unsafe/memory/OnHeapMemoryBlock.java | 2 +- .../java/org/apache/spark/unsafe/types/UTF8String.java | 6 +----- .../apache/spark/unsafe/memory/MemoryBlockSuite.java | 2 +- .../org/apache/spark/unsafe/types/UTF8StringSuite.java | 10 ++++------ .../spark/shuffle/sort/ShuffleInMemorySorter.java | 4 +--- .../spark/shuffle/sort/ShuffleSortDataFormat.java | 9 ++------- .../collection/unsafe/sort/UnsafeInMemorySorter.java | 4 +--- .../sql/execution/vectorized/OffHeapColumnVector.java | 2 +- .../apache/spark/sql/vectorized/ArrowColumnVector.java | 2 +- 12 files changed, 17 insertions(+), 31 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java index 282ab81bf8022..4a6d1656150e4 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -33,7 +33,7 @@ public ByteArrayMemoryBlock(byte[] obj, long offset, long length) { @Override public MemoryBlock allocate(long offset, long size) { - return new ByteArrayMemoryBlock(array, offset, size); + return new ByteArrayMemoryBlock(array, this.offset + offset, size); } public byte[] getByteArray() { return array; } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index ccb615684fd5d..ddb049e86bd7e 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -160,7 +160,8 @@ public final static MemoryBlock allocateFromObject(Object obj, long offset, long public static final void copyMemory( MemoryBlock src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { - Platform.copyMemory(src.getBaseObject(), srcOffset, dst.getBaseObject(), dstOffset, length); + Platform.copyMemory(src.getBaseObject(), src.getBaseOffset() + srcOffset, + dst.getBaseObject(), dst.getBaseOffset() + dstOffset, length); } public static final void copyMemory(MemoryBlock src, MemoryBlock dst, long length) { diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java index cee88a94d4cab..4fea544126218 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java @@ -33,7 +33,7 @@ public void setAddressAndSize(long address, long size) { @Override public MemoryBlock allocate(long offset, long size) { - return new OffHeapMemoryBlock(offset, size); + return new OffHeapMemoryBlock(this.offset + offset, size); } public final int getInt(long offset) { diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java index f51c473df3d82..87003b04ef88e 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java @@ -34,7 +34,7 @@ public OnHeapMemoryBlock(long[] obj, long offset, long size) { @Override public MemoryBlock allocate(long offset, long size) { - return new OnHeapMemoryBlock(array, offset, size); + return new OnHeapMemoryBlock(array, this.offset + offset, size); } public long[] getLongArray() { return array; } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 14098a6a72772..b45f54b5e51c0 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -138,7 +138,7 @@ protected UTF8String(MemoryBlock base, int numBytes) { // for serialization public UTF8String() { - this((MemoryBlock)null,0); + this(null,0); } /** @@ -150,10 +150,6 @@ public void writeToMemory(byte[] target, long targetOffset) { base.writeTo(base.getBaseOffset(), target, targetOffset, numBytes); } - public void writeToMemory(MemoryBlock target, long targetOffset) { - MemoryBlock.copyMemory(base, base.getBaseOffset(), target, targetOffset, numBytes); - } - public void writeTo(ByteBuffer buffer) { assert(buffer.hasArray()); byte[] target = buffer.array(); diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java index 52398e3d49635..d13f37b996e06 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java @@ -37,7 +37,7 @@ private void check(MemoryBlock memory, Object obj, long offset, int length) { memory.putLong(offset + 8, -1L); memory.putFloat(offset + 16, 1.0F); memory.putDouble(offset + 20, 2.0); - MemoryBlock.copyMemory(memory, offset, memory, offset + 28, 4); + MemoryBlock.copyMemory(memory, 0L, memory, 28, 4); int[] a = new int[2]; a[0] = 0x12345678; a[1] = 0x13579BDF; diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java index 3b174a45d7b0a..039f496e9fbba 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java @@ -518,7 +518,7 @@ public void writeToOutputStreamUnderflow() throws IOException { for (int i = 1; i <= Platform.BYTE_ARRAY_OFFSET; ++i) { UTF8String.fromAddress(ByteArrayMemoryBlock.fromArray(test), - Platform.BYTE_ARRAY_OFFSET - i, test.length + i) + - i, test.length + i) .writeTo(outputStream); final ByteBuffer buffer = ByteBuffer.wrap(outputStream.toByteArray(), i, test.length); assertEquals("01234567", StandardCharsets.UTF_8.decode(buffer).toString()); @@ -533,8 +533,7 @@ public void writeToOutputStreamSlice() throws IOException { for (int i = 0; i < test.length; ++i) { for (int j = 0; j < test.length - i; ++j) { - UTF8String.fromAddress(ByteArrayMemoryBlock.fromArray(test), - Platform.BYTE_ARRAY_OFFSET + i, j) + UTF8String.fromAddress(ByteArrayMemoryBlock.fromArray(test), i, j) .writeTo(outputStream); assertArrayEquals(Arrays.copyOfRange(test, i, i + j), outputStream.toByteArray()); @@ -565,8 +564,7 @@ public void writeToOutputStreamOverflow() throws IOException { for (final long offset : offsets) { try { - fromAddress(ByteArrayMemoryBlock.fromArray(test), - BYTE_ARRAY_OFFSET + offset, test.length) + fromAddress(ByteArrayMemoryBlock.fromArray(test), offset, test.length) .writeTo(outputStream); throw new IllegalStateException(Long.toString(offset)); @@ -610,7 +608,7 @@ public void writeToOutputStreamLongArray() throws IOException { } final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - fromAddress(OnHeapMemoryBlock.fromArray(array), Platform.LONG_ARRAY_OFFSET, length) + fromAddress(OnHeapMemoryBlock.fromArray(array), 0, length) .writeTo(outputStream); assertEquals("3千大千世界", outputStream.toString("UTF-8")); } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java index 5f253ee9a4fbb..61b332adc174a 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java @@ -173,9 +173,7 @@ public ShuffleSorterIterator getSortedIterator() { PackedRecordPointer.PARTITION_ID_START_BYTE_INDEX, PackedRecordPointer.PARTITION_ID_END_BYTE_INDEX, false, false); } else { - MemoryBlock unused = array.memoryBlock().allocate( - array.getBaseOffset() + pos * 8L, - (array.size() - pos) * 8L); + MemoryBlock unused = array.memoryBlock().allocate(pos * 8L,(array.size() - pos) * 8L); LongArray buffer = new LongArray(unused); Sorter sorter = new Sorter<>(new ShuffleSortDataFormat(buffer)); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java index 8dbe06905ef57..086e169a42c59 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java @@ -60,13 +60,8 @@ public void copyElement(LongArray src, int srcPos, LongArray dst, int dstPos) { @Override public void copyRange(LongArray src, int srcPos, LongArray dst, int dstPos, int length) { - MemoryBlock.copyMemory( - src.memoryBlock(), - src.getBaseOffset() + srcPos * 8L, - dst.memoryBlock(), - dst.getBaseOffset() + dstPos * 8L, - length * 8L - ); + MemoryBlock.copyMemory(src.memoryBlock(),srcPos * 8L, + dst.memoryBlock(),dstPos * 8L,length * 8L); } @Override diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index afa210aaafc66..638384caebeb8 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -342,9 +342,7 @@ public UnsafeSorterIterator getSortedIterator() { array, nullBoundaryPos, (pos - nullBoundaryPos) / 2L, 0, 7, radixSortSupport.sortDescending(), radixSortSupport.sortSigned()); } else { - MemoryBlock unused = array.memoryBlock().allocate( - array.getBaseOffset() + pos * 8L, - (array.size() - pos) * 8L); + MemoryBlock unused = array.memoryBlock().allocate(pos * 8L,(array.size() - pos) * 8L); LongArray buffer = new LongArray(unused); Sorter sorter = new Sorter<>(new UnsafeSortDataFormat(buffer)); diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java index fe6f310bad90c..388045245017e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -209,7 +209,7 @@ public byte[] getBytes(int rowId, int count) { @Override protected UTF8String getBytesAsUTF8String(int rowId, int count) { - return UTF8String.fromAddress(data, data.getBaseOffset() + rowId, count); + return UTF8String.fromAddress(data, rowId, count); } // diff --git a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java index 80b189d820166..0f60071a07d49 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java @@ -381,7 +381,7 @@ final UTF8String getUTF8String(int rowId) { } else { mb.setAddressAndSize(stringResult.buffer.memoryAddress(), stringResult.buffer.capacity()); return UTF8String.fromAddress(mb, - stringResult.buffer.memoryAddress() + stringResult.start, + stringResult.start, stringResult.end - stringResult.start); } } From 8cd4853f611ab50395a8255b7316f147815f1bec Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 25 Feb 2018 09:47:20 +0000 Subject: [PATCH 33/65] provide generic copyFrom() and writeTo() in MemoryBlock remove MemoryBlock offset at caller site for copyFrom() and writeTo() --- .../unsafe/memory/ByteArrayMemoryBlock.java | 72 ------------------- .../spark/unsafe/memory/MemoryBlock.java | 28 ++------ .../unsafe/memory/OffHeapMemoryBlock.java | 72 ------------------- .../unsafe/memory/OnHeapMemoryBlock.java | 72 ------------------- .../apache/spark/unsafe/types/UTF8String.java | 39 +++++----- .../spark/unsafe/memory/MemoryBlockSuite.java | 4 +- .../vectorized/OffHeapColumnVector.java | 41 ++++++----- 7 files changed, 46 insertions(+), 282 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java index 4a6d1656150e4..7a0ddc1f81e0b 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -115,76 +115,4 @@ public final void putDouble(long offset, double value) { assert(offset + 8 - Platform.BYTE_ARRAY_OFFSET <= array.length); Platform.putDouble(array, offset, value); } - - public final void copyFrom(byte[] src, long srcOffset, long dstOffset, long length) { - assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= src.length); - assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); - Platform.copyMemory(src, srcOffset, array, dstOffset, length); - } - - public final void copyFrom(short[] src, long srcOffset, long dstOffset, long length) { - assert(srcOffset - Platform.SHORT_ARRAY_OFFSET + length <= src.length * 2); - assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); - Platform.copyMemory(src, srcOffset, array, dstOffset, length); - } - - public final void copyFrom(int[] src, long srcOffset, long dstOffset, long length) { - assert(srcOffset - Platform.INT_ARRAY_OFFSET + length <= src.length * 4); - assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); - Platform.copyMemory(src, srcOffset, array, dstOffset, length); - } - - public final void copyFrom(long[] src, long srcOffset, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= src.length * 8); - assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); - Platform.copyMemory(src, srcOffset, array, dstOffset, length); - } - - public final void copyFrom(float[] src, long srcOffset, long dstOffset, long length) { - assert(srcOffset - Platform.FLOAT_ARRAY_OFFSET + length <= src.length * 4); - assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); - Platform.copyMemory(src, srcOffset, array, dstOffset, length); - } - - public final void copyFrom(double[] src, long srcOffset, long dstOffset, long length) { - assert(srcOffset - Platform.DOUBLE_ARRAY_OFFSET + length <= src.length * 8); - assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length); - Platform.copyMemory(src, srcOffset, array, dstOffset, length); - } - - public final void writeTo(long srcOffset, byte[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); - assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= dst.length); - Platform.copyMemory(array, srcOffset, dst, dstOffset, length); - } - - public final void writeTo(long srcOffset, short[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); - assert(dstOffset - Platform.SHORT_ARRAY_OFFSET + length <= dst.length * 2); - Platform.copyMemory(array, srcOffset, dst, dstOffset, length); - } - - public final void writeTo(long srcOffset, int[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); - assert(dstOffset - Platform.INT_ARRAY_OFFSET + length <= dst.length * 4); - Platform.copyMemory(array, srcOffset, dst, dstOffset, length); - } - - public final void writeTo(long srcOffset, long[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); - assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= dst.length * 8); - Platform.copyMemory(array, srcOffset, dst, dstOffset, length); - } - - public final void writeTo(long srcOffset, float[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); - assert(dstOffset - Platform.FLOAT_ARRAY_OFFSET + length <= dst.length * 4); - Platform.copyMemory(array, srcOffset, dst, dstOffset, length); - } - - public final void writeTo(long srcOffset, double[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= array.length); - assert(dstOffset - Platform.DOUBLE_ARRAY_OFFSET + length <= dst.length * 8); - Platform.copyMemory(array, srcOffset, dst, dstOffset, length); - } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index ddb049e86bd7e..3c3d0d8f5ec18 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -169,27 +169,11 @@ public static final void copyMemory(MemoryBlock src, MemoryBlock dst, long lengt dst.getBaseObject(), dst.getBaseOffset(), length); } - public abstract void copyFrom(byte[] src, long srcOffset, long dstOffset, long length); - - public abstract void copyFrom(short[] src, long srcOffset, long dstOffset, long length); - - public abstract void copyFrom(int[] src, long srcOffset, long dstOffset, long length); - - public abstract void copyFrom(long[] src, long srcOffset, long dstOffset, long length); - - public abstract void copyFrom(float[] src, long srcOffset, long dstOffset, long length); - - public abstract void copyFrom(double[] src, long srcOffset, long dstOffset, long length); - - public abstract void writeTo(long srcOffset, byte[] dst, long dstOffset, long length); - - public abstract void writeTo(long srcOffset, short[] dst, long dstOffset, long length); - - public abstract void writeTo(long srcOffset, int[] dst, long dstOffset, long length); - - public abstract void writeTo(long srcOffset, long[] dst, long dstOffset, long length); - - public abstract void writeTo(long srcOffset, float[] dst, long dstOffset, long length); + public final void copyFrom(Object src, long srcOffset, long dstOffset, long length) { + Platform.copyMemory(src, srcOffset, obj, offset + dstOffset, length); + } - public abstract void writeTo(long srcOffset, double[] dst, long dstOffset, long length); + public final void writeTo(long srcOffset, Object dst, long dstOffset, long length) { + Platform.copyMemory(obj, offset + srcOffset, dst, dstOffset, length); + } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java index 4fea544126218..a87cd52a84f00 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java @@ -105,76 +105,4 @@ public final void putDouble(long offset, double value) { assert(offset + 8 <= this.offset + this.length); Platform.putDouble(null, offset, value); } - - public final void copyFrom(byte[] src, long srcOffset, long dstOffset, long length) { - assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= src.length); - assert(dstOffset + length <= this.offset + this.length); - Platform.copyMemory(src, srcOffset, null, dstOffset, length); - } - - public final void copyFrom(short[] src, long srcOffset, long dstOffset, long length) { - assert(srcOffset - Platform.SHORT_ARRAY_OFFSET + length <= src.length * 2); - assert(dstOffset + length <= this.offset + this.length); - Platform.copyMemory(src, srcOffset, null, dstOffset, length); - } - - public final void copyFrom(int[] src, long srcOffset, long dstOffset, long length) { - assert(srcOffset - Platform.INT_ARRAY_OFFSET + length <= src.length * 4); - assert(dstOffset + length <= this.offset + this.length); - Platform.copyMemory(src, srcOffset, null, dstOffset, length); - } - - public final void copyFrom(long[] src, long srcOffset, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= src.length * 8); - assert(dstOffset + length <= this.offset + this.length); - Platform.copyMemory(src, srcOffset, null, dstOffset, length); - } - - public final void copyFrom(float[] src, long srcOffset, long dstOffset, long length) { - assert(srcOffset - Platform.FLOAT_ARRAY_OFFSET + length <= src.length * 4); - assert(dstOffset + length <= this.offset + this.length); - Platform.copyMemory(src, srcOffset, null, dstOffset, length); - } - - public final void copyFrom(double[] src, long srcOffset, long dstOffset, long length) { - assert(srcOffset - Platform.DOUBLE_ARRAY_OFFSET + length <= src.length * 8); - assert(dstOffset + length <= this.offset + this.length); - Platform.copyMemory(src, srcOffset, null, dstOffset, length); - } - - public final void writeTo(long srcOffset, byte[] dst, long dstOffset, long length) { - assert(srcOffset + length <= this.offset + this.length); - assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= dst.length); - Platform.copyMemory(null, srcOffset, dst, dstOffset, length); - } - - public final void writeTo(long srcOffset, short[] dst, long dstOffset, long length) { - assert(srcOffset + length <= this.offset + this.length); - assert(dstOffset - Platform.SHORT_ARRAY_OFFSET + length <= dst.length * 2); - Platform.copyMemory(null, srcOffset, dst, dstOffset, length); - } - - public final void writeTo(long srcOffset, int[] dst, long dstOffset, long length) { - assert(srcOffset + length <= this.offset + this.length); - assert(dstOffset - Platform.INT_ARRAY_OFFSET + length <= dst.length * 4); - Platform.copyMemory(null, srcOffset, dst, dstOffset, length); - } - - public final void writeTo(long srcOffset, long[] dst, long dstOffset, long length) { - assert(srcOffset + length <= this.offset + this.length); - assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= dst.length * 8); - Platform.copyMemory(null, srcOffset, dst, dstOffset, length); - } - - public final void writeTo(long srcOffset, float[] dst, long dstOffset, long length) { - assert(srcOffset + length <= this.offset + this.length); - assert(dstOffset - Platform.FLOAT_ARRAY_OFFSET + length <= dst.length * 4); - Platform.copyMemory(null, srcOffset, dst, dstOffset, length); - } - - public final void writeTo(long srcOffset, double[] dst, long dstOffset, long length) { - assert(srcOffset + length <= this.offset + this.length); - assert(dstOffset - Platform.DOUBLE_ARRAY_OFFSET + length <= dst.length * 8); - Platform.copyMemory(null, srcOffset, dst, dstOffset, length); - } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java index 87003b04ef88e..47cafd2d60ccc 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java @@ -119,76 +119,4 @@ public final void putDouble(long offset, double value) { assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); Platform.putDouble(array, offset, value); } - - public final void copyFrom(byte[] src, long srcOffset, long dstOffset, long length) { - assert(srcOffset - Platform.BYTE_ARRAY_OFFSET + length <= src.length); - assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); - Platform.copyMemory(src, srcOffset, array, dstOffset, length); - } - - public final void copyFrom(short[] src, long srcOffset, long dstOffset, long length) { - assert(srcOffset - Platform.SHORT_ARRAY_OFFSET + length <= src.length * 2); - assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); - Platform.copyMemory(src, srcOffset, array, dstOffset, length); - } - - public final void copyFrom(int[] src, long srcOffset, long dstOffset, long length) { - assert(srcOffset - Platform.INT_ARRAY_OFFSET + length <= src.length * 4); - assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); - Platform.copyMemory(src, srcOffset, array, dstOffset, length); - } - - public final void copyFrom(long[] src, long srcOffset, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= src.length * 8); - assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); - Platform.copyMemory(src, srcOffset, array, dstOffset, length); - } - - public final void copyFrom(float[] src, long srcOffset, long dstOffset, long length) { - assert(srcOffset - Platform.FLOAT_ARRAY_OFFSET + length <= src.length * 4); - assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); - Platform.copyMemory(src, srcOffset, array, dstOffset, length); - } - - public final void copyFrom(double[] src, long srcOffset, long dstOffset, long length) { - assert(srcOffset - Platform.DOUBLE_ARRAY_OFFSET + length <= src.length * 8); - assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); - Platform.copyMemory(src, srcOffset, array, dstOffset, length); - } - - public final void writeTo(long srcOffset, byte[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); - assert(dstOffset - Platform.BYTE_ARRAY_OFFSET + length <= dst.length); - Platform.copyMemory(array, srcOffset, dst, dstOffset, length); - } - - public final void writeTo(long srcOffset, short[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); - assert(dstOffset - Platform.SHORT_ARRAY_OFFSET + length <= dst.length * 2); - Platform.copyMemory(array, srcOffset, dst, dstOffset, length); - } - - public final void writeTo(long srcOffset, int[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); - assert(dstOffset - Platform.INT_ARRAY_OFFSET + length <= dst.length * 4); - Platform.copyMemory(array, srcOffset, dst, dstOffset, length); - } - - public final void writeTo(long srcOffset, long[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); - assert(dstOffset - Platform.LONG_ARRAY_OFFSET + length <= dst.length * 8); - Platform.copyMemory(array, srcOffset, dst, dstOffset, length); - } - - public final void writeTo(long srcOffset, float[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); - assert(dstOffset - Platform.FLOAT_ARRAY_OFFSET + length <= dst.length * 4); - Platform.copyMemory(array, srcOffset, dst, dstOffset, length); - } - - public final void writeTo(long srcOffset, double[] dst, long dstOffset, long length) { - assert(srcOffset - Platform.LONG_ARRAY_OFFSET + length <= array.length * 8); - assert(dstOffset - Platform.DOUBLE_ARRAY_OFFSET + length <= dst.length * 8); - Platform.copyMemory(array, srcOffset, dst, dstOffset, length); - } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index b45f54b5e51c0..5a24e4b0099ff 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -147,7 +147,7 @@ public UTF8String() { * bytes in this string. */ public void writeToMemory(byte[] target, long targetOffset) { - base.writeTo(base.getBaseOffset(), target, targetOffset, numBytes); + base.writeTo(0, target, targetOffset, numBytes); } public void writeTo(ByteBuffer buffer) { @@ -276,7 +276,7 @@ public byte[] getBytes() { return ((ByteArrayMemoryBlock) base).getByteArray(); } else { byte[] bytes = new byte[numBytes]; - base.writeTo(offset, bytes, BYTE_ARRAY_OFFSET, numBytes); + base.writeTo(0, bytes, BYTE_ARRAY_OFFSET, numBytes); return bytes; } } @@ -306,7 +306,7 @@ public UTF8String substring(final int start, final int until) { if (i > j) { byte[] bytes = new byte[i - j]; - base.writeTo(base.getBaseOffset() + j, bytes, BYTE_ARRAY_OFFSET, i - j); + base.writeTo(j, bytes, BYTE_ARRAY_OFFSET, i - j); return fromBytes(bytes); } else { return EMPTY_UTF8; @@ -508,7 +508,7 @@ public int findInSet(UTF8String match) { private UTF8String copyUTF8String(int start, int end) { int len = end - start + 1; byte[] newBytes = new byte[len]; - base.writeTo(getBaseOffset() + start, newBytes, BYTE_ARRAY_OFFSET, len); + base.writeTo(start, newBytes, BYTE_ARRAY_OFFSET, len); return UTF8String.fromBytes(newBytes); } @@ -652,11 +652,10 @@ public UTF8String trimRight(UTF8String trimString) { public UTF8String reverse() { byte[] result = new byte[this.numBytes]; - long offset = getBaseOffset(); int i = 0; // position in byte while (i < numBytes) { int len = numBytesForFirstByte(getByte(i)); - base.writeTo(offset + i, result, BYTE_ARRAY_OFFSET + result.length - i - len, len); + base.writeTo(i, result, BYTE_ARRAY_OFFSET + result.length - i - len, len); i += len; } @@ -670,7 +669,7 @@ public UTF8String repeat(int times) { } byte[] newBytes = new byte[numBytes * times]; - base.writeTo(this.getBaseOffset(), newBytes, BYTE_ARRAY_OFFSET, numBytes); + base.writeTo(0, newBytes, BYTE_ARRAY_OFFSET, numBytes); int copied = 1; while (copied < times) { @@ -776,7 +775,7 @@ public UTF8String subStringIndex(UTF8String delim, int count) { return EMPTY_UTF8; } byte[] bytes = new byte[idx]; - base.writeTo(getBaseOffset(), bytes, BYTE_ARRAY_OFFSET, idx); + base.writeTo(0, bytes, BYTE_ARRAY_OFFSET, idx); return fromBytes(bytes); } else { @@ -796,7 +795,7 @@ public UTF8String subStringIndex(UTF8String delim, int count) { } int size = numBytes - delim.numBytes - idx; byte[] bytes = new byte[size]; - base.writeTo(getBaseOffset() + idx + delim.numBytes, bytes, BYTE_ARRAY_OFFSET, size); + base.writeTo(idx + delim.numBytes, bytes, BYTE_ARRAY_OFFSET, size); return fromBytes(bytes); } } @@ -819,16 +818,15 @@ public UTF8String rpad(int len, UTF8String pad) { UTF8String remain = pad.substring(0, spaces - padChars * count); byte[] data = new byte[this.numBytes + pad.numBytes * count + remain.numBytes]; - base.writeTo(this.getBaseOffset(), data, BYTE_ARRAY_OFFSET, this.numBytes); + base.writeTo(0, data, BYTE_ARRAY_OFFSET, this.numBytes); int offset = this.numBytes; int idx = 0; - long padOffset = pad.getBaseOffset(); while (idx < count) { - pad.base.writeTo(padOffset, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); + pad.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); ++ idx; offset += pad.numBytes; } - remain.base.writeTo(remain.getBaseOffset(), data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); + remain.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); return UTF8String.fromBytes(data); } @@ -855,15 +853,14 @@ public UTF8String lpad(int len, UTF8String pad) { int offset = 0; int idx = 0; - long padOffset = pad.getBaseOffset(); while (idx < count) { - pad.base.writeTo(padOffset, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); + pad.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); ++ idx; offset += pad.numBytes; } - remain.base.writeTo(remain.getBaseOffset(), data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); + remain.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); offset += remain.numBytes; - base.writeTo(this.getBaseOffset(), data, BYTE_ARRAY_OFFSET + offset, numBytes()); + base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, numBytes()); return UTF8String.fromBytes(data); } @@ -889,7 +886,7 @@ public static UTF8String concat(UTF8String... inputs) { for (int i = 0; i < inputs.length; i++) { int len = inputs[i].numBytes; inputs[i].base.writeTo( - inputs[i].getBaseOffset(), + 0, result, BYTE_ARRAY_OFFSET + offset, len); offset += len; @@ -929,7 +926,7 @@ public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) { if (inputs[i] != null) { int len = inputs[i].numBytes; inputs[i].base.writeTo( - inputs[i].getBaseOffset(), + 0, result, BYTE_ARRAY_OFFSET + offset, len); offset += len; @@ -938,7 +935,7 @@ public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) { // Add separator if this is not the last input. if (j < numInputs) { separator.base.writeTo( - separator.getBaseOffset(), + 0, result, BYTE_ARRAY_OFFSET + offset, separator.numBytes); offset += separator.numBytes; @@ -1212,7 +1209,7 @@ public UTF8String clone() { public UTF8String copy() { byte[] bytes = new byte[numBytes]; - base.writeTo(getBaseOffset(), bytes, BYTE_ARRAY_OFFSET, numBytes); + base.writeTo(0, bytes, BYTE_ARRAY_OFFSET, numBytes); return fromBytes(bytes); } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java index d13f37b996e06..97146dbf8432b 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java @@ -41,9 +41,9 @@ private void check(MemoryBlock memory, Object obj, long offset, int length) { int[] a = new int[2]; a[0] = 0x12345678; a[1] = 0x13579BDF; - memory.copyFrom(a, Platform.INT_ARRAY_OFFSET, offset + 32, 8); + memory.copyFrom(a, Platform.INT_ARRAY_OFFSET, 32, 8); byte[] b = new byte[8]; - memory.writeTo(offset + 32, b, Platform.BYTE_ARRAY_OFFSET, 8); + memory.writeTo(32, b, Platform.BYTE_ARRAY_OFFSET, 8); Assert.assertEquals(obj, memory.getBaseObject()); Assert.assertEquals(offset, memory.getBaseOffset()); diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java index 388045245017e..a3067e16d40f8 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -25,7 +25,6 @@ import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.types.UTF8String; import org.apache.spark.unsafe.memory.MemoryAllocator; -import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.unsafe.memory.OffHeapMemoryBlock; /** @@ -187,7 +186,7 @@ public void putBytes(int rowId, int count, byte value) { @Override public void putBytes(int rowId, int count, byte[] src, int srcIndex) { - data.copyFrom(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, data.getBaseOffset() + rowId, count); + data.copyFrom(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, rowId, count); } @Override @@ -203,7 +202,7 @@ public byte getByte(int rowId) { public byte[] getBytes(int rowId, int count) { assert(dictionary == null); byte[] array = new byte[count]; - data.writeTo(data.getBaseOffset() + rowId, array, Platform.BYTE_ARRAY_OFFSET, count); + data.writeTo(rowId, array, Platform.BYTE_ARRAY_OFFSET, count); return array; } @@ -232,13 +231,13 @@ public void putShorts(int rowId, int count, short value) { @Override public void putShorts(int rowId, int count, short[] src, int srcIndex) { data.copyFrom(src, Platform.SHORT_ARRAY_OFFSET + srcIndex * 2, - data.getBaseOffset() + 2 * rowId, count * 2); + 2 * rowId, count * 2); } @Override public void putShorts(int rowId, int count, byte[] src, int srcIndex) { - Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - null, data.getBaseOffset() + rowId * 2, count * 2); + data.copyFrom(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + rowId * 2, count * 2); } @Override @@ -254,7 +253,7 @@ public short getShort(int rowId) { public short[] getShorts(int rowId, int count) { assert(dictionary == null); short[] array = new short[count]; - data.writeTo(data.getBaseOffset() + rowId * 2, array, Platform.SHORT_ARRAY_OFFSET, count * 2); + data.writeTo(rowId * 2, array, Platform.SHORT_ARRAY_OFFSET, count * 2); return array; } @@ -278,20 +277,20 @@ public void putInts(int rowId, int count, int value) { @Override public void putInts(int rowId, int count, int[] src, int srcIndex) { data.copyFrom(src, Platform.INT_ARRAY_OFFSET + srcIndex * 4, - data.getBaseOffset() + 4 * rowId, count * 4); + 4 * rowId, count * 4); } @Override public void putInts(int rowId, int count, byte[] src, int srcIndex) { data.copyFrom(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - data.getBaseOffset() + rowId * 4, count * 4); + rowId * 4, count * 4); } @Override public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { data.copyFrom(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, - data.getBaseOffset() + 4 * rowId, count * 4); + 4 * rowId, count * 4); } else { int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; long offset = data.getBaseOffset() + 4 * rowId; @@ -314,7 +313,7 @@ public int getInt(int rowId) { public int[] getInts(int rowId, int count) { assert(dictionary == null); int[] array = new int[count]; - data.writeTo(data.getBaseOffset() + rowId * 4, array, Platform.INT_ARRAY_OFFSET, count * 4); + data.writeTo(rowId * 4, array, Platform.INT_ARRAY_OFFSET, count * 4); return array; } @@ -349,20 +348,20 @@ public void putLongs(int rowId, int count, long value) { @Override public void putLongs(int rowId, int count, long[] src, int srcIndex) { data.copyFrom(src, Platform.LONG_ARRAY_OFFSET + srcIndex * 8, - data.getBaseOffset() + 8 * rowId, count * 8); + 8 * rowId, count * 8); } @Override public void putLongs(int rowId, int count, byte[] src, int srcIndex) { data.copyFrom(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - data.getBaseOffset() + rowId * 8, count * 8); + rowId * 8, count * 8); } @Override public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { data.copyFrom(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, - data.getBaseOffset() + 8 * rowId, count * 8); + 8 * rowId, count * 8); } else { int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; long offset = data.getBaseOffset() + 8 * rowId; @@ -385,7 +384,7 @@ public long getLong(int rowId) { public long[] getLongs(int rowId, int count) { assert(dictionary == null); long[] array = new long[count]; - data.writeTo(data.getBaseOffset() + rowId * 8, array, Platform.LONG_ARRAY_OFFSET, count * 8); + data.writeTo(rowId * 8, array, Platform.LONG_ARRAY_OFFSET, count * 8); return array; } @@ -409,14 +408,14 @@ public void putFloats(int rowId, int count, float value) { @Override public void putFloats(int rowId, int count, float[] src, int srcIndex) { data.copyFrom(src, Platform.FLOAT_ARRAY_OFFSET + srcIndex * 4, - data.getBaseOffset() + 4 * rowId, count * 4); + 4 * rowId, count * 4); } @Override public void putFloats(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { data.copyFrom(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - data.getBaseOffset() + rowId * 4, count * 4); + rowId * 4, count * 4); } else { ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); long offset = data.getBaseOffset() + 4 * rowId; @@ -439,7 +438,7 @@ public float getFloat(int rowId) { public float[] getFloats(int rowId, int count) { assert(dictionary == null); float[] array = new float[count]; - data.writeTo(data.getBaseOffset() + rowId * 4, array, Platform.FLOAT_ARRAY_OFFSET, count * 4); + data.writeTo(rowId * 4, array, Platform.FLOAT_ARRAY_OFFSET, count * 4); return array; } @@ -464,14 +463,14 @@ public void putDoubles(int rowId, int count, double value) { @Override public void putDoubles(int rowId, int count, double[] src, int srcIndex) { data.copyFrom(src, Platform.DOUBLE_ARRAY_OFFSET + srcIndex * 8, - data.getBaseOffset() + 8 * rowId, count * 8); + 8 * rowId, count * 8); } @Override public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { data.copyFrom(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - data.getBaseOffset() + rowId * 8, count * 8); + rowId * 8, count * 8); } else { ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); long offset = data.getBaseOffset() + 8 * rowId; @@ -494,7 +493,7 @@ public double getDouble(int rowId) { public double[] getDoubles(int rowId, int count) { assert(dictionary == null); double[] array = new double[count]; - data.writeTo(data.getBaseOffset() + rowId * 8, array, Platform.DOUBLE_ARRAY_OFFSET, count * 8); + data.writeTo(rowId * 8, array, Platform.DOUBLE_ARRAY_OFFSET, count * 8); return array; } From 1bed04800beec4b7f51cae0032aea4e956b80423 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Mon, 26 Feb 2018 16:04:07 +0000 Subject: [PATCH 34/65] address review comments --- .../unsafe/memory/UnsafeMemoryAllocator.java | 3 - .../apache/spark/unsafe/types/UTF8String.java | 171 ++++++++++-------- 2 files changed, 97 insertions(+), 77 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java index 45d84954706e2..d06b6a855d418 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -32,9 +32,6 @@ public class UnsafeMemoryAllocator implements MemoryAllocator { @Override public OffHeapMemoryBlock allocate(long size) throws OutOfMemoryError { - // No usage of DirectByteBuffer.allocateDirect is current design - // Platform.allocateMemory is used here. - // http://downloads.typesafe.com/website/presentations/ScalaDaysSF2015/T4_Xin_Performance_Optimization.pdf#page=26 long address = Platform.allocateMemory(size); OffHeapMemoryBlock memory = new OffHeapMemoryBlock(address, size); if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 5a24e4b0099ff..668d860e7d4c4 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -53,7 +53,6 @@ public final class UTF8String implements Comparable, Externalizable, // These are only updated by readExternal() or read() @Nonnull private MemoryBlock base; - private int numBytes; public MemoryBlock getBaseObject() { return base; } public long getBaseOffset() { return base.getBaseOffset(); } @@ -101,14 +100,14 @@ public static UTF8String fromBytes(byte[] bytes, int offset, int numBytes) { * Creates an UTF8String from given address (base and offset) and length. */ public static UTF8String fromAddress(MemoryBlock base, long offset, int numBytes) { - return new UTF8String(base.allocate(offset, numBytes), numBytes); + return new UTF8String(base.allocate(offset, numBytes)); } /** * Creates an UTF8String from given memory block. */ public static UTF8String fromMemoryBlock(MemoryBlock base) { - return new UTF8String(base, (int)base.size()); + return new UTF8String(base); } /** @@ -128,17 +127,16 @@ public static UTF8String blankString(int length) { } protected UTF8String(byte[] bytes, long offset, int numBytes) { - this(new ByteArrayMemoryBlock(bytes, offset, numBytes), numBytes); + this(new ByteArrayMemoryBlock(bytes, offset, numBytes)); } - protected UTF8String(MemoryBlock base, int numBytes) { + protected UTF8String(MemoryBlock base) { this.base = base; - this.numBytes = numBytes; } // for serialization public UTF8String() { - this(null,0); + this(null); } /** @@ -147,7 +145,7 @@ public UTF8String() { * bytes in this string. */ public void writeToMemory(byte[] target, long targetOffset) { - base.writeTo(0, target, targetOffset, numBytes); + base.writeTo(0, target, targetOffset, numBytes()); } public void writeTo(ByteBuffer buffer) { @@ -156,7 +154,7 @@ public void writeTo(ByteBuffer buffer) { int offset = buffer.arrayOffset(); int pos = buffer.position(); writeToMemory(target, Platform.BYTE_ARRAY_OFFSET + offset + pos); - buffer.position(pos + numBytes); + buffer.position(pos + numBytes()); } /** @@ -176,11 +174,11 @@ public ByteBuffer getByteBuffer() { // verify that the offset and length points somewhere inside the byte array // and that the offset can safely be truncated to a 32-bit integer - if ((long) bytes.length < arrayOffset + numBytes) { + if ((long) bytes.length < arrayOffset + numBytes()) { throw new ArrayIndexOutOfBoundsException(); } - return ByteBuffer.wrap(bytes, (int) arrayOffset, numBytes); + return ByteBuffer.wrap(bytes, (int) arrayOffset, numBytes()); } else { return ByteBuffer.wrap(getBytes()); } @@ -207,7 +205,7 @@ private static int numBytesForFirstByte(final byte b) { * Returns the number of bytes */ public int numBytes() { - return numBytes; + return (int)base.size(); } /** @@ -215,7 +213,7 @@ public int numBytes() { */ public int numChars() { int len = 0; - for (int i = 0; i < numBytes; i += numBytesForFirstByte(getByte(i))) { + for (int i = 0; i < numBytes(); i += numBytesForFirstByte(getByte(i))) { len += 1; } return len; @@ -234,6 +232,7 @@ public long getPrefix() { long p; long mask = 0; long offset = base.getBaseOffset(); + int numBytes = numBytes(); if (IS_LITTLE_ENDIAN) { if (numBytes >= 8) { p = base.getLong(offset); @@ -272,11 +271,11 @@ public byte[] getBytes() { // avoid copy if `base` is `byte[]` long offset = base.getBaseOffset(); if (offset == BYTE_ARRAY_OFFSET && base instanceof ByteArrayMemoryBlock - && (((ByteArrayMemoryBlock) base).getByteArray()).length == numBytes) { + && (((ByteArrayMemoryBlock) base).getByteArray()).length == numBytes()) { return ((ByteArrayMemoryBlock) base).getByteArray(); } else { - byte[] bytes = new byte[numBytes]; - base.writeTo(0, bytes, BYTE_ARRAY_OFFSET, numBytes); + byte[] bytes = new byte[numBytes()]; + base.writeTo(0, bytes, BYTE_ARRAY_OFFSET, numBytes()); return bytes; } } @@ -287,6 +286,7 @@ public byte[] getBytes() { * @param until the position after last code point, exclusive. */ public UTF8String substring(final int start, final int until) { + int numBytes = numBytes(); if (until <= start || start >= numBytes) { return EMPTY_UTF8; } @@ -330,12 +330,12 @@ public UTF8String substringSQL(int pos, int length) { * Returns whether this contains `substring` or not. */ public boolean contains(final UTF8String substring) { - if (substring.numBytes == 0) { + if (substring.numBytes() == 0) { return true; } byte first = substring.getByte(0); - for (int i = 0; i <= numBytes - substring.numBytes; i++) { + for (int i = 0; i <= numBytes() - substring.numBytes(); i++) { if (getByte(i) == first && matchAt(substring, i)) { return true; } @@ -351,10 +351,10 @@ private byte getByte(int i) { } private boolean matchAt(final UTF8String s, int pos) { - if (s.numBytes + pos > numBytes || pos < 0) { + if (s.numBytes() + pos > numBytes() || pos < 0) { return false; } - return ByteArrayMethods.arrayEqualsBlock(base, getBaseOffset() + pos, s.base, s.getBaseOffset(), s.numBytes); + return ByteArrayMethods.arrayEqualsBlock(base, getBaseOffset() + pos, s.base, s.getBaseOffset(), s.numBytes()); } public boolean startsWith(final UTF8String prefix) { @@ -362,13 +362,14 @@ public boolean startsWith(final UTF8String prefix) { } public boolean endsWith(final UTF8String suffix) { - return matchAt(suffix, numBytes - suffix.numBytes); + return matchAt(suffix, numBytes() - suffix.numBytes()); } /** * Returns the upper case of this string */ public UTF8String toUpperCase() { + int numBytes = numBytes(); if (numBytes == 0) { return EMPTY_UTF8; } @@ -399,6 +400,7 @@ private UTF8String toUpperCaseSlow() { * Returns the lower case of this string */ public UTF8String toLowerCase() { + int numBytes = numBytes(); if (numBytes == 0) { return EMPTY_UTF8; } @@ -429,6 +431,7 @@ private UTF8String toLowerCaseSlow() { * Returns the title case of this string, that could be used as title. */ public UTF8String toTitleCase() { + int numBytes = numBytes(); if (numBytes == 0) { return EMPTY_UTF8; } @@ -477,23 +480,25 @@ public int findInSet(UTF8String match) { return 0; } + int numBytes = numBytes(); + int matchNumBytes = match.numBytes(); long offset = getBaseOffset(); long matchOffset = match.getBaseOffset(); int n = 1, lastComma = -1; for (int i = 0; i < numBytes; i++) { if (getByte(i) == (byte) ',') { - if (i - (lastComma + 1) == match.numBytes && + if (i - (lastComma + 1) == matchNumBytes && ByteArrayMethods.arrayEqualsBlock(base, offset + (lastComma + 1), match.base, matchOffset, - match.numBytes)) { + matchNumBytes)) { return n; } lastComma = i; n++; } } - if (numBytes - (lastComma + 1) == match.numBytes && + if (numBytes - (lastComma + 1) == matchNumBytes && ByteArrayMethods.arrayEqualsBlock(base, offset + (lastComma + 1), match.base, matchOffset, - match.numBytes)) { + matchNumBytes)) { return n; } return 0; @@ -513,15 +518,16 @@ private UTF8String copyUTF8String(int start, int end) { } public UTF8String trim() { + int numBytes = numBytes(); int s = 0; // skip all of the space (0x20) in the left side - while (s < this.numBytes && getByte(s) == 0x20) s++; - if (s == this.numBytes) { + while (s < numBytes && getByte(s) == 0x20) s++; + if (s == numBytes) { // empty string return EMPTY_UTF8; } // skip all of the space (0x20) in the right side - int e = this.numBytes - 1; + int e = numBytes - 1; while (e > s && getByte(e) == 0x20) e--; return copyUTF8String(s, e); } @@ -542,14 +548,15 @@ public UTF8String trim(UTF8String trimString) { } public UTF8String trimLeft() { + int numBytes = numBytes(); int s = 0; // skip all of the space (0x20) in the left side - while (s < this.numBytes && getByte(s) == 0x20) s++; - if (s == this.numBytes) { + while (s < numBytes && getByte(s) == 0x20) s++; + if (s == numBytes) { // empty string return EMPTY_UTF8; } else { - return copyUTF8String(s, this.numBytes - 1); + return copyUTF8String(s, numBytes - 1); } } @@ -562,6 +569,7 @@ public UTF8String trimLeft() { */ public UTF8String trimLeft(UTF8String trimString) { if (trimString == null) return null; + int numBytes = numBytes(); // the searching byte position in the source string int srchIdx = 0; // the first beginning byte position of a non-matching character @@ -570,7 +578,7 @@ public UTF8String trimLeft(UTF8String trimString) { while (srchIdx < numBytes) { UTF8String searchChar = copyUTF8String( srchIdx, srchIdx + numBytesForFirstByte(this.getByte(srchIdx)) - 1); - int searchCharBytes = searchChar.numBytes; + int searchCharBytes = searchChar.numBytes(); // try to find the matching for the searchChar in the trimString set if (trimString.find(searchChar, 0) >= 0) { trimIdx += searchCharBytes; @@ -590,7 +598,7 @@ public UTF8String trimLeft(UTF8String trimString) { } public UTF8String trimRight() { - int e = numBytes - 1; + int e = numBytes() - 1; // skip all of the space (0x20) in the right side while (e >= 0 && getByte(e) == 0x20) e--; @@ -611,6 +619,7 @@ public UTF8String trimRight() { */ public UTF8String trimRight(UTF8String trimString) { if (trimString == null) return null; + int numBytes = numBytes(); int charIdx = 0; // number of characters from the source string int numChars = 0; @@ -650,7 +659,8 @@ public UTF8String trimRight(UTF8String trimString) { } public UTF8String reverse() { - byte[] result = new byte[this.numBytes]; + int numBytes = numBytes(); + byte[] result = new byte[numBytes]; int i = 0; // position in byte while (i < numBytes) { @@ -668,6 +678,7 @@ public UTF8String repeat(int times) { return EMPTY_UTF8; } + int numBytes = numBytes(); byte[] newBytes = new byte[numBytes * times]; base.writeTo(0, newBytes, BYTE_ARRAY_OFFSET, numBytes); @@ -694,6 +705,8 @@ public int indexOf(UTF8String v, int start) { return 0; } + int numBytes = numBytes(); + int vNumBytes = v.numBytes(); // locate to the start position. int i = 0; // position in byte int c = 0; // position in character @@ -705,10 +718,10 @@ public int indexOf(UTF8String v, int start) { long offset = getBaseOffset(); long vOffset = v.getBaseOffset(); do { - if (i + v.numBytes > numBytes) { + if (i + vNumBytes > numBytes) { return -1; } - if (ByteArrayMethods.arrayEqualsBlock(base, offset + i, v.base, vOffset, v.numBytes)) { + if (ByteArrayMethods.arrayEqualsBlock(base, offset + i, v.base, vOffset, vNumBytes)) { return c; } i += numBytesForFirstByte(getByte(i)); @@ -722,11 +735,11 @@ public int indexOf(UTF8String v, int start) { * Find the `str` from left to right. */ private int find(UTF8String str, int start) { - assert (str.numBytes > 0); + assert (str.numBytes() > 0); long offset = getBaseOffset(); long strOffset = str.getBaseOffset(); - while (start <= numBytes - str.numBytes) { - if (ByteArrayMethods.arrayEqualsBlock(base, offset + start, str.base, strOffset, str.numBytes)) { + while (start <= numBytes() - str.numBytes()) { + if (ByteArrayMethods.arrayEqualsBlock(base, offset + start, str.base, strOffset, str.numBytes())) { return start; } start += 1; @@ -738,11 +751,11 @@ private int find(UTF8String str, int start) { * Find the `str` from right to left. */ private int rfind(UTF8String str, int start) { - assert (str.numBytes > 0); + assert (str.numBytes() > 0); long offset = getBaseOffset(); long strOffset = str.getBaseOffset(); while (start >= 0) { - if (ByteArrayMethods.arrayEqualsBlock(base, offset + start, str.base, strOffset, str.numBytes)) { + if (ByteArrayMethods.arrayEqualsBlock(base, offset + start, str.base, strOffset, str.numBytes())) { return start; } start -= 1; @@ -757,7 +770,9 @@ private int rfind(UTF8String str, int start) { * right) is returned. subStringIndex performs a case-sensitive match when searching for delim. */ public UTF8String subStringIndex(UTF8String delim, int count) { - if (delim.numBytes == 0 || count == 0) { + int numBytes = numBytes(); + int delimNumBytes = delim.numBytes(); + if (delimNumBytes == 0 || count == 0) { return EMPTY_UTF8; } if (count > 0) { @@ -779,7 +794,7 @@ public UTF8String subStringIndex(UTF8String delim, int count) { return fromBytes(bytes); } else { - int idx = numBytes - delim.numBytes + 1; + int idx = numBytes - delimNumBytes + 1; count = -count; while (count > 0) { idx = rfind(delim, idx - 1); @@ -790,12 +805,12 @@ public UTF8String subStringIndex(UTF8String delim, int count) { return this; } } - if (idx + delim.numBytes == numBytes) { + if (idx + delimNumBytes == numBytes) { return EMPTY_UTF8; } - int size = numBytes - delim.numBytes - idx; + int size = numBytes - delimNumBytes - idx; byte[] bytes = new byte[size]; - base.writeTo(idx + delim.numBytes, bytes, BYTE_ARRAY_OFFSET, size); + base.writeTo(idx + delimNumBytes, bytes, BYTE_ARRAY_OFFSET, size); return fromBytes(bytes); } } @@ -817,16 +832,19 @@ public UTF8String rpad(int len, UTF8String pad) { // the partial string of the padding UTF8String remain = pad.substring(0, spaces - padChars * count); - byte[] data = new byte[this.numBytes + pad.numBytes * count + remain.numBytes]; - base.writeTo(0, data, BYTE_ARRAY_OFFSET, this.numBytes); - int offset = this.numBytes; + int numBytes = numBytes(); + int padNumBytes = pad.numBytes(); + int remainNumBytes = remain.numBytes(); + byte[] data = new byte[numBytes + padNumBytes * count + remainNumBytes]; + base.writeTo(0, data, BYTE_ARRAY_OFFSET, numBytes); + int offset = numBytes; int idx = 0; while (idx < count) { - pad.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); + pad.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, padNumBytes); ++ idx; - offset += pad.numBytes; + offset += padNumBytes; } - remain.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); + remain.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, remainNumBytes); return UTF8String.fromBytes(data); } @@ -839,8 +857,9 @@ public UTF8String rpad(int len, UTF8String pad) { * ('hi', 1, '??') => 'h' */ public UTF8String lpad(int len, UTF8String pad) { + int padNumBytes = pad.numBytes(); int spaces = len - this.numChars(); // number of char need to pad - if (spaces <= 0 || pad.numBytes() == 0) { + if (spaces <= 0 || padNumBytes == 0) { // no padding at all, return the substring of the current string return substring(0, len); } else { @@ -848,18 +867,20 @@ public UTF8String lpad(int len, UTF8String pad) { int count = spaces / padChars; // how many padding string needed // the partial string of the padding UTF8String remain = pad.substring(0, spaces - padChars * count); + int remainNumBytes = remain.numBytes(); + int numBytes = numBytes(); - byte[] data = new byte[this.numBytes + pad.numBytes * count + remain.numBytes]; + byte[] data = new byte[numBytes + padNumBytes * count + remainNumBytes]; int offset = 0; int idx = 0; while (idx < count) { - pad.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); + pad.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, padNumBytes); ++ idx; - offset += pad.numBytes; + offset += padNumBytes; } - remain.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); - offset += remain.numBytes; + remain.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, remainNumBytes); + offset += remainNumBytes; base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, numBytes()); return UTF8String.fromBytes(data); @@ -874,7 +895,7 @@ public static UTF8String concat(UTF8String... inputs) { int totalLength = 0; for (int i = 0; i < inputs.length; i++) { if (inputs[i] != null) { - totalLength += inputs[i].numBytes; + totalLength += inputs[i].numBytes(); } else { return null; } @@ -884,7 +905,7 @@ public static UTF8String concat(UTF8String... inputs) { final byte[] result = new byte[totalLength]; int offset = 0; for (int i = 0; i < inputs.length; i++) { - int len = inputs[i].numBytes; + int len = inputs[i].numBytes(); inputs[i].base.writeTo( 0, result, BYTE_ARRAY_OFFSET + offset, @@ -907,7 +928,7 @@ public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) { int numInputs = 0; // number of non-null inputs for (int i = 0; i < inputs.length; i++) { if (inputs[i] != null) { - numInputBytes += inputs[i].numBytes; + numInputBytes += inputs[i].numBytes(); numInputs++; } } @@ -919,12 +940,12 @@ public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) { // Allocate a new byte array, and copy the inputs one by one into it. // The size of the new array is the size of all inputs, plus the separators. - final byte[] result = new byte[numInputBytes + (numInputs - 1) * separator.numBytes]; + final byte[] result = new byte[numInputBytes + (numInputs - 1) * separator.numBytes()]; int offset = 0; for (int i = 0, j = 0; i < inputs.length; i++) { if (inputs[i] != null) { - int len = inputs[i].numBytes; + int len = inputs[i].numBytes(); inputs[i].base.writeTo( 0, result, BYTE_ARRAY_OFFSET + offset, @@ -937,8 +958,8 @@ public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) { separator.base.writeTo( 0, result, BYTE_ARRAY_OFFSET + offset, - separator.numBytes); - offset += separator.numBytes; + separator.numBytes()); + offset += separator.numBytes(); } } } @@ -1012,6 +1033,7 @@ public static class IntWrapper implements Serializable { * @return true if the parsing was successful else false */ public boolean toLong(LongWrapper toLongResult) { + int numBytes = numBytes(); if (numBytes == 0) { return false; } @@ -1103,6 +1125,7 @@ public boolean toLong(LongWrapper toLongResult) { * @return true if the parsing was successful else false */ public boolean toInt(IntWrapper intWrapper) { + int numBytes = numBytes(); if (numBytes == 0) { return false; } @@ -1208,6 +1231,7 @@ public UTF8String clone() { } public UTF8String copy() { + int numBytes = numBytes(); byte[] bytes = new byte[numBytes]; base.writeTo(0, bytes, BYTE_ARRAY_OFFSET, numBytes); return fromBytes(bytes); @@ -1215,7 +1239,7 @@ public UTF8String copy() { @Override public int compareTo(@Nonnull final UTF8String other) { - int len = Math.min(numBytes, other.numBytes); + int len = Math.min(numBytes(), other.numBytes()); int wordMax = (len / 8) * 8; long offset = getBaseOffset(); long roffset = other.getBaseOffset(); @@ -1238,7 +1262,7 @@ public int compareTo(@Nonnull final UTF8String other) { return res; } } - return numBytes - other.numBytes; + return numBytes() - other.numBytes(); } public int compare(final UTF8String other) { @@ -1249,10 +1273,10 @@ public int compare(final UTF8String other) { public boolean equals(final Object other) { if (other instanceof UTF8String) { UTF8String o = (UTF8String) other; - if (numBytes != o.numBytes) { + if (numBytes() != o.numBytes()) { return false; } - return ByteArrayMethods.arrayEqualsBlock(base, getBaseOffset(), o.base, o.getBaseOffset(), numBytes); + return ByteArrayMethods.arrayEqualsBlock(base, getBaseOffset(), o.base, o.getBaseOffset(), numBytes()); } else { return false; } @@ -1326,7 +1350,7 @@ public int levenshteinDistance(UTF8String other) { @Override public int hashCode() { - return Murmur3_x86_32.hashUnsafeBytesBlock(base, getBaseOffset(), numBytes, 42); + return Murmur3_x86_32.hashUnsafeBytesBlock(base, getBaseOffset(), numBytes(), 42); } /** @@ -1341,6 +1365,7 @@ public int hashCode() { * https://en.wikipedia.org/wiki/Soundex */ public UTF8String soundex() { + int numBytes = numBytes(); if (numBytes == 0) { return EMPTY_UTF8; } @@ -1389,8 +1414,7 @@ public void writeExternal(ObjectOutput out) throws IOException { } public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - // offset = BYTE_ARRAY_OFFSET; - numBytes = in.readInt(); + int numBytes = in.readInt(); base = ByteArrayMemoryBlock.fromArray(new byte[numBytes]); in.readFully(((ByteArrayMemoryBlock)base).getByteArray()); } @@ -1404,8 +1428,7 @@ public void write(Kryo kryo, Output out) { @Override public void read(Kryo kryo, Input in) { - // this.offset = BYTE_ARRAY_OFFSET; - this.numBytes = in.readInt(); + int numBytes = in.readInt(); base = ByteArrayMemoryBlock.fromArray(new byte[numBytes]); in.read(((ByteArrayMemoryBlock)base).getByteArray()); } From c79585f401667bf2a57333e72042322a1e83b5dd Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 27 Feb 2018 08:30:45 +0000 Subject: [PATCH 35/65] address review comments --- .../java/org/apache/spark/unsafe/types/UTF8String.java | 4 +++- .../apache/spark/sql/vectorized/ArrowColumnVector.java | 8 ++++---- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 668d860e7d4c4..378d355680a8e 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -205,7 +205,9 @@ private static int numBytesForFirstByte(final byte b) { * Returns the number of bytes */ public int numBytes() { - return (int)base.size(); + long size = base.size(); + assert(size <= (long)Integer.MAX_VALUE); + return (int)size; } /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java index 0f60071a07d49..b871a85f70c89 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java @@ -379,10 +379,10 @@ final UTF8String getUTF8String(int rowId) { if (stringResult.isSet == 0) { return null; } else { - mb.setAddressAndSize(stringResult.buffer.memoryAddress(), stringResult.buffer.capacity()); - return UTF8String.fromAddress(mb, - stringResult.start, - stringResult.end - stringResult.start); + int size = stringResult.end - stringResult.start; + mb.setAddressAndSize( + stringResult.buffer.memoryAddress() + stringResult.start, size); + return UTF8String.fromAddress(mb, 0, size); } } } From 77cdb81a457bb6fe81f13af9df88c2998695645f Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 27 Feb 2018 16:09:07 +0000 Subject: [PATCH 36/65] fix indent --- .../main/java/org/apache/spark/unsafe/memory/MemoryBlock.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index 3c3d0d8f5ec18..1249a72632f7f 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -161,7 +161,7 @@ public final static MemoryBlock allocateFromObject(Object obj, long offset, long public static final void copyMemory( MemoryBlock src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { Platform.copyMemory(src.getBaseObject(), src.getBaseOffset() + srcOffset, - dst.getBaseObject(), dst.getBaseOffset() + dstOffset, length); + dst.getBaseObject(), dst.getBaseOffset() + dstOffset, length); } public static final void copyMemory(MemoryBlock src, MemoryBlock dst, long length) { From ee5a79861c18725fb1cd9b518cdfd2489c05b81d Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 27 Feb 2018 16:09:27 +0000 Subject: [PATCH 37/65] remove assert for performance --- .../spark/unsafe/memory/ByteArrayMemoryBlock.java | 14 -------------- .../spark/unsafe/memory/OffHeapMemoryBlock.java | 14 -------------- .../org/apache/spark/unsafe/types/UTF8String.java | 4 +--- 3 files changed, 1 insertion(+), 31 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java index 7a0ddc1f81e0b..67b6a7925f14e 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -47,23 +47,18 @@ public static ByteArrayMemoryBlock fromArray(final byte[] array) { public final int getInt(long offset) { - // UTF8String.getPrefix() assumes data is 4-byte aligned - assert(offset + 4 - Platform.BYTE_ARRAY_OFFSET <= ((array.length + 3) / 4) * 4); return Platform.getInt(array, offset); } public final void putInt(long offset, int value) { - assert(offset + 4 - Platform.BYTE_ARRAY_OFFSET <= array.length); Platform.putInt(array, offset, value); } public final boolean getBoolean(long offset) { - assert(offset + 1 - Platform.BYTE_ARRAY_OFFSET <= array.length); return Platform.getBoolean(array, offset); } public final void putBoolean(long offset, boolean value) { - assert(offset + 1 - Platform.BYTE_ARRAY_OFFSET <= array.length); Platform.putBoolean(array, offset, value); } @@ -76,43 +71,34 @@ public final void putByte(long offset, byte value) { } public final short getShort(long offset) { - assert(offset + 2 - Platform.BYTE_ARRAY_OFFSET <= array.length); return Platform.getShort(array, offset); } public final void putShort(long offset, short value) { - assert(offset + 2 - Platform.BYTE_ARRAY_OFFSET <= array.length); Platform.putShort(array, offset, value); } public final long getLong(long offset) { - // UTF8String.getPrefix() assumes data is 8-byte aligned - assert(offset + 8 - Platform.BYTE_ARRAY_OFFSET <= ((array.length + 7) / 8) * 8); return Platform.getLong(array, offset); } public final void putLong(long offset, long value) { - assert(offset + 8 - Platform.BYTE_ARRAY_OFFSET <= array.length); Platform.putLong(array, offset, value); } public final float getFloat(long offset) { - assert(offset + 4 - Platform.BYTE_ARRAY_OFFSET <= array.length); return Platform.getFloat(array, offset); } public final void putFloat(long offset, float value) { - assert(offset + 4 - Platform.BYTE_ARRAY_OFFSET <= array.length); Platform.putFloat(array, offset, value); } public final double getDouble(long offset) { - assert(offset + 8 - Platform.BYTE_ARRAY_OFFSET <= array.length); return Platform.getDouble(array, offset); } public final void putDouble(long offset, double value) { - assert(offset + 8 - Platform.BYTE_ARRAY_OFFSET <= array.length); Platform.putDouble(array, offset, value); } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java index a87cd52a84f00..4b812be8f0d50 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java @@ -37,72 +37,58 @@ public MemoryBlock allocate(long offset, long size) { } public final int getInt(long offset) { - assert(offset + 4 <= this.offset + this.length); return Platform.getInt(null, offset); } public final void putInt(long offset, int value) { - assert(offset + 4 <= this.offset + this.length); Platform.putInt(null, offset, value); } public final boolean getBoolean(long offset) { - assert(offset + 1 <= this.offset + this.length); return Platform.getBoolean(null, offset); } public final void putBoolean(long offset, boolean value) { - assert(offset + 1 <= this.offset + this.length); Platform.putBoolean(null, offset, value); } public final byte getByte(long offset) { - assert(offset + 1 <= this.offset + this.length); return Platform.getByte(null, offset); } public final void putByte(long offset, byte value) { - assert(offset + 1 <= this.offset + this.length); Platform.putByte(null, offset, value); } public final short getShort(long offset) { - assert(offset + 2 <= this.offset + this.length); return Platform.getShort(null, offset); } public final void putShort(long offset, short value) { - assert(offset + 2 <= this.offset + this.length); Platform.putShort(null, offset, value); } public final long getLong(long offset) { - assert(offset + 8 <= this.offset + this.length); return Platform.getLong(null, offset); } public final void putLong(long offset, long value) { - assert(offset + 8 <= this.offset + this.length); Platform.putLong(null, offset, value); } public final float getFloat(long offset) { - assert(offset + 4 <= this.offset + this.length); return Platform.getFloat(null, offset); } public final void putFloat(long offset, float value) { - assert(offset + 4 <= this.offset + this.length); Platform.putFloat(null, offset, value); } public final double getDouble(long offset) { - assert(offset + 8 <= this.offset + this.length); return Platform.getDouble(null, offset); } public final void putDouble(long offset, double value) { - assert(offset + 8 <= this.offset + this.length); Platform.putDouble(null, offset, value); } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 378d355680a8e..668d860e7d4c4 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -205,9 +205,7 @@ private static int numBytesForFirstByte(final byte b) { * Returns the number of bytes */ public int numBytes() { - long size = base.size(); - assert(size <= (long)Integer.MAX_VALUE); - return (int)size; + return (int)base.size(); } /** From c9f401ab5ff2a94950dd57643d2192b47c175d3a Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 2 Mar 2018 05:36:22 +0000 Subject: [PATCH 38/65] address review comments --- .../sql/catalyst/expressions/HiveHasher.java | 4 +- .../spark/unsafe/hash/Murmur3_x86_32.java | 4 +- .../unsafe/memory/ByteArrayMemoryBlock.java | 9 +- .../spark/unsafe/memory/MemoryBlock.java | 7 +- .../unsafe/memory/OffHeapMemoryBlock.java | 7 +- .../unsafe/memory/OnHeapMemoryBlock.java | 37 ++-- .../unsafe/memory/UnsafeMemoryAllocator.java | 3 +- .../apache/spark/unsafe/types/UTF8String.java | 187 ++++++++---------- .../spark/unsafe/array/LongArraySuite.java | 3 +- .../spark/unsafe/types/UTF8StringSuite.java | 13 +- .../shuffle/sort/ShuffleInMemorySorter.java | 2 +- .../unsafe/sort/UnsafeInMemorySorter.java | 2 +- .../util/collection/ExternalSorterSuite.scala | 3 +- .../unsafe/sort/RadixSortSuite.scala | 3 +- .../spark/mllib/feature/HashingTF.scala | 2 +- .../spark/sql/catalyst/expressions/XXH64.java | 4 +- .../spark/sql/catalyst/expressions/hash.scala | 32 ++- .../catalyst/expressions/HiveHasherSuite.java | 2 +- .../vectorized/OffHeapColumnVector.java | 2 +- .../sql/vectorized/ArrowColumnVector.java | 5 +- .../execution/benchmark/SortBenchmark.scala | 3 +- .../sql/execution/python/RowQueueSuite.scala | 2 +- 22 files changed, 152 insertions(+), 184 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java index 147c367b75898..523cb85d3d28e 100644 --- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java @@ -39,8 +39,8 @@ public static int hashLong(long input) { return (int) ((input >>> 32) ^ input); } - public static int hashUnsafeBytesBlock(MemoryBlock base, long offset, int lengthInBytes) { - return hashUnsafeBytes(base.getBaseObject(), offset, lengthInBytes); + public static int hashUnsafeBytesBlock(MemoryBlock base) { + return hashUnsafeBytes(base.getBaseObject(), base.getBaseOffset(), (int)base.size()); } public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes) { diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index 26c7980a5bbea..c8b6320287e73 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -64,7 +64,9 @@ public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, i return fmix(h1, lengthInBytes); } - public static int hashUnsafeBytesBlock(MemoryBlock base, long offset, int lengthInBytes, int seed) { + public static int hashUnsafeBytesBlock(MemoryBlock base, int seed) { + long offset = base.getBaseOffset(); + int lengthInBytes = (int)base.size(); assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; int lengthAligned = lengthInBytes - lengthInBytes % 4; int h1 = hashBytesByIntBlock(base, offset, lengthAligned, seed); diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java index 67b6a7925f14e..4f2b28675fe12 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -27,12 +27,16 @@ public final class ByteArrayMemoryBlock extends MemoryBlock { private final byte[] array; public ByteArrayMemoryBlock(byte[] obj, long offset, long length) { - super(obj, offset, length); + super(obj, offset, (long)length); this.array = obj; } + public ByteArrayMemoryBlock(long length) { + this(new byte[(int)length], Platform.BYTE_ARRAY_OFFSET, length); + } + @Override - public MemoryBlock allocate(long offset, long size) { + public MemoryBlock subBlock(long offset, long size) { return new ByteArrayMemoryBlock(array, this.offset + offset, size); } @@ -45,7 +49,6 @@ public static ByteArrayMemoryBlock fromArray(final byte[] array) { return new ByteArrayMemoryBlock(array, Platform.BYTE_ARRAY_OFFSET, array.length); } - public final int getInt(long offset) { return Platform.getInt(array, offset); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index 1249a72632f7f..46cffece5b081 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -50,7 +50,7 @@ public abstract class MemoryBlock { protected long offset; - protected long length; + protected final long length; /** * Optional page number; used when this MemoryBlock represents a page allocated by a @@ -125,9 +125,10 @@ public final static MemoryBlock allocateFromObject(Object obj, long offset, long } /** - * Instantiate the same type of MemoryBlock with new offset and size + * Just instantiate the same type of MemoryBlock with new offset and size. The data is not + * copied. */ - public abstract MemoryBlock allocate(long offset, long size); + public abstract MemoryBlock subBlock(long offset, long size); public abstract int getInt(long offset); diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java index 4b812be8f0d50..336569f3864f9 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java @@ -26,13 +26,8 @@ public OffHeapMemoryBlock(long address, long size) { super(null, address, size); } - public void setAddressAndSize(long address, long size) { - this.offset = address; - this.length = size; - } - @Override - public MemoryBlock allocate(long offset, long size) { + public MemoryBlock subBlock(long offset, long size) { return new OffHeapMemoryBlock(this.offset + offset, size); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java index 47cafd2d60ccc..5f9e3554b1067 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java @@ -32,8 +32,13 @@ public OnHeapMemoryBlock(long[] obj, long offset, long size) { assert(offset - Platform.LONG_ARRAY_OFFSET + size <= obj.length * 8L); } + public OnHeapMemoryBlock(long size) { + this(new long[(int)((size + 7) / 8)], Platform.LONG_ARRAY_OFFSET, + ((size + 7) / 8) * 8L); + } + @Override - public MemoryBlock allocate(long offset, long size) { + public MemoryBlock subBlock(long offset, long size) { return new OnHeapMemoryBlock(array, this.offset + offset, size); } @@ -43,7 +48,7 @@ public MemoryBlock allocate(long offset, long size) { * Creates a memory block pointing to the memory used by the long array. */ public static OnHeapMemoryBlock fromArray(final long[] array) { - return new OnHeapMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, array.length * 8); + return new OnHeapMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, array.length * 8L); } public static OnHeapMemoryBlock fromArray(final long[] array, long size) { @@ -51,72 +56,72 @@ public static OnHeapMemoryBlock fromArray(final long[] array, long size) { } public final int getInt(long offset) { - assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getInt(array, offset); } public final void putInt(long offset, int value) { - assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putInt(array, offset, value); } public final boolean getBoolean(long offset) { - assert(offset + 1 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + assert(offset + 1 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getBoolean(array, offset); } public final void putBoolean(long offset, boolean value) { - assert(offset + 1 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + assert(offset + 1 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putBoolean(array, offset, value); } public final byte getByte(long offset) { - assert(offset + 1 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + assert(offset + 1 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getByte(array, offset); } public final void putByte(long offset, byte value) { - assert(offset + 1 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + assert(offset + 1 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putByte(array, offset, value); } public final short getShort(long offset) { - assert(offset + 2 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + assert(offset + 2 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getShort(array, offset); } public final void putShort(long offset, short value) { - assert(offset + 2 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + assert(offset + 2 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putShort(array, offset, value); } public final long getLong(long offset) { - assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getLong(array, offset); } public final void putLong(long offset, long value) { - assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putLong(array, offset, value); } public final float getFloat(long offset) { - assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getFloat(array, offset); } public final void putFloat(long offset, float value) { - assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putFloat(array, offset, value); } public final double getDouble(long offset) { - assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getDouble(array, offset); } public final void putDouble(long offset, double value) { - assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8); + assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putDouble(array, offset, value); } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java index d06b6a855d418..2824a2fc29a08 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -69,7 +69,8 @@ public OffHeapMemoryBlock reallocate(OffHeapMemoryBlock block, long oldSize, lon OffHeapMemoryBlock mb = this.allocate(newSize); if (block.getBaseOffset() != 0) MemoryBlock.copyMemory(block, mb, oldSize); - free(block); + if (block != mb) + free(block); return mb; } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 668d860e7d4c4..715a4cb4cfacb 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -53,9 +53,11 @@ public final class UTF8String implements Comparable, Externalizable, // These are only updated by readExternal() or read() @Nonnull private MemoryBlock base; + // While numBytes has the same value as base,length, to keep as int avoids cast from long to int + private int numBytes; - public MemoryBlock getBaseObject() { return base; } - public long getBaseOffset() { return base.getBaseOffset(); } + public MemoryBlock getMemoryBlock() { return base; } + private long getBaseOffset() { return base.getBaseOffset(); } private static int[] bytesOfCodePointInUTF8 = {2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, @@ -96,13 +98,6 @@ public static UTF8String fromBytes(byte[] bytes, int offset, int numBytes) { } } - /** - * Creates an UTF8String from given address (base and offset) and length. - */ - public static UTF8String fromAddress(MemoryBlock base, long offset, int numBytes) { - return new UTF8String(base.allocate(offset, numBytes)); - } - /** * Creates an UTF8String from given memory block. */ @@ -132,6 +127,13 @@ protected UTF8String(byte[] bytes, long offset, int numBytes) { protected UTF8String(MemoryBlock base) { this.base = base; + if (base != null) { + if ((long) Integer.MAX_VALUE < base.size()) { + throw new ArrayIndexOutOfBoundsException( + "MemoryBlock.size " + base.size() + " should be less than " + Integer.MAX_VALUE); + } + this.numBytes = (int) base.size(); + } } // for serialization @@ -145,7 +147,7 @@ public UTF8String() { * bytes in this string. */ public void writeToMemory(byte[] target, long targetOffset) { - base.writeTo(0, target, targetOffset, numBytes()); + base.writeTo(0, target, targetOffset, numBytes); } public void writeTo(ByteBuffer buffer) { @@ -154,7 +156,7 @@ public void writeTo(ByteBuffer buffer) { int offset = buffer.arrayOffset(); int pos = buffer.position(); writeToMemory(target, Platform.BYTE_ARRAY_OFFSET + offset + pos); - buffer.position(pos + numBytes()); + buffer.position(pos + numBytes); } /** @@ -174,11 +176,11 @@ public ByteBuffer getByteBuffer() { // verify that the offset and length points somewhere inside the byte array // and that the offset can safely be truncated to a 32-bit integer - if ((long) bytes.length < arrayOffset + numBytes()) { + if ((long) bytes.length < arrayOffset + numBytes) { throw new ArrayIndexOutOfBoundsException(); } - return ByteBuffer.wrap(bytes, (int) arrayOffset, numBytes()); + return ByteBuffer.wrap(bytes, (int) arrayOffset, numBytes); } else { return ByteBuffer.wrap(getBytes()); } @@ -205,7 +207,7 @@ private static int numBytesForFirstByte(final byte b) { * Returns the number of bytes */ public int numBytes() { - return (int)base.size(); + return numBytes; } /** @@ -213,7 +215,7 @@ public int numBytes() { */ public int numChars() { int len = 0; - for (int i = 0; i < numBytes(); i += numBytesForFirstByte(getByte(i))) { + for (int i = 0; i < numBytes; i += numBytesForFirstByte(getByte(i))) { len += 1; } return len; @@ -232,7 +234,6 @@ public long getPrefix() { long p; long mask = 0; long offset = base.getBaseOffset(); - int numBytes = numBytes(); if (IS_LITTLE_ENDIAN) { if (numBytes >= 8) { p = base.getLong(offset); @@ -271,11 +272,11 @@ public byte[] getBytes() { // avoid copy if `base` is `byte[]` long offset = base.getBaseOffset(); if (offset == BYTE_ARRAY_OFFSET && base instanceof ByteArrayMemoryBlock - && (((ByteArrayMemoryBlock) base).getByteArray()).length == numBytes()) { + && (((ByteArrayMemoryBlock) base).getByteArray()).length == numBytes) { return ((ByteArrayMemoryBlock) base).getByteArray(); } else { - byte[] bytes = new byte[numBytes()]; - base.writeTo(0, bytes, BYTE_ARRAY_OFFSET, numBytes()); + byte[] bytes = new byte[numBytes]; + base.writeTo(0, bytes, BYTE_ARRAY_OFFSET, numBytes); return bytes; } } @@ -286,7 +287,6 @@ public byte[] getBytes() { * @param until the position after last code point, exclusive. */ public UTF8String substring(final int start, final int until) { - int numBytes = numBytes(); if (until <= start || start >= numBytes) { return EMPTY_UTF8; } @@ -330,12 +330,12 @@ public UTF8String substringSQL(int pos, int length) { * Returns whether this contains `substring` or not. */ public boolean contains(final UTF8String substring) { - if (substring.numBytes() == 0) { + if (substring.numBytes == 0) { return true; } byte first = substring.getByte(0); - for (int i = 0; i <= numBytes() - substring.numBytes(); i++) { + for (int i = 0; i <= numBytes - substring.numBytes; i++) { if (getByte(i) == first && matchAt(substring, i)) { return true; } @@ -351,10 +351,10 @@ private byte getByte(int i) { } private boolean matchAt(final UTF8String s, int pos) { - if (s.numBytes() + pos > numBytes() || pos < 0) { + if (s.numBytes + pos > numBytes || pos < 0) { return false; } - return ByteArrayMethods.arrayEqualsBlock(base, getBaseOffset() + pos, s.base, s.getBaseOffset(), s.numBytes()); + return ByteArrayMethods.arrayEqualsBlock(base, getBaseOffset() + pos, s.base, s.getBaseOffset(), s.numBytes); } public boolean startsWith(final UTF8String prefix) { @@ -362,14 +362,13 @@ public boolean startsWith(final UTF8String prefix) { } public boolean endsWith(final UTF8String suffix) { - return matchAt(suffix, numBytes() - suffix.numBytes()); + return matchAt(suffix, numBytes - suffix.numBytes); } /** * Returns the upper case of this string */ public UTF8String toUpperCase() { - int numBytes = numBytes(); if (numBytes == 0) { return EMPTY_UTF8; } @@ -400,7 +399,6 @@ private UTF8String toUpperCaseSlow() { * Returns the lower case of this string */ public UTF8String toLowerCase() { - int numBytes = numBytes(); if (numBytes == 0) { return EMPTY_UTF8; } @@ -431,7 +429,6 @@ private UTF8String toLowerCaseSlow() { * Returns the title case of this string, that could be used as title. */ public UTF8String toTitleCase() { - int numBytes = numBytes(); if (numBytes == 0) { return EMPTY_UTF8; } @@ -480,25 +477,23 @@ public int findInSet(UTF8String match) { return 0; } - int numBytes = numBytes(); - int matchNumBytes = match.numBytes(); long offset = getBaseOffset(); long matchOffset = match.getBaseOffset(); int n = 1, lastComma = -1; for (int i = 0; i < numBytes; i++) { if (getByte(i) == (byte) ',') { - if (i - (lastComma + 1) == matchNumBytes && + if (i - (lastComma + 1) == match.numBytes && ByteArrayMethods.arrayEqualsBlock(base, offset + (lastComma + 1), match.base, matchOffset, - matchNumBytes)) { + match.numBytes)) { return n; } lastComma = i; n++; } } - if (numBytes - (lastComma + 1) == matchNumBytes && + if (numBytes - (lastComma + 1) == match.numBytes && ByteArrayMethods.arrayEqualsBlock(base, offset + (lastComma + 1), match.base, matchOffset, - matchNumBytes)) { + match.numBytes)) { return n; } return 0; @@ -518,16 +513,15 @@ private UTF8String copyUTF8String(int start, int end) { } public UTF8String trim() { - int numBytes = numBytes(); int s = 0; // skip all of the space (0x20) in the left side - while (s < numBytes && getByte(s) == 0x20) s++; - if (s == numBytes) { + while (s < this.numBytes && getByte(s) == 0x20) s++; + if (s == this.numBytes) { // empty string return EMPTY_UTF8; } // skip all of the space (0x20) in the right side - int e = numBytes - 1; + int e = this.numBytes - 1; while (e > s && getByte(e) == 0x20) e--; return copyUTF8String(s, e); } @@ -548,15 +542,14 @@ public UTF8String trim(UTF8String trimString) { } public UTF8String trimLeft() { - int numBytes = numBytes(); int s = 0; // skip all of the space (0x20) in the left side - while (s < numBytes && getByte(s) == 0x20) s++; - if (s == numBytes) { + while (s < this.numBytes && getByte(s) == 0x20) s++; + if (s == this.numBytes) { // empty string return EMPTY_UTF8; } else { - return copyUTF8String(s, numBytes - 1); + return copyUTF8String(s, this.numBytes - 1); } } @@ -569,7 +562,6 @@ public UTF8String trimLeft() { */ public UTF8String trimLeft(UTF8String trimString) { if (trimString == null) return null; - int numBytes = numBytes(); // the searching byte position in the source string int srchIdx = 0; // the first beginning byte position of a non-matching character @@ -578,7 +570,7 @@ public UTF8String trimLeft(UTF8String trimString) { while (srchIdx < numBytes) { UTF8String searchChar = copyUTF8String( srchIdx, srchIdx + numBytesForFirstByte(this.getByte(srchIdx)) - 1); - int searchCharBytes = searchChar.numBytes(); + int searchCharBytes = searchChar.numBytes; // try to find the matching for the searchChar in the trimString set if (trimString.find(searchChar, 0) >= 0) { trimIdx += searchCharBytes; @@ -598,7 +590,7 @@ public UTF8String trimLeft(UTF8String trimString) { } public UTF8String trimRight() { - int e = numBytes() - 1; + int e = numBytes - 1; // skip all of the space (0x20) in the right side while (e >= 0 && getByte(e) == 0x20) e--; @@ -619,7 +611,6 @@ public UTF8String trimRight() { */ public UTF8String trimRight(UTF8String trimString) { if (trimString == null) return null; - int numBytes = numBytes(); int charIdx = 0; // number of characters from the source string int numChars = 0; @@ -659,8 +650,7 @@ public UTF8String trimRight(UTF8String trimString) { } public UTF8String reverse() { - int numBytes = numBytes(); - byte[] result = new byte[numBytes]; + byte[] result = new byte[this.numBytes]; int i = 0; // position in byte while (i < numBytes) { @@ -678,7 +668,6 @@ public UTF8String repeat(int times) { return EMPTY_UTF8; } - int numBytes = numBytes(); byte[] newBytes = new byte[numBytes * times]; base.writeTo(0, newBytes, BYTE_ARRAY_OFFSET, numBytes); @@ -705,8 +694,6 @@ public int indexOf(UTF8String v, int start) { return 0; } - int numBytes = numBytes(); - int vNumBytes = v.numBytes(); // locate to the start position. int i = 0; // position in byte int c = 0; // position in character @@ -718,10 +705,10 @@ public int indexOf(UTF8String v, int start) { long offset = getBaseOffset(); long vOffset = v.getBaseOffset(); do { - if (i + vNumBytes > numBytes) { + if (i + v.numBytes > numBytes) { return -1; } - if (ByteArrayMethods.arrayEqualsBlock(base, offset + i, v.base, vOffset, vNumBytes)) { + if (ByteArrayMethods.arrayEqualsBlock(base, offset + i, v.base, vOffset, v.numBytes)) { return c; } i += numBytesForFirstByte(getByte(i)); @@ -735,11 +722,11 @@ public int indexOf(UTF8String v, int start) { * Find the `str` from left to right. */ private int find(UTF8String str, int start) { - assert (str.numBytes() > 0); + assert (str.numBytes > 0); long offset = getBaseOffset(); long strOffset = str.getBaseOffset(); - while (start <= numBytes() - str.numBytes()) { - if (ByteArrayMethods.arrayEqualsBlock(base, offset + start, str.base, strOffset, str.numBytes())) { + while (start <= numBytes - str.numBytes) { + if (ByteArrayMethods.arrayEqualsBlock(base, offset + start, str.base, strOffset, str.numBytes)) { return start; } start += 1; @@ -751,11 +738,11 @@ private int find(UTF8String str, int start) { * Find the `str` from right to left. */ private int rfind(UTF8String str, int start) { - assert (str.numBytes() > 0); + assert (str.numBytes > 0); long offset = getBaseOffset(); long strOffset = str.getBaseOffset(); while (start >= 0) { - if (ByteArrayMethods.arrayEqualsBlock(base, offset + start, str.base, strOffset, str.numBytes())) { + if (ByteArrayMethods.arrayEqualsBlock(base, offset + start, str.base, strOffset, str.numBytes)) { return start; } start -= 1; @@ -770,9 +757,7 @@ private int rfind(UTF8String str, int start) { * right) is returned. subStringIndex performs a case-sensitive match when searching for delim. */ public UTF8String subStringIndex(UTF8String delim, int count) { - int numBytes = numBytes(); - int delimNumBytes = delim.numBytes(); - if (delimNumBytes == 0 || count == 0) { + if (delim.numBytes == 0 || count == 0) { return EMPTY_UTF8; } if (count > 0) { @@ -794,7 +779,7 @@ public UTF8String subStringIndex(UTF8String delim, int count) { return fromBytes(bytes); } else { - int idx = numBytes - delimNumBytes + 1; + int idx = numBytes - delim.numBytes + 1; count = -count; while (count > 0) { idx = rfind(delim, idx - 1); @@ -805,12 +790,12 @@ public UTF8String subStringIndex(UTF8String delim, int count) { return this; } } - if (idx + delimNumBytes == numBytes) { + if (idx + delim.numBytes == numBytes) { return EMPTY_UTF8; } - int size = numBytes - delimNumBytes - idx; + int size = numBytes - delim.numBytes - idx; byte[] bytes = new byte[size]; - base.writeTo(idx + delimNumBytes, bytes, BYTE_ARRAY_OFFSET, size); + base.writeTo(idx + delim.numBytes, bytes, BYTE_ARRAY_OFFSET, size); return fromBytes(bytes); } } @@ -832,19 +817,16 @@ public UTF8String rpad(int len, UTF8String pad) { // the partial string of the padding UTF8String remain = pad.substring(0, spaces - padChars * count); - int numBytes = numBytes(); - int padNumBytes = pad.numBytes(); - int remainNumBytes = remain.numBytes(); - byte[] data = new byte[numBytes + padNumBytes * count + remainNumBytes]; - base.writeTo(0, data, BYTE_ARRAY_OFFSET, numBytes); - int offset = numBytes; + byte[] data = new byte[this.numBytes + pad.numBytes * count + remain.numBytes]; + base.writeTo(0, data, BYTE_ARRAY_OFFSET, this.numBytes); + int offset = this.numBytes; int idx = 0; while (idx < count) { - pad.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, padNumBytes); + pad.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); ++ idx; - offset += padNumBytes; + offset += pad.numBytes; } - remain.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, remainNumBytes); + remain.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); return UTF8String.fromBytes(data); } @@ -857,9 +839,8 @@ public UTF8String rpad(int len, UTF8String pad) { * ('hi', 1, '??') => 'h' */ public UTF8String lpad(int len, UTF8String pad) { - int padNumBytes = pad.numBytes(); int spaces = len - this.numChars(); // number of char need to pad - if (spaces <= 0 || padNumBytes == 0) { + if (spaces <= 0 || pad.numBytes() == 0) { // no padding at all, return the substring of the current string return substring(0, len); } else { @@ -867,20 +848,18 @@ public UTF8String lpad(int len, UTF8String pad) { int count = spaces / padChars; // how many padding string needed // the partial string of the padding UTF8String remain = pad.substring(0, spaces - padChars * count); - int remainNumBytes = remain.numBytes(); - int numBytes = numBytes(); - byte[] data = new byte[numBytes + padNumBytes * count + remainNumBytes]; + byte[] data = new byte[this.numBytes + pad.numBytes * count + remain.numBytes]; int offset = 0; int idx = 0; while (idx < count) { - pad.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, padNumBytes); + pad.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); ++ idx; - offset += padNumBytes; + offset += pad.numBytes; } - remain.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, remainNumBytes); - offset += remainNumBytes; + remain.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); + offset += remain.numBytes; base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, numBytes()); return UTF8String.fromBytes(data); @@ -895,7 +874,7 @@ public static UTF8String concat(UTF8String... inputs) { int totalLength = 0; for (int i = 0; i < inputs.length; i++) { if (inputs[i] != null) { - totalLength += inputs[i].numBytes(); + totalLength += inputs[i].numBytes; } else { return null; } @@ -905,7 +884,7 @@ public static UTF8String concat(UTF8String... inputs) { final byte[] result = new byte[totalLength]; int offset = 0; for (int i = 0; i < inputs.length; i++) { - int len = inputs[i].numBytes(); + int len = inputs[i].numBytes; inputs[i].base.writeTo( 0, result, BYTE_ARRAY_OFFSET + offset, @@ -928,7 +907,7 @@ public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) { int numInputs = 0; // number of non-null inputs for (int i = 0; i < inputs.length; i++) { if (inputs[i] != null) { - numInputBytes += inputs[i].numBytes(); + numInputBytes += inputs[i].numBytes; numInputs++; } } @@ -940,12 +919,12 @@ public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) { // Allocate a new byte array, and copy the inputs one by one into it. // The size of the new array is the size of all inputs, plus the separators. - final byte[] result = new byte[numInputBytes + (numInputs - 1) * separator.numBytes()]; + final byte[] result = new byte[numInputBytes + (numInputs - 1) * separator.numBytes]; int offset = 0; for (int i = 0, j = 0; i < inputs.length; i++) { if (inputs[i] != null) { - int len = inputs[i].numBytes(); + int len = inputs[i].numBytes; inputs[i].base.writeTo( 0, result, BYTE_ARRAY_OFFSET + offset, @@ -958,8 +937,8 @@ public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) { separator.base.writeTo( 0, result, BYTE_ARRAY_OFFSET + offset, - separator.numBytes()); - offset += separator.numBytes(); + separator.numBytes); + offset += separator.numBytes; } } } @@ -1033,7 +1012,6 @@ public static class IntWrapper implements Serializable { * @return true if the parsing was successful else false */ public boolean toLong(LongWrapper toLongResult) { - int numBytes = numBytes(); if (numBytes == 0) { return false; } @@ -1125,7 +1103,6 @@ public boolean toLong(LongWrapper toLongResult) { * @return true if the parsing was successful else false */ public boolean toInt(IntWrapper intWrapper) { - int numBytes = numBytes(); if (numBytes == 0) { return false; } @@ -1231,7 +1208,6 @@ public UTF8String clone() { } public UTF8String copy() { - int numBytes = numBytes(); byte[] bytes = new byte[numBytes]; base.writeTo(0, bytes, BYTE_ARRAY_OFFSET, numBytes); return fromBytes(bytes); @@ -1239,7 +1215,7 @@ public UTF8String copy() { @Override public int compareTo(@Nonnull final UTF8String other) { - int len = Math.min(numBytes(), other.numBytes()); + int len = Math.min(numBytes, other.numBytes); int wordMax = (len / 8) * 8; long offset = getBaseOffset(); long roffset = other.getBaseOffset(); @@ -1262,7 +1238,7 @@ public int compareTo(@Nonnull final UTF8String other) { return res; } } - return numBytes() - other.numBytes(); + return numBytes - other.numBytes; } public int compare(final UTF8String other) { @@ -1273,10 +1249,10 @@ public int compare(final UTF8String other) { public boolean equals(final Object other) { if (other instanceof UTF8String) { UTF8String o = (UTF8String) other; - if (numBytes() != o.numBytes()) { + if (numBytes != o.numBytes) { return false; } - return ByteArrayMethods.arrayEqualsBlock(base, getBaseOffset(), o.base, o.getBaseOffset(), numBytes()); + return ByteArrayMethods.arrayEqualsBlock(base, getBaseOffset(), o.base, o.getBaseOffset(), numBytes); } else { return false; } @@ -1350,7 +1326,7 @@ public int levenshteinDistance(UTF8String other) { @Override public int hashCode() { - return Murmur3_x86_32.hashUnsafeBytesBlock(base, getBaseOffset(), numBytes(), 42); + return Murmur3_x86_32.hashUnsafeBytesBlock(base,42); } /** @@ -1365,7 +1341,6 @@ public int hashCode() { * https://en.wikipedia.org/wiki/Soundex */ public UTF8String soundex() { - int numBytes = numBytes(); if (numBytes == 0) { return EMPTY_UTF8; } @@ -1414,9 +1389,10 @@ public void writeExternal(ObjectOutput out) throws IOException { } public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - int numBytes = in.readInt(); - base = ByteArrayMemoryBlock.fromArray(new byte[numBytes]); - in.readFully(((ByteArrayMemoryBlock)base).getByteArray()); + numBytes = in.readInt(); + byte[] bytes = new byte[numBytes]; + in.readFully(bytes); + base = ByteArrayMemoryBlock.fromArray(bytes); } @Override @@ -1428,9 +1404,10 @@ public void write(Kryo kryo, Output out) { @Override public void read(Kryo kryo, Input in) { - int numBytes = in.readInt(); - base = ByteArrayMemoryBlock.fromArray(new byte[numBytes]); - in.read(((ByteArrayMemoryBlock)base).getByteArray()); + numBytes = in.readInt(); + byte[] bytes = new byte[numBytes]; + in.read(bytes); + base = ByteArrayMemoryBlock.fromArray(bytes); } } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java index 623fddaede9fe..8c2e98c2bfc54 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java @@ -26,8 +26,7 @@ public class LongArraySuite { @Test public void basicTest() { - long[] bytes = new long[2]; - LongArray arr = new LongArray(OnHeapMemoryBlock.fromArray(bytes)); + LongArray arr = new LongArray(new OnHeapMemoryBlock(16)); arr.set(0, 1L); arr.set(1, 2L); arr.set(1, 3L); diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java index 039f496e9fbba..9bf2c5ed3370b 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java @@ -27,6 +27,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.memory.ByteArrayMemoryBlock; +import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.unsafe.memory.OnHeapMemoryBlock; import org.junit.Test; @@ -517,9 +518,8 @@ public void writeToOutputStreamUnderflow() throws IOException { final byte[] test = "01234567".getBytes(StandardCharsets.UTF_8); for (int i = 1; i <= Platform.BYTE_ARRAY_OFFSET; ++i) { - UTF8String.fromAddress(ByteArrayMemoryBlock.fromArray(test), - - i, test.length + i) - .writeTo(outputStream); + UTF8String.fromMemoryBlock(ByteArrayMemoryBlock.fromArray(test) + .subBlock(-i, test.length + i)).writeTo(outputStream); final ByteBuffer buffer = ByteBuffer.wrap(outputStream.toByteArray(), i, test.length); assertEquals("01234567", StandardCharsets.UTF_8.decode(buffer).toString()); outputStream.reset(); @@ -533,7 +533,7 @@ public void writeToOutputStreamSlice() throws IOException { for (int i = 0; i < test.length; ++i) { for (int j = 0; j < test.length - i; ++j) { - UTF8String.fromAddress(ByteArrayMemoryBlock.fromArray(test), i, j) + UTF8String.fromMemoryBlock(ByteArrayMemoryBlock.fromArray(test).subBlock(i, j)) .writeTo(outputStream); assertArrayEquals(Arrays.copyOfRange(test, i, i + j), outputStream.toByteArray()); @@ -564,7 +564,7 @@ public void writeToOutputStreamOverflow() throws IOException { for (final long offset : offsets) { try { - fromAddress(ByteArrayMemoryBlock.fromArray(test), offset, test.length) + fromMemoryBlock(ByteArrayMemoryBlock.fromArray(test).subBlock(offset, test.length)) .writeTo(outputStream); throw new IllegalStateException(Long.toString(offset)); @@ -608,8 +608,7 @@ public void writeToOutputStreamLongArray() throws IOException { } final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - fromAddress(OnHeapMemoryBlock.fromArray(array), 0, length) - .writeTo(outputStream); + fromMemoryBlock(OnHeapMemoryBlock.fromArray(array)).writeTo(outputStream); assertEquals("3千大千世界", outputStream.toString("UTF-8")); } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java index 61b332adc174a..c9329d69fd742 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java @@ -173,7 +173,7 @@ public ShuffleSorterIterator getSortedIterator() { PackedRecordPointer.PARTITION_ID_START_BYTE_INDEX, PackedRecordPointer.PARTITION_ID_END_BYTE_INDEX, false, false); } else { - MemoryBlock unused = array.memoryBlock().allocate(pos * 8L,(array.size() - pos) * 8L); + MemoryBlock unused = array.memoryBlock().subBlock(pos * 8L,(array.size() - pos) * 8L); LongArray buffer = new LongArray(unused); Sorter sorter = new Sorter<>(new ShuffleSortDataFormat(buffer)); diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index 638384caebeb8..db06e0e65fe6b 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -342,7 +342,7 @@ public UnsafeSorterIterator getSortedIterator() { array, nullBoundaryPos, (pos - nullBoundaryPos) / 2L, 0, 7, radixSortSupport.sortDescending(), radixSortSupport.sortSigned()); } else { - MemoryBlock unused = array.memoryBlock().allocate(pos * 8L,(array.size() - pos) * 8L); + MemoryBlock unused = array.memoryBlock().subBlock(pos * 8L,(array.size() - pos) * 8L); LongArray buffer = new LongArray(unused); Sorter sorter = new Sorter<>(new UnsafeSortDataFormat(buffer)); diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala index 507ec191f307b..3e56db5ea116a 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -106,8 +106,7 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { // that can trigger copyRange() in TimSort.mergeLo() or TimSort.mergeHi() val ref = Array.tabulate[Long](size) { i => if (i < size / 2) size / 2 + i else i } val buf = new LongArray(OnHeapMemoryBlock.fromArray(ref)) - val tmp = new Array[Long](size/2) - val tmpBuf = new LongArray(OnHeapMemoryBlock.fromArray(tmp)) + val tmpBuf = new LongArray(new OnHeapMemoryBlock((size/2) * 8L)) new Sorter(new UnsafeSortDataFormat(tmpBuf)).sort( buf, 0, size, new Comparator[RecordPointerAndKeyPrefix] { diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala index 639fbc0179103..ddf3740e76a7a 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala @@ -110,8 +110,7 @@ class RadixSortSuite extends SparkFunSuite with Logging { } private def referenceKeyPrefixSort(buf: LongArray, lo: Long, hi: Long, refCmp: PrefixComparator) { - val sortBuffer = - new LongArray(OnHeapMemoryBlock.fromArray(new Array[Long](buf.size().toInt))) + val sortBuffer = new LongArray(new OnHeapMemoryBlock(buf.size() * 8L)) new Sorter(new UnsafeSortDataFormat(sortBuffer)).sort( buf, Ints.checkedCast(lo), Ints.checkedCast(hi), new Comparator[RecordPointerAndKeyPrefix] { override def compare( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala index b02fa48e3e1be..7b73b286fb91c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala @@ -160,7 +160,7 @@ object HashingTF { case d: Double => hashLong(java.lang.Double.doubleToLongBits(d), seed) case s: String => val utf8 = UTF8String.fromString(s) - hashUnsafeBytesBlock(utf8.getBaseObject, utf8.getBaseOffset, utf8.numBytes(), seed) + hashUnsafeBytesBlock(utf8.getMemoryBlock(), seed) case _ => throw new SparkException("HashingTF with murmur3 algorithm does not " + s"support type ${term.getClass.getCanonicalName} of input data.") } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java index 9db6b94b2b8e4..c9a722d1261a3 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java @@ -86,8 +86,8 @@ public long hashUnsafeBytes(Object base, long offset, int length) { return hashUnsafeBytes(base, offset, length, seed); } - public static long hashUnsafeBytesBlock(MemoryBlock base, long offset, int length, long seed) { - return hashUnsafeBytes(base.getBaseObject(), offset, length, seed); + public static long hashUnsafeBytesBlock(MemoryBlock base, long seed) { + return hashUnsafeBytes(base.getBaseObject(), base.getBaseOffset(), (int)base.size(), seed); } public static long hashUnsafeBytes(Object base, long offset, int length, long seed) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala index 9dcda27f79e49..dbe3dff7ff6ba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala @@ -360,11 +360,8 @@ abstract class HashExpression[E] extends Expression { } protected def genHashString(input: String, result: String): String = { - val baseObject = s"$input.getBaseObject()" - val baseOffset = s"$input.getBaseOffset()" - val numBytes = s"$input.numBytes()" - s"$result = $hasherClassName.hashUnsafeBytesBlock(" + - s"$baseObject, $baseOffset, $numBytes, $result);" + val mb = s"$input.getMemoryBlock()" + s"$result = $hasherClassName.hashUnsafeBytesBlock($mb, $result);" } protected def genHashForMap( @@ -466,8 +463,7 @@ abstract class InterpretedHashFunction { protected def hashUnsafeBytes(base: AnyRef, offset: Long, length: Int, seed: Long): Long - protected def hashUnsafeBytesBlock( - base: MemoryBlock, offset: Long, length: Int, seed: Long): Long + protected def hashUnsafeBytesBlock(base: MemoryBlock, seed: Long): Long /** * Computes hash of a given `value` of type `dataType`. The caller needs to check the validity @@ -494,8 +490,7 @@ abstract class InterpretedHashFunction { case c: CalendarInterval => hashInt(c.months, hashLong(c.microseconds, seed)) case a: Array[Byte] => hashUnsafeBytes(a, Platform.BYTE_ARRAY_OFFSET, a.length, seed) - case s: UTF8String => - hashUnsafeBytesBlock(s.getBaseObject, s.getBaseOffset, s.numBytes(), seed) + case s: UTF8String => hashUnsafeBytesBlock(s.getMemoryBlock(), seed) case array: ArrayData => val elementType = dataType match { @@ -588,8 +583,8 @@ object Murmur3HashFunction extends InterpretedHashFunction { } override protected def hashUnsafeBytesBlock( - base: MemoryBlock, offset: Long, len: Int, seed: Long): Long = { - Murmur3_x86_32.hashUnsafeBytesBlock(base, offset, len, seed.toInt) + base: MemoryBlock, seed: Long): Long = { + Murmur3_x86_32.hashUnsafeBytesBlock(base, seed.toInt) } } @@ -620,9 +615,8 @@ object XxHash64Function extends InterpretedHashFunction { XXH64.hashUnsafeBytes(base, offset, len, seed) } - override protected def hashUnsafeBytesBlock( - base: MemoryBlock, offset: Long, len: Int, seed: Long): Long = { - XXH64.hashUnsafeBytesBlock(base, offset, len, seed) + override protected def hashUnsafeBytesBlock(base: MemoryBlock, seed: Long): Long = { + XXH64.hashUnsafeBytesBlock(base, seed) } } @@ -730,10 +724,8 @@ case class HiveHash(children: Seq[Expression]) extends HashExpression[Int] { """ override protected def genHashString(input: String, result: String): String = { - val baseObject = s"$input.getBaseObject()" - val baseOffset = s"$input.getBaseOffset()" - val numBytes = s"$input.numBytes()" - s"$result = $hasherClassName.hashUnsafeBytesBlock($baseObject, $baseOffset, $numBytes);" + val mb = s"$input.getMemoryBlock()" + s"$result = $hasherClassName.hashUnsafeBytesBlock($mb);" } override protected def genHashForArray( @@ -827,9 +819,7 @@ object HiveHashFunction extends InterpretedHashFunction { } override protected def hashUnsafeBytesBlock( - base: MemoryBlock, offset: Long, len: Int, seed: Long): Long = { - HiveHasher.hashUnsafeBytesBlock(base, offset, len) - } + base: MemoryBlock, seed: Long): Long = HiveHasher.hashUnsafeBytesBlock(base) private val HIVE_DECIMAL_MAX_PRECISION = 38 private val HIVE_DECIMAL_MAX_SCALE = 38 diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java index 0f1b692ea43be..b4fd21ecbb8b9 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java @@ -53,7 +53,7 @@ public void testKnownStringAndIntInputs() { for (int i = 0; i < inputs.length; i++) { UTF8String s = UTF8String.fromString("val_" + inputs[i]); - int hash = HiveHasher.hashUnsafeBytesBlock(s.getBaseObject(), s.getBaseOffset(), s.numBytes()); + int hash = HiveHasher.hashUnsafeBytesBlock(s.getMemoryBlock()); Assert.assertEquals(expected[i], ((31 * inputs[i]) + hash)); } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java index a3067e16d40f8..ceaa6aa052af7 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -208,7 +208,7 @@ public byte[] getBytes(int rowId, int count) { @Override protected UTF8String getBytesAsUTF8String(int rowId, int count) { - return UTF8String.fromAddress(data, rowId, count); + return UTF8String.fromMemoryBlock(data.subBlock(rowId, count)); } // diff --git a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java index b871a85f70c89..58ab5c08ed278 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java @@ -366,7 +366,6 @@ private static class StringAccessor extends ArrowVectorAccessor { private final VarCharVector accessor; private final NullableVarCharHolder stringResult = new NullableVarCharHolder(); - private final OffHeapMemoryBlock mb = new OffHeapMemoryBlock(0L, 0L); StringAccessor(VarCharVector vector) { super(vector); @@ -380,9 +379,9 @@ final UTF8String getUTF8String(int rowId) { return null; } else { int size = stringResult.end - stringResult.start; - mb.setAddressAndSize( + OffHeapMemoryBlock mb = new OffHeapMemoryBlock( stringResult.buffer.memoryAddress() + stringResult.start, size); - return UTF8String.fromAddress(mb, 0, size); + return UTF8String.fromMemoryBlock(mb); } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala index e1cea8a88ea4b..470b93efd1974 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala @@ -36,8 +36,7 @@ import org.apache.spark.util.random.XORShiftRandom class SortBenchmark extends BenchmarkBase { private def referenceKeyPrefixSort(buf: LongArray, lo: Int, hi: Int, refCmp: PrefixComparator) { - val sortBuffer = - new LongArray(OnHeapMemoryBlock.fromArray(new Array[Long](buf.size().toInt))) + val sortBuffer = new LongArray(new OnHeapMemoryBlock(buf.size() * 8L)) new Sorter(new UnsafeSortDataFormat(sortBuffer)).sort( buf, lo, hi, new Comparator[RecordPointerAndKeyPrefix] { override def compare( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala index 52f9fba08d88b..25ee95daa034c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.util.Utils class RowQueueSuite extends SparkFunSuite { test("in-memory queue") { - val page = OnHeapMemoryBlock.fromArray(new Array[Long](1<<10)) + val page = new OnHeapMemoryBlock((1<<10) * 8L) val queue = new InMemoryRowQueue(page, 1) { override def close() {} } From cf2d532ae9c8688ef314a51a89c76abe2fd5d857 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 2 Mar 2018 06:03:41 +0000 Subject: [PATCH 39/65] fix compilation failure --- .../apache/spark/unsafe/hash/Murmur3_x86_32.java | 13 ++++++++----- .../org/apache/spark/ml/feature/FeatureHasher.scala | 3 +-- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index c8b6320287e73..ec6c4942cf63f 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -69,7 +69,7 @@ public static int hashUnsafeBytesBlock(MemoryBlock base, int seed) { int lengthInBytes = (int)base.size(); assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; int lengthAligned = lengthInBytes - lengthInBytes % 4; - int h1 = hashBytesByIntBlock(base, offset, lengthAligned, seed); + int h1 = hashBytesByIntBlock(base.subBlock(offset, lengthAligned), seed); for (int i = lengthAligned; i < lengthInBytes; i++) { int halfWord = base.getByte(offset + i); int k1 = mixK1(halfWord); @@ -106,13 +106,14 @@ public static int hashUnsafeBytes2(Object base, long offset, int lengthInBytes, return fmix(h1, lengthInBytes); } - public static int hashUnsafeBytes2Block( - MemoryBlock base, long offset, int lengthInBytes, int seed) { + public static int hashUnsafeBytes2Block(MemoryBlock base, int seed) { // This is compatible with original and another implementations. // Use this method for new components after Spark 2.3. + long offset = base.getBaseOffset(); + int lengthInBytes = (int)base.size(); assert (lengthInBytes >= 0) : "lengthInBytes cannot be negative"; int lengthAligned = lengthInBytes - lengthInBytes % 4; - int h1 = hashBytesByIntBlock(base, offset, lengthAligned, seed); + int h1 = hashBytesByIntBlock(base.subBlock(offset, lengthAligned), seed); int k1 = 0; for (int i = lengthAligned, shift = 0; i < lengthInBytes; i++, shift += 8) { k1 ^= (base.getByte(offset + i) & 0xFF) << shift; @@ -121,7 +122,9 @@ public static int hashUnsafeBytes2Block( return fmix(h1, lengthInBytes); } - private static int hashBytesByIntBlock(MemoryBlock base, long offset, int lengthInBytes, int seed) { + private static int hashBytesByIntBlock(MemoryBlock base, int seed) { + long offset = base.getBaseOffset(); + int lengthInBytes = (int)base.size(); assert (lengthInBytes % 4 == 0); int h1 = seed; for (int i = 0; i < lengthInBytes; i += 4) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala index d5694cd416fe0..d67e4819b161a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala @@ -243,8 +243,7 @@ object FeatureHasher extends DefaultParamsReadable[FeatureHasher] { case f: Float => hashInt(java.lang.Float.floatToIntBits(f), seed) case d: Double => hashLong(java.lang.Double.doubleToLongBits(d), seed) case s: String => - val utf8 = UTF8String.fromString(s) - hashUnsafeBytes2Block(utf8.getBaseObject, utf8.getBaseOffset, utf8.numBytes(), seed) + hashUnsafeBytes2Block(UTF8String.fromString(s).getMemoryBlock, seed) case _ => throw new SparkException("FeatureHasher with murmur3 algorithm does not " + s"support type ${term.getClass.getCanonicalName} of input data.") } From eb0cc6d2ddb3fc648e1d449dad6e53abbabcf2fc Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 2 Mar 2018 13:00:31 +0000 Subject: [PATCH 40/65] address review comment --- .../spark/unsafe/array/ByteArrayMethods.java | 9 +++-- .../spark/unsafe/bitset/BitSetMethods.java | 34 ------------------- .../unsafe/memory/ByteArrayMemoryBlock.java | 14 ++++++++ .../spark/unsafe/memory/MemoryBlock.java | 9 ++++- .../unsafe/memory/OffHeapMemoryBlock.java | 14 ++++++++ .../unsafe/memory/OnHeapMemoryBlock.java | 14 ++++++++ .../unsafe/memory/UnsafeMemoryAllocator.java | 2 +- .../apache/spark/unsafe/types/UTF8String.java | 31 ++++++----------- 8 files changed, 68 insertions(+), 59 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java index fc71953463552..c334c9651cf6b 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java @@ -50,14 +50,19 @@ public static int roundNumberOfBytesToNearestWord(int numBytes) { private static final boolean unaligned = Platform.unaligned(); /** - * Optimized byte array equality check for byte arrays. + * MemoryBlock equality check for MemoryBlocks. * @return true if the arrays are equal, false otherwise */ public static boolean arrayEqualsBlock( MemoryBlock leftBase, long leftOffset, MemoryBlock rightBase, long rightOffset, final long length) { - return arrayEquals(leftBase.getBaseObject(), leftOffset, rightBase.getBaseObject(), rightOffset, length); + return arrayEquals(leftBase.getBaseObject(), leftBase.getBaseOffset() + leftOffset, + rightBase.getBaseObject(), rightBase.getBaseOffset() + rightOffset, length); } + /** + * Optimized byte array equality check for byte arrays. + * @return true if the arrays are equal, false otherwise + */ public static boolean arrayEquals( Object leftBase, long leftOffset, Object rightBase, long rightOffset, final long length) { int i = 0; diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java index b2665715785ce..6d06f4ab9b3b0 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java @@ -46,14 +46,6 @@ public static void set(Object baseObject, long baseOffset, int index) { Platform.putLong(baseObject, wordOffset, word | mask); } - public static void setBlock(MemoryBlock base, long baseOffset, int index) { - assert index >= 0 : "index (" + index + ") should >= 0"; - final long mask = 1L << (index & 0x3f); // mod 64 and shift - final long wordOffset = baseOffset + (index >> 6) * WORD_SIZE; - final long word = base.getLong(wordOffset); - base.putLong(wordOffset, word | mask); - } - /** * Sets the bit at the specified index to {@code false}. */ @@ -65,14 +57,6 @@ public static void unset(Object baseObject, long baseOffset, int index) { Platform.putLong(baseObject, wordOffset, word & ~mask); } - public static void unsetBlock(MemoryBlock base, long baseOffset, int index) { - assert index >= 0 : "index (" + index + ") should >= 0"; - final long mask = 1L << (index & 0x3f); // mod 64 and shift - final long wordOffset = baseOffset + (index >> 6) * WORD_SIZE; - final long word = base.getLong(wordOffset); - base.putLong(wordOffset, word & ~mask); - } - /** * Returns {@code true} if the bit is set at the specified index. */ @@ -84,14 +68,6 @@ public static boolean isSet(Object baseObject, long baseOffset, int index) { return (word & mask) != 0; } - public static boolean isSetBlock(MemoryBlock base, long baseOffset, int index) { - assert index >= 0 : "index (" + index + ") should >= 0"; - final long mask = 1L << (index & 0x3f); // mod 64 and shift - final long wordOffset = baseOffset + (index >> 6) * WORD_SIZE; - final long word = base.getLong(wordOffset); - return (word & mask) != 0; - } - /** * Returns {@code true} if any bit is set. */ @@ -105,16 +81,6 @@ public static boolean anySet(Object baseObject, long baseOffset, long bitSetWidt return false; } - public static boolean anySetBlock(MemoryBlock base, long baseOffset, long bitSetWidthInWords) { - long addr = baseOffset; - for (int i = 0; i < bitSetWidthInWords; i++, addr += WORD_SIZE) { - if (base.getLong(addr) != 0) { - return true; - } - } - return false; - } - /** * Returns the index of the first bit that is set to true that occurs on or after the * specified starting index. If no such bit exists then {@code -1} is returned. diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java index 4f2b28675fe12..844a6ce6ce712 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -49,58 +49,72 @@ public static ByteArrayMemoryBlock fromArray(final byte[] array) { return new ByteArrayMemoryBlock(array, Platform.BYTE_ARRAY_OFFSET, array.length); } + @Override public final int getInt(long offset) { return Platform.getInt(array, offset); } + @Override public final void putInt(long offset, int value) { Platform.putInt(array, offset, value); } + @Override public final boolean getBoolean(long offset) { return Platform.getBoolean(array, offset); } + @Override public final void putBoolean(long offset, boolean value) { Platform.putBoolean(array, offset, value); } + @Override public final byte getByte(long offset) { return array[(int)(offset - Platform.BYTE_ARRAY_OFFSET)]; } + @Override public final void putByte(long offset, byte value) { array[(int)(offset - Platform.BYTE_ARRAY_OFFSET)] = value; } + @Override public final short getShort(long offset) { return Platform.getShort(array, offset); } + @Override public final void putShort(long offset, short value) { Platform.putShort(array, offset, value); } + @Override public final long getLong(long offset) { return Platform.getLong(array, offset); } + @Override public final void putLong(long offset, long value) { Platform.putLong(array, offset, value); } + @Override public final float getFloat(long offset) { return Platform.getFloat(array, offset); } + @Override public final void putFloat(long offset, float value) { Platform.putFloat(array, offset, value); } + @Override public final double getDouble(long offset) { return Platform.getDouble(array, offset); } + @Override public final void putDouble(long offset, double value) { Platform.putDouble(array, offset, value); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index 46cffece5b081..936fe35a98594 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -130,7 +130,10 @@ public final static MemoryBlock allocateFromObject(Object obj, long offset, long */ public abstract MemoryBlock subBlock(long offset, long size); - + /** + * getXXX/putXXX does not ensure guarantee behavior if the offset is invalid. e.g cause illegal + * memory access, throw an exception, or etc. + */ public abstract int getInt(long offset); public abstract void putInt(long offset, int value); @@ -161,20 +164,24 @@ public final static MemoryBlock allocateFromObject(Object obj, long offset, long public static final void copyMemory( MemoryBlock src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { + assert(length <= src.length && length <= dst.length); Platform.copyMemory(src.getBaseObject(), src.getBaseOffset() + srcOffset, dst.getBaseObject(), dst.getBaseOffset() + dstOffset, length); } public static final void copyMemory(MemoryBlock src, MemoryBlock dst, long length) { + assert(length <= src.length && length <= dst.length); Platform.copyMemory(src.getBaseObject(), src.getBaseOffset(), dst.getBaseObject(), dst.getBaseOffset(), length); } public final void copyFrom(Object src, long srcOffset, long dstOffset, long length) { + assert(length <= length); Platform.copyMemory(src, srcOffset, obj, offset + dstOffset, length); } public final void writeTo(long srcOffset, Object dst, long dstOffset, long length) { + assert(length <= length); Platform.copyMemory(obj, offset + srcOffset, dst, dstOffset, length); } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java index 336569f3864f9..6ca68e6575c99 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java @@ -31,58 +31,72 @@ public MemoryBlock subBlock(long offset, long size) { return new OffHeapMemoryBlock(this.offset + offset, size); } + @Override public final int getInt(long offset) { return Platform.getInt(null, offset); } + @Override public final void putInt(long offset, int value) { Platform.putInt(null, offset, value); } + @Override public final boolean getBoolean(long offset) { return Platform.getBoolean(null, offset); } + @Override public final void putBoolean(long offset, boolean value) { Platform.putBoolean(null, offset, value); } + @Override public final byte getByte(long offset) { return Platform.getByte(null, offset); } + @Override public final void putByte(long offset, byte value) { Platform.putByte(null, offset, value); } + @Override public final short getShort(long offset) { return Platform.getShort(null, offset); } + @Override public final void putShort(long offset, short value) { Platform.putShort(null, offset, value); } + @Override public final long getLong(long offset) { return Platform.getLong(null, offset); } + @Override public final void putLong(long offset, long value) { Platform.putLong(null, offset, value); } + @Override public final float getFloat(long offset) { return Platform.getFloat(null, offset); } + @Override public final void putFloat(long offset, float value) { Platform.putFloat(null, offset, value); } + @Override public final double getDouble(long offset) { return Platform.getDouble(null, offset); } + @Override public final void putDouble(long offset, double value) { Platform.putDouble(null, offset, value); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java index 5f9e3554b1067..8f9ba828784c8 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java @@ -55,71 +55,85 @@ public static OnHeapMemoryBlock fromArray(final long[] array, long size) { return new OnHeapMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, size); } + @Override public final int getInt(long offset) { assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getInt(array, offset); } + @Override public final void putInt(long offset, int value) { assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putInt(array, offset, value); } + @Override public final boolean getBoolean(long offset) { assert(offset + 1 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getBoolean(array, offset); } + @Override public final void putBoolean(long offset, boolean value) { assert(offset + 1 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putBoolean(array, offset, value); } + @Override public final byte getByte(long offset) { assert(offset + 1 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getByte(array, offset); } + @Override public final void putByte(long offset, byte value) { assert(offset + 1 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putByte(array, offset, value); } + @Override public final short getShort(long offset) { assert(offset + 2 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getShort(array, offset); } + @Override public final void putShort(long offset, short value) { assert(offset + 2 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putShort(array, offset, value); } + @Override public final long getLong(long offset) { assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getLong(array, offset); } + @Override public final void putLong(long offset, long value) { assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putLong(array, offset, value); } + @Override public final float getFloat(long offset) { assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getFloat(array, offset); } + @Override public final void putFloat(long offset, float value) { assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putFloat(array, offset, value); } + @Override public final double getDouble(long offset) { assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getDouble(array, offset); } + @Override public final void putDouble(long offset, double value) { assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putDouble(array, offset, value); diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java index 2824a2fc29a08..55321f2856ed3 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -42,8 +42,8 @@ public OffHeapMemoryBlock allocate(long size) throws OutOfMemoryError { @Override public void free(MemoryBlock memory) { - if (memory == OffHeapMemoryBlock.NULL) return; assert(memory instanceof OffHeapMemoryBlock); + if (memory == OffHeapMemoryBlock.NULL) return; assert (memory.getBaseObject() == null) : "baseObject not null; are you trying to use the off-heap allocator to free on-heap memory?"; assert (memory.getPageNumber() != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 715a4cb4cfacb..059eb73ff1f17 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -53,7 +53,7 @@ public final class UTF8String implements Comparable, Externalizable, // These are only updated by readExternal() or read() @Nonnull private MemoryBlock base; - // While numBytes has the same value as base,length, to keep as int avoids cast from long to int + // While numBytes has the same value as base, length, to keep as int avoids cast from long to int private int numBytes; public MemoryBlock getMemoryBlock() { return base; } @@ -354,7 +354,7 @@ private boolean matchAt(final UTF8String s, int pos) { if (s.numBytes + pos > numBytes || pos < 0) { return false; } - return ByteArrayMethods.arrayEqualsBlock(base, getBaseOffset() + pos, s.base, s.getBaseOffset(), s.numBytes); + return ByteArrayMethods.arrayEqualsBlock(base, pos, s.base, 0, s.numBytes); } public boolean startsWith(final UTF8String prefix) { @@ -477,13 +477,11 @@ public int findInSet(UTF8String match) { return 0; } - long offset = getBaseOffset(); - long matchOffset = match.getBaseOffset(); int n = 1, lastComma = -1; for (int i = 0; i < numBytes; i++) { if (getByte(i) == (byte) ',') { if (i - (lastComma + 1) == match.numBytes && - ByteArrayMethods.arrayEqualsBlock(base, offset + (lastComma + 1), match.base, matchOffset, + ByteArrayMethods.arrayEqualsBlock(base, (lastComma + 1), match.base, 0, match.numBytes)) { return n; } @@ -492,8 +490,7 @@ public int findInSet(UTF8String match) { } } if (numBytes - (lastComma + 1) == match.numBytes && - ByteArrayMethods.arrayEqualsBlock(base, offset + (lastComma + 1), match.base, matchOffset, - match.numBytes)) { + ByteArrayMethods.arrayEqualsBlock(base, (lastComma + 1), match.base, 0, match.numBytes)) { return n; } return 0; @@ -702,13 +699,11 @@ public int indexOf(UTF8String v, int start) { c += 1; } - long offset = getBaseOffset(); - long vOffset = v.getBaseOffset(); do { if (i + v.numBytes > numBytes) { return -1; } - if (ByteArrayMethods.arrayEqualsBlock(base, offset + i, v.base, vOffset, v.numBytes)) { + if (ByteArrayMethods.arrayEqualsBlock(base, i, v.base, 0, v.numBytes)) { return c; } i += numBytesForFirstByte(getByte(i)); @@ -723,10 +718,8 @@ public int indexOf(UTF8String v, int start) { */ private int find(UTF8String str, int start) { assert (str.numBytes > 0); - long offset = getBaseOffset(); - long strOffset = str.getBaseOffset(); while (start <= numBytes - str.numBytes) { - if (ByteArrayMethods.arrayEqualsBlock(base, offset + start, str.base, strOffset, str.numBytes)) { + if (ByteArrayMethods.arrayEqualsBlock(base, start, str.base, 0, str.numBytes)) { return start; } start += 1; @@ -739,10 +732,8 @@ private int find(UTF8String str, int start) { */ private int rfind(UTF8String str, int start) { assert (str.numBytes > 0); - long offset = getBaseOffset(); - long strOffset = str.getBaseOffset(); while (start >= 0) { - if (ByteArrayMethods.arrayEqualsBlock(base, offset + start, str.base, strOffset, str.numBytes)) { + if (ByteArrayMethods.arrayEqualsBlock(base, start, str.base, 0, str.numBytes)) { return start; } start -= 1; @@ -1252,7 +1243,7 @@ public boolean equals(final Object other) { if (numBytes != o.numBytes) { return false; } - return ByteArrayMethods.arrayEqualsBlock(base, getBaseOffset(), o.base, o.getBaseOffset(), numBytes); + return ByteArrayMethods.arrayEqualsBlock(base, 0, o.base, 0, numBytes); } else { return false; } @@ -1299,8 +1290,6 @@ public int levenshteinDistance(UTF8String other) { p[i] = i; } - long sOffset = s.getBaseOffset(); - long tOffset = t.getBaseOffset(); for (j = 0, j_bytes = 0; j < m; j_bytes += num_bytes_j, j++) { num_bytes_j = numBytesForFirstByte(t.getByte(j_bytes)); d[0] = j + 1; @@ -1310,8 +1299,8 @@ public int levenshteinDistance(UTF8String other) { num_bytes_j != numBytesForFirstByte(s.getByte(i_bytes))) { cost = 1; } else { - cost = (ByteArrayMethods.arrayEqualsBlock(t.base, tOffset + j_bytes, s.base, - sOffset + i_bytes, num_bytes_j)) ? 0 : 1; + cost = (ByteArrayMethods.arrayEqualsBlock(t.base, j_bytes, s.base, + i_bytes, num_bytes_j)) ? 0 : 1; } d[i + 1] = Math.min(Math.min(d[i] + 1, p[i + 1] + 1), p[i] + cost); } From 6f57994c6a7c52dc2cfe2fd55cd63269b469ae39 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 2 Mar 2018 17:10:26 +0000 Subject: [PATCH 41/65] fix test failures fix incorrect asserion --- .../java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java | 4 ++-- .../java/org/apache/spark/unsafe/memory/MemoryBlock.java | 4 ++-- .../main/java/org/apache/spark/unsafe/types/UTF8String.java | 5 ++--- 3 files changed, 6 insertions(+), 7 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index ec6c4942cf63f..1105899d3461c 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -69,7 +69,7 @@ public static int hashUnsafeBytesBlock(MemoryBlock base, int seed) { int lengthInBytes = (int)base.size(); assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; int lengthAligned = lengthInBytes - lengthInBytes % 4; - int h1 = hashBytesByIntBlock(base.subBlock(offset, lengthAligned), seed); + int h1 = hashBytesByIntBlock(base, seed); for (int i = lengthAligned; i < lengthInBytes; i++) { int halfWord = base.getByte(offset + i); int k1 = mixK1(halfWord); @@ -113,7 +113,7 @@ public static int hashUnsafeBytes2Block(MemoryBlock base, int seed) { int lengthInBytes = (int)base.size(); assert (lengthInBytes >= 0) : "lengthInBytes cannot be negative"; int lengthAligned = lengthInBytes - lengthInBytes % 4; - int h1 = hashBytesByIntBlock(base.subBlock(offset, lengthAligned), seed); + int h1 = hashBytesByIntBlock(base, seed); int k1 = 0; for (int i = lengthAligned, shift = 0; i < lengthInBytes; i++, shift += 8) { k1 ^= (base.getByte(offset + i) & 0xFF) << shift; diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index 936fe35a98594..dacdec01a9f84 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -176,12 +176,12 @@ public static final void copyMemory(MemoryBlock src, MemoryBlock dst, long lengt } public final void copyFrom(Object src, long srcOffset, long dstOffset, long length) { - assert(length <= length); + assert(length <= this.length); Platform.copyMemory(src, srcOffset, obj, offset + dstOffset, length); } public final void writeTo(long srcOffset, Object dst, long dstOffset, long length) { - assert(length <= length); + assert(length <= this.length); Platform.copyMemory(obj, offset + srcOffset, dst, dstOffset, length); } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 059eb73ff1f17..bcfd09b50d7dd 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -481,8 +481,7 @@ public int findInSet(UTF8String match) { for (int i = 0; i < numBytes; i++) { if (getByte(i) == (byte) ',') { if (i - (lastComma + 1) == match.numBytes && - ByteArrayMethods.arrayEqualsBlock(base, (lastComma + 1), match.base, 0, - match.numBytes)) { + ByteArrayMethods.arrayEqualsBlock(base, lastComma + 1, match.base, 0, match.numBytes)) { return n; } lastComma = i; @@ -490,7 +489,7 @@ public int findInSet(UTF8String match) { } } if (numBytes - (lastComma + 1) == match.numBytes && - ByteArrayMethods.arrayEqualsBlock(base, (lastComma + 1), match.base, 0, match.numBytes)) { + ByteArrayMethods.arrayEqualsBlock(base, lastComma + 1, match.base, 0, match.numBytes)) { return n; } return 0; From 3a93d6163b659f5ebb76a48b4948d46d44750878 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 3 Mar 2018 01:17:25 +0000 Subject: [PATCH 42/65] fix test failures --- .../main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index 1105899d3461c..911dbf32baaab 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -69,7 +69,7 @@ public static int hashUnsafeBytesBlock(MemoryBlock base, int seed) { int lengthInBytes = (int)base.size(); assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; int lengthAligned = lengthInBytes - lengthInBytes % 4; - int h1 = hashBytesByIntBlock(base, seed); + int h1 = hashBytesByIntBlock(base.subBlock(0, lengthAligned), seed); for (int i = lengthAligned; i < lengthInBytes; i++) { int halfWord = base.getByte(offset + i); int k1 = mixK1(halfWord); From abf6ba02554091d974ec7a289d318cae559bc3cb Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 3 Mar 2018 02:27:43 +0000 Subject: [PATCH 43/65] reduce duplicated code in hash --- .../sql/catalyst/expressions/HiveHasher.java | 14 +++--- .../spark/unsafe/hash/Murmur3_x86_32.java | 49 ++++++------------- .../spark/sql/catalyst/expressions/XXH64.java | 17 ++++--- 3 files changed, 32 insertions(+), 48 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java index 523cb85d3d28e..c6fd22486ab79 100644 --- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java @@ -39,16 +39,18 @@ public static int hashLong(long input) { return (int) ((input >>> 32) ^ input); } - public static int hashUnsafeBytesBlock(MemoryBlock base) { - return hashUnsafeBytes(base.getBaseObject(), base.getBaseOffset(), (int)base.size()); - } - - public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes) { + public static int hashUnsafeBytesBlock(MemoryBlock mb) { + long offset = mb.getBaseOffset(); + int lengthInBytes = (int)mb.size(); assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; int result = 0; for (int i = 0; i < lengthInBytes; i++) { - result = (result * 31) + (int) Platform.getByte(base, offset + i); + result = (result * 31) + (int) mb.getByte(offset + i); } return result; } + + public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes) { + return hashUnsafeBytesBlock(MemoryBlock.allocateFromObject(base, offset, lengthInBytes)); + } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index 911dbf32baaab..cb81178bcb14f 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -50,21 +50,27 @@ public static int hashInt(int input, int seed) { } public int hashUnsafeWords(Object base, long offset, int lengthInBytes) { - return hashUnsafeWords(base, offset, lengthInBytes, seed); + return hashUnsafeWordsBlock(MemoryBlock.allocateFromObject(base, offset, lengthInBytes), seed); } - public static int hashUnsafeWordsBlock(MemoryBlock base, long offset, int lengthInBytes, int seed) { - return hashUnsafeWords(base.getBaseObject(), offset, lengthInBytes, seed); + public static int hashUnsafeWordsBlock(MemoryBlock base, int seed) { + // This is based on Guava's `Murmur32_Hasher.processRemaining(ByteBuffer)` method. + int lengthInBytes = (int)base.size(); + assert (lengthInBytes % 8 == 0): "lengthInBytes must be a multiple of 8 (word-aligned)"; + int h1 = hashBytesByIntBlock(base, seed); + return fmix(h1, lengthInBytes); } public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, int seed) { // This is based on Guava's `Murmur32_Hasher.processRemaining(ByteBuffer)` method. assert (lengthInBytes % 8 == 0): "lengthInBytes must be a multiple of 8 (word-aligned)"; - int h1 = hashBytesByInt(base, offset, lengthInBytes, seed); + int h1 = hashBytesByIntBlock(MemoryBlock.allocateFromObject(base, offset, lengthInBytes), seed); return fmix(h1, lengthInBytes); } public static int hashUnsafeBytesBlock(MemoryBlock base, int seed) { + // This is not compatible with original and another implementations. + // But remain it for backward compatibility for the components existing before 2.3. long offset = base.getBaseOffset(); int lengthInBytes = (int)base.size(); assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; @@ -79,31 +85,11 @@ public static int hashUnsafeBytesBlock(MemoryBlock base, int seed) { } public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes, int seed) { - // This is not compatible with original and another implementations. - // But remain it for backward compatibility for the components existing before 2.3. - assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; - int lengthAligned = lengthInBytes - lengthInBytes % 4; - int h1 = hashBytesByInt(base, offset, lengthAligned, seed); - for (int i = lengthAligned; i < lengthInBytes; i++) { - int halfWord = Platform.getByte(base, offset + i); - int k1 = mixK1(halfWord); - h1 = mixH1(h1, k1); - } - return fmix(h1, lengthInBytes); + return hashUnsafeBytesBlock(MemoryBlock.allocateFromObject(base, offset, lengthInBytes), seed); } public static int hashUnsafeBytes2(Object base, long offset, int lengthInBytes, int seed) { - // This is compatible with original and another implementations. - // Use this method for new components after Spark 2.3. - assert (lengthInBytes >= 0) : "lengthInBytes cannot be negative"; - int lengthAligned = lengthInBytes - lengthInBytes % 4; - int h1 = hashBytesByInt(base, offset, lengthAligned, seed); - int k1 = 0; - for (int i = lengthAligned, shift = 0; i < lengthInBytes; i++, shift += 8) { - k1 ^= (Platform.getByte(base, offset + i) & 0xFF) << shift; - } - h1 ^= mixK1(k1); - return fmix(h1, lengthInBytes); + return hashUnsafeBytes2Block(MemoryBlock.allocateFromObject(base, offset, lengthInBytes), seed); } public static int hashUnsafeBytes2Block(MemoryBlock base, int seed) { @@ -113,7 +99,7 @@ public static int hashUnsafeBytes2Block(MemoryBlock base, int seed) { int lengthInBytes = (int)base.size(); assert (lengthInBytes >= 0) : "lengthInBytes cannot be negative"; int lengthAligned = lengthInBytes - lengthInBytes % 4; - int h1 = hashBytesByIntBlock(base, seed); + int h1 = hashBytesByIntBlock(base.subBlock(0, lengthAligned), seed); int k1 = 0; for (int i = lengthAligned, shift = 0; i < lengthInBytes; i++, shift += 8) { k1 ^= (base.getByte(offset + i) & 0xFF) << shift; @@ -136,14 +122,7 @@ private static int hashBytesByIntBlock(MemoryBlock base, int seed) { } private static int hashBytesByInt(Object base, long offset, int lengthInBytes, int seed) { - assert (lengthInBytes % 4 == 0); - int h1 = seed; - for (int i = 0; i < lengthInBytes; i += 4) { - int halfWord = Platform.getInt(base, offset + i); - int k1 = mixK1(halfWord); - h1 = mixH1(h1, k1); - } - return h1; + return hashBytesByIntBlock(MemoryBlock.allocateFromObject(base, offset, lengthInBytes), seed); } public int hashLong(long input) { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java index c9a722d1261a3..5c9eb3dc99712 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java @@ -86,30 +86,33 @@ public long hashUnsafeBytes(Object base, long offset, int length) { return hashUnsafeBytes(base, offset, length, seed); } - public static long hashUnsafeBytesBlock(MemoryBlock base, long seed) { - return hashUnsafeBytes(base.getBaseObject(), base.getBaseOffset(), (int)base.size(), seed); - } - - public static long hashUnsafeBytes(Object base, long offset, int length, long seed) { + public static long hashUnsafeBytesBlock(MemoryBlock mb, long seed) { + Object base = mb.getBaseObject(); + long offset = mb.getBaseOffset(); + int length = (int)mb.size(); assert (length >= 0) : "lengthInBytes cannot be negative"; long hash = hashBytesByWords(base, offset, length, seed); long end = offset + length; offset += length & -8; if (offset + 4L <= end) { - hash ^= (Platform.getInt(base, offset) & 0xFFFFFFFFL) * PRIME64_1; + hash ^= (mb.getInt(offset) & 0xFFFFFFFFL) * PRIME64_1; hash = Long.rotateLeft(hash, 23) * PRIME64_2 + PRIME64_3; offset += 4L; } while (offset < end) { - hash ^= (Platform.getByte(base, offset) & 0xFFL) * PRIME64_5; + hash ^= (mb.getByte(offset) & 0xFFL) * PRIME64_5; hash = Long.rotateLeft(hash, 11) * PRIME64_1; offset++; } return fmix(hash); } + public static long hashUnsafeBytes(Object base, long offset, int length, long seed) { + return hashUnsafeBytesBlock(MemoryBlock.allocateFromObject(base, offset, length), seed); + } + private static long fmix(long hash) { hash ^= hash >>> 33; hash *= PRIME64_2; From 4567781000618be81121140404df2a01430582ea Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 3 Mar 2018 12:45:06 +0000 Subject: [PATCH 44/65] address review comments --- .../java/org/apache/spark/unsafe/memory/MemoryBlock.java | 3 ++- .../org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java | 7 ++++--- .../apache/spark/unsafe/memory/UnsafeMemoryAllocator.java | 3 ++- 3 files changed, 8 insertions(+), 5 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index dacdec01a9f84..65ce79e94621a 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -119,7 +119,8 @@ public final static MemoryBlock allocateFromObject(Object obj, long offset, long // we assume that to pass null pointer means off-heap mb = new OffHeapMemoryBlock(offset, length); } else { - throw new UnsupportedOperationException(obj.getClass() + " is not supported now"); + throw new UnsupportedOperationException( + "Instantiate MemoryBlock for type " + obj.getClass() + " is not supported now"); } return mb; } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java index 8f9ba828784c8..6cb521be6a4f6 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java @@ -29,12 +29,13 @@ public final class OnHeapMemoryBlock extends MemoryBlock { public OnHeapMemoryBlock(long[] obj, long offset, long size) { super(obj, offset, size); this.array = obj; - assert(offset - Platform.LONG_ARRAY_OFFSET + size <= obj.length * 8L); + assert(offset - Platform.LONG_ARRAY_OFFSET + size <= obj.length * 8L) : + "The size " + size + " and size " + size + " are larger than the array size " + + ((obj.length * 8L) - Platform.LONG_ARRAY_OFFSET); } public OnHeapMemoryBlock(long size) { - this(new long[(int)((size + 7) / 8)], Platform.LONG_ARRAY_OFFSET, - ((size + 7) / 8) * 8L); + this(new long[(int)((size + 7) / 8)], Platform.LONG_ARRAY_OFFSET, size); } @Override diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java index 55321f2856ed3..28c053af5bcae 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -42,7 +42,8 @@ public OffHeapMemoryBlock allocate(long size) throws OutOfMemoryError { @Override public void free(MemoryBlock memory) { - assert(memory instanceof OffHeapMemoryBlock); + assert(memory instanceof OffHeapMemoryBlock) : + "UnsafeMemoryAllocator can only free OffHeapMemoryBlock."; if (memory == OffHeapMemoryBlock.NULL) return; assert (memory.getBaseObject() == null) : "baseObject not null; are you trying to use the off-heap allocator to free on-heap memory?"; From 95ffd0f4385423b87810977ba8d531eba644f2f5 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 3 Mar 2018 16:07:01 +0000 Subject: [PATCH 45/65] address review comments --- .../spark/unsafe/memory/OnHeapMemoryBlock.java | 18 ++---------------- 1 file changed, 2 insertions(+), 16 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java index 6cb521be6a4f6..73733b2413c89 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java @@ -30,8 +30,8 @@ public OnHeapMemoryBlock(long[] obj, long offset, long size) { super(obj, offset, size); this.array = obj; assert(offset - Platform.LONG_ARRAY_OFFSET + size <= obj.length * 8L) : - "The size " + size + " and size " + size + " are larger than the array size " + - ((obj.length * 8L) - Platform.LONG_ARRAY_OFFSET); + "The sum of size " + size + " and offset " + offset + " should not be larger than " + + "the array size " + ((obj.length * 8L) - Platform.LONG_ARRAY_OFFSET); } public OnHeapMemoryBlock(long size) { @@ -58,85 +58,71 @@ public static OnHeapMemoryBlock fromArray(final long[] array, long size) { @Override public final int getInt(long offset) { - assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getInt(array, offset); } @Override public final void putInt(long offset, int value) { - assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putInt(array, offset, value); } @Override public final boolean getBoolean(long offset) { - assert(offset + 1 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getBoolean(array, offset); } @Override public final void putBoolean(long offset, boolean value) { - assert(offset + 1 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putBoolean(array, offset, value); } @Override public final byte getByte(long offset) { - assert(offset + 1 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getByte(array, offset); } @Override public final void putByte(long offset, byte value) { - assert(offset + 1 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putByte(array, offset, value); } @Override public final short getShort(long offset) { - assert(offset + 2 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getShort(array, offset); } @Override public final void putShort(long offset, short value) { - assert(offset + 2 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putShort(array, offset, value); } @Override public final long getLong(long offset) { - assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getLong(array, offset); } @Override public final void putLong(long offset, long value) { - assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putLong(array, offset, value); } @Override public final float getFloat(long offset) { - assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getFloat(array, offset); } @Override public final void putFloat(long offset, float value) { - assert(offset + 4 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putFloat(array, offset, value); } @Override public final double getDouble(long offset) { - assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); return Platform.getDouble(array, offset); } @Override public final void putDouble(long offset, double value) { - assert(offset + 8 - Platform.LONG_ARRAY_OFFSET <= array.length * 8L); Platform.putDouble(array, offset, value); } } From 9cbb8762e0cc3791580e0e316930479cc83b55d7 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 4 Mar 2018 05:49:51 +0000 Subject: [PATCH 46/65] add review comment --- .../java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java index 73733b2413c89..5f052aabcfb91 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java @@ -29,9 +29,6 @@ public final class OnHeapMemoryBlock extends MemoryBlock { public OnHeapMemoryBlock(long[] obj, long offset, long size) { super(obj, offset, size); this.array = obj; - assert(offset - Platform.LONG_ARRAY_OFFSET + size <= obj.length * 8L) : - "The sum of size " + size + " and offset " + offset + " should not be larger than " + - "the array size " + ((obj.length * 8L) - Platform.LONG_ARRAY_OFFSET); } public OnHeapMemoryBlock(long size) { From 291203cd9f473cd6a5a90858eeb6a19bb053fd5d Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 4 Mar 2018 07:11:09 +0000 Subject: [PATCH 47/65] add review comments --- .../spark/unsafe/memory/ByteArrayMemoryBlock.java | 12 ++++++++++-- .../org/apache/spark/unsafe/memory/MemoryBlock.java | 2 +- .../spark/unsafe/memory/OffHeapMemoryBlock.java | 5 +++++ .../spark/unsafe/memory/OnHeapMemoryBlock.java | 8 ++++++++ .../spark/unsafe/memory/UnsafeMemoryAllocator.java | 2 -- 5 files changed, 24 insertions(+), 5 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java index 844a6ce6ce712..dead372942dff 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -26,9 +26,12 @@ public final class ByteArrayMemoryBlock extends MemoryBlock { private final byte[] array; - public ByteArrayMemoryBlock(byte[] obj, long offset, long length) { - super(obj, offset, (long)length); + public ByteArrayMemoryBlock(byte[] obj, long offset, long size) { + super(obj, offset, size); this.array = obj; + assert(offset - Platform.BYTE_ARRAY_OFFSET + size <= obj.length * 8L) : + "The sum of size " + size + " and offset " + offset + " should not be larger than " + + "the array size " + ((obj.length) - Platform.BYTE_ARRAY_OFFSET); } public ByteArrayMemoryBlock(long length) { @@ -37,6 +40,11 @@ public ByteArrayMemoryBlock(long length) { @Override public MemoryBlock subBlock(long offset, long size) { + if (offset - Platform.BYTE_ARRAY_OFFSET + size > length) { + throw new ArrayIndexOutOfBoundsException( + "The sum of size " + size + ", offset " + offset + ", and -" + Platform.BYTE_ARRAY_OFFSET + + " should not be larger than MemoryBlock length " + length); + } return new ByteArrayMemoryBlock(array, this.offset + offset, size); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index 65ce79e94621a..f0c581474ef49 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -127,7 +127,7 @@ public final static MemoryBlock allocateFromObject(Object obj, long offset, long /** * Just instantiate the same type of MemoryBlock with new offset and size. The data is not - * copied. + * copied. If parameters are invalid, an exception is thrown */ public abstract MemoryBlock subBlock(long offset, long size); diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java index 6ca68e6575c99..354198607f14f 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java @@ -28,6 +28,11 @@ public OffHeapMemoryBlock(long address, long size) { @Override public MemoryBlock subBlock(long offset, long size) { + if (offset + size > this.offset + length) { + throw new ArrayIndexOutOfBoundsException("The sum of size " + size + " and offset " + + offset + " should not be larger than " + "the sum of length " + length + " and offset " + + this.offset + " in the MemoryBlock"); + } return new OffHeapMemoryBlock(this.offset + offset, size); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java index 5f052aabcfb91..d77032621dbf5 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java @@ -29,6 +29,9 @@ public final class OnHeapMemoryBlock extends MemoryBlock { public OnHeapMemoryBlock(long[] obj, long offset, long size) { super(obj, offset, size); this.array = obj; + assert(offset - Platform.LONG_ARRAY_OFFSET + size <= obj.length * 8L) : + "The sum of size " + size + " and offset " + offset + " should not be larger than " + + "the array size " + ((obj.length * 8L) - Platform.LONG_ARRAY_OFFSET); } public OnHeapMemoryBlock(long size) { @@ -37,6 +40,11 @@ public OnHeapMemoryBlock(long size) { @Override public MemoryBlock subBlock(long offset, long size) { + if (offset - Platform.LONG_ARRAY_OFFSET + size > length) { + throw new ArrayIndexOutOfBoundsException( + "The sum of size " + size + ", offset " + offset + ", and -" + Platform.LONG_ARRAY_OFFSET + + " should not be larger than MemoryBlock length " + length); + } return new OnHeapMemoryBlock(array, this.offset + offset, size); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java index 28c053af5bcae..a78b52388bb0e 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -45,8 +45,6 @@ public void free(MemoryBlock memory) { assert(memory instanceof OffHeapMemoryBlock) : "UnsafeMemoryAllocator can only free OffHeapMemoryBlock."; if (memory == OffHeapMemoryBlock.NULL) return; - assert (memory.getBaseObject() == null) : - "baseObject not null; are you trying to use the off-heap allocator to free on-heap memory?"; assert (memory.getPageNumber() != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : "page has already been freed"; assert ((memory.getPageNumber() == MemoryBlock.NO_PAGE_NUMBER) From a62770bdcd2cd83dc19d5f39a55b5186201ddc34 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 6 Mar 2018 02:57:29 +0000 Subject: [PATCH 48/65] address review comments --- .../spark/unsafe/bitset/BitSetMethods.java | 1 - .../unsafe/memory/ByteArrayMemoryBlock.java | 4 +++- .../spark/unsafe/memory/OnHeapMemoryBlock.java | 4 +++- .../unsafe/memory/UnsafeMemoryAllocator.java | 3 +-- .../apache/spark/unsafe/types/UTF8String.java | 16 +++++----------- .../spark/unsafe/types/UTF8StringSuite.java | 8 ++++---- .../catalyst/expressions/UnsafeArrayData.java | 2 +- .../sql/catalyst/expressions/UnsafeRow.java | 2 +- .../vectorized/OffHeapColumnVector.java | 2 +- .../spark/sql/vectorized/ArrowColumnVector.java | 2 +- 10 files changed, 20 insertions(+), 24 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java index 6d06f4ab9b3b0..c8c57381f332f 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java @@ -18,7 +18,6 @@ package org.apache.spark.unsafe.bitset; import org.apache.spark.unsafe.Platform; -import org.apache.spark.unsafe.memory.MemoryBlock; /** * Methods for working with fixed-size uncompressed bitsets. diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java index dead372942dff..367020627eca3 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -17,6 +17,8 @@ package org.apache.spark.unsafe.memory; +import com.google.common.primitives.Ints; + import org.apache.spark.unsafe.Platform; /** @@ -35,7 +37,7 @@ public ByteArrayMemoryBlock(byte[] obj, long offset, long size) { } public ByteArrayMemoryBlock(long length) { - this(new byte[(int)length], Platform.BYTE_ARRAY_OFFSET, length); + this(new byte[Ints.checkedCast(length)], Platform.BYTE_ARRAY_OFFSET, length); } @Override diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java index d77032621dbf5..2dfd69c673fd4 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java @@ -17,6 +17,8 @@ package org.apache.spark.unsafe.memory; +import com.google.common.primitives.Ints; + import org.apache.spark.unsafe.Platform; /** @@ -35,7 +37,7 @@ public OnHeapMemoryBlock(long[] obj, long offset, long size) { } public OnHeapMemoryBlock(long size) { - this(new long[(int)((size + 7) / 8)], Platform.LONG_ARRAY_OFFSET, size); + this(new long[Ints.checkedCast((size + 7) / 8)], Platform.LONG_ARRAY_OFFSET, size); } @Override diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java index a78b52388bb0e..d36f7d3b8d821 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -68,8 +68,7 @@ public OffHeapMemoryBlock reallocate(OffHeapMemoryBlock block, long oldSize, lon OffHeapMemoryBlock mb = this.allocate(newSize); if (block.getBaseOffset() != 0) MemoryBlock.copyMemory(block, mb, oldSize); - if (block != mb) - free(block); + free(block); return mb; } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index bcfd09b50d7dd..f15fd24c57208 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -53,11 +53,12 @@ public final class UTF8String implements Comparable, Externalizable, // These are only updated by readExternal() or read() @Nonnull private MemoryBlock base; - // While numBytes has the same value as base, length, to keep as int avoids cast from long to int + // While numBytes has the same value as base.size(), to keep as int avoids cast from long to int private int numBytes; public MemoryBlock getMemoryBlock() { return base; } - private long getBaseOffset() { return base.getBaseOffset(); } + public Object getBaseObject() { return base.getBaseObject(); } + public long getBaseOffset() { return base.getBaseOffset(); } private static int[] bytesOfCodePointInUTF8 = {2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, @@ -98,13 +99,6 @@ public static UTF8String fromBytes(byte[] bytes, int offset, int numBytes) { } } - /** - * Creates an UTF8String from given memory block. - */ - public static UTF8String fromMemoryBlock(MemoryBlock base) { - return new UTF8String(base); - } - /** * Creates an UTF8String from String. */ @@ -125,7 +119,7 @@ protected UTF8String(byte[] bytes, long offset, int numBytes) { this(new ByteArrayMemoryBlock(bytes, offset, numBytes)); } - protected UTF8String(MemoryBlock base) { + public UTF8String(MemoryBlock base) { this.base = base; if (base != null) { if ((long) Integer.MAX_VALUE < base.size()) { @@ -146,7 +140,7 @@ public UTF8String() { * The target memory address must already been allocated, and have enough space to hold all the * bytes in this string. */ - public void writeToMemory(byte[] target, long targetOffset) { + public void writeToMemory(Object target, long targetOffset) { base.writeTo(0, target, targetOffset, numBytes); } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java index 9bf2c5ed3370b..6646e9cd208fa 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java @@ -518,7 +518,7 @@ public void writeToOutputStreamUnderflow() throws IOException { final byte[] test = "01234567".getBytes(StandardCharsets.UTF_8); for (int i = 1; i <= Platform.BYTE_ARRAY_OFFSET; ++i) { - UTF8String.fromMemoryBlock(ByteArrayMemoryBlock.fromArray(test) + new UTF8String(ByteArrayMemoryBlock.fromArray(test) .subBlock(-i, test.length + i)).writeTo(outputStream); final ByteBuffer buffer = ByteBuffer.wrap(outputStream.toByteArray(), i, test.length); assertEquals("01234567", StandardCharsets.UTF_8.decode(buffer).toString()); @@ -533,7 +533,7 @@ public void writeToOutputStreamSlice() throws IOException { for (int i = 0; i < test.length; ++i) { for (int j = 0; j < test.length - i; ++j) { - UTF8String.fromMemoryBlock(ByteArrayMemoryBlock.fromArray(test).subBlock(i, j)) + new UTF8String(ByteArrayMemoryBlock.fromArray(test).subBlock(i, j)) .writeTo(outputStream); assertArrayEquals(Arrays.copyOfRange(test, i, i + j), outputStream.toByteArray()); @@ -564,7 +564,7 @@ public void writeToOutputStreamOverflow() throws IOException { for (final long offset : offsets) { try { - fromMemoryBlock(ByteArrayMemoryBlock.fromArray(test).subBlock(offset, test.length)) + new UTF8String(ByteArrayMemoryBlock.fromArray(test).subBlock(offset, test.length)) .writeTo(outputStream); throw new IllegalStateException(Long.toString(offset)); @@ -608,7 +608,7 @@ public void writeToOutputStreamLongArray() throws IOException { } final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - fromMemoryBlock(OnHeapMemoryBlock.fromArray(array)).writeTo(outputStream); + new UTF8String(OnHeapMemoryBlock.fromArray(array)).writeTo(outputStream); assertEquals("3千大千世界", outputStream.toString("UTF-8")); } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java index 7fd3d18cb5046..f80806c5f55e1 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java @@ -234,7 +234,7 @@ public UTF8String getUTF8String(int ordinal) { final int offset = (int) (offsetAndSize >> 32); final int size = (int) offsetAndSize; MemoryBlock mb = MemoryBlock.allocateFromObject(baseObject, baseOffset + offset, size); - return UTF8String.fromMemoryBlock(mb); + return new UTF8String(mb); } @Override diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index e612af931c351..72097ba9c9abb 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -416,7 +416,7 @@ public UTF8String getUTF8String(int ordinal) { final int offset = (int) (offsetAndSize >> 32); final int size = (int) offsetAndSize; MemoryBlock mb = MemoryBlock.allocateFromObject(baseObject, baseOffset + offset, size); - return UTF8String.fromMemoryBlock(mb); + return new UTF8String(mb); } @Override diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java index ceaa6aa052af7..880d322ef2c64 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -208,7 +208,7 @@ public byte[] getBytes(int rowId, int count) { @Override protected UTF8String getBytesAsUTF8String(int rowId, int count) { - return UTF8String.fromMemoryBlock(data.subBlock(rowId, count)); + return new UTF8String(data.subBlock(rowId, count)); } // diff --git a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java index 58ab5c08ed278..8357942abb26d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java @@ -381,7 +381,7 @@ final UTF8String getUTF8String(int rowId) { int size = stringResult.end - stringResult.start; OffHeapMemoryBlock mb = new OffHeapMemoryBlock( stringResult.buffer.memoryAddress() + stringResult.start, size); - return UTF8String.fromMemoryBlock(mb); + return new UTF8String(mb); } } } From f9bc4d6b7f8e4f7dc43ee944d3889b6449f28496 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 7 Mar 2018 12:37:30 +0000 Subject: [PATCH 49/65] address review comments --- .../org/apache/spark/unsafe/Platform.java | 7 + .../unsafe/memory/UnsafeMemoryAllocator.java | 5 +- .../vectorized/OffHeapColumnVector.java | 219 +++++++++--------- 3 files changed, 120 insertions(+), 111 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java index e39959594ccc4..54dcadf3a7754 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java @@ -147,6 +147,13 @@ public static void freeMemory(long address) { _UNSAFE.freeMemory(address); } + public static long reallocateMemory(long address, long oldSize, long newSize) { + long newMemory = _UNSAFE.allocateMemory(newSize); + copyMemory(null, address, null, newMemory, oldSize); + freeMemory(address); + return newMemory; + } + /** * Uses internal JDK APIs to allocate a DirectByteBuffer while ignoring the JVM's * MaxDirectMemorySize limit (the default limit is too low and we do not want to require users diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java index d36f7d3b8d821..e6c89334e72d4 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -66,9 +66,10 @@ public void free(MemoryBlock memory) { public OffHeapMemoryBlock reallocate(OffHeapMemoryBlock block, long oldSize, long newSize) { OffHeapMemoryBlock mb = this.allocate(newSize); - if (block.getBaseOffset() != 0) + if (block.getBaseOffset() != 0) { MemoryBlock.copyMemory(block, mb, oldSize); - free(block); + free(block); + } return mb; } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java index 880d322ef2c64..4733f36174f42 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -23,9 +23,8 @@ import org.apache.spark.sql.types.*; import org.apache.spark.unsafe.Platform; -import org.apache.spark.unsafe.types.UTF8String; -import org.apache.spark.unsafe.memory.MemoryAllocator; import org.apache.spark.unsafe.memory.OffHeapMemoryBlock; +import org.apache.spark.unsafe.types.UTF8String; /** * Column data backed using offheap memory. @@ -59,20 +58,20 @@ public static OffHeapColumnVector[] allocateColumns(int capacity, StructField[] // The data stored in these two allocations need to maintain binary compatible. We can // directly pass this buffer to external components. - private OffHeapMemoryBlock nulls; - private OffHeapMemoryBlock data; + private long nulls; + private long data; // Only set if type is Array or Map. - private OffHeapMemoryBlock lengthData; - private OffHeapMemoryBlock offsetData; + private long lengthData; + private long offsetData; public OffHeapColumnVector(int capacity, DataType type) { super(capacity, type); - nulls = OffHeapMemoryBlock.NULL; - data = OffHeapMemoryBlock.NULL; - lengthData = OffHeapMemoryBlock.NULL; - offsetData = OffHeapMemoryBlock.NULL; + nulls = 0; + data = 0; + lengthData = 0; + offsetData = 0; reserveInternal(capacity); reset(); @@ -83,20 +82,20 @@ public OffHeapColumnVector(int capacity, DataType type) { */ @VisibleForTesting public long valuesNativeAddress() { - return data.getBaseOffset(); + return data; } @Override public void close() { super.close(); - MemoryAllocator.UNSAFE.free(nulls); - MemoryAllocator.UNSAFE.free(data); - MemoryAllocator.UNSAFE.free(lengthData); - MemoryAllocator.UNSAFE.free(offsetData); - nulls = OffHeapMemoryBlock.NULL; - data = OffHeapMemoryBlock.NULL; - lengthData = OffHeapMemoryBlock.NULL; - offsetData = OffHeapMemoryBlock.NULL; + Platform.freeMemory(nulls); + Platform.freeMemory(data); + Platform.freeMemory(lengthData); + Platform.freeMemory(offsetData); + nulls = 0; + data = 0; + lengthData = 0; + offsetData = 0; } // @@ -105,20 +104,20 @@ public void close() { @Override public void putNotNull(int rowId) { - nulls.putByte(nulls.getBaseOffset() + rowId, (byte) 0); + Platform.putByte(null, nulls + rowId, (byte) 0); } @Override public void putNull(int rowId) { - nulls.putByte(nulls.getBaseOffset() + rowId, (byte) 1); + Platform.putByte(null, nulls + rowId, (byte) 1); ++numNulls; } @Override public void putNulls(int rowId, int count) { - long offset = nulls.getBaseOffset() + rowId; + long offset = nulls + rowId; for (int i = 0; i < count; ++i, ++offset) { - nulls.putByte(offset, (byte) 1); + Platform.putByte(null, offset, (byte) 1); } numNulls += count; } @@ -126,15 +125,15 @@ public void putNulls(int rowId, int count) { @Override public void putNotNulls(int rowId, int count) { if (!hasNull()) return; - long offset = nulls.getBaseOffset() + rowId; + long offset = nulls + rowId; for (int i = 0; i < count; ++i, ++offset) { - nulls.putByte(offset, (byte) 0); + Platform.putByte(null, offset, (byte) 0); } } @Override public boolean isNullAt(int rowId) { - return nulls.getByte(nulls.getBaseOffset() + rowId) == 1; + return Platform.getByte(null, nulls + rowId) == 1; } // @@ -143,26 +142,26 @@ public boolean isNullAt(int rowId) { @Override public void putBoolean(int rowId, boolean value) { - data.putByte(data.getBaseOffset() + rowId, (byte)((value) ? 1 : 0)); + Platform.putByte(null, data + rowId, (byte)((value) ? 1 : 0)); } @Override public void putBooleans(int rowId, int count, boolean value) { byte v = (byte)((value) ? 1 : 0); for (int i = 0; i < count; ++i) { - data.putByte(data.getBaseOffset() + rowId + i, v); + Platform.putByte(null, data + rowId + i, v); } } @Override - public boolean getBoolean(int rowId) { return data.getByte(data.getBaseOffset() + rowId) == 1; } + public boolean getBoolean(int rowId) { return Platform.getByte(null, data + rowId) == 1; } @Override public boolean[] getBooleans(int rowId, int count) { assert(dictionary == null); boolean[] array = new boolean[count]; for (int i = 0; i < count; ++i) { - array[i] = (data.getByte(data.getBaseOffset() + rowId + i) == 1); + array[i] = (Platform.getByte(null, data + rowId + i) == 1); } return array; } @@ -173,26 +172,26 @@ public boolean[] getBooleans(int rowId, int count) { @Override public void putByte(int rowId, byte value) { - data.putByte(data.getBaseOffset() + rowId, value); + Platform.putByte(null, data + rowId, value); } @Override public void putBytes(int rowId, int count, byte value) { for (int i = 0; i < count; ++i) { - data.putByte(data.getBaseOffset() + rowId + i, value); + Platform.putByte(null, data + rowId + i, value); } } @Override public void putBytes(int rowId, int count, byte[] src, int srcIndex) { - data.copyFrom(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, rowId, count); + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, null, data + rowId, count); } @Override public byte getByte(int rowId) { if (dictionary == null) { - return data.getByte(data.getBaseOffset() + rowId); + return Platform.getByte(null, data + rowId); } else { return (byte) dictionary.decodeToInt(dictionaryIds.getDictId(rowId)); } @@ -202,13 +201,13 @@ public byte getByte(int rowId) { public byte[] getBytes(int rowId, int count) { assert(dictionary == null); byte[] array = new byte[count]; - data.writeTo(rowId, array, Platform.BYTE_ARRAY_OFFSET, count); + Platform.copyMemory(null, data + rowId, array, Platform.BYTE_ARRAY_OFFSET, count); return array; } @Override protected UTF8String getBytesAsUTF8String(int rowId, int count) { - return new UTF8String(data.subBlock(rowId, count)); + return new UTF8String(new OffHeapMemoryBlock(data + rowId, count)); } // @@ -217,33 +216,33 @@ protected UTF8String getBytesAsUTF8String(int rowId, int count) { @Override public void putShort(int rowId, short value) { - data.putShort(data.getBaseOffset() + 2 * rowId, value); + Platform.putShort(null, data + 2 * rowId, value); } @Override public void putShorts(int rowId, int count, short value) { - long offset = data.getBaseOffset() + 2 * rowId; + long offset = data + 2 * rowId; for (int i = 0; i < count; ++i, offset += 2) { - data.putShort(offset, value); + Platform.putShort(null, offset, value); } } @Override public void putShorts(int rowId, int count, short[] src, int srcIndex) { - data.copyFrom(src, Platform.SHORT_ARRAY_OFFSET + srcIndex * 2, - 2 * rowId, count * 2); + Platform.copyMemory(src, Platform.SHORT_ARRAY_OFFSET + srcIndex * 2, + null, data + 2 * rowId, count * 2); } @Override public void putShorts(int rowId, int count, byte[] src, int srcIndex) { - data.copyFrom(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - rowId * 2, count * 2); + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + null, data + rowId * 2, count * 2); } @Override public short getShort(int rowId) { if (dictionary == null) { - return data.getShort(data.getBaseOffset() + 2 * rowId); + return Platform.getShort(null, data + 2 * rowId); } else { return (short) dictionary.decodeToInt(dictionaryIds.getDictId(rowId)); } @@ -253,7 +252,7 @@ public short getShort(int rowId) { public short[] getShorts(int rowId, int count) { assert(dictionary == null); short[] array = new short[count]; - data.writeTo(rowId * 2, array, Platform.SHORT_ARRAY_OFFSET, count * 2); + Platform.copyMemory(null, data + rowId * 2, array, Platform.SHORT_ARRAY_OFFSET, count * 2); return array; } @@ -263,39 +262,40 @@ public short[] getShorts(int rowId, int count) { @Override public void putInt(int rowId, int value) { - data.putInt(data.getBaseOffset() + 4 * rowId, value); + Platform.putInt(null, data + 4 * rowId, value); } @Override public void putInts(int rowId, int count, int value) { - long offset = data.getBaseOffset() + 4 * rowId; + long offset = data + 4 * rowId; for (int i = 0; i < count; ++i, offset += 4) { - data.putInt(offset, value); + Platform.putInt(null, offset, value); } } @Override public void putInts(int rowId, int count, int[] src, int srcIndex) { - data.copyFrom(src, Platform.INT_ARRAY_OFFSET + srcIndex * 4, - 4 * rowId, count * 4); + Platform.copyMemory(src, Platform.INT_ARRAY_OFFSET + srcIndex * 4, + null, data + 4 * rowId, count * 4); } @Override public void putInts(int rowId, int count, byte[] src, int srcIndex) { - data.copyFrom(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - rowId * 4, count * 4); + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + null, data + rowId * 4, count * 4); } @Override public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { - data.copyFrom(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, - 4 * rowId, count * 4); + Platform.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, + null, data + 4 * rowId, count * 4); } else { int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; - long offset = data.getBaseOffset() + 4 * rowId; + long offset = data + 4 * rowId; for (int i = 0; i < count; ++i, offset += 4, srcOffset += 4) { - data.putInt(offset, java.lang.Integer.reverseBytes(Platform.getInt(src, srcOffset))); + Platform.putInt(null, offset, + java.lang.Integer.reverseBytes(Platform.getInt(src, srcOffset))); } } } @@ -303,7 +303,7 @@ public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) @Override public int getInt(int rowId) { if (dictionary == null) { - return data.getInt(data.getBaseOffset() + 4 * rowId); + return Platform.getInt(null, data + 4 * rowId); } else { return dictionary.decodeToInt(dictionaryIds.getDictId(rowId)); } @@ -313,7 +313,7 @@ public int getInt(int rowId) { public int[] getInts(int rowId, int count) { assert(dictionary == null); int[] array = new int[count]; - data.writeTo(rowId * 4, array, Platform.INT_ARRAY_OFFSET, count * 4); + Platform.copyMemory(null, data + rowId * 4, array, Platform.INT_ARRAY_OFFSET, count * 4); return array; } @@ -325,7 +325,7 @@ public int[] getInts(int rowId, int count) { public int getDictId(int rowId) { assert(dictionary == null) : "A ColumnVector dictionary should not have a dictionary for itself."; - return data.getInt(data.getBaseOffset() + 4 * rowId); + return Platform.getInt(null, data + 4 * rowId); } // @@ -334,39 +334,40 @@ public int getDictId(int rowId) { @Override public void putLong(int rowId, long value) { - data.putLong(data.getBaseOffset() + 8 * rowId, value); + Platform.putLong(null, data + 8 * rowId, value); } @Override public void putLongs(int rowId, int count, long value) { - long offset = data.getBaseOffset() + 8 * rowId; + long offset = data + 8 * rowId; for (int i = 0; i < count; ++i, offset += 8) { - data.putLong(offset, value); + Platform.putLong(null, offset, value); } } @Override public void putLongs(int rowId, int count, long[] src, int srcIndex) { - data.copyFrom(src, Platform.LONG_ARRAY_OFFSET + srcIndex * 8, - 8 * rowId, count * 8); + Platform.copyMemory(src, Platform.LONG_ARRAY_OFFSET + srcIndex * 8, + null, data + 8 * rowId, count * 8); } @Override public void putLongs(int rowId, int count, byte[] src, int srcIndex) { - data.copyFrom(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - rowId * 8, count * 8); + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + null, data + rowId * 8, count * 8); } @Override public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { - data.copyFrom(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, - 8 * rowId, count * 8); + Platform.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, + null, data + 8 * rowId, count * 8); } else { int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; - long offset = data.getBaseOffset() + 8 * rowId; + long offset = data + 8 * rowId; for (int i = 0; i < count; ++i, offset += 8, srcOffset += 8) { - data.putLong(offset, java.lang.Long.reverseBytes(Platform.getLong(src, srcOffset))); + Platform.putLong(null, offset, + java.lang.Long.reverseBytes(Platform.getLong(src, srcOffset))); } } } @@ -374,7 +375,7 @@ public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) @Override public long getLong(int rowId) { if (dictionary == null) { - return data.getLong(data.getBaseOffset() + 8 * rowId); + return Platform.getLong(null, data + 8 * rowId); } else { return dictionary.decodeToLong(dictionaryIds.getDictId(rowId)); } @@ -384,7 +385,7 @@ public long getLong(int rowId) { public long[] getLongs(int rowId, int count) { assert(dictionary == null); long[] array = new long[count]; - data.writeTo(rowId * 8, array, Platform.LONG_ARRAY_OFFSET, count * 8); + Platform.copyMemory(null, data + rowId * 8, array, Platform.LONG_ARRAY_OFFSET, count * 8); return array; } @@ -394,33 +395,33 @@ public long[] getLongs(int rowId, int count) { @Override public void putFloat(int rowId, float value) { - data.putFloat(data.getBaseOffset() + rowId * 4, value); + Platform.putFloat(null, data + rowId * 4, value); } @Override public void putFloats(int rowId, int count, float value) { - long offset = data.getBaseOffset() + 4 * rowId; + long offset = data + 4 * rowId; for (int i = 0; i < count; ++i, offset += 4) { - data.putFloat(offset, value); + Platform.putFloat(null, offset, value); } } @Override public void putFloats(int rowId, int count, float[] src, int srcIndex) { - data.copyFrom(src, Platform.FLOAT_ARRAY_OFFSET + srcIndex * 4, - 4 * rowId, count * 4); + Platform.copyMemory(src, Platform.FLOAT_ARRAY_OFFSET + srcIndex * 4, + null, data + 4 * rowId, count * 4); } @Override public void putFloats(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { - data.copyFrom(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - rowId * 4, count * 4); + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + null, data + rowId * 4, count * 4); } else { ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); - long offset = data.getBaseOffset() + 4 * rowId; + long offset = data + 4 * rowId; for (int i = 0; i < count; ++i, offset += 4) { - data.putFloat(offset, bb.getFloat(srcIndex + (4 * i))); + Platform.putFloat(null, offset, bb.getFloat(srcIndex + (4 * i))); } } } @@ -428,7 +429,7 @@ public void putFloats(int rowId, int count, byte[] src, int srcIndex) { @Override public float getFloat(int rowId) { if (dictionary == null) { - return data.getFloat(data.getBaseOffset() + rowId * 4); + return Platform.getFloat(null, data + rowId * 4); } else { return dictionary.decodeToFloat(dictionaryIds.getDictId(rowId)); } @@ -438,7 +439,7 @@ public float getFloat(int rowId) { public float[] getFloats(int rowId, int count) { assert(dictionary == null); float[] array = new float[count]; - data.writeTo(rowId * 4, array, Platform.FLOAT_ARRAY_OFFSET, count * 4); + Platform.copyMemory(null, data + rowId * 4, array, Platform.FLOAT_ARRAY_OFFSET, count * 4); return array; } @@ -449,33 +450,33 @@ public float[] getFloats(int rowId, int count) { @Override public void putDouble(int rowId, double value) { - data.putDouble(data.getBaseOffset() + rowId * 8, value); + Platform.putDouble(null, data + rowId * 8, value); } @Override public void putDoubles(int rowId, int count, double value) { - long offset = data.getBaseOffset() + 8 * rowId; + long offset = data + 8 * rowId; for (int i = 0; i < count; ++i, offset += 8) { - data.putDouble(offset, value); + Platform.putDouble(null, offset, value); } } @Override public void putDoubles(int rowId, int count, double[] src, int srcIndex) { - data.copyFrom(src, Platform.DOUBLE_ARRAY_OFFSET + srcIndex * 8, - 8 * rowId, count * 8); + Platform.copyMemory(src, Platform.DOUBLE_ARRAY_OFFSET + srcIndex * 8, + null, data + 8 * rowId, count * 8); } @Override public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { - data.copyFrom(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - rowId * 8, count * 8); + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + null, data + rowId * 8, count * 8); } else { ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); - long offset = data.getBaseOffset() + 8 * rowId; + long offset = data + 8 * rowId; for (int i = 0; i < count; ++i, offset += 8) { - data.putDouble(offset, bb.getDouble(srcIndex + (8 * i))); + Platform.putDouble(null, offset, bb.getDouble(srcIndex + (8 * i))); } } } @@ -483,7 +484,7 @@ public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { @Override public double getDouble(int rowId) { if (dictionary == null) { - return data.getDouble(data.getBaseOffset() + rowId * 8); + return Platform.getDouble(null, data + rowId * 8); } else { return dictionary.decodeToDouble(dictionaryIds.getDictId(rowId)); } @@ -493,7 +494,7 @@ public double getDouble(int rowId) { public double[] getDoubles(int rowId, int count) { assert(dictionary == null); double[] array = new double[count]; - data.writeTo(rowId * 8, array, Platform.DOUBLE_ARRAY_OFFSET, count * 8); + Platform.copyMemory(null, data + rowId * 8, array, Platform.DOUBLE_ARRAY_OFFSET, count * 8); return array; } @@ -503,55 +504,55 @@ public double[] getDoubles(int rowId, int count) { @Override public void putArray(int rowId, int offset, int length) { assert(offset >= 0 && offset + length <= childColumns[0].capacity); - lengthData.putInt(lengthData.getBaseOffset() + 4 * rowId, length); - offsetData.putInt(offsetData.getBaseOffset() + 4 * rowId, offset); + Platform.putInt(null, lengthData + 4 * rowId, length); + Platform.putInt(null, offsetData + 4 * rowId, offset); } @Override public int getArrayLength(int rowId) { - return lengthData.getInt(lengthData.getBaseOffset() + 4 * rowId); + return Platform.getInt(null, lengthData + 4 * rowId); } @Override public int getArrayOffset(int rowId) { - return offsetData.getInt(offsetData.getBaseOffset() + 4 * rowId); + return Platform.getInt(null, offsetData + 4 * rowId); } // APIs dealing with ByteArrays @Override public int putByteArray(int rowId, byte[] value, int offset, int length) { int result = arrayData().appendBytes(length, value, offset); - lengthData.putInt(lengthData.getBaseOffset() + 4 * rowId, length); - offsetData.putInt(offsetData.getBaseOffset() + 4 * rowId, result); + Platform.putInt(null, lengthData + 4 * rowId, length); + Platform.putInt(null, offsetData + 4 * rowId, result); return result; } // Split out the slow path. @Override protected void reserveInternal(int newCapacity) { - int oldCapacity = (nulls == OffHeapMemoryBlock.NULL) ? 0 : capacity; + int oldCapacity = (nulls == 0L) ? 0 : capacity; if (isArray() || type instanceof MapType) { this.lengthData = - MemoryAllocator.UNSAFE.reallocate(lengthData, oldCapacity * 4, newCapacity * 4); + Platform.reallocateMemory(lengthData, oldCapacity * 4, newCapacity * 4); this.offsetData = - MemoryAllocator.UNSAFE.reallocate(offsetData, oldCapacity * 4, newCapacity * 4); + Platform.reallocateMemory(offsetData, oldCapacity * 4, newCapacity * 4); } else if (type instanceof ByteType || type instanceof BooleanType) { - this.data = MemoryAllocator.UNSAFE.reallocate(data, oldCapacity, newCapacity); + this.data = Platform.reallocateMemory(data, oldCapacity, newCapacity); } else if (type instanceof ShortType) { - this.data = MemoryAllocator.UNSAFE.reallocate(data, oldCapacity * 2, newCapacity * 2); + this.data = Platform.reallocateMemory(data, oldCapacity * 2, newCapacity * 2); } else if (type instanceof IntegerType || type instanceof FloatType || type instanceof DateType || DecimalType.is32BitDecimalType(type)) { - this.data = MemoryAllocator.UNSAFE.reallocate(data, oldCapacity * 4, newCapacity * 4); + this.data = Platform.reallocateMemory(data, oldCapacity * 4, newCapacity * 4); } else if (type instanceof LongType || type instanceof DoubleType || DecimalType.is64BitDecimalType(type) || type instanceof TimestampType) { - this.data = MemoryAllocator.UNSAFE.reallocate(data, oldCapacity * 8, newCapacity * 8); + this.data = Platform.reallocateMemory(data, oldCapacity * 8, newCapacity * 8); } else if (childColumns != null) { // Nothing to store. } else { throw new RuntimeException("Unhandled " + type); } - this.nulls = MemoryAllocator.UNSAFE.reallocate(nulls, oldCapacity, newCapacity); - Platform.setMemory(nulls.getBaseOffset() + oldCapacity, (byte)0, newCapacity - oldCapacity); + this.nulls = Platform.reallocateMemory(nulls, oldCapacity, newCapacity); + Platform.setMemory(nulls + oldCapacity, (byte)0, newCapacity - oldCapacity); capacity = newCapacity; } From c6d45ea3eed791dbd67246068d77b9b239c209e6 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 16 Mar 2018 06:07:10 +0100 Subject: [PATCH 50/65] address review comments --- .../sql/catalyst/expressions/HiveHasher.java | 4 +-- .../spark/unsafe/hash/Murmur3_x86_32.java | 25 ++++++++-------- .../unsafe/memory/ByteArrayMemoryBlock.java | 8 ++--- .../spark/unsafe/memory/MemoryBlock.java | 20 +++++++++++-- .../unsafe/memory/OffHeapMemoryBlock.java | 6 +--- .../unsafe/memory/OnHeapMemoryBlock.java | 6 +--- .../spark/unsafe/memory/MemoryBlockSuite.java | 30 +++++++++++++++++++ .../spark/sql/catalyst/expressions/XXH64.java | 6 ++-- 8 files changed, 71 insertions(+), 34 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java index c6fd22486ab79..bd53553a3d4d4 100644 --- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java @@ -41,10 +41,10 @@ public static int hashLong(long input) { public static int hashUnsafeBytesBlock(MemoryBlock mb) { long offset = mb.getBaseOffset(); - int lengthInBytes = (int)mb.size(); + long lengthInBytes = mb.size(); assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; int result = 0; - for (int i = 0; i < lengthInBytes; i++) { + for (long i = 0; i < lengthInBytes; i++) { result = (result * 31) + (int) mb.getByte(offset + i); } return result; diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index cb81178bcb14f..1fb7343a0ade6 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -17,7 +17,8 @@ package org.apache.spark.unsafe.hash; -import org.apache.spark.unsafe.Platform; +import com.google.common.primitives.Ints; + import org.apache.spark.unsafe.memory.MemoryBlock; /** @@ -55,7 +56,7 @@ public int hashUnsafeWords(Object base, long offset, int lengthInBytes) { public static int hashUnsafeWordsBlock(MemoryBlock base, int seed) { // This is based on Guava's `Murmur32_Hasher.processRemaining(ByteBuffer)` method. - int lengthInBytes = (int)base.size(); + int lengthInBytes = Ints.checkedCast(base.size()); assert (lengthInBytes % 8 == 0): "lengthInBytes must be a multiple of 8 (word-aligned)"; int h1 = hashBytesByIntBlock(base, seed); return fmix(h1, lengthInBytes); @@ -72,16 +73,16 @@ public static int hashUnsafeBytesBlock(MemoryBlock base, int seed) { // This is not compatible with original and another implementations. // But remain it for backward compatibility for the components existing before 2.3. long offset = base.getBaseOffset(); - int lengthInBytes = (int)base.size(); + long lengthInBytes = base.size(); assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; - int lengthAligned = lengthInBytes - lengthInBytes % 4; + long lengthAligned = lengthInBytes - lengthInBytes % 4; int h1 = hashBytesByIntBlock(base.subBlock(0, lengthAligned), seed); - for (int i = lengthAligned; i < lengthInBytes; i++) { + for (long i = lengthAligned; i < lengthInBytes; i++) { int halfWord = base.getByte(offset + i); int k1 = mixK1(halfWord); h1 = mixH1(h1, k1); } - return fmix(h1, lengthInBytes); + return fmix(h1, Ints.checkedCast(lengthInBytes)); } public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes, int seed) { @@ -96,24 +97,24 @@ public static int hashUnsafeBytes2Block(MemoryBlock base, int seed) { // This is compatible with original and another implementations. // Use this method for new components after Spark 2.3. long offset = base.getBaseOffset(); - int lengthInBytes = (int)base.size(); + long lengthInBytes = base.size(); assert (lengthInBytes >= 0) : "lengthInBytes cannot be negative"; - int lengthAligned = lengthInBytes - lengthInBytes % 4; + long lengthAligned = lengthInBytes - lengthInBytes % 4; int h1 = hashBytesByIntBlock(base.subBlock(0, lengthAligned), seed); int k1 = 0; - for (int i = lengthAligned, shift = 0; i < lengthInBytes; i++, shift += 8) { + for (long i = lengthAligned, shift = 0; i < lengthInBytes; i++, shift += 8) { k1 ^= (base.getByte(offset + i) & 0xFF) << shift; } h1 ^= mixK1(k1); - return fmix(h1, lengthInBytes); + return fmix(h1, Ints.checkedCast(lengthInBytes)); } private static int hashBytesByIntBlock(MemoryBlock base, int seed) { long offset = base.getBaseOffset(); - int lengthInBytes = (int)base.size(); + long lengthInBytes = base.size(); assert (lengthInBytes % 4 == 0); int h1 = seed; - for (int i = 0; i < lengthInBytes; i += 4) { + for (long i = 0; i < lengthInBytes; i += 4) { int halfWord = base.getInt(offset + i); int k1 = mixK1(halfWord); h1 = mixH1(h1, k1); diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java index 367020627eca3..67e2f71df61f6 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -31,7 +31,7 @@ public final class ByteArrayMemoryBlock extends MemoryBlock { public ByteArrayMemoryBlock(byte[] obj, long offset, long size) { super(obj, offset, size); this.array = obj; - assert(offset - Platform.BYTE_ARRAY_OFFSET + size <= obj.length * 8L) : + assert(offset - Platform.BYTE_ARRAY_OFFSET + size <= obj.length) : "The sum of size " + size + " and offset " + offset + " should not be larger than " + "the array size " + ((obj.length) - Platform.BYTE_ARRAY_OFFSET); } @@ -42,11 +42,7 @@ public ByteArrayMemoryBlock(long length) { @Override public MemoryBlock subBlock(long offset, long size) { - if (offset - Platform.BYTE_ARRAY_OFFSET + size > length) { - throw new ArrayIndexOutOfBoundsException( - "The sum of size " + size + ", offset " + offset + ", and -" + Platform.BYTE_ARRAY_OFFSET + - " should not be larger than MemoryBlock length " + length); - } + checkSubBlockRange(offset, size); return new ByteArrayMemoryBlock(array, this.offset + offset, size); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index f0c581474ef49..e0949acd75223 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -59,13 +59,17 @@ public abstract class MemoryBlock { */ private int pageNumber = NO_PAGE_NUMBER; - public MemoryBlock(@Nullable Object obj, long offset, long length) { + protected MemoryBlock(@Nullable Object obj, long offset, long length) { + if (offset < 0 || length < 0) { + throw new ArrayIndexOutOfBoundsException( + "Length " + length + " and offset " + offset + "must be non-negative"); + } this.obj = obj; this.offset = offset; this.length = length; } - public MemoryBlock() { + protected MemoryBlock() { this(null, 0, 0); } @@ -131,6 +135,18 @@ public final static MemoryBlock allocateFromObject(Object obj, long offset, long */ public abstract MemoryBlock subBlock(long offset, long size); + protected void checkSubBlockRange(long offset, long size) { + if (offset < 0 || length < 0) { + throw new ArrayIndexOutOfBoundsException( + "Length " + length + " and offset " + offset + "must be non-negative"); + } + if (offset + size > this.offset + length) { + throw new ArrayIndexOutOfBoundsException("The sum of size " + size + " and offset " + + offset + " should not be larger than " + "the sum of length " + length + " and offset " + + this.offset + " in the MemoryBlock"); + } + } + /** * getXXX/putXXX does not ensure guarantee behavior if the offset is invalid. e.g cause illegal * memory access, throw an exception, or etc. diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java index 354198607f14f..3a97061c9a4a7 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java @@ -28,11 +28,7 @@ public OffHeapMemoryBlock(long address, long size) { @Override public MemoryBlock subBlock(long offset, long size) { - if (offset + size > this.offset + length) { - throw new ArrayIndexOutOfBoundsException("The sum of size " + size + " and offset " + - offset + " should not be larger than " + "the sum of length " + length + " and offset " + - this.offset + " in the MemoryBlock"); - } + checkSubBlockRange(offset, size); return new OffHeapMemoryBlock(this.offset + offset, size); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java index 2dfd69c673fd4..361039d1dbd9e 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java @@ -42,11 +42,7 @@ public OnHeapMemoryBlock(long size) { @Override public MemoryBlock subBlock(long offset, long size) { - if (offset - Platform.LONG_ARRAY_OFFSET + size > length) { - throw new ArrayIndexOutOfBoundsException( - "The sum of size " + size + ", offset " + offset + ", and -" + Platform.LONG_ARRAY_OFFSET + - " should not be larger than MemoryBlock length " + length); - } + checkSubBlockRange(offset, size); return new OnHeapMemoryBlock(array, this.offset + offset, size); } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java index 97146dbf8432b..5f0f46e8dbb5c 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java @@ -23,6 +23,8 @@ import java.nio.ByteOrder; +import static org.hamcrest.core.StringContains.containsString; + public class MemoryBlockSuite { private static final boolean bigEndianPlatform = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); @@ -79,6 +81,20 @@ private void check(MemoryBlock memory, Object obj, long offset, int length) { for (int i = 40; i < memory.size(); i++) { Assert.assertEquals((byte) -1, memory.getByte(offset + i)); } + + try { + memory.subBlock(-1, -1); + Assert.fail(); + } catch (Exception expected) { + Assert.assertThat(expected.getMessage(), containsString("non-negative")); + } + + try { + memory.subBlock(offset + 16, length - 8); + Assert.fail(); + } catch (Exception expected) { + Assert.assertThat(expected.getMessage(), containsString("should not be larger than")); + } } @Test @@ -92,6 +108,10 @@ public void ByteArrayMemoryBlockTest() { memory = ByteArrayMemoryBlock.fromArray(obj); check(memory, obj, offset, length); + + obj = new byte[96]; + memory = new ByteArrayMemoryBlock(obj, offset, length); + check(memory, obj, offset, length); } @Test @@ -105,6 +125,10 @@ public void OnHeapMemoryBlockTest() { memory = OnHeapMemoryBlock.fromArray(obj); check(memory, obj, offset, length); + + obj = new long[12]; + memory = new OnHeapMemoryBlock(obj, offset, length); + check(memory, obj, offset, length); } @Test @@ -116,5 +140,11 @@ public void OffHeapArrayMemoryBlockTest() { int length = 48; check(memory, obj, offset, length); + + long address = Platform.allocateMemory(96); + memory = new OffHeapMemoryBlock(address, length); + obj = memory.getBaseObject(); + offset = memory.getBaseOffset(); + check(memory, obj, offset, length); } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java index 5c9eb3dc99712..2d5160d9973a0 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.catalyst.expressions; +import com.google.common.primitives.Ints; + import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.memory.MemoryBlock; @@ -89,9 +91,9 @@ public long hashUnsafeBytes(Object base, long offset, int length) { public static long hashUnsafeBytesBlock(MemoryBlock mb, long seed) { Object base = mb.getBaseObject(); long offset = mb.getBaseOffset(); - int length = (int)mb.size(); + long length = mb.size(); assert (length >= 0) : "lengthInBytes cannot be negative"; - long hash = hashBytesByWords(base, offset, length, seed); + long hash = hashBytesByWords(base, offset, Ints.checkedCast(length), seed); long end = offset + length; offset += length & -8; From 5284593aeda08700073711e79e78c74f8a58a060 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 17 Mar 2018 02:14:54 +0100 Subject: [PATCH 51/65] address review comment --- .../apache/spark/unsafe/memory/ByteArrayMemoryBlock.java | 4 ++-- .../java/org/apache/spark/unsafe/memory/MemoryBlock.java | 9 ++++----- .../apache/spark/unsafe/memory/OnHeapMemoryBlock.java | 4 ++-- 3 files changed, 8 insertions(+), 9 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java index 67e2f71df61f6..fafe34fb35422 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -31,9 +31,9 @@ public final class ByteArrayMemoryBlock extends MemoryBlock { public ByteArrayMemoryBlock(byte[] obj, long offset, long size) { super(obj, offset, size); this.array = obj; - assert(offset - Platform.BYTE_ARRAY_OFFSET + size <= obj.length) : + assert(offset + size <= Platform.BYTE_ARRAY_OFFSET + obj.length) : "The sum of size " + size + " and offset " + offset + " should not be larger than " + - "the array size " + ((obj.length) - Platform.BYTE_ARRAY_OFFSET); + "the array size " + (obj.length + Platform.BYTE_ARRAY_OFFSET); } public ByteArrayMemoryBlock(long length) { diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index e0949acd75223..898ea22ca0655 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -136,14 +136,13 @@ public final static MemoryBlock allocateFromObject(Object obj, long offset, long public abstract MemoryBlock subBlock(long offset, long size); protected void checkSubBlockRange(long offset, long size) { - if (offset < 0 || length < 0) { + if (this.offset + offset < 0 || size < 0) { throw new ArrayIndexOutOfBoundsException( - "Length " + length + " and offset " + offset + "must be non-negative"); + "Size " + size + " and offset " + (this.offset + offset) + " must be non-negative"); } - if (offset + size > this.offset + length) { + if (offset + size > length) { throw new ArrayIndexOutOfBoundsException("The sum of size " + size + " and offset " + - offset + " should not be larger than " + "the sum of length " + length + " and offset " + - this.offset + " in the MemoryBlock"); + offset + " should not be larger than the length " + length + " in the MemoryBlock"); } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java index 361039d1dbd9e..b48b4cc12d51e 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java @@ -31,9 +31,9 @@ public final class OnHeapMemoryBlock extends MemoryBlock { public OnHeapMemoryBlock(long[] obj, long offset, long size) { super(obj, offset, size); this.array = obj; - assert(offset - Platform.LONG_ARRAY_OFFSET + size <= obj.length * 8L) : + assert(offset + size <= obj.length * 8L + Platform.LONG_ARRAY_OFFSET) : "The sum of size " + size + " and offset " + offset + " should not be larger than " + - "the array size " + ((obj.length * 8L) - Platform.LONG_ARRAY_OFFSET); + "the array size " + (obj.length * 8L + Platform.LONG_ARRAY_OFFSET); } public OnHeapMemoryBlock(long size) { From b1750a1f47f2986af5e589b27e04195fceaa4b39 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 20 Mar 2018 19:11:40 +0100 Subject: [PATCH 52/65] address review comments --- .../org/apache/spark/unsafe/hash/Murmur3_x86_32.java | 2 +- .../spark/unsafe/memory/UnsafeMemoryAllocator.java | 9 --------- .../java/org/apache/spark/unsafe/types/UTF8String.java | 7 ++----- 3 files changed, 3 insertions(+), 15 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index 1fb7343a0ade6..e14f8159468a5 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -65,7 +65,7 @@ public static int hashUnsafeWordsBlock(MemoryBlock base, int seed) { public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, int seed) { // This is based on Guava's `Murmur32_Hasher.processRemaining(ByteBuffer)` method. assert (lengthInBytes % 8 == 0): "lengthInBytes must be a multiple of 8 (word-aligned)"; - int h1 = hashBytesByIntBlock(MemoryBlock.allocateFromObject(base, offset, lengthInBytes), seed); + int h1 = hashUnsafeWordsBlock(MemoryBlock.allocateFromObject(base, offset, lengthInBytes), seed); return fmix(h1, lengthInBytes); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java index e6c89334e72d4..43bd1c63c6bf2 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -63,13 +63,4 @@ public void free(MemoryBlock memory) { // Mark the page as freed (so we can detect double-frees). memory.setPageNumber(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER); } - - public OffHeapMemoryBlock reallocate(OffHeapMemoryBlock block, long oldSize, long newSize) { - OffHeapMemoryBlock mb = this.allocate(newSize); - if (block.getBaseOffset() != 0) { - MemoryBlock.copyMemory(block, mb, oldSize); - free(block); - } - return mb; - } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index f15fd24c57208..11c6c5714ab82 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -30,6 +30,7 @@ import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; +import com.google.common.primitives.Ints; import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; @@ -122,11 +123,7 @@ protected UTF8String(byte[] bytes, long offset, int numBytes) { public UTF8String(MemoryBlock base) { this.base = base; if (base != null) { - if ((long) Integer.MAX_VALUE < base.size()) { - throw new ArrayIndexOutOfBoundsException( - "MemoryBlock.size " + base.size() + " should be less than " + Integer.MAX_VALUE); - } - this.numBytes = (int) base.size(); + this.numBytes = Ints.checkedCast(base.size()); } } From 38ddf4894eb0d51a1cbf651e23cb29b3ca7986c1 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 20 Mar 2018 20:34:03 +0100 Subject: [PATCH 53/65] address review comment --- .../spark/unsafe/hash/Murmur3_x86_32.java | 7 +------ .../spark/unsafe/hash/Murmur3_x86_32Suite.java | 18 ++++++++++++++++++ 2 files changed, 19 insertions(+), 6 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index e14f8159468a5..247f917539560 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -65,8 +65,7 @@ public static int hashUnsafeWordsBlock(MemoryBlock base, int seed) { public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, int seed) { // This is based on Guava's `Murmur32_Hasher.processRemaining(ByteBuffer)` method. assert (lengthInBytes % 8 == 0): "lengthInBytes must be a multiple of 8 (word-aligned)"; - int h1 = hashUnsafeWordsBlock(MemoryBlock.allocateFromObject(base, offset, lengthInBytes), seed); - return fmix(h1, lengthInBytes); + return hashUnsafeWordsBlock(MemoryBlock.allocateFromObject(base, offset, lengthInBytes), seed); } public static int hashUnsafeBytesBlock(MemoryBlock base, int seed) { @@ -122,10 +121,6 @@ private static int hashBytesByIntBlock(MemoryBlock base, int seed) { return h1; } - private static int hashBytesByInt(Object base, long offset, int lengthInBytes, int seed) { - return hashBytesByIntBlock(MemoryBlock.allocateFromObject(base, offset, lengthInBytes), seed); - } - public int hashLong(long input) { return hashLong(input, seed); } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java index 6348a73bf3895..d7ed005db1891 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java @@ -70,6 +70,24 @@ public void testKnownBytesInputs() { Murmur3_x86_32.hashUnsafeBytes2(tes, Platform.BYTE_ARRAY_OFFSET, tes.length, 0)); } + @Test + public void testKnownWordsInputs() { + byte[] bytes = new byte[16]; + long offset = Platform.BYTE_ARRAY_OFFSET; + for (int i = 0; i < 16; i++) { + bytes[i] = 0; + } + Assert.assertEquals(-300363099, hasher.hashUnsafeWords(bytes, offset, 16, 42)); + for (int i = 0; i < 16; i++) { + bytes[i] = -1; + } + Assert.assertEquals(-1210324667, hasher.hashUnsafeWords(bytes, offset, 16, 42)); + for (int i = 0; i < 16; i++) { + bytes[i] = (byte)i; + } + Assert.assertEquals(-634919701, hasher.hashUnsafeWords(bytes, offset, 16, 42)); + } + @Test public void randomizedStressTest() { int size = 65536; From 4e46a187dca8a0e976dcdfb5f1588f8bdf4adb3b Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 22 Mar 2018 14:58:03 +0900 Subject: [PATCH 54/65] make MemoryBlock.length mutable for reuse --- .../main/java/org/apache/spark/unsafe/memory/MemoryBlock.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index 898ea22ca0655..7612a17f5eeda 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -50,7 +50,7 @@ public abstract class MemoryBlock { protected long offset; - protected final long length; + protected long length; /** * Optional page number; used when this MemoryBlock represents a page allocated by a From 511d58d8046db5e69ffe1a79fbb775def0fde943 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Mon, 26 Mar 2018 19:41:40 +0100 Subject: [PATCH 55/65] address review comment --- .../apache/spark/unsafe/hash/Murmur3_x86_32.java | 16 ++++++++-------- .../apache/spark/unsafe/types/UTF8String.java | 10 ++++------ 2 files changed, 12 insertions(+), 14 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index 247f917539560..28306a86b57e5 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -72,16 +72,16 @@ public static int hashUnsafeBytesBlock(MemoryBlock base, int seed) { // This is not compatible with original and another implementations. // But remain it for backward compatibility for the components existing before 2.3. long offset = base.getBaseOffset(); - long lengthInBytes = base.size(); + int lengthInBytes = Ints.checkedCast(base.size()); assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; - long lengthAligned = lengthInBytes - lengthInBytes % 4; + int lengthAligned = lengthInBytes - lengthInBytes % 4; int h1 = hashBytesByIntBlock(base.subBlock(0, lengthAligned), seed); - for (long i = lengthAligned; i < lengthInBytes; i++) { + for (int i = lengthAligned; i < lengthInBytes; i++) { int halfWord = base.getByte(offset + i); int k1 = mixK1(halfWord); h1 = mixH1(h1, k1); } - return fmix(h1, Ints.checkedCast(lengthInBytes)); + return fmix(h1, lengthInBytes); } public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes, int seed) { @@ -96,16 +96,16 @@ public static int hashUnsafeBytes2Block(MemoryBlock base, int seed) { // This is compatible with original and another implementations. // Use this method for new components after Spark 2.3. long offset = base.getBaseOffset(); - long lengthInBytes = base.size(); + int lengthInBytes = Ints.checkedCast(base.size()); assert (lengthInBytes >= 0) : "lengthInBytes cannot be negative"; - long lengthAligned = lengthInBytes - lengthInBytes % 4; + int lengthAligned = lengthInBytes - lengthInBytes % 4; int h1 = hashBytesByIntBlock(base.subBlock(0, lengthAligned), seed); int k1 = 0; - for (long i = lengthAligned, shift = 0; i < lengthInBytes; i++, shift += 8) { + for (int i = lengthAligned, shift = 0; i < lengthInBytes; i++, shift += 8) { k1 ^= (base.getByte(offset + i) & 0xFF) << shift; } h1 ^= mixK1(k1); - return fmix(h1, Ints.checkedCast(lengthInBytes)); + return fmix(h1, lengthInBytes); } private static int hashBytesByIntBlock(MemoryBlock base, int seed) { diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 11c6c5714ab82..408f9231fe7e7 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -81,7 +81,8 @@ public final class UTF8String implements Comparable, Externalizable, */ public static UTF8String fromBytes(byte[] bytes) { if (bytes != null) { - return new UTF8String(bytes, BYTE_ARRAY_OFFSET, bytes.length); + return new UTF8String( + MemoryBlock.allocateFromObject(bytes, BYTE_ARRAY_OFFSET, bytes.length)); } else { return null; } @@ -94,7 +95,8 @@ public static UTF8String fromBytes(byte[] bytes) { */ public static UTF8String fromBytes(byte[] bytes, int offset, int numBytes) { if (bytes != null) { - return new UTF8String(bytes, BYTE_ARRAY_OFFSET + offset, numBytes); + return new UTF8String( + MemoryBlock.allocateFromObject(bytes, BYTE_ARRAY_OFFSET + offset, numBytes)); } else { return null; } @@ -116,10 +118,6 @@ public static UTF8String blankString(int length) { return fromBytes(spaces); } - protected UTF8String(byte[] bytes, long offset, int numBytes) { - this(new ByteArrayMemoryBlock(bytes, offset, numBytes)); - } - public UTF8String(MemoryBlock base) { this.base = base; if (base != null) { From 1939fda215a8570ea3290c3658b1faa0fe17aa5f Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Mon, 26 Mar 2018 20:59:51 +0100 Subject: [PATCH 56/65] include MemoryBlock.offset in get/putXXX --- .../sql/catalyst/expressions/HiveHasher.java | 3 +- .../spark/unsafe/hash/Murmur3_x86_32.java | 9 ++--- .../unsafe/memory/ByteArrayMemoryBlock.java | 28 ++++++++-------- .../spark/unsafe/memory/MemoryBlock.java | 2 ++ .../unsafe/memory/OffHeapMemoryBlock.java | 28 ++++++++-------- .../unsafe/memory/OnHeapMemoryBlock.java | 28 ++++++++-------- .../apache/spark/unsafe/types/UTF8String.java | 23 ++++++------- .../spark/sql/catalyst/expressions/XXH64.java | 33 ++++++++++--------- .../catalyst/expressions/HiveHasherSuite.java | 18 +++++----- .../sql/catalyst/expressions/XXH64Suite.java | 18 +++++----- 10 files changed, 95 insertions(+), 95 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java index bd53553a3d4d4..5d905943a3aa7 100644 --- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java @@ -40,12 +40,11 @@ public static int hashLong(long input) { } public static int hashUnsafeBytesBlock(MemoryBlock mb) { - long offset = mb.getBaseOffset(); long lengthInBytes = mb.size(); assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; int result = 0; for (long i = 0; i < lengthInBytes; i++) { - result = (result * 31) + (int) mb.getByte(offset + i); + result = (result * 31) + (int) mb.getByte(i); } return result; } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index 28306a86b57e5..de0a3c63674bc 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -71,13 +71,12 @@ public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, i public static int hashUnsafeBytesBlock(MemoryBlock base, int seed) { // This is not compatible with original and another implementations. // But remain it for backward compatibility for the components existing before 2.3. - long offset = base.getBaseOffset(); int lengthInBytes = Ints.checkedCast(base.size()); assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; int lengthAligned = lengthInBytes - lengthInBytes % 4; int h1 = hashBytesByIntBlock(base.subBlock(0, lengthAligned), seed); for (int i = lengthAligned; i < lengthInBytes; i++) { - int halfWord = base.getByte(offset + i); + int halfWord = base.getByte(i); int k1 = mixK1(halfWord); h1 = mixH1(h1, k1); } @@ -95,26 +94,24 @@ public static int hashUnsafeBytes2(Object base, long offset, int lengthInBytes, public static int hashUnsafeBytes2Block(MemoryBlock base, int seed) { // This is compatible with original and another implementations. // Use this method for new components after Spark 2.3. - long offset = base.getBaseOffset(); int lengthInBytes = Ints.checkedCast(base.size()); assert (lengthInBytes >= 0) : "lengthInBytes cannot be negative"; int lengthAligned = lengthInBytes - lengthInBytes % 4; int h1 = hashBytesByIntBlock(base.subBlock(0, lengthAligned), seed); int k1 = 0; for (int i = lengthAligned, shift = 0; i < lengthInBytes; i++, shift += 8) { - k1 ^= (base.getByte(offset + i) & 0xFF) << shift; + k1 ^= (base.getByte(i) & 0xFF) << shift; } h1 ^= mixK1(k1); return fmix(h1, lengthInBytes); } private static int hashBytesByIntBlock(MemoryBlock base, int seed) { - long offset = base.getBaseOffset(); long lengthInBytes = base.size(); assert (lengthInBytes % 4 == 0); int h1 = seed; for (long i = 0; i < lengthInBytes; i += 4) { - int halfWord = base.getInt(offset + i); + int halfWord = base.getInt(i); int k1 = mixK1(halfWord); h1 = mixH1(h1, k1); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java index fafe34fb35422..e7ee2e8fb7e78 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -57,71 +57,71 @@ public static ByteArrayMemoryBlock fromArray(final byte[] array) { @Override public final int getInt(long offset) { - return Platform.getInt(array, offset); + return Platform.getInt(array, this.offset + offset); } @Override public final void putInt(long offset, int value) { - Platform.putInt(array, offset, value); + Platform.putInt(array, this.offset + offset, value); } @Override public final boolean getBoolean(long offset) { - return Platform.getBoolean(array, offset); + return Platform.getBoolean(array, this.offset + offset); } @Override public final void putBoolean(long offset, boolean value) { - Platform.putBoolean(array, offset, value); + Platform.putBoolean(array, this.offset + offset, value); } @Override public final byte getByte(long offset) { - return array[(int)(offset - Platform.BYTE_ARRAY_OFFSET)]; + return array[(int)(this.offset + offset - Platform.BYTE_ARRAY_OFFSET)]; } @Override public final void putByte(long offset, byte value) { - array[(int)(offset - Platform.BYTE_ARRAY_OFFSET)] = value; + array[(int)(this.offset + offset - Platform.BYTE_ARRAY_OFFSET)] = value; } @Override public final short getShort(long offset) { - return Platform.getShort(array, offset); + return Platform.getShort(array, this.offset + offset); } @Override public final void putShort(long offset, short value) { - Platform.putShort(array, offset, value); + Platform.putShort(array, this.offset + offset, value); } @Override public final long getLong(long offset) { - return Platform.getLong(array, offset); + return Platform.getLong(array, this.offset + offset); } @Override public final void putLong(long offset, long value) { - Platform.putLong(array, offset, value); + Platform.putLong(array, this.offset + offset, value); } @Override public final float getFloat(long offset) { - return Platform.getFloat(array, offset); + return Platform.getFloat(array, this.offset + offset); } @Override public final void putFloat(long offset, float value) { - Platform.putFloat(array, offset, value); + Platform.putFloat(array, this.offset + offset, value); } @Override public final double getDouble(long offset) { - return Platform.getDouble(array, offset); + return Platform.getDouble(array, this.offset + offset); } @Override public final void putDouble(long offset, double value) { - Platform.putDouble(array, offset, value); + Platform.putDouble(array, this.offset + offset, value); } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index 898ea22ca0655..d720219d15844 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -149,6 +149,8 @@ protected void checkSubBlockRange(long offset, long size) { /** * getXXX/putXXX does not ensure guarantee behavior if the offset is invalid. e.g cause illegal * memory access, throw an exception, or etc. + * getXXX/putXXX uses an index based on this.offset that includes the size of metadata such as + * JVM object header. Thus, the offset is expected as an logical offset in the memory block. */ public abstract int getInt(long offset); diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java index 3a97061c9a4a7..d18e4744a5980 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java @@ -34,71 +34,71 @@ public MemoryBlock subBlock(long offset, long size) { @Override public final int getInt(long offset) { - return Platform.getInt(null, offset); + return Platform.getInt(null, this.offset + offset); } @Override public final void putInt(long offset, int value) { - Platform.putInt(null, offset, value); + Platform.putInt(null, this.offset + offset, value); } @Override public final boolean getBoolean(long offset) { - return Platform.getBoolean(null, offset); + return Platform.getBoolean(null, this.offset + offset); } @Override public final void putBoolean(long offset, boolean value) { - Platform.putBoolean(null, offset, value); + Platform.putBoolean(null, this.offset + offset, value); } @Override public final byte getByte(long offset) { - return Platform.getByte(null, offset); + return Platform.getByte(null, this.offset + offset); } @Override public final void putByte(long offset, byte value) { - Platform.putByte(null, offset, value); + Platform.putByte(null, this.offset + offset, value); } @Override public final short getShort(long offset) { - return Platform.getShort(null, offset); + return Platform.getShort(null, this.offset + offset); } @Override public final void putShort(long offset, short value) { - Platform.putShort(null, offset, value); + Platform.putShort(null, this.offset + offset, value); } @Override public final long getLong(long offset) { - return Platform.getLong(null, offset); + return Platform.getLong(null, this.offset + offset); } @Override public final void putLong(long offset, long value) { - Platform.putLong(null, offset, value); + Platform.putLong(null, this.offset + offset, value); } @Override public final float getFloat(long offset) { - return Platform.getFloat(null, offset); + return Platform.getFloat(null, this.offset + offset); } @Override public final void putFloat(long offset, float value) { - Platform.putFloat(null, offset, value); + Platform.putFloat(null, this.offset + offset, value); } @Override public final double getDouble(long offset) { - return Platform.getDouble(null, offset); + return Platform.getDouble(null, this.offset + offset); } @Override public final void putDouble(long offset, double value) { - Platform.putDouble(null, offset, value); + Platform.putDouble(null, this.offset + offset, value); } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java index b48b4cc12d51e..1537f62d78217 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java @@ -61,71 +61,71 @@ public static OnHeapMemoryBlock fromArray(final long[] array, long size) { @Override public final int getInt(long offset) { - return Platform.getInt(array, offset); + return Platform.getInt(array, this.offset + offset); } @Override public final void putInt(long offset, int value) { - Platform.putInt(array, offset, value); + Platform.putInt(array, this.offset + offset, value); } @Override public final boolean getBoolean(long offset) { - return Platform.getBoolean(array, offset); + return Platform.getBoolean(array, this.offset + offset); } @Override public final void putBoolean(long offset, boolean value) { - Platform.putBoolean(array, offset, value); + Platform.putBoolean(array, this.offset + offset, value); } @Override public final byte getByte(long offset) { - return Platform.getByte(array, offset); + return Platform.getByte(array, this.offset + offset); } @Override public final void putByte(long offset, byte value) { - Platform.putByte(array, offset, value); + Platform.putByte(array, this.offset + offset, value); } @Override public final short getShort(long offset) { - return Platform.getShort(array, offset); + return Platform.getShort(array, this.offset + offset); } @Override public final void putShort(long offset, short value) { - Platform.putShort(array, offset, value); + Platform.putShort(array, this.offset + offset, value); } @Override public final long getLong(long offset) { - return Platform.getLong(array, offset); + return Platform.getLong(array, this.offset + offset); } @Override public final void putLong(long offset, long value) { - Platform.putLong(array, offset, value); + Platform.putLong(array, this.offset + offset, value); } @Override public final float getFloat(long offset) { - return Platform.getFloat(array, offset); + return Platform.getFloat(array, this.offset + offset); } @Override public final void putFloat(long offset, float value) { - Platform.putFloat(array, offset, value); + Platform.putFloat(array, this.offset + offset, value); } @Override public final double getDouble(long offset) { - return Platform.getDouble(array, offset); + return Platform.getDouble(array, this.offset + offset); } @Override public final void putDouble(long offset, double value) { - Platform.putDouble(array, offset, value); + Platform.putDouble(array, this.offset + offset, value); } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 408f9231fe7e7..f801f973383a2 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -222,15 +222,14 @@ public long getPrefix() { // After getting the data, we use a mask to mask out data that is not part of the string. long p; long mask = 0; - long offset = base.getBaseOffset(); if (IS_LITTLE_ENDIAN) { if (numBytes >= 8) { - p = base.getLong(offset); + p = base.getLong(0); } else if (numBytes > 4) { - p = base.getLong(offset); + p = base.getLong(0); mask = (1L << (8 - numBytes) * 8) - 1; } else if (numBytes > 0) { - p = (long) base.getInt(offset); + p = (long) base.getInt(0); mask = (1L << (8 - numBytes) * 8) - 1; } else { p = 0; @@ -239,12 +238,12 @@ public long getPrefix() { } else { // byteOrder == ByteOrder.BIG_ENDIAN if (numBytes >= 8) { - p = base.getLong(offset); + p = base.getLong(0); } else if (numBytes > 4) { - p = base.getLong(offset); + p = base.getLong(0); mask = (1L << (8 - numBytes) * 8) - 1; } else if (numBytes > 0) { - p = ((long) base.getInt(offset)) << 32; + p = ((long) base.getInt(0)) << 32; mask = (1L << (8 - numBytes) * 8) - 1; } else { p = 0; @@ -336,7 +335,7 @@ public boolean contains(final UTF8String substring) { * Returns the byte at position `i`. */ private byte getByte(int i) { - return base.getByte(getBaseOffset() + i); + return base.getByte(i); } private boolean matchAt(final UTF8String s, int pos) { @@ -1196,12 +1195,10 @@ public UTF8String copy() { public int compareTo(@Nonnull final UTF8String other) { int len = Math.min(numBytes, other.numBytes); int wordMax = (len / 8) * 8; - long offset = getBaseOffset(); - long roffset = other.getBaseOffset(); MemoryBlock rbase = other.base; for (int i = 0; i < wordMax; i += 8) { - long left = base.getLong(offset + i); - long right = rbase.getLong(roffset + i); + long left = base.getLong(i); + long right = rbase.getLong(i); if (left != right) { if (IS_LITTLE_ENDIAN) { return Long.compareUnsigned(Long.reverseBytes(left), Long.reverseBytes(right)); @@ -1212,7 +1209,7 @@ public int compareTo(@Nonnull final UTF8String other) { } for (int i = wordMax; i < len; i++) { // In UTF-8, the byte should be unsigned, so we should compare them as unsigned int. - int res = (getByte(i) & 0xFF) - (rbase.getByte(roffset + i) & 0xFF); + int res = (getByte(i) & 0xFF) - (rbase.getByte(i) & 0xFF); if (res != 0) { return res; } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java index 2d5160d9973a0..d5766bcbec64f 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java @@ -74,13 +74,13 @@ public static long hashLong(long input, long seed) { return fmix(hash); } - public long hashUnsafeWords(Object base, long offset, int length) { - return hashUnsafeWords(base, offset, length, seed); + public long hashUnsafeWordsBlock(MemoryBlock mb) { + return hashUnsafeWordsBlock(mb, seed); } - public static long hashUnsafeWords(Object base, long offset, int length, long seed) { - assert (length % 8 == 0) : "lengthInBytes must be a multiple of 8 (word-aligned)"; - long hash = hashBytesByWords(base, offset, length, seed); + public static long hashUnsafeWordsBlock(MemoryBlock mb, long seed) { + assert (mb.size() % 8 == 0) : "lengthInBytes must be a multiple of 8 (word-aligned)"; + long hash = hashBytesByWordsBlock(mb, seed); return fmix(hash); } @@ -90,10 +90,10 @@ public long hashUnsafeBytes(Object base, long offset, int length) { public static long hashUnsafeBytesBlock(MemoryBlock mb, long seed) { Object base = mb.getBaseObject(); - long offset = mb.getBaseOffset(); + long offset = 0; long length = mb.size(); assert (length >= 0) : "lengthInBytes cannot be negative"; - long hash = hashBytesByWords(base, offset, Ints.checkedCast(length), seed); + long hash = hashBytesByWordsBlock(mb, seed); long end = offset + length; offset += length & -8; @@ -124,30 +124,31 @@ private static long fmix(long hash) { return hash; } - private static long hashBytesByWords(Object base, long offset, int length, long seed) { - long end = offset + length; + private static long hashBytesByWordsBlock(MemoryBlock mb, long seed) { + long offset = 0; + long length = mb.size(); long hash; if (length >= 32) { - long limit = end - 32; + long limit = length - 32; long v1 = seed + PRIME64_1 + PRIME64_2; long v2 = seed + PRIME64_2; long v3 = seed; long v4 = seed - PRIME64_1; do { - v1 += Platform.getLong(base, offset) * PRIME64_2; + v1 += mb.getLong(offset) * PRIME64_2; v1 = Long.rotateLeft(v1, 31); v1 *= PRIME64_1; - v2 += Platform.getLong(base, offset + 8) * PRIME64_2; + v2 += mb.getLong(offset + 8) * PRIME64_2; v2 = Long.rotateLeft(v2, 31); v2 *= PRIME64_1; - v3 += Platform.getLong(base, offset + 16) * PRIME64_2; + v3 += mb.getLong(offset + 16) * PRIME64_2; v3 = Long.rotateLeft(v3, 31); v3 *= PRIME64_1; - v4 += Platform.getLong(base, offset + 24) * PRIME64_2; + v4 += mb.getLong(offset + 24) * PRIME64_2; v4 = Long.rotateLeft(v4, 31); v4 *= PRIME64_1; @@ -188,9 +189,9 @@ private static long hashBytesByWords(Object base, long offset, int length, long hash += length; - long limit = end - 8; + long limit = length - 8; while (offset <= limit) { - long k1 = Platform.getLong(base, offset); + long k1 = mb.getLong(offset); hash ^= Long.rotateLeft(k1 * PRIME64_2, 31) * PRIME64_1; hash = Long.rotateLeft(hash, 27) * PRIME64_1 + PRIME64_4; offset += 8L; diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java index b4fd21ecbb8b9..8ffc1d7c24d61 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java @@ -18,6 +18,8 @@ package org.apache.spark.sql.catalyst.expressions; import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.memory.ByteArrayMemoryBlock; +import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.unsafe.types.UTF8String; import org.junit.Assert; import org.junit.Test; @@ -89,13 +91,13 @@ public void randomizedStressTestBytes() { int byteArrSize = rand.nextInt(100) * 8; byte[] bytes = new byte[byteArrSize]; rand.nextBytes(bytes); + MemoryBlock mb = ByteArrayMemoryBlock.fromArray(bytes); Assert.assertEquals( - HiveHasher.hashUnsafeBytes(bytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize), - HiveHasher.hashUnsafeBytes(bytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize)); + HiveHasher.hashUnsafeBytesBlock(mb), + HiveHasher.hashUnsafeBytesBlock(mb)); - hashcodes.add(HiveHasher.hashUnsafeBytes( - bytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize)); + hashcodes.add(HiveHasher.hashUnsafeBytesBlock(mb)); } // A very loose bound. @@ -112,13 +114,13 @@ public void randomizedStressTestPaddedStrings() { byte[] strBytes = String.valueOf(i).getBytes(StandardCharsets.UTF_8); byte[] paddedBytes = new byte[byteArrSize]; System.arraycopy(strBytes, 0, paddedBytes, 0, strBytes.length); + MemoryBlock mb = ByteArrayMemoryBlock.fromArray(paddedBytes); Assert.assertEquals( - HiveHasher.hashUnsafeBytes(paddedBytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize), - HiveHasher.hashUnsafeBytes(paddedBytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize)); + HiveHasher.hashUnsafeBytesBlock(mb), + HiveHasher.hashUnsafeBytesBlock(mb)); - hashcodes.add(HiveHasher.hashUnsafeBytes( - paddedBytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize)); + hashcodes.add(HiveHasher.hashUnsafeBytesBlock(mb)); } // A very loose bound. diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/XXH64Suite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/XXH64Suite.java index 711887f02832a..bb7cad1212a26 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/XXH64Suite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/XXH64Suite.java @@ -24,6 +24,8 @@ import java.util.Set; import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.memory.ByteArrayMemoryBlock; +import org.apache.spark.unsafe.memory.MemoryBlock; import org.junit.Assert; import org.junit.Test; @@ -128,13 +130,13 @@ public void randomizedStressTestBytes() { int byteArrSize = rand.nextInt(100) * 8; byte[] bytes = new byte[byteArrSize]; rand.nextBytes(bytes); + MemoryBlock mb = ByteArrayMemoryBlock.fromArray(bytes); Assert.assertEquals( - hasher.hashUnsafeWords(bytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize), - hasher.hashUnsafeWords(bytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize)); + hasher.hashUnsafeWordsBlock(mb), + hasher.hashUnsafeWordsBlock(mb)); - hashcodes.add(hasher.hashUnsafeWords( - bytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize)); + hashcodes.add(hasher.hashUnsafeWordsBlock(mb)); } // A very loose bound. @@ -151,13 +153,13 @@ public void randomizedStressTestPaddedStrings() { byte[] strBytes = String.valueOf(i).getBytes(StandardCharsets.UTF_8); byte[] paddedBytes = new byte[byteArrSize]; System.arraycopy(strBytes, 0, paddedBytes, 0, strBytes.length); + MemoryBlock mb = ByteArrayMemoryBlock.fromArray(paddedBytes); Assert.assertEquals( - hasher.hashUnsafeWords(paddedBytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize), - hasher.hashUnsafeWords(paddedBytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize)); + hasher.hashUnsafeWordsBlock(mb), + hasher.hashUnsafeWordsBlock(mb)); - hashcodes.add(hasher.hashUnsafeWords( - paddedBytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize)); + hashcodes.add(hasher.hashUnsafeWordsBlock(mb)); } // A very loose bound. From c53b6b839be6a69e9f05d2be258d5342bdeb4632 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 27 Mar 2018 06:48:27 +0100 Subject: [PATCH 57/65] address review comments --- .../apache/spark/unsafe/array/LongArray.java | 7 ++-- .../spark/unsafe/memory/MemoryBlock.java | 7 ++-- .../unsafe/memory/OnHeapMemoryBlock.java | 4 +- .../spark/unsafe/memory/MemoryBlockSuite.java | 42 +++++++++---------- 4 files changed, 30 insertions(+), 30 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java index 206cc6b5c4440..b74d2de0691d5 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java @@ -64,8 +64,7 @@ public long size() { * Fill this all with 0L. */ public void zeroOut() { - long baseOffset = memory.getBaseOffset(); - for (long off = baseOffset; off < baseOffset + length * WIDTH; off += WIDTH) { + for (long off = 0; off < length * WIDTH; off += WIDTH) { memory.putLong(off, 0); } } @@ -76,7 +75,7 @@ public void zeroOut() { public void set(int index, long value) { assert index >= 0 : "index (" + index + ") should >= 0"; assert index < length : "index (" + index + ") should < length (" + length + ")"; - memory.putLong(memory.getBaseOffset() + index * WIDTH, value); + memory.putLong(index * WIDTH, value); } /** @@ -85,6 +84,6 @@ public void set(int index, long value) { public long get(int index) { assert index >= 0 : "index (" + index + ") should >= 0"; assert index < length : "index (" + index + ") should < length (" + length + ")"; - return memory.getLong(memory.getBaseOffset() + index * WIDTH); + return memory.getLong(index * WIDTH); } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index f746a29949629..123a4e21d48c7 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -136,9 +136,9 @@ public final static MemoryBlock allocateFromObject(Object obj, long offset, long public abstract MemoryBlock subBlock(long offset, long size); protected void checkSubBlockRange(long offset, long size) { - if (this.offset + offset < 0 || size < 0) { + if (offset < 0 || size < 0 || this.offset + offset < 0) { throw new ArrayIndexOutOfBoundsException( - "Size " + size + " and offset " + (this.offset + offset) + " must be non-negative"); + "Size " + size + " and offset " + offset + " must be non-negative"); } if (offset + size > length) { throw new ArrayIndexOutOfBoundsException("The sum of size " + size + " and offset " + @@ -150,7 +150,8 @@ protected void checkSubBlockRange(long offset, long size) { * getXXX/putXXX does not ensure guarantee behavior if the offset is invalid. e.g cause illegal * memory access, throw an exception, or etc. * getXXX/putXXX uses an index based on this.offset that includes the size of metadata such as - * JVM object header. Thus, the offset is expected as an logical offset in the memory block. + * JVM object header. The offset is 0-based and is expected as an logical offset in the memory + * block. */ public abstract int getInt(long offset); diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java index 1537f62d78217..c325d05fa0957 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java @@ -101,12 +101,12 @@ public final void putShort(long offset, short value) { @Override public final long getLong(long offset) { - return Platform.getLong(array, this.offset + offset); + return array[(int)((this.offset + offset - Platform.LONG_ARRAY_OFFSET) / 8)]; } @Override public final void putLong(long offset, long value) { - Platform.putLong(array, this.offset + offset, value); + array[(int)((this.offset + offset - Platform.LONG_ARRAY_OFFSET) / 8)] = value; } @Override diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java index 5f0f46e8dbb5c..97b1ad499c046 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java @@ -32,13 +32,13 @@ public class MemoryBlockSuite { private void check(MemoryBlock memory, Object obj, long offset, int length) { memory.setPageNumber(1); memory.fill((byte)-1); - memory.putBoolean(offset, true); - memory.putByte(offset + 1, (byte)127); - memory.putShort(offset + 2, (short)257); - memory.putInt(offset + 4, 0x20000002); - memory.putLong(offset + 8, -1L); - memory.putFloat(offset + 16, 1.0F); - memory.putDouble(offset + 20, 2.0); + memory.putBoolean(0, true); + memory.putByte(1, (byte)127); + memory.putShort(2, (short)257); + memory.putInt(4, 0x20000002); + memory.putLong(8, -1L); + memory.putFloat(16, 1.0F); + memory.putDouble(20, 2.0); MemoryBlock.copyMemory(memory, 0L, memory, 28, 4); int[] a = new int[2]; a[0] = 0x12345678; @@ -51,18 +51,18 @@ private void check(MemoryBlock memory, Object obj, long offset, int length) { Assert.assertEquals(offset, memory.getBaseOffset()); Assert.assertEquals(length, memory.size()); Assert.assertEquals(1, memory.getPageNumber()); - Assert.assertEquals(true, memory.getBoolean(offset)); - Assert.assertEquals((byte)127, memory.getByte(offset + 1 )); - Assert.assertEquals((short)257, memory.getShort(offset + 2)); - Assert.assertEquals(0x20000002, memory.getInt(offset + 4)); - Assert.assertEquals(-1L, memory.getLong(offset + 8)); - Assert.assertEquals(1.0F, memory.getFloat(offset + 16), 0); - Assert.assertEquals(2.0, memory.getDouble(offset + 20), 0); - Assert.assertEquals(true, memory.getBoolean(offset + 28)); - Assert.assertEquals((byte)127, memory.getByte(offset + 29 )); - Assert.assertEquals((short)257, memory.getShort(offset + 30)); - Assert.assertEquals(a[0], memory.getInt(offset + 32)); - Assert.assertEquals(a[1], memory.getInt(offset + 36)); + Assert.assertEquals(true, memory.getBoolean(0)); + Assert.assertEquals((byte)127, memory.getByte(1 )); + Assert.assertEquals((short)257, memory.getShort(2)); + Assert.assertEquals(0x20000002, memory.getInt(4)); + Assert.assertEquals(-1L, memory.getLong(8)); + Assert.assertEquals(1.0F, memory.getFloat(16), 0); + Assert.assertEquals(2.0, memory.getDouble(20), 0); + Assert.assertEquals(true, memory.getBoolean(28)); + Assert.assertEquals((byte)127, memory.getByte(29 )); + Assert.assertEquals((short)257, memory.getShort(30)); + Assert.assertEquals(a[0], memory.getInt(32)); + Assert.assertEquals(a[1], memory.getInt(36)); if (bigEndianPlatform) { Assert.assertEquals(a[0], ((int)b[0] & 0xff) << 24 | ((int)b[1] & 0xff) << 16 | @@ -79,7 +79,7 @@ private void check(MemoryBlock memory, Object obj, long offset, int length) { ((int)b[5] & 0xff) << 8 | ((int)b[4] & 0xff)); } for (int i = 40; i < memory.size(); i++) { - Assert.assertEquals((byte) -1, memory.getByte(offset + i)); + Assert.assertEquals((byte) -1, memory.getByte(i)); } try { @@ -90,7 +90,7 @@ private void check(MemoryBlock memory, Object obj, long offset, int length) { } try { - memory.subBlock(offset + 16, length - 8); + memory.subBlock(16, length - 8); Assert.fail(); } catch (Exception expected) { Assert.assertThat(expected.getMessage(), containsString("should not be larger than")); From 45aa73699932317611142a8ee3fae758e397e272 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 27 Mar 2018 08:54:07 +0100 Subject: [PATCH 58/65] fix test failure --- .../java/org/apache/spark/unsafe/types/UTF8StringSuite.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java index 6646e9cd208fa..b394091300d14 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java @@ -518,8 +518,9 @@ public void writeToOutputStreamUnderflow() throws IOException { final byte[] test = "01234567".getBytes(StandardCharsets.UTF_8); for (int i = 1; i <= Platform.BYTE_ARRAY_OFFSET; ++i) { - new UTF8String(ByteArrayMemoryBlock.fromArray(test) - .subBlock(-i, test.length + i)).writeTo(outputStream); + new UTF8String( + new ByteArrayMemoryBlock(test, Platform.BYTE_ARRAY_OFFSET - i, test.length + i)) + .writeTo(outputStream); final ByteBuffer buffer = ByteBuffer.wrap(outputStream.toByteArray(), i, test.length); assertEquals("01234567", StandardCharsets.UTF_8.decode(buffer).toString()); outputStream.reset(); From 8690e4350f73fe172cb3b265504b913bb63a8422 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 29 Mar 2018 05:15:21 +0100 Subject: [PATCH 59/65] address review comment --- .../java/org/apache/spark/sql/catalyst/expressions/XXH64.java | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java index d5766bcbec64f..883748932ad33 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java @@ -89,7 +89,6 @@ public long hashUnsafeBytes(Object base, long offset, int length) { } public static long hashUnsafeBytesBlock(MemoryBlock mb, long seed) { - Object base = mb.getBaseObject(); long offset = 0; long length = mb.size(); assert (length >= 0) : "lengthInBytes cannot be negative"; From 59fd393cb4e378550f90aaa5f5ceb2c9e3d85fef Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 29 Mar 2018 05:16:23 +0100 Subject: [PATCH 60/65] fix test failures --- .../unsafe/memory/OnHeapMemoryBlock.java | 4 +- .../spark/unsafe/memory/MemoryBlockSuite.java | 42 ++++++++++--------- 2 files changed, 24 insertions(+), 22 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java index c325d05fa0957..1537f62d78217 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java @@ -101,12 +101,12 @@ public final void putShort(long offset, short value) { @Override public final long getLong(long offset) { - return array[(int)((this.offset + offset - Platform.LONG_ARRAY_OFFSET) / 8)]; + return Platform.getLong(array, this.offset + offset); } @Override public final void putLong(long offset, long value) { - array[(int)((this.offset + offset - Platform.LONG_ARRAY_OFFSET) / 8)] = value; + Platform.putLong(array, this.offset + offset, value); } @Override diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java index 97b1ad499c046..18eca9331965a 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java @@ -36,16 +36,17 @@ private void check(MemoryBlock memory, Object obj, long offset, int length) { memory.putByte(1, (byte)127); memory.putShort(2, (short)257); memory.putInt(4, 0x20000002); - memory.putLong(8, -1L); + memory.putLong(8, 0x1234567089ABCDEFL); memory.putFloat(16, 1.0F); - memory.putDouble(20, 2.0); - MemoryBlock.copyMemory(memory, 0L, memory, 28, 4); + memory.putLong(20, 0x1234567089ABCDEFL); + memory.putDouble(28, 2.0); + MemoryBlock.copyMemory(memory, 0L, memory, 36, 4); int[] a = new int[2]; a[0] = 0x12345678; a[1] = 0x13579BDF; - memory.copyFrom(a, Platform.INT_ARRAY_OFFSET, 32, 8); + memory.copyFrom(a, Platform.INT_ARRAY_OFFSET, 40, 8); byte[] b = new byte[8]; - memory.writeTo(32, b, Platform.BYTE_ARRAY_OFFSET, 8); + memory.writeTo(40, b, Platform.BYTE_ARRAY_OFFSET, 8); Assert.assertEquals(obj, memory.getBaseObject()); Assert.assertEquals(offset, memory.getBaseOffset()); @@ -55,14 +56,15 @@ private void check(MemoryBlock memory, Object obj, long offset, int length) { Assert.assertEquals((byte)127, memory.getByte(1 )); Assert.assertEquals((short)257, memory.getShort(2)); Assert.assertEquals(0x20000002, memory.getInt(4)); - Assert.assertEquals(-1L, memory.getLong(8)); + Assert.assertEquals(0x1234567089ABCDEFL, memory.getLong(8)); Assert.assertEquals(1.0F, memory.getFloat(16), 0); - Assert.assertEquals(2.0, memory.getDouble(20), 0); - Assert.assertEquals(true, memory.getBoolean(28)); - Assert.assertEquals((byte)127, memory.getByte(29 )); - Assert.assertEquals((short)257, memory.getShort(30)); - Assert.assertEquals(a[0], memory.getInt(32)); - Assert.assertEquals(a[1], memory.getInt(36)); + Assert.assertEquals(0x1234567089ABCDEFL, memory.getLong(20)); + Assert.assertEquals(2.0, memory.getDouble(28), 0); + Assert.assertEquals(true, memory.getBoolean(36)); + Assert.assertEquals((byte)127, memory.getByte(37 )); + Assert.assertEquals((short)257, memory.getShort(38)); + Assert.assertEquals(a[0], memory.getInt(40)); + Assert.assertEquals(a[1], memory.getInt(44)); if (bigEndianPlatform) { Assert.assertEquals(a[0], ((int)b[0] & 0xff) << 24 | ((int)b[1] & 0xff) << 16 | @@ -78,7 +80,7 @@ private void check(MemoryBlock memory, Object obj, long offset, int length) { ((int)b[7] & 0xff) << 24 | ((int)b[6] & 0xff) << 16 | ((int)b[5] & 0xff) << 8 | ((int)b[4] & 0xff)); } - for (int i = 40; i < memory.size(); i++) { + for (int i = 48; i < memory.size(); i++) { Assert.assertEquals((byte) -1, memory.getByte(i)); } @@ -99,7 +101,7 @@ private void check(MemoryBlock memory, Object obj, long offset, int length) { @Test public void ByteArrayMemoryBlockTest() { - byte[] obj = new byte[48]; + byte[] obj = new byte[56]; long offset = Platform.BYTE_ARRAY_OFFSET; int length = obj.length; @@ -109,14 +111,14 @@ public void ByteArrayMemoryBlockTest() { memory = ByteArrayMemoryBlock.fromArray(obj); check(memory, obj, offset, length); - obj = new byte[96]; + obj = new byte[112]; memory = new ByteArrayMemoryBlock(obj, offset, length); check(memory, obj, offset, length); } @Test public void OnHeapMemoryBlockTest() { - long[] obj = new long[6]; + long[] obj = new long[7]; long offset = Platform.LONG_ARRAY_OFFSET; int length = obj.length * 8; @@ -126,7 +128,7 @@ public void OnHeapMemoryBlockTest() { memory = OnHeapMemoryBlock.fromArray(obj); check(memory, obj, offset, length); - obj = new long[12]; + obj = new long[14]; memory = new OnHeapMemoryBlock(obj, offset, length); check(memory, obj, offset, length); } @@ -134,14 +136,14 @@ public void OnHeapMemoryBlockTest() { @Test public void OffHeapArrayMemoryBlockTest() { MemoryAllocator memoryAllocator = new UnsafeMemoryAllocator(); - MemoryBlock memory = memoryAllocator.allocate(48); + MemoryBlock memory = memoryAllocator.allocate(56); Object obj = memory.getBaseObject(); long offset = memory.getBaseOffset(); - int length = 48; + int length = 56; check(memory, obj, offset, length); - long address = Platform.allocateMemory(96); + long address = Platform.allocateMemory(112); memory = new OffHeapMemoryBlock(address, length); obj = memory.getBaseObject(); offset = memory.getBaseOffset(); From b69cb6430d71fe6ce7a39f9d6a13bdcfa8704ccf Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 29 Mar 2018 07:31:50 +0100 Subject: [PATCH 61/65] address yesterday's review comment --- .../apache/spark/unsafe/hash/Murmur3_x86_32.java | 1 - .../org/apache/spark/unsafe/memory/MemoryBlock.java | 2 +- .../org/apache/spark/unsafe/types/UTF8String.java | 13 +++++-------- .../spark/shuffle/sort/ShuffleInMemorySorter.java | 4 ++-- .../spark/shuffle/sort/ShuffleSortDataFormat.java | 2 +- .../unsafe/sort/UnsafeInMemorySorter.java | 2 +- .../sql/catalyst/expressions/UnsafeArrayData.java | 2 -- .../spark/sql/catalyst/expressions/UnsafeRow.java | 2 +- .../spark/sql/vectorized/ArrowColumnVector.java | 8 ++++---- 9 files changed, 15 insertions(+), 21 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index de0a3c63674bc..f372b19fac119 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -64,7 +64,6 @@ public static int hashUnsafeWordsBlock(MemoryBlock base, int seed) { public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, int seed) { // This is based on Guava's `Murmur32_Hasher.processRemaining(ByteBuffer)` method. - assert (lengthInBytes % 8 == 0): "lengthInBytes must be a multiple of 8 (word-aligned)"; return hashUnsafeWordsBlock(MemoryBlock.allocateFromObject(base, offset, lengthInBytes), seed); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index 123a4e21d48c7..b4a4ef9f07826 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -136,7 +136,7 @@ public final static MemoryBlock allocateFromObject(Object obj, long offset, long public abstract MemoryBlock subBlock(long offset, long size); protected void checkSubBlockRange(long offset, long size) { - if (offset < 0 || size < 0 || this.offset + offset < 0) { + if (offset < 0 || size < 0) { throw new ArrayIndexOutOfBoundsException( "Size " + size + " and offset " + offset + " must be non-negative"); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index f801f973383a2..c02bb4546a871 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -82,7 +82,7 @@ public final class UTF8String implements Comparable, Externalizable, public static UTF8String fromBytes(byte[] bytes) { if (bytes != null) { return new UTF8String( - MemoryBlock.allocateFromObject(bytes, BYTE_ARRAY_OFFSET, bytes.length)); + new ByteArrayMemoryBlock(bytes, BYTE_ARRAY_OFFSET, bytes.length)); } else { return null; } @@ -96,7 +96,7 @@ public static UTF8String fromBytes(byte[] bytes) { public static UTF8String fromBytes(byte[] bytes, int offset, int numBytes) { if (bytes != null) { return new UTF8String( - MemoryBlock.allocateFromObject(bytes, BYTE_ARRAY_OFFSET + offset, numBytes)); + new ByteArrayMemoryBlock(bytes, BYTE_ARRAY_OFFSET + offset, numBytes)); } else { return null; } @@ -120,15 +120,11 @@ public static UTF8String blankString(int length) { public UTF8String(MemoryBlock base) { this.base = base; - if (base != null) { - this.numBytes = Ints.checkedCast(base.size()); - } + this.numBytes = Ints.checkedCast(base.size()); } // for serialization - public UTF8String() { - this(null); - } + public UTF8String() {} /** * Writes the content of this string into a memory address, identified by an object and an offset. @@ -250,6 +246,7 @@ public long getPrefix() { } } p &= ~mask; + System.out.println("P: "+Long.toHexString(p)+", obj="+base.getBaseObject()+", offset="+base.getBaseOffset()+", size="+base.size()+", numBytes="+numBytes); return p; } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java index c9329d69fd742..8f49859746b89 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java @@ -104,7 +104,7 @@ public void reset() { public void expandPointerArray(LongArray newArray) { assert(newArray.size() > array.size()); - MemoryBlock.copyMemory(array.memoryBlock(), newArray.memoryBlock(),pos * 8L); + MemoryBlock.copyMemory(array.memoryBlock(), newArray.memoryBlock(), pos * 8L); consumer.freeArray(array); array = newArray; usableCapacity = getUsableCapacity(); @@ -173,7 +173,7 @@ public ShuffleSorterIterator getSortedIterator() { PackedRecordPointer.PARTITION_ID_START_BYTE_INDEX, PackedRecordPointer.PARTITION_ID_END_BYTE_INDEX, false, false); } else { - MemoryBlock unused = array.memoryBlock().subBlock(pos * 8L,(array.size() - pos) * 8L); + MemoryBlock unused = array.memoryBlock().subBlock(pos * 8L, (array.size() - pos) * 8L); LongArray buffer = new LongArray(unused); Sorter sorter = new Sorter<>(new ShuffleSortDataFormat(buffer)); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java index 086e169a42c59..254449e95443e 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java @@ -60,7 +60,7 @@ public void copyElement(LongArray src, int srcPos, LongArray dst, int dstPos) { @Override public void copyRange(LongArray src, int srcPos, LongArray dst, int dstPos, int length) { - MemoryBlock.copyMemory(src.memoryBlock(),srcPos * 8L, + MemoryBlock.copyMemory(src.memoryBlock(), srcPos * 8L, dst.memoryBlock(),dstPos * 8L,length * 8L); } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index db06e0e65fe6b..20a7a8b267438 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -342,7 +342,7 @@ public UnsafeSorterIterator getSortedIterator() { array, nullBoundaryPos, (pos - nullBoundaryPos) / 2L, 0, 7, radixSortSupport.sortDescending(), radixSortSupport.sortSigned()); } else { - MemoryBlock unused = array.memoryBlock().subBlock(pos * 8L,(array.size() - pos) * 8L); + MemoryBlock unused = array.memoryBlock().subBlock(pos * 8L, (array.size() - pos) * 8L); LongArray buffer = new LongArray(unused); Sorter sorter = new Sorter<>(new UnsafeSortDataFormat(buffer)); diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java index f80806c5f55e1..8546c28335536 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java @@ -27,8 +27,6 @@ import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.bitset.BitSetMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; -import org.apache.spark.unsafe.memory.ByteArrayMemoryBlock; -import org.apache.spark.unsafe.memory.OnHeapMemoryBlock; import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index 72097ba9c9abb..29a1411241cf6 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -37,7 +37,7 @@ import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.bitset.BitSetMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; -import org.apache.spark.unsafe.memory.*; +import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; diff --git a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java index 8357942abb26d..227a16f7e69e9 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java @@ -378,10 +378,10 @@ final UTF8String getUTF8String(int rowId) { if (stringResult.isSet == 0) { return null; } else { - int size = stringResult.end - stringResult.start; - OffHeapMemoryBlock mb = new OffHeapMemoryBlock( - stringResult.buffer.memoryAddress() + stringResult.start, size); - return new UTF8String(mb); + return new UTF8String(new OffHeapMemoryBlock( + stringResult.buffer.memoryAddress() + stringResult.start, + stringResult.end - stringResult.start + )); } } } From 94c9648220a61975a98dac36b2b053888ecc05ae Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 29 Mar 2018 17:27:02 +0100 Subject: [PATCH 62/65] address review comment --- .../src/main/java/org/apache/spark/unsafe/types/UTF8String.java | 1 - 1 file changed, 1 deletion(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index c02bb4546a871..302be6407d5db 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -246,7 +246,6 @@ public long getPrefix() { } } p &= ~mask; - System.out.println("P: "+Long.toHexString(p)+", obj="+base.getBaseObject()+", offset="+base.getBaseOffset()+", size="+base.size()+", numBytes="+numBytes); return p; } From e4a701681290b824241bc18f4d897e435d108693 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 5 Apr 2018 07:30:37 +0100 Subject: [PATCH 63/65] address review comments --- .../unsafe/memory/ByteArrayMemoryBlock.java | 2 +- .../apache/spark/unsafe/memory/MemoryBlock.java | 16 +++++++++------- .../spark/unsafe/memory/OnHeapMemoryBlock.java | 2 +- .../unsafe/memory/UnsafeMemoryAllocator.java | 6 ------ 4 files changed, 11 insertions(+), 15 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java index e7ee2e8fb7e78..a381f16c85c35 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -33,7 +33,7 @@ public ByteArrayMemoryBlock(byte[] obj, long offset, long size) { this.array = obj; assert(offset + size <= Platform.BYTE_ARRAY_OFFSET + obj.length) : "The sum of size " + size + " and offset " + offset + " should not be larger than " + - "the array size " + (obj.length + Platform.BYTE_ARRAY_OFFSET); + "the size of the given memory space " + (obj.length + Platform.BYTE_ARRAY_OFFSET); } public ByteArrayMemoryBlock(long length) { diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index b4a4ef9f07826..c9bf50ca3b850 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -61,7 +61,7 @@ public abstract class MemoryBlock { protected MemoryBlock(@Nullable Object obj, long offset, long length) { if (offset < 0 || length < 0) { - throw new ArrayIndexOutOfBoundsException( + throw new IllegalArgumentException( "Length " + length + " and offset " + offset + "must be non-negative"); } this.obj = obj; @@ -130,8 +130,8 @@ public final static MemoryBlock allocateFromObject(Object obj, long offset, long } /** - * Just instantiate the same type of MemoryBlock with new offset and size. The data is not - * copied. If parameters are invalid, an exception is thrown + * Just instantiate the same type of MemoryBlock with the new absolute offset and size. The data + * is not copied. If parameters are invalid, an exception is thrown */ public abstract MemoryBlock subBlock(long offset, long size); @@ -183,24 +183,26 @@ protected void checkSubBlockRange(long offset, long size) { public static final void copyMemory( MemoryBlock src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { - assert(length <= src.length && length <= dst.length); + assert(length <= (src.length - src.getBaseOffset()) && + length <= (dst.length - dst.getBaseOffset())); Platform.copyMemory(src.getBaseObject(), src.getBaseOffset() + srcOffset, dst.getBaseObject(), dst.getBaseOffset() + dstOffset, length); } public static final void copyMemory(MemoryBlock src, MemoryBlock dst, long length) { - assert(length <= src.length && length <= dst.length); + assert(length <= (src.length - src.getBaseOffset()) && + length <= (dst.length - dst.getBaseOffset())); Platform.copyMemory(src.getBaseObject(), src.getBaseOffset(), dst.getBaseObject(), dst.getBaseOffset(), length); } public final void copyFrom(Object src, long srcOffset, long dstOffset, long length) { - assert(length <= this.length); + assert(length <= this.length - srcOffset); Platform.copyMemory(src, srcOffset, obj, offset + dstOffset, length); } public final void writeTo(long srcOffset, Object dst, long dstOffset, long length) { - assert(length <= this.length); + assert(length <= this.length - srcOffset); Platform.copyMemory(obj, offset + srcOffset, dst, dstOffset, length); } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java index 1537f62d78217..bb2ca1b501e15 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java @@ -33,7 +33,7 @@ public OnHeapMemoryBlock(long[] obj, long offset, long size) { this.array = obj; assert(offset + size <= obj.length * 8L + Platform.LONG_ARRAY_OFFSET) : "The sum of size " + size + " and offset " + offset + " should not be larger than " + - "the array size " + (obj.length * 8L + Platform.LONG_ARRAY_OFFSET); + "the size of the given memory space " + (obj.length * 8L + Platform.LONG_ARRAY_OFFSET); } public OnHeapMemoryBlock(long size) { diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java index 43bd1c63c6bf2..5310bdf2779a9 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -19,12 +19,6 @@ import org.apache.spark.unsafe.Platform; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.nio.ByteBuffer; - -import sun.nio.ch.DirectBuffer; - /** * A simple {@link MemoryAllocator} that uses {@code Unsafe} to allocate off-heap memory. */ From 3d03f92d4cb0158fa608d8a3dc35962d3e0f0861 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 5 Apr 2018 10:29:36 +0100 Subject: [PATCH 64/65] Address review comments --- .../unsafe/memory/ByteArrayMemoryBlock.java | 1 + .../spark/unsafe/memory/MemoryBlock.java | 13 ++++----- .../unsafe/memory/OffHeapMemoryBlock.java | 1 + .../unsafe/memory/OnHeapMemoryBlock.java | 1 + .../spark/unsafe/memory/MemoryBlockSuite.java | 29 +++++++++++++++++-- 5 files changed, 35 insertions(+), 10 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java index a381f16c85c35..99a9868a49a79 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java @@ -43,6 +43,7 @@ public ByteArrayMemoryBlock(long length) { @Override public MemoryBlock subBlock(long offset, long size) { checkSubBlockRange(offset, size); + if (offset == 0 && size == this.size()) return this; return new ByteArrayMemoryBlock(array, this.offset + offset, size); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index c9bf50ca3b850..49d5384c5e611 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -130,8 +130,9 @@ public final static MemoryBlock allocateFromObject(Object obj, long offset, long } /** - * Just instantiate the same type of MemoryBlock with the new absolute offset and size. The data - * is not copied. If parameters are invalid, an exception is thrown + * Just instantiate the sub-block with the same type of MemoryBlock with the new size and relative + * offset from the original offset. The data is not copied. + * If parameters are invalid, an exception is thrown. */ public abstract MemoryBlock subBlock(long offset, long size); @@ -142,7 +143,7 @@ protected void checkSubBlockRange(long offset, long size) { } if (offset + size > length) { throw new ArrayIndexOutOfBoundsException("The sum of size " + size + " and offset " + - offset + " should not be larger than the length " + length + " in the MemoryBlock"); + offset + " should be equal to or subset of the original MemoryBlock"); } } @@ -183,15 +184,13 @@ protected void checkSubBlockRange(long offset, long size) { public static final void copyMemory( MemoryBlock src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { - assert(length <= (src.length - src.getBaseOffset()) && - length <= (dst.length - dst.getBaseOffset())); + assert(length <= src.length && length <= dst.length); Platform.copyMemory(src.getBaseObject(), src.getBaseOffset() + srcOffset, dst.getBaseObject(), dst.getBaseOffset() + dstOffset, length); } public static final void copyMemory(MemoryBlock src, MemoryBlock dst, long length) { - assert(length <= (src.length - src.getBaseOffset()) && - length <= (dst.length - dst.getBaseOffset())); + assert(length <= src.length && length <= dst.length); Platform.copyMemory(src.getBaseObject(), src.getBaseOffset(), dst.getBaseObject(), dst.getBaseOffset(), length); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java index d18e4744a5980..f90f62bf21dcb 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java @@ -29,6 +29,7 @@ public OffHeapMemoryBlock(long address, long size) { @Override public MemoryBlock subBlock(long offset, long size) { checkSubBlockRange(offset, size); + if (offset == 0 && size == this.size()) return this; return new OffHeapMemoryBlock(this.offset + offset, size); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java index bb2ca1b501e15..12f67c7bd593e 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java @@ -43,6 +43,7 @@ public OnHeapMemoryBlock(long size) { @Override public MemoryBlock subBlock(long offset, long size) { checkSubBlockRange(offset, size); + if (offset == 0 && size == this.size()) return this; return new OnHeapMemoryBlock(array, this.offset + offset, size); } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java index 18eca9331965a..0cfba0a2a0e88 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java @@ -84,18 +84,41 @@ private void check(MemoryBlock memory, Object obj, long offset, int length) { Assert.assertEquals((byte) -1, memory.getByte(i)); } + assert(memory.subBlock(0, memory.size()) == memory); + try { - memory.subBlock(-1, -1); + memory.subBlock(-8, 8); Assert.fail(); } catch (Exception expected) { Assert.assertThat(expected.getMessage(), containsString("non-negative")); } try { - memory.subBlock(16, length - 8); + memory.subBlock(0, -8); + Assert.fail(); + } catch (Exception expected) { + Assert.assertThat(expected.getMessage(), containsString("non-negative")); + } + + try { + memory.subBlock(0, length + 8); + Assert.fail(); + } catch (Exception expected) { + Assert.assertThat(expected.getMessage(), containsString("subset")); + } + + try { + memory.subBlock(8, length - 4); + Assert.fail(); + } catch (Exception expected) { + Assert.assertThat(expected.getMessage(), containsString("subset")); + } + + try { + memory.subBlock(length + 8, 4); Assert.fail(); } catch (Exception expected) { - Assert.assertThat(expected.getMessage(), containsString("should not be larger than")); + Assert.assertThat(expected.getMessage(), containsString("subset")); } } From 50326ca2fff63ef0a88f3c7924fa8f1d53d17ec1 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 5 Apr 2018 16:16:29 +0100 Subject: [PATCH 65/65] Address review comments --- .../java/org/apache/spark/unsafe/memory/MemoryBlock.java | 4 ++-- .../org/apache/spark/unsafe/memory/MemoryBlockSuite.java | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index 49d5384c5e611..b086941108522 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -143,7 +143,7 @@ protected void checkSubBlockRange(long offset, long size) { } if (offset + size > length) { throw new ArrayIndexOutOfBoundsException("The sum of size " + size + " and offset " + - offset + " should be equal to or subset of the original MemoryBlock"); + offset + " should not be larger than the length " + length + " in the MemoryBlock"); } } @@ -184,7 +184,7 @@ protected void checkSubBlockRange(long offset, long size) { public static final void copyMemory( MemoryBlock src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { - assert(length <= src.length && length <= dst.length); + assert(srcOffset + length <= src.length && dstOffset + length <= dst.length); Platform.copyMemory(src.getBaseObject(), src.getBaseOffset() + srcOffset, dst.getBaseObject(), dst.getBaseOffset() + dstOffset, length); } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java index 0cfba0a2a0e88..47f05c928f2e5 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java @@ -104,21 +104,21 @@ private void check(MemoryBlock memory, Object obj, long offset, int length) { memory.subBlock(0, length + 8); Assert.fail(); } catch (Exception expected) { - Assert.assertThat(expected.getMessage(), containsString("subset")); + Assert.assertThat(expected.getMessage(), containsString("should not be larger than")); } try { memory.subBlock(8, length - 4); Assert.fail(); } catch (Exception expected) { - Assert.assertThat(expected.getMessage(), containsString("subset")); + Assert.assertThat(expected.getMessage(), containsString("should not be larger than")); } try { memory.subBlock(length + 8, 4); Assert.fail(); } catch (Exception expected) { - Assert.assertThat(expected.getMessage(), containsString("subset")); + Assert.assertThat(expected.getMessage(), containsString("should not be larger than")); } }